spark git commit: [SPARK-8881] [SPARK-9260] Fix algorithm for scheduling executors on workers
Repository: spark Updated Branches: refs/heads/master b1f4b4abf -> 41a7cdf85 [SPARK-8881] [SPARK-9260] Fix algorithm for scheduling executors on workers Current scheduling algorithm allocates one core at a time and in doing so ends up ignoring spark.executor.cores. As a result, when spark.cores.max/spark.executor.cores (i.e, num_executors) < num_workers, executors are not launched and the app hangs. This PR fixes and refactors the scheduling algorithm. andrewor14 Author: Nishkam Ravi Author: nishkamravi2 Closes #7274 from nishkamravi2/master_scheduler and squashes the following commits: b998097 [nishkamravi2] Update Master.scala da0f491 [Nishkam Ravi] Update Master.scala 79084e8 [Nishkam Ravi] Update Master.scala 1daf25f [Nishkam Ravi] Update Master.scala f279cdf [Nishkam Ravi] Update Master.scala adec84b [Nishkam Ravi] Update Master.scala a06da76 [nishkamravi2] Update Master.scala 40c8f9f [nishkamravi2] Update Master.scala (to trigger retest) c11c689 [nishkamravi2] Update EventLoggingListenerSuite.scala 5d6a19c [nishkamravi2] Update Master.scala (for the purpose of issuing a retest) 2d6371c [Nishkam Ravi] Update Master.scala 66362d5 [nishkamravi2] Update Master.scala ee7cf0e [Nishkam Ravi] Improved scheduling algorithm for executors Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/41a7cdf8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/41a7cdf8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/41a7cdf8 Branch: refs/heads/master Commit: 41a7cdf85de2d583d8b8759941a9d6c6e98cae4d Parents: b1f4b4a Author: Nishkam Ravi Authored: Sat Jul 25 22:56:25 2015 -0700 Committer: Andrew Or Committed: Sat Jul 25 22:56:25 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 112 +-- 1 file changed, 75 insertions(+), 37 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/41a7cdf8/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 4615feb..029f94d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -541,6 +541,7 @@ private[master] class Master( /** * Schedule executors to be launched on the workers. + * Returns an array containing number of cores assigned to each worker. * * There are two modes of launching executors. The first attempts to spread out an application's * executors on as many workers as possible, while the second does the opposite (i.e. launch them @@ -551,39 +552,73 @@ private[master] class Master( * multiple executors from the same application may be launched on the same worker if the worker * has enough cores and memory. Otherwise, each executor grabs all the cores available on the * worker by default, in which case only one executor may be launched on each worker. + * + * It is important to allocate coresPerExecutor on each worker at a time (instead of 1 core + * at a time). Consider the following example: cluster has 4 workers with 16 cores each. + * User requests 3 executors (spark.cores.max = 48, spark.executor.cores = 16). If 1 core is + * allocated at a time, 12 cores from each worker would be assigned to each executor. + * Since 12 < 16, no executors would launch [SPARK-8881]. */ - private def startExecutorsOnWorkers(): Unit = { -// Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app -// in the queue, then the second app, etc. -if (spreadOutApps) { - // Try to spread out each app among all the workers, until it has all its cores - for (app <- waitingApps if app.coresLeft > 0) { -val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && -worker.coresFree >= app.desc.coresPerExecutor.getOrElse(1)) - .sortBy(_.coresFree).reverse -val numUsable = usableWorkers.length -val assigned = new Array[Int](numUsable) // Number of cores to give on each node -var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) -var pos = 0 -while (toAssign > 0) { - if (usableWorkers(pos).coresFree - assigned(pos) > 0) { -toAssign -= 1 -assigned(pos) += 1 + private[master] def scheduleExecutorsOnWorkers( + app: ApplicationInfo, + usableWorkers: Array[WorkerInfo], + spreadOutApps: Boolean): Array[Int] = { +// If the number o
spark git commit: [SPARK-8881] [SPARK-9260] Fix algorithm for scheduling executors on workers
Repository: spark Updated Branches: refs/heads/branch-1.4 712e13bba -> a37796be7 [SPARK-8881] [SPARK-9260] Fix algorithm for scheduling executors on workers Current scheduling algorithm allocates one core at a time and in doing so ends up ignoring spark.executor.cores. As a result, when spark.cores.max/spark.executor.cores (i.e, num_executors) < num_workers, executors are not launched and the app hangs. This PR fixes and refactors the scheduling algorithm. andrewor14 Author: Nishkam Ravi Author: nishkamravi2 Closes #7274 from nishkamravi2/master_scheduler and squashes the following commits: b998097 [nishkamravi2] Update Master.scala da0f491 [Nishkam Ravi] Update Master.scala 79084e8 [Nishkam Ravi] Update Master.scala 1daf25f [Nishkam Ravi] Update Master.scala f279cdf [Nishkam Ravi] Update Master.scala adec84b [Nishkam Ravi] Update Master.scala a06da76 [nishkamravi2] Update Master.scala 40c8f9f [nishkamravi2] Update Master.scala (to trigger retest) c11c689 [nishkamravi2] Update EventLoggingListenerSuite.scala 5d6a19c [nishkamravi2] Update Master.scala (for the purpose of issuing a retest) 2d6371c [Nishkam Ravi] Update Master.scala 66362d5 [nishkamravi2] Update Master.scala ee7cf0e [Nishkam Ravi] Improved scheduling algorithm for executors (cherry picked from commit 41a7cdf85de2d583d8b8759941a9d6c6e98cae4d) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a37796be Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a37796be Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a37796be Branch: refs/heads/branch-1.4 Commit: a37796be7366bfa4b51f5f4675537c10de4469fe Parents: 712e13b Author: Nishkam Ravi Authored: Sat Jul 25 22:56:25 2015 -0700 Committer: Andrew Or Committed: Sat Jul 25 22:56:36 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 112 +-- 1 file changed, 75 insertions(+), 37 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a37796be/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index fccceb3..ac74eba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -533,6 +533,7 @@ private[master] class Master( /** * Schedule executors to be launched on the workers. + * Returns an array containing number of cores assigned to each worker. * * There are two modes of launching executors. The first attempts to spread out an application's * executors on as many workers as possible, while the second does the opposite (i.e. launch them @@ -543,39 +544,73 @@ private[master] class Master( * multiple executors from the same application may be launched on the same worker if the worker * has enough cores and memory. Otherwise, each executor grabs all the cores available on the * worker by default, in which case only one executor may be launched on each worker. + * + * It is important to allocate coresPerExecutor on each worker at a time (instead of 1 core + * at a time). Consider the following example: cluster has 4 workers with 16 cores each. + * User requests 3 executors (spark.cores.max = 48, spark.executor.cores = 16). If 1 core is + * allocated at a time, 12 cores from each worker would be assigned to each executor. + * Since 12 < 16, no executors would launch [SPARK-8881]. */ - private def startExecutorsOnWorkers(): Unit = { -// Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app -// in the queue, then the second app, etc. -if (spreadOutApps) { - // Try to spread out each app among all the workers, until it has all its cores - for (app <- waitingApps if app.coresLeft > 0) { -val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && -worker.coresFree >= app.desc.coresPerExecutor.getOrElse(1)) - .sortBy(_.coresFree).reverse -val numUsable = usableWorkers.length -val assigned = new Array[Int](numUsable) // Number of cores to give on each node -var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) -var pos = 0 -while (toAssign > 0) { - if (usableWorkers(pos).coresFree - assigned(pos) > 0) { -toAssign -= 1 -assigned(pos) += 1 + private[master] def scheduleExecutorsOnWorkers( + app: ApplicationI
spark git commit: [SPARK-9352] [SPARK-9353] Add tests for standalone scheduling code
Repository: spark Updated Branches: refs/heads/master fb5d43fb2 -> 1cf19760d [SPARK-9352] [SPARK-9353] Add tests for standalone scheduling code This also fixes a small issue in the standalone Master that was uncovered by the new tests. For more detail, read the description of SPARK-9353. Author: Andrew Or Closes #7668 from andrewor14/standalone-scheduling-tests and squashes the following commits: d852faf [Andrew Or] Add tests + fix scheduling with memory limits Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1cf19760 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1cf19760 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1cf19760 Branch: refs/heads/master Commit: 1cf19760d61a5a17bd175a906d34a2940141b76d Parents: fb5d43f Author: Andrew Or Authored: Sun Jul 26 13:03:13 2015 -0700 Committer: Andrew Or Committed: Sun Jul 26 13:03:13 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 8 +- .../spark/deploy/master/MasterSuite.scala | 199 ++- 2 files changed, 202 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1cf19760/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 029f94d..51b3f0d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -559,7 +559,7 @@ private[master] class Master( * allocated at a time, 12 cores from each worker would be assigned to each executor. * Since 12 < 16, no executors would launch [SPARK-8881]. */ - private[master] def scheduleExecutorsOnWorkers( + private def scheduleExecutorsOnWorkers( app: ApplicationInfo, usableWorkers: Array[WorkerInfo], spreadOutApps: Boolean): Array[Int] = { @@ -585,7 +585,11 @@ private[master] class Master( while (keepScheduling && canLaunchExecutor(pos) && coresToAssign >= coresPerExecutor) { coresToAssign -= coresPerExecutor assignedCores(pos) += coresPerExecutor - assignedMemory(pos) += memoryPerExecutor + // If cores per executor is not set, we are assigning 1 core at a time + // without actually meaning to launch 1 executor for each core assigned + if (app.desc.coresPerExecutor.isDefined) { +assignedMemory(pos) += memoryPerExecutor + } // Spreading out an application means spreading out its executors across as // many workers as possible. If we are not spreading out, then we should keep http://git-wip-us.apache.org/repos/asf/spark/blob/1cf19760/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index a8fbaf1..4d7016d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -25,14 +25,15 @@ import scala.language.postfixOps import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.scalatest.Matchers +import org.scalatest.{Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy._ +import org.apache.spark.rpc.RpcEnv -class MasterSuite extends SparkFunSuite with Matchers with Eventually { +class MasterSuite extends SparkFunSuite with Matchers with Eventually with PrivateMethodTester { test("can use a custom recovery mode factory") { val conf = new SparkConf(loadDefaults = false) @@ -142,4 +143,196 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually { } } + test("basic scheduling - spread out") { +testBasicScheduling(spreadOut = true) + } + + test("basic scheduling - no spread out") { +testBasicScheduling(spreadOut = false) + } + + test("scheduling with max cores - spread out") { +testSchedulingWithMaxCores(spreadOut = true) + } + + test("scheduling with max cores - no spread out") { +testSchedulingWithMaxCores(spreadOut = false) + } + + test("scheduling with cores per executor - spread out") { +testSchedulingWith
spark git commit: [SPARK-9352] [SPARK-9353] Add tests for standalone scheduling code
Repository: spark Updated Branches: refs/heads/branch-1.4 a37796be7 -> a4b80e6db [SPARK-9352] [SPARK-9353] Add tests for standalone scheduling code This also fixes a small issue in the standalone Master that was uncovered by the new tests. For more detail, read the description of SPARK-9353. Author: Andrew Or Closes #7668 from andrewor14/standalone-scheduling-tests and squashes the following commits: d852faf [Andrew Or] Add tests + fix scheduling with memory limits (cherry picked from commit 1cf19760d61a5a17bd175a906d34a2940141b76d) Signed-off-by: Andrew Or Conflicts: core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a4b80e6d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a4b80e6d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a4b80e6d Branch: refs/heads/branch-1.4 Commit: a4b80e6dbd007c3d0cc012bacc19361d9eb00b02 Parents: a37796b Author: Andrew Or Authored: Sun Jul 26 13:03:13 2015 -0700 Committer: Andrew Or Committed: Sun Jul 26 13:04:20 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 8 +- .../spark/deploy/master/MasterSuite.scala | 199 ++- 2 files changed, 202 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a4b80e6d/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index ac74eba..b4c8771 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -551,7 +551,7 @@ private[master] class Master( * allocated at a time, 12 cores from each worker would be assigned to each executor. * Since 12 < 16, no executors would launch [SPARK-8881]. */ - private[master] def scheduleExecutorsOnWorkers( + private def scheduleExecutorsOnWorkers( app: ApplicationInfo, usableWorkers: Array[WorkerInfo], spreadOutApps: Boolean): Array[Int] = { @@ -577,7 +577,11 @@ private[master] class Master( while (keepScheduling && canLaunchExecutor(pos) && coresToAssign >= coresPerExecutor) { coresToAssign -= coresPerExecutor assignedCores(pos) += coresPerExecutor - assignedMemory(pos) += memoryPerExecutor + // If cores per executor is not set, we are assigning 1 core at a time + // without actually meaning to launch 1 executor for each core assigned + if (app.desc.coresPerExecutor.isDefined) { +assignedMemory(pos) += memoryPerExecutor + } // Spreading out an application means spreading out its executors across as // many workers as possible. If we are not spreading out, then we should keep http://git-wip-us.apache.org/repos/asf/spark/blob/a4b80e6d/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 014e87b..bde349c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -27,14 +27,15 @@ import scala.language.postfixOps import akka.actor.Address import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.scalatest.Matchers +import org.scalatest.{Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy._ +import org.apache.spark.rpc.RpcEnv -class MasterSuite extends SparkFunSuite with Matchers with Eventually { +class MasterSuite extends SparkFunSuite with Matchers with Eventually with PrivateMethodTester { test("toAkkaUrl") { val conf = new SparkConf(loadDefaults = false) @@ -184,4 +185,196 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually { } } + test("basic scheduling - spread out") { +testBasicScheduling(spreadOut = true) + } + + test("basic scheduling - no spread out") { +testBasicScheduling(spreadOut = false) + } + + test("scheduling with max cores - spread out") { +testSchedulingWithMaxCores(spreadOut
spark git commit: [HOTFIX] Fix compile in MasterSuite
Repository: spark Updated Branches: refs/heads/branch-1.4 d0ef42c4e -> 5aed92613 [HOTFIX] Fix compile in MasterSuite Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5aed9261 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5aed9261 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5aed9261 Branch: refs/heads/branch-1.4 Commit: 5aed9261380a183cc7aa292e1cd63f22e772fa5c Parents: d0ef42c Author: Andrew Or Authored: Sun Jul 26 21:45:29 2015 -0700 Committer: Andrew Or Committed: Sun Jul 26 21:45:29 2015 -0700 -- .../test/scala/org/apache/spark/deploy/master/MasterSuite.scala| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5aed9261/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index bde349c..e585c98 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -357,7 +357,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva private def makeMaster(conf: SparkConf = new SparkConf): Master = { val securityMgr = new SecurityManager(conf) -val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 7077, conf, securityMgr) +val rpcEnv = RpcEnv.create(Master.systemName, "localhost", 7077, conf, securityMgr) val master = new Master(rpcEnv, rpcEnv.address, 8080, securityMgr, conf) master } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9352] [SPARK-9353] Add tests for standalone scheduling code
Repository: spark Updated Branches: refs/heads/branch-1.4 43035b4b4 -> c103c99d2 [SPARK-9352] [SPARK-9353] Add tests for standalone scheduling code This also fixes a small issue in the standalone Master that was uncovered by the new tests. For more detail, read the description of SPARK-9353. Author: Andrew Or Closes #7668 from andrewor14/standalone-scheduling-tests and squashes the following commits: d852faf [Andrew Or] Add tests + fix scheduling with memory limits Conflicts: core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c103c99d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c103c99d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c103c99d Branch: refs/heads/branch-1.4 Commit: c103c99d23952a329ae56bde405e194ae7aac301 Parents: 43035b4 Author: Andrew Or Authored: Sun Jul 26 13:03:13 2015 -0700 Committer: Andrew Or Committed: Mon Jul 27 17:58:58 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 135 --- .../spark/deploy/master/MasterSuite.scala | 170 +++ 2 files changed, 242 insertions(+), 63 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c103c99d/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index ac74eba..821c430 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -532,67 +532,6 @@ private[master] class Master( } /** - * Schedule executors to be launched on the workers. - * Returns an array containing number of cores assigned to each worker. - * - * There are two modes of launching executors. The first attempts to spread out an application's - * executors on as many workers as possible, while the second does the opposite (i.e. launch them - * on as few workers as possible). The former is usually better for data locality purposes and is - * the default. - * - * The number of cores assigned to each executor is configurable. When this is explicitly set, - * multiple executors from the same application may be launched on the same worker if the worker - * has enough cores and memory. Otherwise, each executor grabs all the cores available on the - * worker by default, in which case only one executor may be launched on each worker. - * - * It is important to allocate coresPerExecutor on each worker at a time (instead of 1 core - * at a time). Consider the following example: cluster has 4 workers with 16 cores each. - * User requests 3 executors (spark.cores.max = 48, spark.executor.cores = 16). If 1 core is - * allocated at a time, 12 cores from each worker would be assigned to each executor. - * Since 12 < 16, no executors would launch [SPARK-8881]. - */ - private[master] def scheduleExecutorsOnWorkers( - app: ApplicationInfo, - usableWorkers: Array[WorkerInfo], - spreadOutApps: Boolean): Array[Int] = { -// If the number of cores per executor is not specified, then we can just schedule -// 1 core at a time since we expect a single executor to be launched on each worker -val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(1) -val memoryPerExecutor = app.desc.memoryPerExecutorMB -val numUsable = usableWorkers.length -val assignedCores = new Array[Int](numUsable) // Number of cores to give to each worker -val assignedMemory = new Array[Int](numUsable) // Amount of memory to give to each worker -var coresToAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) -var freeWorkers = (0 until numUsable).toIndexedSeq - -def canLaunchExecutor(pos: Int): Boolean = { - usableWorkers(pos).coresFree - assignedCores(pos) >= coresPerExecutor && - usableWorkers(pos).memoryFree - assignedMemory(pos) >= memoryPerExecutor -} - -while (coresToAssign >= coresPerExecutor && freeWorkers.nonEmpty) { - freeWorkers = freeWorkers.filter(canLaunchExecutor) - freeWorkers.foreach { pos => -var keepScheduling = true -while (keepScheduling && canLaunchExecutor(pos) && coresToAssign >= coresPerExecutor) { - coresToAssign -= coresPerExecutor - assignedCores(pos) += coresPerExecutor - assignedMemory(pos) += memoryPerExecutor - - // Spreading out an application means spreading out its executors across as - // many workers as possible. If we are not sp
spark git commit: [SPARK-4751] Dynamic allocation in standalone mode
Repository: spark Updated Branches: refs/heads/master c5166f7a6 -> 6688ba6e6 [SPARK-4751] Dynamic allocation in standalone mode Dynamic allocation is a feature that allows a Spark application to scale the number of executors up and down dynamically based on the workload. Support was first introduced in YARN since 1.2, and then extended to Mesos coarse-grained mode recently. Today, it is finally supported in standalone mode as well! I tested this locally and it works as expected. This is WIP because unit tests are coming. Author: Andrew Or Closes #7532 from andrewor14/standalone-da and squashes the following commits: b3c1736 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da 879e928 [Andrew Or] Add end-to-end tests for standalone dynamic allocation accc8f6 [Andrew Or] Address comments ee686a8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da c0a2c02 [Andrew Or] Fix build after merge conflict 24149eb [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da 2e762d6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da 6832bd7 [Andrew Or] Add tests for scheduling with executor limit a82e907 [Andrew Or] Fix comments 0a8be79 [Andrew Or] Simplify logic by removing the worker blacklist b7742af [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da 2eb5f3f [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da 1334e9a [Andrew Or] Fix MiMa 32abe44 [Andrew Or] Fix style 58cb06f [Andrew Or] Privatize worker blacklist for cleanliness 42ac215 [Andrew Or] Clean up comments and rewrite code for readability 49702d1 [Andrew Or] Clean up shuffle files after application exits 80047aa [Andrew Or] First working implementation Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6688ba6e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6688ba6e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6688ba6e Branch: refs/heads/master Commit: 6688ba6e68e342201b81ea09cc2c6ba216f90f3e Parents: c5166f7 Author: Andrew Or Authored: Sat Aug 1 11:57:14 2015 -0700 Committer: Andrew Or Committed: Sat Aug 1 11:57:14 2015 -0700 -- .../org/apache/spark/HeartbeatReceiver.scala| 16 +- .../scala/org/apache/spark/SparkContext.scala | 27 +- .../org/apache/spark/deploy/DeployMessage.scala | 4 + .../spark/deploy/ExternalShuffleService.scala | 5 + .../apache/spark/deploy/client/AppClient.scala | 45 +++ .../spark/deploy/master/ApplicationInfo.scala | 13 +- .../org/apache/spark/deploy/master/Master.scala | 144 +++- .../org/apache/spark/deploy/worker/Worker.scala | 13 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 5 +- .../cluster/SparkDeploySchedulerBackend.scala | 28 ++ .../StandaloneDynamicAllocationSuite.scala | 363 +++ .../spark/deploy/master/MasterSuite.scala | 256 +++-- project/MimaExcludes.scala | 4 + 13 files changed, 753 insertions(+), 170 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6688ba6e/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala -- diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 43dd4a1..ee60d69 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -177,16 +177,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms") scheduler.executorLost(executorId, SlaveLost("Executor heartbeat " + s"timed out after ${now - lastSeenMs} ms")) -if (sc.supportDynamicAllocation) { // Asynchronously kill the executor to avoid blocking the current thread - killExecutorThread.submit(new Runnable { -override def run(): Unit = Utils.tryLogNonFatalError { - // Note: we want to get an executor back after expiring this one, - // so do not simply call `sc.killExecutor` here (SPARK-8119) - sc.killAndReplaceExecutor(executorId) -} - }) -} +killExecutorThread.submit(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { +// Note: we want to get an executor back after expiring this one, +// so do not simply call `sc.killExecutor` here (SPARK-8119) +sc.killAndReplaceExecutor(executorId) +
spark git commit: [SPARK-8873] [MESOS] Clean up shuffle files if external shuffle service is used
Repository: spark Updated Branches: refs/heads/master 1ebd41b14 -> 95dccc633 [SPARK-8873] [MESOS] Clean up shuffle files if external shuffle service is used This patch builds directly on #7820, which is largely written by tnachen. The only addition is one commit for cleaning up the code. There should be no functional differences between this and #7820. Author: Timothy Chen Author: Andrew Or Closes #7881 from andrewor14/tim-cleanup-mesos-shuffle and squashes the following commits: 8894f7d [Andrew Or] Clean up code 2a5fa10 [Andrew Or] Merge branch 'mesos_shuffle_clean' of github.com:tnachen/spark into tim-cleanup-mesos-shuffle fadff89 [Timothy Chen] Address comments. e4d0f1d [Timothy Chen] Clean up external shuffle data on driver exit with Mesos. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/95dccc63 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/95dccc63 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/95dccc63 Branch: refs/heads/master Commit: 95dccc63350c45045f038bab9f8a5080b4e1f8cc Parents: 1ebd41b Author: Timothy Chen Authored: Mon Aug 3 01:55:58 2015 -0700 Committer: Andrew Or Committed: Mon Aug 3 01:55:58 2015 -0700 -- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/deploy/ExternalShuffleService.scala | 17 ++- .../mesos/MesosExternalShuffleService.scala | 107 +++ .../org/apache/spark/rpc/RpcEndpoint.scala | 6 +- .../mesos/CoarseMesosSchedulerBackend.scala | 52 - .../CoarseMesosSchedulerBackendSuite.scala | 5 +- .../launcher/SparkClassCommandBuilder.java | 3 +- .../spark/network/client/TransportClient.java | 5 + .../shuffle/ExternalShuffleBlockHandler.java| 6 ++ .../network/shuffle/ExternalShuffleClient.java | 12 ++- .../mesos/MesosExternalShuffleClient.java | 72 + .../shuffle/protocol/BlockTransferMessage.java | 4 +- .../shuffle/protocol/mesos/RegisterDriver.java | 60 +++ sbin/start-mesos-shuffle-service.sh | 35 ++ sbin/stop-mesos-shuffle-service.sh | 25 + 15 files changed, 394 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/95dccc63/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a1c66ef..6f336a7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2658,7 +2658,7 @@ object SparkContext extends Logging { val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false) val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, sc, url) + new CoarseMesosSchedulerBackend(scheduler, sc, url, sc.env.securityManager) } else { new MesosSchedulerBackend(scheduler, sc, url) } http://git-wip-us.apache.org/repos/asf/spark/blob/95dccc63/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index 4089c3e..20a9faa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -27,6 +27,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.sasl.SaslServerBootstrap import org.apache.spark.network.server.TransportServer import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler +import org.apache.spark.network.util.TransportConf import org.apache.spark.util.Utils /** @@ -45,11 +46,16 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana private val useSasl: Boolean = securityManager.isAuthenticationEnabled() private val transportConf = SparkTransportConf.fromSparkConf(sparkConf, numUsableCores = 0) - private val blockHandler = new ExternalShuffleBlockHandler(transportConf) + private val blockHandler = newShuffleBlockHandler(transportConf) private val transportContext: TransportContext = new TransportContext(transportConf, blockHandler) private var server: TransportServer = _ + /** Create a new shuffle block handler. Factored out for subclasses to override. */ + protected def newShuffl
spark git commit: [SPARK-9731] Standalone scheduling incorrect cores if spark.executor.cores is not set
Repository: spark Updated Branches: refs/heads/master c564b2744 -> ef062c159 [SPARK-9731] Standalone scheduling incorrect cores if spark.executor.cores is not set The issue only happens if `spark.executor.cores` is not set and executor memory is set to a high value. For example, if we have a worker with 4G and 10 cores and we set `spark.executor.memory` to 3G, then only 1 core is assigned to the executor. The correct number should be 10 cores. I've added a unit test to illustrate the issue. Author: Carson Wang Closes #8017 from carsonwang/SPARK-9731 and squashes the following commits: d09ec48 [Carson Wang] Fix code style 86b651f [Carson Wang] Simplify the code 943cc4c [Carson Wang] fix scheduling correct cores to executors Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ef062c15 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ef062c15 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ef062c15 Branch: refs/heads/master Commit: ef062c15992b0d08554495b8ea837bef3fabf6e9 Parents: c564b27 Author: Carson Wang Authored: Fri Aug 7 23:36:26 2015 -0700 Committer: Andrew Or Committed: Fri Aug 7 23:36:26 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 26 +++- .../spark/deploy/master/MasterSuite.scala | 15 +++ 2 files changed, 29 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ef062c15/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index e38e437..9217202 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -581,20 +581,22 @@ private[deploy] class Master( /** Return whether the specified worker can launch an executor for this app. */ def canLaunchExecutor(pos: Int): Boolean = { + val keepScheduling = coresToAssign >= minCoresPerExecutor + val enoughCores = usableWorkers(pos).coresFree - assignedCores(pos) >= minCoresPerExecutor + // If we allow multiple executors per worker, then we can always launch new executors. - // Otherwise, we may have already started assigning cores to the executor on this worker. + // Otherwise, if there is already an executor on this worker, just give it more cores. val launchingNewExecutor = !oneExecutorPerWorker || assignedExecutors(pos) == 0 - val underLimit = -if (launchingNewExecutor) { - assignedExecutors.sum + app.executors.size < app.executorLimit -} else { - true -} - val assignedMemory = assignedExecutors(pos) * memoryPerExecutor - usableWorkers(pos).memoryFree - assignedMemory >= memoryPerExecutor && - usableWorkers(pos).coresFree - assignedCores(pos) >= minCoresPerExecutor && - coresToAssign >= minCoresPerExecutor && - underLimit + if (launchingNewExecutor) { +val assignedMemory = assignedExecutors(pos) * memoryPerExecutor +val enoughMemory = usableWorkers(pos).memoryFree - assignedMemory >= memoryPerExecutor +val underLimit = assignedExecutors.sum + app.executors.size < app.executorLimit +keepScheduling && enoughCores && enoughMemory && underLimit + } else { +// We're adding cores to an existing executor, so no need +// to check memory and executor limits +keepScheduling && enoughCores + } } // Keep launching executors until no more workers can accommodate any http://git-wip-us.apache.org/repos/asf/spark/blob/ef062c15/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index ae0e037..20d0201 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -151,6 +151,14 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva basicScheduling(spreadOut = false) } + test("basic scheduling with more memory - spread out") { +basicSchedulingWithMoreMemory(spreadOut = true) + } + + test("basic scheduling with more memory - no spread out") { +basicSchedulingWithMoreMemory(spreadOut = false) + } + test("scheduling with max cores - spread out") { schedulingWithMaxCores(spreadOut = true) } @@ -214,6 +222,13 @@ class MasterSuite extends SparkFunSuite
spark git commit: [SPARK-9731] Standalone scheduling incorrect cores if spark.executor.cores is not set
Repository: spark Updated Branches: refs/heads/branch-1.5 47e473550 -> 2ad75d99f [SPARK-9731] Standalone scheduling incorrect cores if spark.executor.cores is not set The issue only happens if `spark.executor.cores` is not set and executor memory is set to a high value. For example, if we have a worker with 4G and 10 cores and we set `spark.executor.memory` to 3G, then only 1 core is assigned to the executor. The correct number should be 10 cores. I've added a unit test to illustrate the issue. Author: Carson Wang Closes #8017 from carsonwang/SPARK-9731 and squashes the following commits: d09ec48 [Carson Wang] Fix code style 86b651f [Carson Wang] Simplify the code 943cc4c [Carson Wang] fix scheduling correct cores to executors (cherry picked from commit ef062c15992b0d08554495b8ea837bef3fabf6e9) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2ad75d99 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2ad75d99 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2ad75d99 Branch: refs/heads/branch-1.5 Commit: 2ad75d99f1652aea2f00d6eee7aa49301f56c67d Parents: 47e4735 Author: Carson Wang Authored: Fri Aug 7 23:36:26 2015 -0700 Committer: Andrew Or Committed: Fri Aug 7 23:36:43 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 26 +++- .../spark/deploy/master/MasterSuite.scala | 15 +++ 2 files changed, 29 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2ad75d99/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index e38e437..9217202 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -581,20 +581,22 @@ private[deploy] class Master( /** Return whether the specified worker can launch an executor for this app. */ def canLaunchExecutor(pos: Int): Boolean = { + val keepScheduling = coresToAssign >= minCoresPerExecutor + val enoughCores = usableWorkers(pos).coresFree - assignedCores(pos) >= minCoresPerExecutor + // If we allow multiple executors per worker, then we can always launch new executors. - // Otherwise, we may have already started assigning cores to the executor on this worker. + // Otherwise, if there is already an executor on this worker, just give it more cores. val launchingNewExecutor = !oneExecutorPerWorker || assignedExecutors(pos) == 0 - val underLimit = -if (launchingNewExecutor) { - assignedExecutors.sum + app.executors.size < app.executorLimit -} else { - true -} - val assignedMemory = assignedExecutors(pos) * memoryPerExecutor - usableWorkers(pos).memoryFree - assignedMemory >= memoryPerExecutor && - usableWorkers(pos).coresFree - assignedCores(pos) >= minCoresPerExecutor && - coresToAssign >= minCoresPerExecutor && - underLimit + if (launchingNewExecutor) { +val assignedMemory = assignedExecutors(pos) * memoryPerExecutor +val enoughMemory = usableWorkers(pos).memoryFree - assignedMemory >= memoryPerExecutor +val underLimit = assignedExecutors.sum + app.executors.size < app.executorLimit +keepScheduling && enoughCores && enoughMemory && underLimit + } else { +// We're adding cores to an existing executor, so no need +// to check memory and executor limits +keepScheduling && enoughCores + } } // Keep launching executors until no more workers can accommodate any http://git-wip-us.apache.org/repos/asf/spark/blob/2ad75d99/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index ae0e037..20d0201 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -151,6 +151,14 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva basicScheduling(spreadOut = false) } + test("basic scheduling with more memory - spread out") { +basicSchedulingWithMoreMemory(spreadOut = true) + } + + test("basic scheduling with more memory - no spread out") { +basicSchedulingWithMoreMemory(spreadOut = false) + } + test("scheduling with max cores - spread out") { sch
spark git commit: [HOTFIX] Fix style error caused by 017b5de
Repository: spark Updated Branches: refs/heads/master 017b5de07 -> 736af95bd [HOTFIX] Fix style error caused by 017b5de Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/736af95b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/736af95b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/736af95b Branch: refs/heads/master Commit: 736af95bd0c41723d455246b634a0fb68b38a7c7 Parents: 017b5de Author: Andrew Or Authored: Tue Aug 11 14:52:52 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 14:52:52 2015 -0700 -- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/736af95b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 26eb84a..11ed231 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -114,7 +114,7 @@ object MLUtils { } // Convenient methods for `loadLibSVMFile`. - + /** * @since 1.0.0 */ - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [HOTFIX] Fix style error caused by ef961ed48a4f45447f0e0ad256b040c7ab2d78d9
Repository: spark Updated Branches: refs/heads/branch-1.5 725e5c7a4 -> 1067c7369 [HOTFIX] Fix style error caused by ef961ed48a4f45447f0e0ad256b040c7ab2d78d9 Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1067c736 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1067c736 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1067c736 Branch: refs/heads/branch-1.5 Commit: 1067c73693c52facddfb5e425e9caaf7a1cb364b Parents: 725e5c7 Author: Andrew Or Authored: Tue Aug 11 14:52:52 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 14:57:23 2015 -0700 -- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1067c736/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 26eb84a..11ed231 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -114,7 +114,7 @@ object MLUtils { } // Convenient methods for `loadLibSVMFile`. - + /** * @since 1.0.0 */ - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8366] maxNumExecutorsNeeded should properly handle failed tasks
Repository: spark Updated Branches: refs/heads/branch-1.5 b994f8928 -> 2f909184e [SPARK-8366] maxNumExecutorsNeeded should properly handle failed tasks Author: xutingjun Author: meiyoula <1039320...@qq.com> Closes #6817 from XuTingjun/SPARK-8366. (cherry picked from commit b85f9a242a12e8096e331fa77d5ebd16e93c844d) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2f909184 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2f909184 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2f909184 Branch: refs/heads/branch-1.5 Commit: 2f909184e2b346ba920129619fe8d45b20ae0573 Parents: b994f89 Author: xutingjun Authored: Tue Aug 11 23:19:35 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:19:45 2015 -0700 -- .../spark/ExecutorAllocationManager.scala | 22 +--- .../spark/ExecutorAllocationManagerSuite.scala | 22 ++-- 2 files changed, 34 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2f909184/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 1877aaf..b93536e 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -599,14 +599,8 @@ private[spark] class ExecutorAllocationManager( // If this is the last pending task, mark the scheduler queue as empty stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex -val numTasksScheduled = stageIdToTaskIndices(stageId).size -val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) -if (numTasksScheduled == numTasksTotal) { - // No more pending tasks for this stage - stageIdToNumTasks -= stageId - if (stageIdToNumTasks.isEmpty) { -allocationManager.onSchedulerQueueEmpty() - } +if (totalPendingTasks() == 0) { + allocationManager.onSchedulerQueueEmpty() } // Mark the executor on which this task is scheduled as busy @@ -618,6 +612,8 @@ private[spark] class ExecutorAllocationManager( override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { val executorId = taskEnd.taskInfo.executorId val taskId = taskEnd.taskInfo.taskId + val taskIndex = taskEnd.taskInfo.index + val stageId = taskEnd.stageId allocationManager.synchronized { numRunningTasks -= 1 // If the executor is no longer running any scheduled tasks, mark it as idle @@ -628,6 +624,16 @@ private[spark] class ExecutorAllocationManager( allocationManager.onExecutorIdle(executorId) } } + +// If the task failed, we expect it to be resubmitted later. To ensure we have +// enough resources to run the resubmitted task, we need to mark the scheduler +// as backlogged again if it's not already marked as such (SPARK-8366) +if (taskEnd.reason != Success) { + if (totalPendingTasks() == 0) { +allocationManager.onSchedulerBacklogged() + } + stageIdToTaskIndices.get(stageId).foreach { _.remove(taskIndex) } +} } } http://git-wip-us.apache.org/repos/asf/spark/blob/2f909184/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 34caca8..f374f97 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -206,8 +206,8 @@ class ExecutorAllocationManagerSuite val task2Info = createTaskInfo(1, 0, "executor-1") sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task2Info)) -sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task1Info, null)) -sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task2Info, null)) +sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, Success, task1Info, null)) +sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, Success, task2Info, null)) assert(adjustRequestedExecutors(manager) === -1) } @@ -787,6 +787,24 @@ class ExecutorAllocationManagerSuite Map("host2" -> 1, "host3" -> 2, "host4" -> 1, "host5" -> 2)) } + test("SPARK-8366: maxNumExec
spark git commit: [SPARK-8366] maxNumExecutorsNeeded should properly handle failed tasks
Repository: spark Updated Branches: refs/heads/master b1581ac28 -> b85f9a242 [SPARK-8366] maxNumExecutorsNeeded should properly handle failed tasks Author: xutingjun Author: meiyoula <1039320...@qq.com> Closes #6817 from XuTingjun/SPARK-8366. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b85f9a24 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b85f9a24 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b85f9a24 Branch: refs/heads/master Commit: b85f9a242a12e8096e331fa77d5ebd16e93c844d Parents: b1581ac Author: xutingjun Authored: Tue Aug 11 23:19:35 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:19:35 2015 -0700 -- .../spark/ExecutorAllocationManager.scala | 22 +--- .../spark/ExecutorAllocationManagerSuite.scala | 22 ++-- 2 files changed, 34 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b85f9a24/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 1877aaf..b93536e 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -599,14 +599,8 @@ private[spark] class ExecutorAllocationManager( // If this is the last pending task, mark the scheduler queue as empty stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex -val numTasksScheduled = stageIdToTaskIndices(stageId).size -val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) -if (numTasksScheduled == numTasksTotal) { - // No more pending tasks for this stage - stageIdToNumTasks -= stageId - if (stageIdToNumTasks.isEmpty) { -allocationManager.onSchedulerQueueEmpty() - } +if (totalPendingTasks() == 0) { + allocationManager.onSchedulerQueueEmpty() } // Mark the executor on which this task is scheduled as busy @@ -618,6 +612,8 @@ private[spark] class ExecutorAllocationManager( override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { val executorId = taskEnd.taskInfo.executorId val taskId = taskEnd.taskInfo.taskId + val taskIndex = taskEnd.taskInfo.index + val stageId = taskEnd.stageId allocationManager.synchronized { numRunningTasks -= 1 // If the executor is no longer running any scheduled tasks, mark it as idle @@ -628,6 +624,16 @@ private[spark] class ExecutorAllocationManager( allocationManager.onExecutorIdle(executorId) } } + +// If the task failed, we expect it to be resubmitted later. To ensure we have +// enough resources to run the resubmitted task, we need to mark the scheduler +// as backlogged again if it's not already marked as such (SPARK-8366) +if (taskEnd.reason != Success) { + if (totalPendingTasks() == 0) { +allocationManager.onSchedulerBacklogged() + } + stageIdToTaskIndices.get(stageId).foreach { _.remove(taskIndex) } +} } } http://git-wip-us.apache.org/repos/asf/spark/blob/b85f9a24/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 34caca8..f374f97 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -206,8 +206,8 @@ class ExecutorAllocationManagerSuite val task2Info = createTaskInfo(1, 0, "executor-1") sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task2Info)) -sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task1Info, null)) -sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task2Info, null)) +sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, Success, task1Info, null)) +sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, Success, task2Info, null)) assert(adjustRequestedExecutors(manager) === -1) } @@ -787,6 +787,24 @@ class ExecutorAllocationManagerSuite Map("host2" -> 1, "host3" -> 2, "host4" -> 1, "host5" -> 2)) } + test("SPARK-8366: maxNumExecutorsNeeded should properly handle failed tasks") { +sc = createSparkContext() +val manager = s
spark git commit: [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple applications
Repository: spark Updated Branches: refs/heads/master b85f9a242 -> a807fcbe5 [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple applications Author: Rohit Agarwal Closes #8088 from mindprince/SPARK-9806. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a807fcbe Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a807fcbe Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a807fcbe Branch: refs/heads/master Commit: a807fcbe50b2ce18751d80d39e9d21842f7da32a Parents: b85f9a2 Author: Rohit Agarwal Authored: Tue Aug 11 23:20:39 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:20:39 2015 -0700 -- .../scala/org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a807fcbe/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e3060ac..53c18ca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -272,9 +272,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Replay the log files in the list and merge the list of old applications with new ones */ private def mergeApplicationListing(logs: Seq[FileStatus]): Unit = { -val bus = new ReplayListenerBus() val newAttempts = logs.flatMap { fileStatus => try { +val bus = new ReplayListenerBus() val res = replay(fileStatus, bus) res match { case Some(r) => logDebug(s"Application log ${r.logPath} loaded successfully.") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple applications
Repository: spark Updated Branches: refs/heads/branch-1.5 2f909184e -> 402c0ca9a [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple applications Author: Rohit Agarwal Closes #8088 from mindprince/SPARK-9806. (cherry picked from commit a807fcbe50b2ce18751d80d39e9d21842f7da32a) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/402c0ca9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/402c0ca9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/402c0ca9 Branch: refs/heads/branch-1.5 Commit: 402c0ca9a36f738fc92d281f69ec5099f19ffbf8 Parents: 2f90918 Author: Rohit Agarwal Authored: Tue Aug 11 23:20:39 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:20:55 2015 -0700 -- .../scala/org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/402c0ca9/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e3060ac..53c18ca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -272,9 +272,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Replay the log files in the list and merge the list of old applications with new ones */ private def mergeApplicationListing(logs: Seq[FileStatus]): Unit = { -val bus = new ReplayListenerBus() val newAttempts = logs.flatMap { fileStatus => try { +val bus = new ReplayListenerBus() val res = replay(fileStatus, bus) res match { case Some(r) => logDebug(s"Application log ${r.logPath} loaded successfully.") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple applications
Repository: spark Updated Branches: refs/heads/branch-1.4 6dde38026 -> 89c8aea94 [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple applications Author: Rohit Agarwal Closes #8088 from mindprince/SPARK-9806. (cherry picked from commit a807fcbe50b2ce18751d80d39e9d21842f7da32a) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/89c8aea9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/89c8aea9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/89c8aea9 Branch: refs/heads/branch-1.4 Commit: 89c8aea94cfc4b2945d0739397a14333bc64c2ae Parents: 6dde380 Author: Rohit Agarwal Authored: Tue Aug 11 23:20:39 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:22:07 2015 -0700 -- .../scala/org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/89c8aea9/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index b1e43fc..4483718 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -217,9 +217,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Replay the log files in the list and merge the list of old applications with new ones */ private def mergeApplicationListing(logs: Seq[FileStatus]): Unit = { -val bus = new ReplayListenerBus() val newAttempts = logs.flatMap { fileStatus => try { +val bus = new ReplayListenerBus() val res = replay(fileStatus, bus) res match { case Some(r) => logDebug(s"Application log ${r.logPath} loaded successfully.") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9829] [WEBUI] Display the update value for peak execution memory
Repository: spark Updated Branches: refs/heads/master a807fcbe5 -> 4e3f4b934 [SPARK-9829] [WEBUI] Display the update value for peak execution memory The peak execution memory is not correct because it shows the sum of finished tasks' values when a task finishes. This PR fixes it by using the update value rather than the accumulator value. Author: zsxwing Closes #8121 from zsxwing/SPARK-9829. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4e3f4b93 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4e3f4b93 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4e3f4b93 Branch: refs/heads/master Commit: 4e3f4b934f74e8c7c06f4940d6381343f9fd4918 Parents: a807fcb Author: zsxwing Authored: Tue Aug 11 23:23:17 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:23:17 2015 -0700 -- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4e3f4b93/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0c94204..fb4556b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -860,7 +860,7 @@ private[ui] class TaskDataSource( } val peakExecutionMemoryUsed = taskInternalAccumulables .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.value.toLong } + .map { acc => acc.update.getOrElse("0").toLong } .getOrElse(0L) val maybeInput = metrics.flatMap(_.inputMetrics) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9829] [WEBUI] Display the update value for peak execution memory
Repository: spark Updated Branches: refs/heads/branch-1.5 402c0ca9a -> d9d4bdea2 [SPARK-9829] [WEBUI] Display the update value for peak execution memory The peak execution memory is not correct because it shows the sum of finished tasks' values when a task finishes. This PR fixes it by using the update value rather than the accumulator value. Author: zsxwing Closes #8121 from zsxwing/SPARK-9829. (cherry picked from commit 4e3f4b934f74e8c7c06f4940d6381343f9fd4918) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d9d4bdea Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d9d4bdea Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d9d4bdea Branch: refs/heads/branch-1.5 Commit: d9d4bdea2d711c50273012206832c6ee1a8d90d6 Parents: 402c0ca Author: zsxwing Authored: Tue Aug 11 23:23:17 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:23:31 2015 -0700 -- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d9d4bdea/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0c94204..fb4556b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -860,7 +860,7 @@ private[ui] class TaskDataSource( } val peakExecutionMemoryUsed = taskInternalAccumulables .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.value.toLong } + .map { acc => acc.update.getOrElse("0").toLong } .getOrElse(0L) val maybeInput = metrics.flatMap(_.inputMetrics) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9426] [WEBUI] Job page DAG visualization is not shown
Repository: spark Updated Branches: refs/heads/branch-1.5 d9d4bdea2 -> 93fc95934 [SPARK-9426] [WEBUI] Job page DAG visualization is not shown To reproduce the issue, go to the stage page and click DAG Visualization once, then go to the job page to show the job DAG visualization. You will only see the first stage of the job. Root cause: the java script use local storage to remember your selection. Once you click the stage DAG visualization, the local storage set `expand-dag-viz-arrow-stage` to true. When you go to the job page, the js checks `expand-dag-viz-arrow-stage` in the local storage first and will try to show stage DAG visualization on the job page. To fix this, I set an id to the DAG span to differ job page and stage page. In the js code, we check the id and local storage together to make sure we show the correct DAG visualization. Author: Carson Wang Closes #8104 from carsonwang/SPARK-9426. (cherry picked from commit bab89232854de7554e88f29cab76f1a1c349edc1) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/93fc9593 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/93fc9593 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/93fc9593 Branch: refs/heads/branch-1.5 Commit: 93fc95934f233a3cfdf744160242092dadeaec8b Parents: d9d4bde Author: Carson Wang Authored: Tue Aug 11 23:25:02 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:25:27 2015 -0700 -- .../resources/org/apache/spark/ui/static/spark-dag-viz.js| 8 core/src/main/scala/org/apache/spark/ui/UIUtils.scala| 3 ++- 2 files changed, 6 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/93fc9593/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js -- diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index 4a893bc..83dbea4 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -109,13 +109,13 @@ function toggleDagViz(forJob) { } $(function (){ - if (window.localStorage.getItem(expandDagVizArrowKey(false)) == "true") { + if ($("#stage-dag-viz").length && + window.localStorage.getItem(expandDagVizArrowKey(false)) == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem(expandDagVizArrowKey(false), "false"); toggleDagViz(false); - } - - if (window.localStorage.getItem(expandDagVizArrowKey(true)) == "true") { + } else if ($("#job-dag-viz").length && + window.localStorage.getItem(expandDagVizArrowKey(true)) == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem(expandDagVizArrowKey(true), "false"); toggleDagViz(true); http://git-wip-us.apache.org/repos/asf/spark/blob/93fc9593/core/src/main/scala/org/apache/spark/ui/UIUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 718aea7..f2da417 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -352,7 +352,8 @@ private[spark] object UIUtils extends Logging { */ private def showDagViz(graphs: Seq[RDDOperationGraph], forJob: Boolean): Seq[Node] = { - + - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9426] [WEBUI] Job page DAG visualization is not shown
Repository: spark Updated Branches: refs/heads/master 4e3f4b934 -> bab892328 [SPARK-9426] [WEBUI] Job page DAG visualization is not shown To reproduce the issue, go to the stage page and click DAG Visualization once, then go to the job page to show the job DAG visualization. You will only see the first stage of the job. Root cause: the java script use local storage to remember your selection. Once you click the stage DAG visualization, the local storage set `expand-dag-viz-arrow-stage` to true. When you go to the job page, the js checks `expand-dag-viz-arrow-stage` in the local storage first and will try to show stage DAG visualization on the job page. To fix this, I set an id to the DAG span to differ job page and stage page. In the js code, we check the id and local storage together to make sure we show the correct DAG visualization. Author: Carson Wang Closes #8104 from carsonwang/SPARK-9426. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bab89232 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bab89232 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bab89232 Branch: refs/heads/master Commit: bab89232854de7554e88f29cab76f1a1c349edc1 Parents: 4e3f4b9 Author: Carson Wang Authored: Tue Aug 11 23:25:02 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:25:02 2015 -0700 -- .../resources/org/apache/spark/ui/static/spark-dag-viz.js| 8 core/src/main/scala/org/apache/spark/ui/UIUtils.scala| 3 ++- 2 files changed, 6 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bab89232/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js -- diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index 4a893bc..83dbea4 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -109,13 +109,13 @@ function toggleDagViz(forJob) { } $(function (){ - if (window.localStorage.getItem(expandDagVizArrowKey(false)) == "true") { + if ($("#stage-dag-viz").length && + window.localStorage.getItem(expandDagVizArrowKey(false)) == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem(expandDagVizArrowKey(false), "false"); toggleDagViz(false); - } - - if (window.localStorage.getItem(expandDagVizArrowKey(true)) == "true") { + } else if ($("#job-dag-viz").length && + window.localStorage.getItem(expandDagVizArrowKey(true)) == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem(expandDagVizArrowKey(true), "false"); toggleDagViz(true); http://git-wip-us.apache.org/repos/asf/spark/blob/bab89232/core/src/main/scala/org/apache/spark/ui/UIUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 718aea7..f2da417 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -352,7 +352,8 @@ private[spark] object UIUtils extends Logging { */ private def showDagViz(graphs: Seq[RDDOperationGraph], forJob: Boolean): Seq[Node] = { - + - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8798] [MESOS] Allow additional uris to be fetched with mesos
Repository: spark Updated Branches: refs/heads/branch-1.5 93fc95934 -> a2f805729 [SPARK-8798] [MESOS] Allow additional uris to be fetched with mesos Some users like to download additional files in their sandbox that they can refer to from their spark program, or even later mount these files to another directory. Author: Timothy Chen Closes #7195 from tnachen/mesos_files. (cherry picked from commit 5c99d8bf98cbf7f568345d02a814fc318cbfca75) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a2f80572 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a2f80572 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a2f80572 Branch: refs/heads/branch-1.5 Commit: a2f805729b401c68b60bd690ad02533b8db57b58 Parents: 93fc959 Author: Timothy Chen Authored: Tue Aug 11 23:26:33 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:26:48 2015 -0700 -- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 5 + .../scheduler/cluster/mesos/MesosClusterScheduler.scala | 3 +++ .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 5 + .../spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala | 6 ++ docs/running-on-mesos.md | 8 5 files changed, 27 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a2f80572/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 15a0915..d6e1e9e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,6 +194,11 @@ private[spark] class CoarseMesosSchedulerBackend( s" --app-id $appId") command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) } + +conf.getOption("spark.mesos.uris").map { uris => + setupUris(uris, command) +} + command.build() } http://git-wip-us.apache.org/repos/asf/spark/blob/a2f80572/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index f078547..64ec2b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -403,6 +403,9 @@ private[spark] class MesosClusterScheduler( } builder.setValue(s"$executable $cmdOptions $jar $appArguments") builder.setEnvironment(envBuilder.build()) +conf.getOption("spark.mesos.uris").map { uris => + setupUris(uris, builder) +} builder.build() } http://git-wip-us.apache.org/repos/asf/spark/blob/a2f80572/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 3f63ec1..5c20606 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -133,6 +133,11 @@ private[spark] class MesosSchedulerBackend( builder.addAllResources(usedCpuResources) builder.addAllResources(usedMemResources) + +sc.conf.getOption("spark.mesos.uris").map { uris => + setupUris(uris, command) +} + val executorInfo = builder .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) http://git-wip-us.apache.org/repos/asf/spark/blob/a2f80572/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index c04920e..5b854aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/cor
spark git commit: [SPARK-8798] [MESOS] Allow additional uris to be fetched with mesos
Repository: spark Updated Branches: refs/heads/master bab892328 -> 5c99d8bf9 [SPARK-8798] [MESOS] Allow additional uris to be fetched with mesos Some users like to download additional files in their sandbox that they can refer to from their spark program, or even later mount these files to another directory. Author: Timothy Chen Closes #7195 from tnachen/mesos_files. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5c99d8bf Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5c99d8bf Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5c99d8bf Branch: refs/heads/master Commit: 5c99d8bf98cbf7f568345d02a814fc318cbfca75 Parents: bab8923 Author: Timothy Chen Authored: Tue Aug 11 23:26:33 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:26:33 2015 -0700 -- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 5 + .../scheduler/cluster/mesos/MesosClusterScheduler.scala | 3 +++ .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 5 + .../spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala | 6 ++ docs/running-on-mesos.md | 8 5 files changed, 27 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5c99d8bf/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 15a0915..d6e1e9e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,6 +194,11 @@ private[spark] class CoarseMesosSchedulerBackend( s" --app-id $appId") command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) } + +conf.getOption("spark.mesos.uris").map { uris => + setupUris(uris, command) +} + command.build() } http://git-wip-us.apache.org/repos/asf/spark/blob/5c99d8bf/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index f078547..64ec2b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -403,6 +403,9 @@ private[spark] class MesosClusterScheduler( } builder.setValue(s"$executable $cmdOptions $jar $appArguments") builder.setEnvironment(envBuilder.build()) +conf.getOption("spark.mesos.uris").map { uris => + setupUris(uris, builder) +} builder.build() } http://git-wip-us.apache.org/repos/asf/spark/blob/5c99d8bf/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 3f63ec1..5c20606 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -133,6 +133,11 @@ private[spark] class MesosSchedulerBackend( builder.addAllResources(usedCpuResources) builder.addAllResources(usedMemResources) + +sc.conf.getOption("spark.mesos.uris").map { uris => + setupUris(uris, command) +} + val executorInfo = builder .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) http://git-wip-us.apache.org/repos/asf/spark/blob/5c99d8bf/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index c04920e..5b854aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -331,4 +331,10 @@
spark git commit: [SPARK-9575] [MESOS] Add docuemntation around Mesos shuffle service.
Repository: spark Updated Branches: refs/heads/master 5c99d8bf9 -> 741a29f98 [SPARK-9575] [MESOS] Add docuemntation around Mesos shuffle service. andrewor14 Author: Timothy Chen Closes #7907 from tnachen/mesos_shuffle. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/741a29f9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/741a29f9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/741a29f9 Branch: refs/heads/master Commit: 741a29f98945538a475579ccc974cd42c1613be4 Parents: 5c99d8b Author: Timothy Chen Authored: Tue Aug 11 23:33:22 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:33:22 2015 -0700 -- docs/running-on-mesos.md | 14 ++ 1 file changed, 14 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/741a29f9/docs/running-on-mesos.md -- diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 55e6d4e..cfd219a 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -216,6 +216,20 @@ node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). In either case, HDFS runs separately from Hadoop MapReduce, without being scheduled through Mesos. +# Dynamic Resource Allocation with Mesos + +Mesos supports dynamic allocation only with coarse grain mode, which can resize the number of executors based on statistics +of the application. While dynamic allocation supports both scaling up and scaling down the number of executors, the coarse grain scheduler only supports scaling down +since it is already designed to run one executor per slave with the configured amount of resources. However, after scaling down the number of executors the coarse grain scheduler +can scale back up to the same amount of executors when Spark signals more executors are needed. + +Users that like to utilize this feature should launch the Mesos Shuffle Service that +provides shuffle data cleanup functionality on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's +termination. To launch/stop the Mesos Shuffle Service please use the provided sbin/start-mesos-shuffle-service.sh and sbin/stop-mesos-shuffle-service.sh +scripts accordingly. + +The Shuffle Service is expected to be running on each slave node that will run Spark executors. One way to easily achieve this with Mesos +is to launch the Shuffle Service with Marathon with a unique host constraint. # Configuration - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9575] [MESOS] Add docuemntation around Mesos shuffle service.
Repository: spark Updated Branches: refs/heads/branch-1.5 a2f805729 -> 5dd0c5cd6 [SPARK-9575] [MESOS] Add docuemntation around Mesos shuffle service. andrewor14 Author: Timothy Chen Closes #7907 from tnachen/mesos_shuffle. (cherry picked from commit 741a29f98945538a475579ccc974cd42c1613be4) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5dd0c5cd Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5dd0c5cd Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5dd0c5cd Branch: refs/heads/branch-1.5 Commit: 5dd0c5cd613c0a6c6836ae8f16d22820010e65f8 Parents: a2f8057 Author: Timothy Chen Authored: Tue Aug 11 23:33:22 2015 -0700 Committer: Andrew Or Committed: Tue Aug 11 23:34:04 2015 -0700 -- docs/running-on-mesos.md | 14 ++ 1 file changed, 14 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5dd0c5cd/docs/running-on-mesos.md -- diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 55e6d4e..cfd219a 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -216,6 +216,20 @@ node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). In either case, HDFS runs separately from Hadoop MapReduce, without being scheduled through Mesos. +# Dynamic Resource Allocation with Mesos + +Mesos supports dynamic allocation only with coarse grain mode, which can resize the number of executors based on statistics +of the application. While dynamic allocation supports both scaling up and scaling down the number of executors, the coarse grain scheduler only supports scaling down +since it is already designed to run one executor per slave with the configured amount of resources. However, after scaling down the number of executors the coarse grain scheduler +can scale back up to the same amount of executors when Spark signals more executors are needed. + +Users that like to utilize this feature should launch the Mesos Shuffle Service that +provides shuffle data cleanup functionality on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's +termination. To launch/stop the Mesos Shuffle Service please use the provided sbin/start-mesos-shuffle-service.sh and sbin/stop-mesos-shuffle-service.sh +scripts accordingly. + +The Shuffle Service is expected to be running on each slave node that will run Spark executors. One way to easily achieve this with Mesos +is to launch the Shuffle Service with Marathon with a unique host constraint. # Configuration - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9795] Dynamic allocation: avoid double counting when killing same executor twice
Repository: spark Updated Branches: refs/heads/master 2e680668f -> be5d19120 [SPARK-9795] Dynamic allocation: avoid double counting when killing same executor twice This is based on KaiXinXiaoLei's changes in #7716. The issue is that when someone calls `sc.killExecutor("1")` on the same executor twice quickly, then the executor target will be adjusted downwards by 2 instead of 1 even though we're only actually killing one executor. In certain cases where we don't adjust the target back upwards quickly, we'll end up with jobs hanging. This is a common danger because there are many places where this is called: - `HeartbeatReceiver` kills an executor that has not been sending heartbeats - `ExecutorAllocationManager` kills an executor that has been idle - The user code might call this, which may interfere with the previous callers While it's not clear whether this fixes SPARK-9745, fixing this potential race condition seems like a strict improvement. I've added a regression test to illustrate the issue. Author: Andrew Or Closes #8078 from andrewor14/da-double-kill. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/be5d1912 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/be5d1912 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/be5d1912 Branch: refs/heads/master Commit: be5d1912076c2ffd21ec88611e53d3b3c59b7ecc Parents: 2e68066 Author: Andrew Or Authored: Wed Aug 12 09:24:50 2015 -0700 Committer: Andrew Or Committed: Wed Aug 12 09:24:50 2015 -0700 -- .../cluster/CoarseGrainedSchedulerBackend.scala | 11 +++ .../StandaloneDynamicAllocationSuite.scala | 20 2 files changed, 27 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/be5d1912/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6acf8a9..5730a87 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -422,16 +422,19 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logWarning(s"Executor to kill $id does not exist!") } +// If an executor is already pending to be removed, do not kill it again (SPARK-9795) +val executorsToKill = knownExecutors.filter { id => !executorsPendingToRemove.contains(id) } +executorsPendingToRemove ++= executorsToKill + // If we do not wish to replace the executors we kill, sync the target number of executors // with the cluster manager to avoid allocating new ones. When computing the new target, // take into account executors that are pending to be added or removed. if (!replace) { - doRequestTotalExecutors(numExistingExecutors + numPendingExecutors -- executorsPendingToRemove.size - knownExecutors.size) + doRequestTotalExecutors( +numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) } -executorsPendingToRemove ++= knownExecutors -doKillExecutors(knownExecutors) +doKillExecutors(executorsToKill) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/be5d1912/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 08c41a8..1f2a0f0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -283,6 +283,26 @@ class StandaloneDynamicAllocationSuite assert(master.apps.head.getExecutorLimit === 1000) } + test("kill the same executor twice (SPARK-9795)") { +sc = new SparkContext(appConf) +val appId = sc.applicationId +assert(master.apps.size === 1) +assert(master.apps.head.id === appId) +assert(master.apps.head.executors.size === 2) +assert(master.apps.head.getExecutorLimit === Int.MaxValue) +// sync executors between the Master and the driver, needed because +// the driver refuses to kill executors it does not know about +syncExecutors(sc) +// kill the same executor twice +v
spark git commit: [SPARK-9795] Dynamic allocation: avoid double counting when killing same executor twice
Repository: spark Updated Branches: refs/heads/branch-1.5 0579f28df -> bc4ac65d4 [SPARK-9795] Dynamic allocation: avoid double counting when killing same executor twice This is based on KaiXinXiaoLei's changes in #7716. The issue is that when someone calls `sc.killExecutor("1")` on the same executor twice quickly, then the executor target will be adjusted downwards by 2 instead of 1 even though we're only actually killing one executor. In certain cases where we don't adjust the target back upwards quickly, we'll end up with jobs hanging. This is a common danger because there are many places where this is called: - `HeartbeatReceiver` kills an executor that has not been sending heartbeats - `ExecutorAllocationManager` kills an executor that has been idle - The user code might call this, which may interfere with the previous callers While it's not clear whether this fixes SPARK-9745, fixing this potential race condition seems like a strict improvement. I've added a regression test to illustrate the issue. Author: Andrew Or Closes #8078 from andrewor14/da-double-kill. (cherry picked from commit be5d1912076c2ffd21ec88611e53d3b3c59b7ecc) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bc4ac65d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bc4ac65d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bc4ac65d Branch: refs/heads/branch-1.5 Commit: bc4ac65d4c0fed93c70582fc74574c5b70aa842d Parents: 0579f28 Author: Andrew Or Authored: Wed Aug 12 09:24:50 2015 -0700 Committer: Andrew Or Committed: Wed Aug 12 09:24:58 2015 -0700 -- .../cluster/CoarseGrainedSchedulerBackend.scala | 11 +++ .../StandaloneDynamicAllocationSuite.scala | 20 2 files changed, 27 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bc4ac65d/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6acf8a9..5730a87 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -422,16 +422,19 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logWarning(s"Executor to kill $id does not exist!") } +// If an executor is already pending to be removed, do not kill it again (SPARK-9795) +val executorsToKill = knownExecutors.filter { id => !executorsPendingToRemove.contains(id) } +executorsPendingToRemove ++= executorsToKill + // If we do not wish to replace the executors we kill, sync the target number of executors // with the cluster manager to avoid allocating new ones. When computing the new target, // take into account executors that are pending to be added or removed. if (!replace) { - doRequestTotalExecutors(numExistingExecutors + numPendingExecutors -- executorsPendingToRemove.size - knownExecutors.size) + doRequestTotalExecutors( +numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) } -executorsPendingToRemove ++= knownExecutors -doKillExecutors(knownExecutors) +doKillExecutors(executorsToKill) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/bc4ac65d/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 08c41a8..1f2a0f0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -283,6 +283,26 @@ class StandaloneDynamicAllocationSuite assert(master.apps.head.getExecutorLimit === 1000) } + test("kill the same executor twice (SPARK-9795)") { +sc = new SparkContext(appConf) +val appId = sc.applicationId +assert(master.apps.size === 1) +assert(master.apps.head.id === appId) +assert(master.apps.head.executors.size === 2) +assert(master.apps.head.getExecutorLimit === Int.MaxValue) +// sync executors between the Master and the driver, needed because +// the driver refuses to kill ex
spark git commit: [SPARK-9870] Disable driver UI and Master REST server in SparkSubmitSuite
Repository: spark Updated Branches: refs/heads/master f4bc01f1f -> 7b13ed27c [SPARK-9870] Disable driver UI and Master REST server in SparkSubmitSuite I think that we should pass additional configuration flags to disable the driver UI and Master REST server in SparkSubmitSuite and HiveSparkSubmitSuite. This might cut down on port-contention-related flakiness in Jenkins. Author: Josh Rosen Closes #8124 from JoshRosen/disable-ui-in-sparksubmitsuite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7b13ed27 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7b13ed27 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7b13ed27 Branch: refs/heads/master Commit: 7b13ed27c1296cf76d0946e400f3449c335c8471 Parents: f4bc01f Author: Josh Rosen Authored: Wed Aug 12 18:52:11 2015 -0700 Committer: Andrew Or Committed: Wed Aug 12 18:52:11 2015 -0700 -- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 7 +++ .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 10 +- 2 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7b13ed27/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 2456c5d..1110ca6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -324,6 +324,8 @@ class SparkSubmitSuite "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -337,6 +339,8 @@ class SparkSubmitSuite "--class", JarCreationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", "--jars", jarsString, unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") runSparkSubmit(args) @@ -355,6 +359,7 @@ class SparkSubmitSuite "--packages", Seq(main, dep).mkString(","), "--repositories", repo, "--conf", "spark.ui.enabled=false", +"--conf", "spark.master.rest.enabled=false", unusedJar.toString, "my.great.lib.MyLib", "my.great.dep.MyLib") runSparkSubmit(args) @@ -500,6 +505,8 @@ class SparkSubmitSuite "--master", "local", "--conf", "spark.driver.extraClassPath=" + systemJar, "--conf", "spark.driver.userClassPathFirst=true", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", userJar.toString) runSparkSubmit(args) } http://git-wip-us.apache.org/repos/asf/spark/blob/7b13ed27/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index b8d4106..1e1972d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -57,6 +57,8 @@ class HiveSparkSubmitSuite "--class", SparkSubmitClassLoaderTest.getClass.getName.stripSuffix("$"), "--name", "SparkSubmitClassLoaderTest", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", "--jars", jarsString, unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") runSparkSubmit(args) @@ -68,6 +70,8 @@ class HiveSparkSubmitSuite "--class", SparkSQLConfTest.getClass.getName.stripSuffix("$"), "--name", "SparkSQLConfTest", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -79,7 +83,11 @@ class HiveSparkSubmitSuite // the HiveContext code mistakenly overrides the class loader that contains user classes. // For more detail, see sql/hive/src/test/resources/regression-test-SPARK-8489/*scala. val testJar = "sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar" -val args = Seq("--class", "Main", testJar) +val args = Seq( + "--conf", "spark.ui.en
spark git commit: [SPARK-9870] Disable driver UI and Master REST server in SparkSubmitSuite
Repository: spark Updated Branches: refs/heads/branch-1.5 ca39c9e91 -> 4b547b91d [SPARK-9870] Disable driver UI and Master REST server in SparkSubmitSuite I think that we should pass additional configuration flags to disable the driver UI and Master REST server in SparkSubmitSuite and HiveSparkSubmitSuite. This might cut down on port-contention-related flakiness in Jenkins. Author: Josh Rosen Closes #8124 from JoshRosen/disable-ui-in-sparksubmitsuite. (cherry picked from commit 7b13ed27c1296cf76d0946e400f3449c335c8471) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4b547b91 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4b547b91 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4b547b91 Branch: refs/heads/branch-1.5 Commit: 4b547b91d6786b2478cf9068023217c979372e79 Parents: ca39c9e Author: Josh Rosen Authored: Wed Aug 12 18:52:11 2015 -0700 Committer: Andrew Or Committed: Wed Aug 12 18:52:20 2015 -0700 -- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 7 +++ .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 10 +- 2 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4b547b91/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 2456c5d..1110ca6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -324,6 +324,8 @@ class SparkSubmitSuite "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -337,6 +339,8 @@ class SparkSubmitSuite "--class", JarCreationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", "--jars", jarsString, unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") runSparkSubmit(args) @@ -355,6 +359,7 @@ class SparkSubmitSuite "--packages", Seq(main, dep).mkString(","), "--repositories", repo, "--conf", "spark.ui.enabled=false", +"--conf", "spark.master.rest.enabled=false", unusedJar.toString, "my.great.lib.MyLib", "my.great.dep.MyLib") runSparkSubmit(args) @@ -500,6 +505,8 @@ class SparkSubmitSuite "--master", "local", "--conf", "spark.driver.extraClassPath=" + systemJar, "--conf", "spark.driver.userClassPathFirst=true", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", userJar.toString) runSparkSubmit(args) } http://git-wip-us.apache.org/repos/asf/spark/blob/4b547b91/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index b8d4106..1e1972d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -57,6 +57,8 @@ class HiveSparkSubmitSuite "--class", SparkSubmitClassLoaderTest.getClass.getName.stripSuffix("$"), "--name", "SparkSubmitClassLoaderTest", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", "--jars", jarsString, unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") runSparkSubmit(args) @@ -68,6 +70,8 @@ class HiveSparkSubmitSuite "--class", SparkSQLConfTest.getClass.getName.stripSuffix("$"), "--name", "SparkSQLConfTest", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -79,7 +83,11 @@ class HiveSparkSubmitSuite // the HiveContext code mistakenly overrides the class loader that contains user classes. // For more detail, see sql/hive/src/test/resources/regression-test-SPARK-8489/*scala. val testJar = "sql/hive/src/test/resources/regression-test-SPARK-8489/test
spark git commit: [SPARK-9649] Fix MasterSuite, third time's a charm
Repository: spark Updated Branches: refs/heads/master 65fec798c -> 8815ba2f6 [SPARK-9649] Fix MasterSuite, third time's a charm This particular test did not load the default configurations so it continued to start the REST server, which causes port bind exceptions. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8815ba2f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8815ba2f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8815ba2f Branch: refs/heads/master Commit: 8815ba2f674dbb18eb499216df9942b411e10daa Parents: 65fec79 Author: Andrew Or Authored: Thu Aug 13 11:31:10 2015 -0700 Committer: Andrew Or Committed: Thu Aug 13 11:31:10 2015 -0700 -- .../src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8815ba2f/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 20d0201..242bf4b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -40,6 +40,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva conf.set("spark.deploy.recoveryMode", "CUSTOM") conf.set("spark.deploy.recoveryMode.factory", classOf[CustomRecoveryModeFactory].getCanonicalName) +conf.set("spark.master.rest.enabled", "false") val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9649] Fix MasterSuite, third time's a charm
Repository: spark Updated Branches: refs/heads/branch-1.5 883c7d35f -> 30460206f [SPARK-9649] Fix MasterSuite, third time's a charm This particular test did not load the default configurations so it continued to start the REST server, which causes port bind exceptions. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/30460206 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/30460206 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/30460206 Branch: refs/heads/branch-1.5 Commit: 30460206f1866548945449381ae463a3e701e81c Parents: 883c7d3 Author: Andrew Or Authored: Thu Aug 13 11:31:10 2015 -0700 Committer: Andrew Or Committed: Thu Aug 13 11:33:01 2015 -0700 -- .../src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/30460206/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 20d0201..242bf4b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -40,6 +40,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva conf.set("spark.deploy.recoveryMode", "CUSTOM") conf.set("spark.deploy.recoveryMode.factory", classOf[CustomRecoveryModeFactory].getCanonicalName) +conf.set("spark.master.rest.enabled", "false") val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9649] Fix flaky test MasterSuite - randomize ports
Repository: spark Updated Branches: refs/heads/branch-1.4 8ce86b23f -> 041e720ec [SPARK-9649] Fix flaky test MasterSuite - randomize ports ``` Error Message Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries! Stacktrace java.net.BindException: Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries! at org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272) at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393) at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389) at scala.util.Success$$anonfun$map$1.apply(Try.scala:206) at scala.util.Try$.apply(Try.scala:161) ``` Author: Andrew Or Closes #7968 from andrewor14/fix-master-flaky-test and squashes the following commits: fcc42ef [Andrew Or] Randomize port Conflicts: core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/041e720e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/041e720e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/041e720e Branch: refs/heads/branch-1.4 Commit: 041e720ecb98197dbeed40c8eb35fff1052ffd0a Parents: 8ce86b2 Author: Andrew Or Authored: Wed Aug 5 14:12:22 2015 -0700 Committer: Andrew Or Committed: Thu Aug 13 11:40:36 2015 -0700 -- .../test/scala/org/apache/spark/deploy/master/MasterSuite.scala | 3 ++- pom.xml | 1 + project/SparkBuild.scala | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/041e720e/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 3eabc1a..d331807 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -81,6 +81,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually { conf.set("spark.deploy.recoveryMode", "CUSTOM") conf.set("spark.deploy.recoveryMode.factory", classOf[CustomRecoveryModeFactory].getCanonicalName) +conf.set("spark.master.rest.enabled", "false") val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts @@ -135,7 +136,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually { ) val (actorSystem, port, uiPort, restPort) = - Master.startSystemAndActor("127.0.0.1", 7077, 8080, conf) + Master.startSystemAndActor("127.0.0.1", 0, 0, conf) try { Await.result(actorSystem.actorSelection("/user/Master").resolveOne(10 seconds), 10 seconds) http://git-wip-us.apache.org/repos/asf/spark/blob/041e720e/pom.xml -- diff --git a/pom.xml b/pom.xml index 6c102da..be8aebd 100644 --- a/pom.xml +++ b/pom.xml @@ -1256,6 +1256,7 @@ ${spark.test.home} 1 false + false false true true http://git-wip-us.apache.org/repos/asf/spark/blob/041e720e/project/SparkBuild.scala -- diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4b00a66..9b53be4 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -514,6 +514,7 @@ object TestSettings { javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.port.maxRetries=100", javaOptions in Test += "-Dspark.ui.enabled=false", +javaOptions in Test += "-Dspark.master.rest.enabled=false", javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9809] Task crashes because the internal accumulators are not properly initialized
Repository: spark Updated Branches: refs/heads/branch-1.5 d92568ae5 -> ff3e9561d [SPARK-9809] Task crashes because the internal accumulators are not properly initialized When a stage failed and another stage was resubmitted with only part of partitions to compute, all the tasks failed with error message: java.util.NoSuchElementException: key not found: peakExecutionMemory. This is because the internal accumulators are not properly initialized for this stage while other codes assume the internal accumulators always exist. Author: Carson Wang Closes #8090 from carsonwang/SPARK-9809. (cherry picked from commit 33bae585d4cb25aed2ac32e0d1248f78cc65318b) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ff3e9561 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ff3e9561 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ff3e9561 Branch: refs/heads/branch-1.5 Commit: ff3e9561d63348076b77b3d16ca1a720461e87ea Parents: d92568a Author: Carson Wang Authored: Fri Aug 14 13:38:25 2015 -0700 Committer: Andrew Or Committed: Fri Aug 14 13:38:34 2015 -0700 -- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ff3e9561/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7ab5ccf..f1c63d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -790,9 +790,10 @@ class DAGScheduler( } } +// Create internal accumulators if the stage has no accumulators initialized. // Reset internal accumulators only if this stage is not partially submitted // Otherwise, we may override existing accumulator values from some tasks -if (allPartitions == partitionsToCompute) { +if (stage.internalAccumulators.isEmpty || allPartitions == partitionsToCompute) { stage.resetInternalAccumulators() } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9809] Task crashes because the internal accumulators are not properly initialized
Repository: spark Updated Branches: refs/heads/master ffa05c84f -> 33bae585d [SPARK-9809] Task crashes because the internal accumulators are not properly initialized When a stage failed and another stage was resubmitted with only part of partitions to compute, all the tasks failed with error message: java.util.NoSuchElementException: key not found: peakExecutionMemory. This is because the internal accumulators are not properly initialized for this stage while other codes assume the internal accumulators always exist. Author: Carson Wang Closes #8090 from carsonwang/SPARK-9809. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/33bae585 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/33bae585 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/33bae585 Branch: refs/heads/master Commit: 33bae585d4cb25aed2ac32e0d1248f78cc65318b Parents: ffa05c8 Author: Carson Wang Authored: Fri Aug 14 13:38:25 2015 -0700 Committer: Andrew Or Committed: Fri Aug 14 13:38:25 2015 -0700 -- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/33bae585/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7ab5ccf..f1c63d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -790,9 +790,10 @@ class DAGScheduler( } } +// Create internal accumulators if the stage has no accumulators initialized. // Reset internal accumulators only if this stage is not partially submitted // Otherwise, we may override existing accumulator values from some tasks -if (allPartitions == partitionsToCompute) { +if (stage.internalAccumulators.isEmpty || allPartitions == partitionsToCompute) { stage.resetInternalAccumulators() } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators
Repository: spark Updated Branches: refs/heads/master 33bae585d -> 6518ef630 [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators In these tests, we use a custom listener and we assert on fields in the stage / task completion events. However, these events are posted in a separate thread so they're not guaranteed to be posted in time. This commit fixes this flakiness through a job end registration callback. Author: Andrew Or Closes #8176 from andrewor14/fix-accumulator-suite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6518ef63 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6518ef63 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6518ef63 Branch: refs/heads/master Commit: 6518ef63037aa56b541927f99ad26744f91098ce Parents: 33bae58 Author: Andrew Or Authored: Fri Aug 14 13:42:53 2015 -0700 Committer: Andrew Or Committed: Fri Aug 14 13:42:53 2015 -0700 -- .../org/apache/spark/AccumulatorSuite.scala | 153 +++ 1 file changed, 92 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6518ef63/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 0eb2293..5b84acf 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -182,26 +182,30 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc = new SparkContext("local", "test") sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator -sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => +val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 iter -}.count() -val stageInfos = listener.getCompletedStageInfos -val taskInfos = listener.getCompletedTaskInfos -assert(stageInfos.size === 1) -assert(taskInfos.size === numPartitions) -// The accumulator values should be merged in the stage -val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) -assert(stageAccum.value.toLong === numPartitions) -// The accumulator should be updated locally on each task -val taskAccumValues = taskInfos.map { taskInfo => - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - taskAccum.value.toLong } -// Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions -assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +// Register asserts in job completion callback to avoid flakiness +listener.registerJobCompletionCallback { _ => + val stageInfos = listener.getCompletedStageInfos + val taskInfos = listener.getCompletedTaskInfos + assert(stageInfos.size === 1) + assert(taskInfos.size === numPartitions) + // The accumulator values should be merged in the stage + val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) + assert(stageAccum.value.toLong === numPartitions) + // The accumulator should be updated locally on each task + val taskAccumValues = taskInfos.map { taskInfo => +val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) +assert(taskAccum.update.isDefined) +assert(taskAccum.update.get.toLong === 1) +taskAccum.value.toLong + } + // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions + assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +} +rdd.count() } test("internal accumulators in multiple stages") { @@ -211,7 +215,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Each stage creates its own set of internal accumulators so the // values for the same metric should not be mixed up across stages -sc.parallelize(1 to 100, numPartitions) +val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 @@ -227,16 +231,20 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex TaskContext.get().internalMe
spark git commit: [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators
Repository: spark Updated Branches: refs/heads/branch-1.5 ff3e9561d -> 1ce0b01f4 [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators In these tests, we use a custom listener and we assert on fields in the stage / task completion events. However, these events are posted in a separate thread so they're not guaranteed to be posted in time. This commit fixes this flakiness through a job end registration callback. Author: Andrew Or Closes #8176 from andrewor14/fix-accumulator-suite. (cherry picked from commit 6518ef63037aa56b541927f99ad26744f91098ce) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1ce0b01f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1ce0b01f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1ce0b01f Branch: refs/heads/branch-1.5 Commit: 1ce0b01f427e5840c2d3e7e0046fe18fed945336 Parents: ff3e956 Author: Andrew Or Authored: Fri Aug 14 13:42:53 2015 -0700 Committer: Andrew Or Committed: Fri Aug 14 13:43:01 2015 -0700 -- .../org/apache/spark/AccumulatorSuite.scala | 153 +++ 1 file changed, 92 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1ce0b01f/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 0eb2293..5b84acf 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -182,26 +182,30 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc = new SparkContext("local", "test") sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator -sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => +val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 iter -}.count() -val stageInfos = listener.getCompletedStageInfos -val taskInfos = listener.getCompletedTaskInfos -assert(stageInfos.size === 1) -assert(taskInfos.size === numPartitions) -// The accumulator values should be merged in the stage -val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) -assert(stageAccum.value.toLong === numPartitions) -// The accumulator should be updated locally on each task -val taskAccumValues = taskInfos.map { taskInfo => - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - taskAccum.value.toLong } -// Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions -assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +// Register asserts in job completion callback to avoid flakiness +listener.registerJobCompletionCallback { _ => + val stageInfos = listener.getCompletedStageInfos + val taskInfos = listener.getCompletedTaskInfos + assert(stageInfos.size === 1) + assert(taskInfos.size === numPartitions) + // The accumulator values should be merged in the stage + val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) + assert(stageAccum.value.toLong === numPartitions) + // The accumulator should be updated locally on each task + val taskAccumValues = taskInfos.map { taskInfo => +val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) +assert(taskAccum.update.isDefined) +assert(taskAccum.update.get.toLong === 1) +taskAccum.value.toLong + } + // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions + assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +} +rdd.count() } test("internal accumulators in multiple stages") { @@ -211,7 +215,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Each stage creates its own set of internal accumulators so the // values for the same metric should not be mixed up across stages -sc.parallelize(1 to 100, numPartitions) +val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 @@ -227,16 +231,20 @@ class AccumulatorSuite
spark git commit: [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application
Repository: spark Updated Branches: refs/heads/master 6518ef630 -> 9407baa2a [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application Detailed exception log can be seen in [SPARK-9877](https://issues.apache.org/jira/browse/SPARK-9877), the problem is when creating `StandaloneRestServer`, `self` (`masterEndpoint`) is null. So this fix is creating `StandaloneRestServer` when `self` is available. Author: jerryshao Closes #8127 from jerryshao/SPARK-9877. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9407baa2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9407baa2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9407baa2 Branch: refs/heads/master Commit: 9407baa2a7c26f527f2d043715d313d75bd765bb Parents: 6518ef6 Author: jerryshao Authored: Fri Aug 14 13:44:38 2015 -0700 Committer: Andrew Or Committed: Fri Aug 14 13:44:38 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 16 1 file changed, 8 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9407baa2/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9217202..26904d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -127,14 +127,8 @@ private[deploy] class Master( // Alternative application submission gateway that is stable across Spark versions private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", true) - private val restServer = -if (restServerEnabled) { - val port = conf.getInt("spark.master.rest.port", 6066) - Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) -} else { - None -} - private val restServerBoundPort = restServer.map(_.start()) + private var restServer: Option[StandaloneRestServer] = None + private var restServerBoundPort: Option[Int] = None override def onStart(): Unit = { logInfo("Starting Spark master at " + masterUrl) @@ -148,6 +142,12 @@ private[deploy] class Master( } }, 0, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) +if (restServerEnabled) { + val port = conf.getInt("spark.master.rest.port", 6066) + restServer = Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) +} +restServerBoundPort = restServer.map(_.start()) + masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() applicationMetricsSystem.start() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application
Repository: spark Updated Branches: refs/heads/branch-1.5 1ce0b01f4 -> 130e06ef1 [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application Detailed exception log can be seen in [SPARK-9877](https://issues.apache.org/jira/browse/SPARK-9877), the problem is when creating `StandaloneRestServer`, `self` (`masterEndpoint`) is null. So this fix is creating `StandaloneRestServer` when `self` is available. Author: jerryshao Closes #8127 from jerryshao/SPARK-9877. (cherry picked from commit 9407baa2a7c26f527f2d043715d313d75bd765bb) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/130e06ef Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/130e06ef Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/130e06ef Branch: refs/heads/branch-1.5 Commit: 130e06ef1324e859f96220c84bc04ee7146ec0a1 Parents: 1ce0b01 Author: jerryshao Authored: Fri Aug 14 13:44:38 2015 -0700 Committer: Andrew Or Committed: Fri Aug 14 13:44:53 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 16 1 file changed, 8 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/130e06ef/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9217202..26904d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -127,14 +127,8 @@ private[deploy] class Master( // Alternative application submission gateway that is stable across Spark versions private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", true) - private val restServer = -if (restServerEnabled) { - val port = conf.getInt("spark.master.rest.port", 6066) - Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) -} else { - None -} - private val restServerBoundPort = restServer.map(_.start()) + private var restServer: Option[StandaloneRestServer] = None + private var restServerBoundPort: Option[Int] = None override def onStart(): Unit = { logInfo("Starting Spark master at " + masterUrl) @@ -148,6 +142,12 @@ private[deploy] class Master( } }, 0, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) +if (restServerEnabled) { + val port = conf.getInt("spark.master.rest.port", 6066) + restServer = Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) +} +restServerBoundPort = restServer.map(_.start()) + masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() applicationMetricsSystem.start() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode.
Repository: spark Updated Branches: refs/heads/master affc8a887 -> 73431d8af [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode. Currently the spark applications can be queued to the Mesos cluster dispatcher, but when multiple jobs are in queue we don't handle removing jobs from the buffer correctly while iterating and causes null pointer exception. This patch copies the buffer before iterating them, so exceptions aren't thrown when the jobs are removed. Author: Timothy Chen Closes #8322 from tnachen/fix_cluster_mode. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/73431d8a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/73431d8a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/73431d8a Branch: refs/heads/master Commit: 73431d8afb41b93888d2642a1ce2d011f03fb740 Parents: affc8a8 Author: Timothy Chen Authored: Wed Aug 19 19:43:26 2015 -0700 Committer: Andrew Or Committed: Wed Aug 19 19:43:26 2015 -0700 -- .../cluster/mesos/MesosClusterScheduler.scala| 19 +++ 1 file changed, 11 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/73431d8a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 64ec2b8..1206f18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -507,14 +507,16 @@ private[spark] class MesosClusterScheduler( val driversToRetry = pendingRetryDrivers.filter { d => d.retryState.get.nextRetry.before(currentTime) } + scheduleTasks( -driversToRetry, +copyBuffer(driversToRetry), removeFromPendingRetryDrivers, currentOffers, tasks) + // Then we walk through the queued drivers and try to schedule them. scheduleTasks( -queuedDrivers, +copyBuffer(queuedDrivers), removeFromQueuedDrivers, currentOffers, tasks) @@ -527,13 +529,14 @@ private[spark] class MesosClusterScheduler( .foreach(o => driver.declineOffer(o.getId)) } + private def copyBuffer( + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { +val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) +buffer.copyToBuffer(newBuffer) +newBuffer + } + def getSchedulerState(): MesosClusterSchedulerState = { -def copyBuffer( -buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { - val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) - buffer.copyToBuffer(newBuffer) - newBuffer -} stateLock.synchronized { new MesosClusterSchedulerState( frameworkId, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode.
Repository: spark Updated Branches: refs/heads/branch-1.5 16414dae0 -> a3ed2c31e [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode. Currently the spark applications can be queued to the Mesos cluster dispatcher, but when multiple jobs are in queue we don't handle removing jobs from the buffer correctly while iterating and causes null pointer exception. This patch copies the buffer before iterating them, so exceptions aren't thrown when the jobs are removed. Author: Timothy Chen Closes #8322 from tnachen/fix_cluster_mode. (cherry picked from commit 73431d8afb41b93888d2642a1ce2d011f03fb740) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a3ed2c31 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a3ed2c31 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a3ed2c31 Branch: refs/heads/branch-1.5 Commit: a3ed2c31e60b11c09f815b42c0cd894be3150c67 Parents: 16414da Author: Timothy Chen Authored: Wed Aug 19 19:43:26 2015 -0700 Committer: Andrew Or Committed: Wed Aug 19 19:43:34 2015 -0700 -- .../cluster/mesos/MesosClusterScheduler.scala| 19 +++ 1 file changed, 11 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a3ed2c31/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 64ec2b8..1206f18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -507,14 +507,16 @@ private[spark] class MesosClusterScheduler( val driversToRetry = pendingRetryDrivers.filter { d => d.retryState.get.nextRetry.before(currentTime) } + scheduleTasks( -driversToRetry, +copyBuffer(driversToRetry), removeFromPendingRetryDrivers, currentOffers, tasks) + // Then we walk through the queued drivers and try to schedule them. scheduleTasks( -queuedDrivers, +copyBuffer(queuedDrivers), removeFromQueuedDrivers, currentOffers, tasks) @@ -527,13 +529,14 @@ private[spark] class MesosClusterScheduler( .foreach(o => driver.declineOffer(o.getId)) } + private def copyBuffer( + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { +val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) +buffer.copyToBuffer(newBuffer) +newBuffer + } + def getSchedulerState(): MesosClusterSchedulerState = { -def copyBuffer( -buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { - val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) - buffer.copyToBuffer(newBuffer) - newBuffer -} stateLock.synchronized { new MesosClusterSchedulerState( frameworkId, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15279][SQL] Catch conflicting SerDe when creating table
Repository: spark Updated Branches: refs/heads/branch-2.0 655d88293 -> c55a39c97 [SPARK-15279][SQL] Catch conflicting SerDe when creating table ## What changes were proposed in this pull request? The user may do something like: ``` CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS PARQUET CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS ... SERDE 'myserde' CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ORC CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ... SERDE 'myserde' ``` None of these should be allowed because the SerDe's conflict. As of this patch: - `ROW FORMAT DELIMITED` is only compatible with `TEXTFILE` - `ROW FORMAT SERDE` is only compatible with `TEXTFILE`, `RCFILE` and `SEQUENCEFILE` ## How was this patch tested? New tests in `DDLCommandSuite`. Author: Andrew Or Closes #13068 from andrewor14/row-format-conflict. (cherry picked from commit 2585d2b322f3b6b85a0a12ddf7dcde957453000d) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c55a39c9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c55a39c9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c55a39c9 Branch: refs/heads/branch-2.0 Commit: c55a39c973ca617122e5f8af40316e259399b9c9 Parents: 655d882 Author: Andrew Or Authored: Mon May 23 11:55:03 2016 -0700 Committer: Andrew Or Committed: Mon May 23 11:55:13 2016 -0700 -- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala| 60 +++-- .../sql/execution/command/DDLCommandSuite.scala | 94 .../spark/sql/hive/HiveDDLCommandSuite.scala| 4 +- 4 files changed, 129 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c55a39c9/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 -- diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 848c59e..8ea8f76 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -267,8 +267,8 @@ createFileFormat ; fileFormat -: INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING (SERDE serdeCls=STRING)? #tableFileFormat -| identifier #genericFileFormat +: INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING#tableFileFormat +| identifier #genericFileFormat ; storageHandler http://git-wip-us.apache.org/repos/asf/spark/blob/c55a39c9/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index c517b8b..6e4af95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -796,14 +796,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * Expected format: * {{{ - * CREATE [TEMPORARY] [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name - * [(col1 data_type [COMMENT col_comment], ...)] + * CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name + * [(col1[:] data_type [COMMENT col_comment], ...)] * [COMMENT table_comment] - * [PARTITIONED BY (col3 data_type [COMMENT col_comment], ...)] - * [CLUSTERED BY (col1, ...) [SORTED BY (col1 [ASC|DESC], ...)] INTO num_buckets BUCKETS] - * [SKEWED BY (col1, col2, ...) ON ((col_value, col_value, ...), ...) [STORED AS DIRECTORIES]] + * [PARTITIONED BY (col2[:] data_type [COMMENT col_comment], ...)] * [ROW FORMAT row_format] - * [STORED AS file_format | STORED BY storage_handler_class [WITH SERDEPROPERTIES (...)]] + * [STORED AS file_format] * [LOCATION path] * [TBLPROPERTIES (property_name=property_value, ...)] * [AS select_statement]; @@ -849,6 +847,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { compressed = false, serdeProperties = Map()) } +validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) .getOrElse(EmptyStorageFormat) val rowStorage = Option(ctx.rowFormat).
spark git commit: [SPARK-15279][SQL] Catch conflicting SerDe when creating table
Repository: spark Updated Branches: refs/heads/master 07c36a2f0 -> 2585d2b32 [SPARK-15279][SQL] Catch conflicting SerDe when creating table ## What changes were proposed in this pull request? The user may do something like: ``` CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS PARQUET CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS ... SERDE 'myserde' CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ORC CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ... SERDE 'myserde' ``` None of these should be allowed because the SerDe's conflict. As of this patch: - `ROW FORMAT DELIMITED` is only compatible with `TEXTFILE` - `ROW FORMAT SERDE` is only compatible with `TEXTFILE`, `RCFILE` and `SEQUENCEFILE` ## How was this patch tested? New tests in `DDLCommandSuite`. Author: Andrew Or Closes #13068 from andrewor14/row-format-conflict. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2585d2b3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2585d2b3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2585d2b3 Branch: refs/heads/master Commit: 2585d2b322f3b6b85a0a12ddf7dcde957453000d Parents: 07c36a2 Author: Andrew Or Authored: Mon May 23 11:55:03 2016 -0700 Committer: Andrew Or Committed: Mon May 23 11:55:03 2016 -0700 -- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala| 60 +++-- .../sql/execution/command/DDLCommandSuite.scala | 94 .../spark/sql/hive/HiveDDLCommandSuite.scala| 4 +- 4 files changed, 129 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2585d2b3/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 -- diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 848c59e..8ea8f76 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -267,8 +267,8 @@ createFileFormat ; fileFormat -: INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING (SERDE serdeCls=STRING)? #tableFileFormat -| identifier #genericFileFormat +: INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING#tableFileFormat +| identifier #genericFileFormat ; storageHandler http://git-wip-us.apache.org/repos/asf/spark/blob/2585d2b3/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index c517b8b..6e4af95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -796,14 +796,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * Expected format: * {{{ - * CREATE [TEMPORARY] [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name - * [(col1 data_type [COMMENT col_comment], ...)] + * CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name + * [(col1[:] data_type [COMMENT col_comment], ...)] * [COMMENT table_comment] - * [PARTITIONED BY (col3 data_type [COMMENT col_comment], ...)] - * [CLUSTERED BY (col1, ...) [SORTED BY (col1 [ASC|DESC], ...)] INTO num_buckets BUCKETS] - * [SKEWED BY (col1, col2, ...) ON ((col_value, col_value, ...), ...) [STORED AS DIRECTORIES]] + * [PARTITIONED BY (col2[:] data_type [COMMENT col_comment], ...)] * [ROW FORMAT row_format] - * [STORED AS file_format | STORED BY storage_handler_class [WITH SERDEPROPERTIES (...)]] + * [STORED AS file_format] * [LOCATION path] * [TBLPROPERTIES (property_name=property_value, ...)] * [AS select_statement]; @@ -849,6 +847,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { compressed = false, serdeProperties = Map()) } +validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) .getOrElse(EmptyStorageFormat) val rowStorage = Option(ctx.rowFormat).map(visitRowFormat).getOrElse(EmptyStorageFormat) @@ -905,6 +904,8 @@ class SparkSqlAstBuilder(conf: SQLCo
spark git commit: [SPARK-15311][SQL] Disallow DML on Regular Tables when Using In-Memory Catalog
Repository: spark Updated Branches: refs/heads/master 01659bc50 -> 5afd927a4 [SPARK-15311][SQL] Disallow DML on Regular Tables when Using In-Memory Catalog What changes were proposed in this pull request? So far, when using In-Memory Catalog, we allow DDL operations for the tables. However, the corresponding DML operations are not supported for the tables that are neither temporary nor data source tables. For example, ```SQL CREATE TABLE tabName(i INT, j STRING) SELECT * FROM tabName INSERT OVERWRITE TABLE tabName SELECT 1, 'a' ``` In the above example, before this PR fix, we will get very confusing exception messages for either `SELECT` or `INSERT` ``` org.apache.spark.sql.AnalysisException: unresolved operator 'SimpleCatalogRelation default, CatalogTable(`default`.`tbl`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(i,int,true,None), CatalogColumn(j,string,true,None)),List(),List(),List(),-1,,1463928681802,-1,Map(),None,None,None,List()), None; ``` This PR is to issue appropriate exceptions in this case. The message will be like ``` org.apache.spark.sql.AnalysisException: Please enable Hive support when operating non-temporary tables: `tbl`; ``` How was this patch tested? Added a test case in `DDLSuite`. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13093 from gatorsmile/selectAfterCreate. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5afd927a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5afd927a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5afd927a Branch: refs/heads/master Commit: 5afd927a47aa7ede3039234f2f7262e2247aa2ae Parents: 01659bc Author: gatorsmile Authored: Mon May 23 18:03:45 2016 -0700 Committer: Andrew Or Committed: Mon May 23 18:03:45 2016 -0700 -- .../sql/catalyst/analysis/CheckAnalysis.scala | 15 ++ .../plans/logical/basicLogicalOperators.scala | 3 +- .../spark/sql/execution/command/DDLSuite.scala | 48 3 files changed, 65 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5afd927a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 28aa249..cd242d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.catalog.SimpleCatalogRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.UsingJoin @@ -305,6 +306,20 @@ trait CheckAnalysis extends PredicateHelper { |Conflicting attributes: ${conflictingAttributes.mkString(",")} """.stripMargin) + case s: SimpleCatalogRelation => +failAnalysis( + s""" + |Please enable Hive support when selecting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + + case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => +failAnalysis( + s""" + |Please enable Hive support when inserting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + case o if !o.resolved => failAnalysis( s"unresolved operator ${operator.simpleString}") http://git-wip-us.apache.org/repos/asf/spark/blob/5afd927a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b1b3e00..ca0096e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -377,7 +377,8 @@ case class InsertIntoTable( } assert(overwrite || !ifNo
spark git commit: [SPARK-15311][SQL] Disallow DML on Regular Tables when Using In-Memory Catalog
Repository: spark Updated Branches: refs/heads/branch-2.0 3def56120 -> 220b9a08e [SPARK-15311][SQL] Disallow DML on Regular Tables when Using In-Memory Catalog What changes were proposed in this pull request? So far, when using In-Memory Catalog, we allow DDL operations for the tables. However, the corresponding DML operations are not supported for the tables that are neither temporary nor data source tables. For example, ```SQL CREATE TABLE tabName(i INT, j STRING) SELECT * FROM tabName INSERT OVERWRITE TABLE tabName SELECT 1, 'a' ``` In the above example, before this PR fix, we will get very confusing exception messages for either `SELECT` or `INSERT` ``` org.apache.spark.sql.AnalysisException: unresolved operator 'SimpleCatalogRelation default, CatalogTable(`default`.`tbl`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(i,int,true,None), CatalogColumn(j,string,true,None)),List(),List(),List(),-1,,1463928681802,-1,Map(),None,None,None,List()), None; ``` This PR is to issue appropriate exceptions in this case. The message will be like ``` org.apache.spark.sql.AnalysisException: Please enable Hive support when operating non-temporary tables: `tbl`; ``` How was this patch tested? Added a test case in `DDLSuite`. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13093 from gatorsmile/selectAfterCreate. (cherry picked from commit 5afd927a47aa7ede3039234f2f7262e2247aa2ae) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/220b9a08 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/220b9a08 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/220b9a08 Branch: refs/heads/branch-2.0 Commit: 220b9a08ef483e9dfd42c69d2822662d013d19c0 Parents: 3def561 Author: gatorsmile Authored: Mon May 23 18:03:45 2016 -0700 Committer: Andrew Or Committed: Mon May 23 18:03:55 2016 -0700 -- .../sql/catalyst/analysis/CheckAnalysis.scala | 15 ++ .../plans/logical/basicLogicalOperators.scala | 3 +- .../spark/sql/execution/command/DDLSuite.scala | 48 3 files changed, 65 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/220b9a08/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 28aa249..cd242d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.catalog.SimpleCatalogRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.UsingJoin @@ -305,6 +306,20 @@ trait CheckAnalysis extends PredicateHelper { |Conflicting attributes: ${conflictingAttributes.mkString(",")} """.stripMargin) + case s: SimpleCatalogRelation => +failAnalysis( + s""" + |Please enable Hive support when selecting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + + case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => +failAnalysis( + s""" + |Please enable Hive support when inserting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + case o if !o.resolved => failAnalysis( s"unresolved operator ${operator.simpleString}") http://git-wip-us.apache.org/repos/asf/spark/blob/220b9a08/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b1b3e00..ca0096e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLo
spark git commit: [SPARK-15464][ML][MLLIB][SQL][TESTS] Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code
Repository: spark Updated Branches: refs/heads/master 5afd927a4 -> a15ca5533 [SPARK-15464][ML][MLLIB][SQL][TESTS] Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code ## What changes were proposed in this pull request? Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code. ## How was this patch tested? Existing test. Author: WeichenXu Closes #13242 from WeichenXu123/python_doctest_update_sparksession. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a15ca553 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a15ca553 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a15ca553 Branch: refs/heads/master Commit: a15ca5533db91fefaf3248255a59c4d94eeda1a9 Parents: 5afd927 Author: WeichenXu Authored: Mon May 23 18:14:48 2016 -0700 Committer: Andrew Or Committed: Mon May 23 18:14:48 2016 -0700 -- python/pyspark/ml/classification.py| 38 +++--- python/pyspark/ml/clustering.py| 22 ++-- python/pyspark/ml/evaluation.py| 20 ++-- python/pyspark/ml/feature.py | 66 +- python/pyspark/ml/recommendation.py| 18 +-- python/pyspark/ml/regression.py| 46 +++ python/pyspark/ml/tuning.py| 18 +-- python/pyspark/mllib/classification.py | 10 +- python/pyspark/mllib/evaluation.py | 10 +- python/pyspark/mllib/feature.py| 10 +- python/pyspark/mllib/fpm.py| 9 +- python/pyspark/mllib/linalg/distributed.py | 12 +- python/pyspark/mllib/random.py | 10 +- python/pyspark/mllib/regression.py | 10 +- python/pyspark/mllib/stat/_statistics.py | 10 +- python/pyspark/mllib/tree.py | 9 +- python/pyspark/mllib/util.py | 10 +- python/pyspark/sql/catalog.py | 14 ++- python/pyspark/sql/column.py | 12 +- python/pyspark/sql/conf.py | 11 +- python/pyspark/sql/functions.py| 153 python/pyspark/sql/group.py| 12 +- 22 files changed, 298 insertions(+), 232 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a15ca553/python/pyspark/ml/classification.py -- diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index a1c3f72..ea660d7 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -498,7 +498,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer ->>> df = sqlContext.createDataFrame([ +>>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -512,7 +512,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred 1 >>> model.featureImportances SparseVector(1, {0: 1.0}) ->>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) +>>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -520,7 +520,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred DenseVector([1.0, 0.0]) >>> result.rawPrediction DenseVector([1.0, 0.0]) ->>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) +>>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 @@ -627,7 +627,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer ->>> df = sqlContext.createDataFrame([ +>>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -639,7 +639,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0, 1.0]) True ->>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) +>>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(te
spark git commit: [SPARK-15464][ML][MLLIB][SQL][TESTS] Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code
Repository: spark Updated Branches: refs/heads/branch-2.0 220b9a08e -> f3162b96d [SPARK-15464][ML][MLLIB][SQL][TESTS] Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code ## What changes were proposed in this pull request? Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code. ## How was this patch tested? Existing test. Author: WeichenXu Closes #13242 from WeichenXu123/python_doctest_update_sparksession. (cherry picked from commit a15ca5533db91fefaf3248255a59c4d94eeda1a9) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f3162b96 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f3162b96 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f3162b96 Branch: refs/heads/branch-2.0 Commit: f3162b96da4f61524c11150904916734c0e949ab Parents: 220b9a0 Author: WeichenXu Authored: Mon May 23 18:14:48 2016 -0700 Committer: Andrew Or Committed: Mon May 23 18:14:58 2016 -0700 -- python/pyspark/ml/classification.py| 38 +++--- python/pyspark/ml/clustering.py| 22 ++-- python/pyspark/ml/evaluation.py| 20 ++-- python/pyspark/ml/feature.py | 66 +- python/pyspark/ml/recommendation.py| 18 +-- python/pyspark/ml/regression.py| 46 +++ python/pyspark/ml/tuning.py| 18 +-- python/pyspark/mllib/classification.py | 10 +- python/pyspark/mllib/evaluation.py | 10 +- python/pyspark/mllib/feature.py| 10 +- python/pyspark/mllib/fpm.py| 9 +- python/pyspark/mllib/linalg/distributed.py | 12 +- python/pyspark/mllib/random.py | 10 +- python/pyspark/mllib/regression.py | 10 +- python/pyspark/mllib/stat/_statistics.py | 10 +- python/pyspark/mllib/tree.py | 9 +- python/pyspark/mllib/util.py | 10 +- python/pyspark/sql/catalog.py | 14 ++- python/pyspark/sql/column.py | 12 +- python/pyspark/sql/conf.py | 11 +- python/pyspark/sql/functions.py| 153 python/pyspark/sql/group.py| 12 +- 22 files changed, 298 insertions(+), 232 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f3162b96/python/pyspark/ml/classification.py -- diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index a1c3f72..ea660d7 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -498,7 +498,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer ->>> df = sqlContext.createDataFrame([ +>>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -512,7 +512,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred 1 >>> model.featureImportances SparseVector(1, {0: 1.0}) ->>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) +>>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -520,7 +520,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred DenseVector([1.0, 0.0]) >>> result.rawPrediction DenseVector([1.0, 0.0]) ->>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) +>>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 @@ -627,7 +627,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer ->>> df = sqlContext.createDataFrame([ +>>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -639,7 +639,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0, 1.0]) True ->>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) +>>> te
spark git commit: Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB"
Repository: spark Updated Branches: refs/heads/master fa244e5a9 -> de726b0d5 Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB" This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/de726b0d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/de726b0d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/de726b0d Branch: refs/heads/master Commit: de726b0d533158d3ca08841bd6976bcfa26ca79d Parents: fa244e5 Author: Andrew Or Authored: Mon May 23 21:43:11 2016 -0700 Committer: Andrew Or Committed: Mon May 23 21:43:11 2016 -0700 -- .../catalyst/expressions/objects/objects.scala | 32 -- .../spark/sql/DataFrameComplexTypeSuite.scala | 35 2 files changed, 6 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/de726b0d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index cc094f2..2f2323f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -232,47 +232,27 @@ case class NewInstance( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) -val argIsNulls = ctx.freshName("argIsNulls") -ctx.addMutableState("boolean[]", argIsNulls, - s"$argIsNulls = new boolean[${arguments.size}];") -val argValues = arguments.zipWithIndex.map { case (e, i) => - val argValue = ctx.freshName("argValue") - ctx.addMutableState(ctx.javaType(e.dataType), argValue, "") - argValue -} - -val argCodes = arguments.zipWithIndex.map { case (e, i) => - val expr = e.genCode(ctx) - expr.code + s""" - $argIsNulls[$i] = ${expr.isNull}; - ${argValues(i)} = ${expr.value}; - """ -} -val argCode = ctx.splitExpressions(ctx.INPUT_ROW, argCodes) +val argGen = arguments.map(_.genCode(ctx)) +val argString = argGen.map(_.value).mkString(", ") val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) var isNull = ev.isNull val setIsNull = if (propagateNull && arguments.nonEmpty) { - s""" - boolean $isNull = false; - for (int idx = 0; idx < ${arguments.length}; idx++) { - if ($argIsNulls[idx]) { $isNull = true; break; } - } - """ + s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" } else { isNull = "false" "" } val constructorCall = outer.map { gen => - s"""${gen.value}.new ${cls.getSimpleName}(${argValues.mkString(", ")})""" + s"""${gen.value}.new ${cls.getSimpleName}($argString)""" }.getOrElse { - s"new $className(${argValues.mkString(", ")})" + s"new $className($argString)" } val code = s""" - $argCode + ${argGen.map(_.code).mkString("\n")} ${outer.map(_.code).getOrElse("")} $setIsNull final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; http://git-wip-us.apache.org/repos/asf/spark/blob/de726b0d/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 07fbaba..72f676e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -58,39 +58,4 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { val nullIntRow = df.selectExpr("i[1]").collect()(0) assert(nullIntRow == org.apache.spark.sql.Row(null)) } - - test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { -val ds100_5 = Seq(S100_5()).toDS() -ds100_5.rdd.count - } } - -case class S100( - s1: String = "1", s2: String = "2", s3: String = "3", s4: String = "4", - s5: String = "5", s6: String = "6", s7: String = "7", s8: String = "8", - s9: String = "9", s10: String = "10", s11: String = "11", s12: String = "12", - s13: String = "13", s14: String = "14", s15: String = "15", s16: String = "16", - s17: String = "17", s18: Stri
spark git commit: Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB"
Repository: spark Updated Branches: refs/heads/branch-2.0 d0bcec157 -> 1890f5fdf Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB" This reverts commit d0bcec157d2bd2ed4eff848f831841bef4745904. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1890f5fd Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1890f5fd Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1890f5fd Branch: refs/heads/branch-2.0 Commit: 1890f5fdf501104171f93c759e20835a61c8927b Parents: d0bcec1 Author: Andrew Or Authored: Mon May 23 21:43:34 2016 -0700 Committer: Andrew Or Committed: Mon May 23 21:43:34 2016 -0700 -- .../catalyst/expressions/objects/objects.scala | 32 -- .../spark/sql/DataFrameComplexTypeSuite.scala | 35 2 files changed, 6 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1890f5fd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index cc094f2..2f2323f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -232,47 +232,27 @@ case class NewInstance( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) -val argIsNulls = ctx.freshName("argIsNulls") -ctx.addMutableState("boolean[]", argIsNulls, - s"$argIsNulls = new boolean[${arguments.size}];") -val argValues = arguments.zipWithIndex.map { case (e, i) => - val argValue = ctx.freshName("argValue") - ctx.addMutableState(ctx.javaType(e.dataType), argValue, "") - argValue -} - -val argCodes = arguments.zipWithIndex.map { case (e, i) => - val expr = e.genCode(ctx) - expr.code + s""" - $argIsNulls[$i] = ${expr.isNull}; - ${argValues(i)} = ${expr.value}; - """ -} -val argCode = ctx.splitExpressions(ctx.INPUT_ROW, argCodes) +val argGen = arguments.map(_.genCode(ctx)) +val argString = argGen.map(_.value).mkString(", ") val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) var isNull = ev.isNull val setIsNull = if (propagateNull && arguments.nonEmpty) { - s""" - boolean $isNull = false; - for (int idx = 0; idx < ${arguments.length}; idx++) { - if ($argIsNulls[idx]) { $isNull = true; break; } - } - """ + s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" } else { isNull = "false" "" } val constructorCall = outer.map { gen => - s"""${gen.value}.new ${cls.getSimpleName}(${argValues.mkString(", ")})""" + s"""${gen.value}.new ${cls.getSimpleName}($argString)""" }.getOrElse { - s"new $className(${argValues.mkString(", ")})" + s"new $className($argString)" } val code = s""" - $argCode + ${argGen.map(_.code).mkString("\n")} ${outer.map(_.code).getOrElse("")} $setIsNull final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; http://git-wip-us.apache.org/repos/asf/spark/blob/1890f5fd/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 07fbaba..72f676e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -58,39 +58,4 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { val nullIntRow = df.selectExpr("i[1]").collect()(0) assert(nullIntRow == org.apache.spark.sql.Row(null)) } - - test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { -val ds100_5 = Seq(S100_5()).toDS() -ds100_5.rdd.count - } } - -case class S100( - s1: String = "1", s2: String = "2", s3: String = "3", s4: String = "4", - s5: String = "5", s6: String = "6", s7: String = "7", s8: String = "8", - s9: String = "9", s10: String = "10", s11: String = "11", s12: String = "12", - s13: String = "13", s14: String = "14", s15: String = "15", s16: String = "16", - s17: String = "17", s
spark git commit: [SPARK-15397][SQL] fix string udf locate as hive
Repository: spark Updated Branches: refs/heads/master de726b0d5 -> d642b2735 [SPARK-15397][SQL] fix string udf locate as hive ## What changes were proposed in this pull request? in hive, `locate("aa", "aaa", 0)` would yield 0, `locate("aa", "aaa", 1)` would yield 1 and `locate("aa", "aaa", 2)` would yield 2, while in Spark, `locate("aa", "aaa", 0)` would yield 1, `locate("aa", "aaa", 1)` would yield 2 and `locate("aa", "aaa", 2)` would yield 0. This results from the different understanding of the third parameter in udf `locate`. It means the starting index and starts from 1, so when we use 0, the return would always be 0. ## How was this patch tested? tested with modified `StringExpressionsSuite` and `StringFunctionsSuite` Author: Daoyuan Wang Closes #13186 from adrian-wang/locate. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d642b273 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d642b273 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d642b273 Branch: refs/heads/master Commit: d642b273544bb77ef7f584326aa2d214649ac61b Parents: de726b0 Author: Daoyuan Wang Authored: Mon May 23 23:29:15 2016 -0700 Committer: Andrew Or Committed: Mon May 23 23:29:15 2016 -0700 -- R/pkg/R/functions.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R| 2 +- python/pyspark/sql/functions.py | 2 +- .../catalyst/expressions/stringExpressions.scala | 19 +-- .../expressions/StringExpressionsSuite.scala | 16 +--- .../apache/spark/sql/StringFunctionsSuite.scala | 10 +- 6 files changed, 30 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d642b273/R/pkg/R/functions.R -- diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 4a0bdf3..2665d1d 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2226,7 +2226,7 @@ setMethod("window", signature(x = "Column"), #' @export #' @examples \dontrun{locate('b', df$c, 1)} setMethod("locate", signature(substr = "character", str = "Column"), - function(substr, str, pos = 0) { + function(substr, str, pos = 1) { jc <- callJStatic("org.apache.spark.sql.functions", "locate", substr, str@jc, as.integer(pos)) http://git-wip-us.apache.org/repos/asf/spark/blob/d642b273/R/pkg/inst/tests/testthat/test_sparkSQL.R -- diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 6a99b43..b2d769f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1152,7 +1152,7 @@ test_that("string operators", { l2 <- list(list(a = "aaads")) df2 <- createDataFrame(sqlContext, l2) expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) - expect_equal(collect(select(df2, locate("aa", df2$a, 1)))[1, 1], 2) + expect_equal(collect(select(df2, locate("aa", df2$a, 2)))[1, 1], 2) expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") # nolint expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") # nolint http://git-wip-us.apache.org/repos/asf/spark/blob/d642b273/python/pyspark/sql/functions.py -- diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 1f15eec..64b8bc4 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1359,7 +1359,7 @@ def levenshtein(left, right): @since(1.5) -def locate(substr, str, pos=0): +def locate(substr, str, pos=1): """ Locate the position of the first occurrence of substr in a string column, after position pos. http://git-wip-us.apache.org/repos/asf/spark/blob/d642b273/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 78e846d..44ff7fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -494,7 +494,7 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) extends TernaryExpression with ImplicitCastInputTypes { def this(substr: Expression, str: Expression) = {
spark git commit: [SPARK-15397][SQL] fix string udf locate as hive
Repository: spark Updated Branches: refs/heads/branch-2.0 1890f5fdf -> 6adbc0613 [SPARK-15397][SQL] fix string udf locate as hive ## What changes were proposed in this pull request? in hive, `locate("aa", "aaa", 0)` would yield 0, `locate("aa", "aaa", 1)` would yield 1 and `locate("aa", "aaa", 2)` would yield 2, while in Spark, `locate("aa", "aaa", 0)` would yield 1, `locate("aa", "aaa", 1)` would yield 2 and `locate("aa", "aaa", 2)` would yield 0. This results from the different understanding of the third parameter in udf `locate`. It means the starting index and starts from 1, so when we use 0, the return would always be 0. ## How was this patch tested? tested with modified `StringExpressionsSuite` and `StringFunctionsSuite` Author: Daoyuan Wang Closes #13186 from adrian-wang/locate. (cherry picked from commit d642b273544bb77ef7f584326aa2d214649ac61b) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6adbc061 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6adbc061 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6adbc061 Branch: refs/heads/branch-2.0 Commit: 6adbc061361bd4fd66bedb5b5a9997cf489d6439 Parents: 1890f5f Author: Daoyuan Wang Authored: Mon May 23 23:29:15 2016 -0700 Committer: Andrew Or Committed: Mon May 23 23:29:28 2016 -0700 -- R/pkg/R/functions.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R| 2 +- python/pyspark/sql/functions.py | 2 +- .../catalyst/expressions/stringExpressions.scala | 19 +-- .../expressions/StringExpressionsSuite.scala | 16 +--- .../apache/spark/sql/StringFunctionsSuite.scala | 10 +- 6 files changed, 30 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6adbc061/R/pkg/R/functions.R -- diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 4a0bdf3..2665d1d 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2226,7 +2226,7 @@ setMethod("window", signature(x = "Column"), #' @export #' @examples \dontrun{locate('b', df$c, 1)} setMethod("locate", signature(substr = "character", str = "Column"), - function(substr, str, pos = 0) { + function(substr, str, pos = 1) { jc <- callJStatic("org.apache.spark.sql.functions", "locate", substr, str@jc, as.integer(pos)) http://git-wip-us.apache.org/repos/asf/spark/blob/6adbc061/R/pkg/inst/tests/testthat/test_sparkSQL.R -- diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 6a99b43..b2d769f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1152,7 +1152,7 @@ test_that("string operators", { l2 <- list(list(a = "aaads")) df2 <- createDataFrame(sqlContext, l2) expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) - expect_equal(collect(select(df2, locate("aa", df2$a, 1)))[1, 1], 2) + expect_equal(collect(select(df2, locate("aa", df2$a, 2)))[1, 1], 2) expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") # nolint expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") # nolint http://git-wip-us.apache.org/repos/asf/spark/blob/6adbc061/python/pyspark/sql/functions.py -- diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 1f15eec..64b8bc4 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1359,7 +1359,7 @@ def levenshtein(left, right): @since(1.5) -def locate(substr, str, pos=0): +def locate(substr, str, pos=1): """ Locate the position of the first occurrence of substr in a string column, after position pos. http://git-wip-us.apache.org/repos/asf/spark/blob/6adbc061/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 78e846d..44ff7fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -494,7 +494,7 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) extend
spark git commit: [SPARK-15388][SQL] Fix spark sql CREATE FUNCTION with hive 1.2.1
Repository: spark Updated Branches: refs/heads/master a313a5ae7 -> 784cc07d1 [SPARK-15388][SQL] Fix spark sql CREATE FUNCTION with hive 1.2.1 ## What changes were proposed in this pull request? spark.sql("CREATE FUNCTION myfunc AS 'com.haizhi.bdp.udf.UDFGetGeoCode'") throws "org.apache.hadoop.hive.ql.metadata.HiveException:MetaException(message:NoSuchObjectException(message:Function default.myfunc does not exist))" with hive 1.2.1. I think it is introduced by pr #12853. Fixing it by catching Exception (not NoSuchObjectException) and string matching. ## How was this patch tested? added a unit test and also tested it manually Author: wangyang Closes #13177 from wangyang1992/fixCreateFunc2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/784cc07d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/784cc07d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/784cc07d Branch: refs/heads/master Commit: 784cc07d1675eb9e0a387673cf86874e1bfc10f9 Parents: a313a5a Author: wangyang Authored: Tue May 24 11:03:12 2016 -0700 Committer: Andrew Or Committed: Tue May 24 11:03:12 2016 -0700 -- .../org/apache/spark/sql/hive/client/HiveShim.scala | 14 -- .../apache/spark/sql/hive/client/VersionsSuite.scala | 1 + 2 files changed, 13 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/784cc07d/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 78713c3..9df4a26 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -24,6 +24,7 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JS import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf @@ -42,7 +43,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{IntegralType, StringType} -import org.apache.spark.util.CausedBy /** @@ -480,11 +480,21 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { try { Option(hive.getFunction(db, name)).map(fromHiveFunction) } catch { - case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + case NonFatal(e) if isCausedBy(e, s"$name does not exist") => None } } + private def isCausedBy(e: Throwable, matchMassage: String): Boolean = { +if (e.getMessage.contains(matchMassage)) { + true +} else if (e.getCause != null) { + isCausedBy(e.getCause, matchMassage) +} else { + false +} + } + override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { hive.getFunctions(db, pattern).asScala } http://git-wip-us.apache.org/repos/asf/spark/blob/784cc07d/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index d46c4e7..8ae4535 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -440,6 +440,7 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.getFunctionOption("default", "func2").isEmpty) } else { assert(client.getFunctionOption("default", "func2").isDefined) +assert(client.getFunctionOption("default", "the_func_not_exists").isEmpty) } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15388][SQL] Fix spark sql CREATE FUNCTION with hive 1.2.1
Repository: spark Updated Branches: refs/heads/branch-2.0 988d4dbf4 -> 1bb0aa4b0 [SPARK-15388][SQL] Fix spark sql CREATE FUNCTION with hive 1.2.1 ## What changes were proposed in this pull request? spark.sql("CREATE FUNCTION myfunc AS 'com.haizhi.bdp.udf.UDFGetGeoCode'") throws "org.apache.hadoop.hive.ql.metadata.HiveException:MetaException(message:NoSuchObjectException(message:Function default.myfunc does not exist))" with hive 1.2.1. I think it is introduced by pr #12853. Fixing it by catching Exception (not NoSuchObjectException) and string matching. ## How was this patch tested? added a unit test and also tested it manually Author: wangyang Closes #13177 from wangyang1992/fixCreateFunc2. (cherry picked from commit 784cc07d1675eb9e0a387673cf86874e1bfc10f9) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1bb0aa4b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1bb0aa4b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1bb0aa4b Branch: refs/heads/branch-2.0 Commit: 1bb0aa4b0a5b680e535cac6305b9eac856606ef4 Parents: 988d4db Author: wangyang Authored: Tue May 24 11:03:12 2016 -0700 Committer: Andrew Or Committed: Tue May 24 11:03:21 2016 -0700 -- .../org/apache/spark/sql/hive/client/HiveShim.scala | 14 -- .../apache/spark/sql/hive/client/VersionsSuite.scala | 1 + 2 files changed, 13 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1bb0aa4b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 78713c3..9df4a26 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -24,6 +24,7 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JS import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf @@ -42,7 +43,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{IntegralType, StringType} -import org.apache.spark.util.CausedBy /** @@ -480,11 +480,21 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { try { Option(hive.getFunction(db, name)).map(fromHiveFunction) } catch { - case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + case NonFatal(e) if isCausedBy(e, s"$name does not exist") => None } } + private def isCausedBy(e: Throwable, matchMassage: String): Boolean = { +if (e.getMessage.contains(matchMassage)) { + true +} else if (e.getCause != null) { + isCausedBy(e.getCause, matchMassage) +} else { + false +} + } + override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { hive.getFunctions(db, pattern).asScala } http://git-wip-us.apache.org/repos/asf/spark/blob/1bb0aa4b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index d46c4e7..8ae4535 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -440,6 +440,7 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.getFunctionOption("default", "func2").isEmpty) } else { assert(client.getFunctionOption("default", "func2").isDefined) +assert(client.getFunctionOption("default", "the_func_not_exists").isEmpty) } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR][CORE][TEST] Update obsolete `takeSample` test case.
Repository: spark Updated Branches: refs/heads/branch-2.0 1bb0aa4b0 -> 2574abea0 [MINOR][CORE][TEST] Update obsolete `takeSample` test case. ## What changes were proposed in this pull request? This PR fixes some obsolete comments and assertion in `takeSample` testcase of `RDDSuite.scala`. ## How was this patch tested? This fixes the testcase only. Author: Dongjoon Hyun Closes #13260 from dongjoon-hyun/SPARK-15481. (cherry picked from commit be99a99fe7976419d727c0cc92e872aa4af58bf1) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2574abea Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2574abea Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2574abea Branch: refs/heads/branch-2.0 Commit: 2574abea088be725b6ecf2c473819535cd1cf0a5 Parents: 1bb0aa4 Author: Dongjoon Hyun Authored: Tue May 24 11:09:54 2016 -0700 Committer: Andrew Or Committed: Tue May 24 11:10:02 2016 -0700 -- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 15 +++ 1 file changed, 7 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2574abea/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 979fb42..a4992fe 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -678,27 +678,26 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } { val sample = data.takeSample(withReplacement = true, num = 20) - assert(sample.size === 20)// Got exactly 100 elements - assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.size === 20)// Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } { val sample = data.takeSample(withReplacement = true, num = n) - assert(sample.size === n)// Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n)// Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, n, seed) - assert(sample.size === n)// Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n)// Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, 2 * n, seed) - assert(sample.size === 2 * n)// Got exactly 200 elements - // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.size === 2 * n)// Got exactly 2 * n elements + // Chance of getting all distinct elements is still quite low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR][CORE][TEST] Update obsolete `takeSample` test case.
Repository: spark Updated Branches: refs/heads/master 784cc07d1 -> be99a99fe [MINOR][CORE][TEST] Update obsolete `takeSample` test case. ## What changes were proposed in this pull request? This PR fixes some obsolete comments and assertion in `takeSample` testcase of `RDDSuite.scala`. ## How was this patch tested? This fixes the testcase only. Author: Dongjoon Hyun Closes #13260 from dongjoon-hyun/SPARK-15481. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/be99a99f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/be99a99f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/be99a99f Branch: refs/heads/master Commit: be99a99fe7976419d727c0cc92e872aa4af58bf1 Parents: 784cc07 Author: Dongjoon Hyun Authored: Tue May 24 11:09:54 2016 -0700 Committer: Andrew Or Committed: Tue May 24 11:09:54 2016 -0700 -- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 15 +++ 1 file changed, 7 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/be99a99f/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 979fb42..a4992fe 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -678,27 +678,26 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } { val sample = data.takeSample(withReplacement = true, num = 20) - assert(sample.size === 20)// Got exactly 100 elements - assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.size === 20)// Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } { val sample = data.takeSample(withReplacement = true, num = n) - assert(sample.size === n)// Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n)// Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, n, seed) - assert(sample.size === n)// Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n)// Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, 2 * n, seed) - assert(sample.size === 2 * n)// Got exactly 200 elements - // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.size === 2 * n)// Got exactly 2 * n elements + // Chance of getting all distinct elements is still quite low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15345][SQL][PYSPARK] SparkSession's conf doesn't take effect when this already an existing SparkContext
Repository: spark Updated Branches: refs/heads/master b120fba6a -> 01e7b9c85 [SPARK-15345][SQL][PYSPARK] SparkSession's conf doesn't take effect when this already an existing SparkContext ## What changes were proposed in this pull request? Override the existing SparkContext is the provided SparkConf is different. PySpark part hasn't been fixed yet, will do that after the first round of review to ensure this is the correct approach. ## How was this patch tested? Manually verify it in spark-shell. rxin Please help review it, I think this is a very critical issue for spark 2.0 Author: Jeff Zhang Closes #13160 from zjffdu/SPARK-15345. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/01e7b9c8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/01e7b9c8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/01e7b9c8 Branch: refs/heads/master Commit: 01e7b9c85bb84924e279021f9748774dce9702c8 Parents: b120fba Author: Jeff Zhang Authored: Wed May 25 10:46:51 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:46:51 2016 -0700 -- .../main/scala/org/apache/spark/SparkContext.scala| 3 +++ .../scala/org/apache/spark/sql/SparkSession.scala | 14 -- .../apache/spark/sql/SparkSessionBuilderSuite.scala | 14 +- 3 files changed, 28 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/01e7b9c8/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 36aa3be..5018eb3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2254,6 +2254,9 @@ object SparkContext extends Logging { if (activeContext.get() == null) { setActiveContext(new SparkContext(config), allowMultipleContexts = false) } + if (config.getAll.nonEmpty) { +logWarning("Use an existing SparkContext, some configuration may not take effect.") + } activeContext.get() } } http://git-wip-us.apache.org/repos/asf/spark/blob/01e7b9c8/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5c87c84..86c97b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -636,7 +636,7 @@ object SparkSession { /** * Builder for [[SparkSession]]. */ - class Builder { + class Builder extends Logging { private[this] val options = new scala.collection.mutable.HashMap[String, String] @@ -753,6 +753,9 @@ object SparkSession { var session = activeThreadSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } +if (options.nonEmpty) { + logWarning("Use an existing SparkSession, some configuration may not take effect.") +} return session } @@ -762,6 +765,9 @@ object SparkSession { session = defaultSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } + if (options.nonEmpty) { +logWarning("Use an existing SparkSession, some configuration may not take effect.") + } return session } @@ -774,7 +780,11 @@ object SparkSession { val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } - SparkContext.getOrCreate(sparkConf) + val sc = SparkContext.getOrCreate(sparkConf) + // maybe this is an existing SparkContext, update its SparkConf which maybe used + // by SparkSession + options.foreach { case (k, v) => sc.conf.set(k, v) } + sc } session = new SparkSession(sparkContext) options.foreach { case (k, v) => session.conf.set(k, v) } http://git-wip-us.apache.org/repos/asf/spark/blob/01e7b9c8/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index ec6a2b3..786956d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSu
spark git commit: [SPARK-15345][SQL][PYSPARK] SparkSession's conf doesn't take effect when this already an existing SparkContext
Repository: spark Updated Branches: refs/heads/branch-2.0 69327667d -> 27f26a39d [SPARK-15345][SQL][PYSPARK] SparkSession's conf doesn't take effect when this already an existing SparkContext ## What changes were proposed in this pull request? Override the existing SparkContext is the provided SparkConf is different. PySpark part hasn't been fixed yet, will do that after the first round of review to ensure this is the correct approach. ## How was this patch tested? Manually verify it in spark-shell. rxin Please help review it, I think this is a very critical issue for spark 2.0 Author: Jeff Zhang Closes #13160 from zjffdu/SPARK-15345. (cherry picked from commit 01e7b9c85bb84924e279021f9748774dce9702c8) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/27f26a39 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/27f26a39 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/27f26a39 Branch: refs/heads/branch-2.0 Commit: 27f26a39db021735bcf75a1f1b89b9481b199341 Parents: 6932766 Author: Jeff Zhang Authored: Wed May 25 10:46:51 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:47:00 2016 -0700 -- .../main/scala/org/apache/spark/SparkContext.scala| 3 +++ .../scala/org/apache/spark/sql/SparkSession.scala | 14 -- .../apache/spark/sql/SparkSessionBuilderSuite.scala | 14 +- 3 files changed, 28 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/27f26a39/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 36aa3be..5018eb3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2254,6 +2254,9 @@ object SparkContext extends Logging { if (activeContext.get() == null) { setActiveContext(new SparkContext(config), allowMultipleContexts = false) } + if (config.getAll.nonEmpty) { +logWarning("Use an existing SparkContext, some configuration may not take effect.") + } activeContext.get() } } http://git-wip-us.apache.org/repos/asf/spark/blob/27f26a39/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5c87c84..86c97b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -636,7 +636,7 @@ object SparkSession { /** * Builder for [[SparkSession]]. */ - class Builder { + class Builder extends Logging { private[this] val options = new scala.collection.mutable.HashMap[String, String] @@ -753,6 +753,9 @@ object SparkSession { var session = activeThreadSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } +if (options.nonEmpty) { + logWarning("Use an existing SparkSession, some configuration may not take effect.") +} return session } @@ -762,6 +765,9 @@ object SparkSession { session = defaultSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } + if (options.nonEmpty) { +logWarning("Use an existing SparkSession, some configuration may not take effect.") + } return session } @@ -774,7 +780,11 @@ object SparkSession { val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } - SparkContext.getOrCreate(sparkConf) + val sc = SparkContext.getOrCreate(sparkConf) + // maybe this is an existing SparkContext, update its SparkConf which maybe used + // by SparkSession + options.foreach { case (k, v) => sc.conf.set(k, v) } + sc } session = new SparkSession(sparkContext) options.foreach { case (k, v) => session.conf.set(k, v) } http://git-wip-us.apache.org/repos/asf/spark/blob/27f26a39/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala
spark git commit: [SPARK-15520][SQL] SparkSession builder in python should also allow overriding confs of existing sessions
Repository: spark Updated Branches: refs/heads/branch-2.0 27f26a39d -> c75ec5eaa [SPARK-15520][SQL] SparkSession builder in python should also allow overriding confs of existing sessions ## What changes were proposed in this pull request? This fixes the python SparkSession builder to allow setting confs correctly. This was a leftover TODO from https://github.com/apache/spark/pull/13200. ## How was this patch tested? Python doc tests. cc andrewor14 Author: Eric Liang Closes #13289 from ericl/spark-15520. (cherry picked from commit 8239fdcb9b54ab6d13c31ad9916b8334dd1462c2) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c75ec5ea Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c75ec5ea Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c75ec5ea Branch: refs/heads/branch-2.0 Commit: c75ec5eaa6c95b3647c80b6595902d16ab3165fa Parents: 27f26a39 Author: Eric Liang Authored: Wed May 25 10:49:11 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:49:19 2016 -0700 -- python/pyspark/sql/session.py | 35 --- 1 file changed, 24 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c75ec5ea/python/pyspark/sql/session.py -- diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 2419475..52e7f3d 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -138,24 +138,37 @@ class SparkSession(object): """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a new one based on the options set in this builder. -This method first checks whether there is a valid thread-local SparkSession, -and if yes, return that one. It then checks whether there is a valid global -default SparkSession, and if yes, return that one. If no valid global default -SparkSession exists, the method creates a new SparkSession and assigns the -newly created SparkSession as the global default. +This method first checks whether there is a valid global default SparkSession, and if +yes, return that one. If no valid global default SparkSession exists, the method +creates a new SparkSession and assigns the newly created SparkSession as the global +default. + +>>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() +>>> s1.conf.get("k1") == "v1" +True In case an existing SparkSession is returned, the config options specified in this builder will be applied to the existing SparkSession. + +>>> s2 = SparkSession.builder.config("k2", "v2").getOrCreate() +>>> s1.conf.get("k1") == s2.conf.get("k1") +True +>>> s1.conf.get("k2") == s2.conf.get("k2") +True """ with self._lock: -from pyspark.conf import SparkConf from pyspark.context import SparkContext -from pyspark.sql.context import SQLContext -sparkConf = SparkConf() +from pyspark.conf import SparkConf +session = SparkSession._instantiatedContext +if session is None: +sparkConf = SparkConf() +for key, value in self._options.items(): +sparkConf.set(key, value) +sc = SparkContext.getOrCreate(sparkConf) +session = SparkSession(sc) for key, value in self._options.items(): -sparkConf.set(key, value) -sparkContext = SparkContext.getOrCreate(sparkConf) -return SQLContext.getOrCreate(sparkContext).sparkSession +session.conf.set(key, value) +return session builder = Builder() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15520][SQL] SparkSession builder in python should also allow overriding confs of existing sessions
Repository: spark Updated Branches: refs/heads/master 01e7b9c85 -> 8239fdcb9 [SPARK-15520][SQL] SparkSession builder in python should also allow overriding confs of existing sessions ## What changes were proposed in this pull request? This fixes the python SparkSession builder to allow setting confs correctly. This was a leftover TODO from https://github.com/apache/spark/pull/13200. ## How was this patch tested? Python doc tests. cc andrewor14 Author: Eric Liang Closes #13289 from ericl/spark-15520. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8239fdcb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8239fdcb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8239fdcb Branch: refs/heads/master Commit: 8239fdcb9b54ab6d13c31ad9916b8334dd1462c2 Parents: 01e7b9c Author: Eric Liang Authored: Wed May 25 10:49:11 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:49:11 2016 -0700 -- python/pyspark/sql/session.py | 35 --- 1 file changed, 24 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8239fdcb/python/pyspark/sql/session.py -- diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 2419475..52e7f3d 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -138,24 +138,37 @@ class SparkSession(object): """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a new one based on the options set in this builder. -This method first checks whether there is a valid thread-local SparkSession, -and if yes, return that one. It then checks whether there is a valid global -default SparkSession, and if yes, return that one. If no valid global default -SparkSession exists, the method creates a new SparkSession and assigns the -newly created SparkSession as the global default. +This method first checks whether there is a valid global default SparkSession, and if +yes, return that one. If no valid global default SparkSession exists, the method +creates a new SparkSession and assigns the newly created SparkSession as the global +default. + +>>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() +>>> s1.conf.get("k1") == "v1" +True In case an existing SparkSession is returned, the config options specified in this builder will be applied to the existing SparkSession. + +>>> s2 = SparkSession.builder.config("k2", "v2").getOrCreate() +>>> s1.conf.get("k1") == s2.conf.get("k1") +True +>>> s1.conf.get("k2") == s2.conf.get("k2") +True """ with self._lock: -from pyspark.conf import SparkConf from pyspark.context import SparkContext -from pyspark.sql.context import SQLContext -sparkConf = SparkConf() +from pyspark.conf import SparkConf +session = SparkSession._instantiatedContext +if session is None: +sparkConf = SparkConf() +for key, value in self._options.items(): +sparkConf.set(key, value) +sc = SparkContext.getOrCreate(sparkConf) +session = SparkSession(sc) for key, value in self._options.items(): -sparkConf.set(key, value) -sparkContext = SparkContext.getOrCreate(sparkConf) -return SQLContext.getOrCreate(sparkContext).sparkSession +session.conf.set(key, value) +return session builder = Builder() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR][CORE] Fix a HadoopRDD log message and remove unused imports in rdd files.
Repository: spark Updated Branches: refs/heads/master 8239fdcb9 -> d6d3e5071 [MINOR][CORE] Fix a HadoopRDD log message and remove unused imports in rdd files. ## What changes were proposed in this pull request? This PR fixes the following typos in log message and comments of `HadoopRDD.scala`. Also, this removes unused imports. ```scala - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + ... - // since its not removed yet + // since it's not removed yet ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13294 from dongjoon-hyun/minor_rdd_fix_log_message. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d6d3e507 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d6d3e507 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d6d3e507 Branch: refs/heads/master Commit: d6d3e50719b01005aa0e77349fc9a6ff88fecce3 Parents: 8239fdc Author: Dongjoon Hyun Authored: Wed May 25 10:51:33 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:51:33 2016 -0700 -- core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala| 2 +- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala| 5 ++--- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 - core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala | 1 - 5 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d6d3e507/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index be0cb17..41832e8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.hadoop.conf.{ Configurable, Configuration } +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.JobContextImpl http://git-wip-us.apache.org/repos/asf/spark/blob/d6d3e507/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index b22134a..515fd6f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -43,7 +43,6 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.Logging import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} @@ -70,7 +69,7 @@ private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: Inp val envVars: Map[String, String] = if (inputSplit.value.isInstanceOf[FileSplit]) { val is: FileSplit = inputSplit.value.asInstanceOf[FileSplit] // map_input_file is deprecated in favor of mapreduce_map_input_file but set both - // since its not removed yet + // since it's not removed yet Map("map_input_file" -> is.getPath().toString(), "mapreduce_map_input_file" -> is.getPath().toString()) } else { @@ -335,7 +334,7 @@ class HadoopRDD[K, V]( override def persist(storageLevel: StorageLevel): this.type = { if (storageLevel.deserialized) { - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + " Use a map transformation to make copies of the records.") } http://git-wip-us.apache.org/repos/asf/spark/blob/d6d3e507/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index ad7c221..189dc7b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala
spark git commit: [MINOR][CORE] Fix a HadoopRDD log message and remove unused imports in rdd files.
Repository: spark Updated Branches: refs/heads/branch-2.0 c75ec5eaa -> 4009ddafd [MINOR][CORE] Fix a HadoopRDD log message and remove unused imports in rdd files. ## What changes were proposed in this pull request? This PR fixes the following typos in log message and comments of `HadoopRDD.scala`. Also, this removes unused imports. ```scala - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + ... - // since its not removed yet + // since it's not removed yet ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13294 from dongjoon-hyun/minor_rdd_fix_log_message. (cherry picked from commit d6d3e50719b01005aa0e77349fc9a6ff88fecce3) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4009ddaf Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4009ddaf Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4009ddaf Branch: refs/heads/branch-2.0 Commit: 4009ddafd810f91f699e52d7822c8c959fe7761e Parents: c75ec5e Author: Dongjoon Hyun Authored: Wed May 25 10:51:33 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:51:41 2016 -0700 -- core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala| 2 +- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala| 5 ++--- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 - core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala | 1 - 5 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4009ddaf/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index be0cb17..41832e8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.hadoop.conf.{ Configurable, Configuration } +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.JobContextImpl http://git-wip-us.apache.org/repos/asf/spark/blob/4009ddaf/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index b22134a..515fd6f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -43,7 +43,6 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.Logging import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} @@ -70,7 +69,7 @@ private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: Inp val envVars: Map[String, String] = if (inputSplit.value.isInstanceOf[FileSplit]) { val is: FileSplit = inputSplit.value.asInstanceOf[FileSplit] // map_input_file is deprecated in favor of mapreduce_map_input_file but set both - // since its not removed yet + // since it's not removed yet Map("map_input_file" -> is.getPath().toString(), "mapreduce_map_input_file" -> is.getPath().toString()) } else { @@ -335,7 +334,7 @@ class HadoopRDD[K, V]( override def persist(storageLevel: StorageLevel): this.type = { if (storageLevel.deserialized) { - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + " Use a map transformation to make copies of the records.") } http://git-wip-us.apache.org/repos/asf/spark/blob/4009ddaf/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index ad7c221..18
spark git commit: [MINOR][MLLIB][STREAMING][SQL] Fix typos
Repository: spark Updated Branches: refs/heads/branch-2.0 4009ddafd -> 6fc367e50 [MINOR][MLLIB][STREAMING][SQL] Fix typos fixed typos for source code for components [mllib] [streaming] and [SQL] None and obvious. Author: lfzCarlosC Closes #13298 from lfzCarlosC/master. (cherry picked from commit 02c8072eea72425e89256347e1f373a3e76e6eba) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6fc367e5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6fc367e5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6fc367e5 Branch: refs/heads/branch-2.0 Commit: 6fc367e50c2de9f68237763919eae12241a2179f Parents: 4009dda Author: lfzCarlosC Authored: Wed May 25 10:53:53 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:54:25 2016 -0700 -- mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala| 2 +- .../org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala | 2 +- .../spark/sql/catalyst/plans/logical/basicLogicalOperators.scala | 2 +- .../org/apache/spark/sql/execution/vectorized/ColumnVector.java | 2 +- .../spark/sql/execution/streaming/state/StateStoreSuite.scala| 2 +- sql/hive-thriftserver/if/TCLIService.thrift | 4 ++-- .../java/org/apache/hive/service/ServiceStateChangeListener.java | 2 +- .../java/org/apache/hive/service/cli/operation/SQLOperation.java | 2 +- .../apache/hive/service/cli/session/HiveSessionHookContext.java | 2 +- .../main/scala/org/apache/spark/sql/hive/HiveSessionState.scala | 2 +- .../apache/spark/streaming/util/WriteAheadLogRecordHandle.java | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6fc367e5/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index 9457c6e..bb4b37e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -204,7 +204,7 @@ private object IDFModel { * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector * * @param idf an IDF vector - * @param v a term frequence vector + * @param v a term frequency vector * @return a TF-IDF vector */ def transform(idf: Vector, v: Vector): Vector = { http://git-wip-us.apache.org/repos/asf/spark/blob/6fc367e5/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala index 9748fbf..c3de5d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala @@ -45,7 +45,7 @@ import org.apache.spark.rdd.RDD * many elements are in each partition. Once these three values have been returned for every * partition, we can collect and operate locally. Locally, we can now adjust each distance by the * appropriate constant (the cumulative sum of number of elements in the prior partitions divided by - * thedata set size). Finally, we take the maximum absolute value, and this is the statistic. + * the data set size). Finally, we take the maximum absolute value, and this is the statistic. */ private[stat] object KolmogorovSmirnovTest extends Logging { http://git-wip-us.apache.org/repos/asf/spark/blob/6fc367e5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 0a9250b..8b7e21b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -591,7 +591,7 @@ case class Expand( } // This operator can reuse attributes (for example making them null when doing a roll up) so - // the contraints of the child may no longer be valid. + // the constraints of the child may no longer be valid. override protected def validConstraints: Set[Expression] = Set.empty[Expression] } http://git-wip-us.apache.org/repos/asf/spark/blob
spark git commit: [MINOR][MLLIB][STREAMING][SQL] Fix typos
Repository: spark Updated Branches: refs/heads/master d6d3e5071 -> 02c8072ee [MINOR][MLLIB][STREAMING][SQL] Fix typos fixed typos for source code for components [mllib] [streaming] and [SQL] None and obvious. Author: lfzCarlosC Closes #13298 from lfzCarlosC/master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/02c8072e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/02c8072e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/02c8072e Branch: refs/heads/master Commit: 02c8072eea72425e89256347e1f373a3e76e6eba Parents: d6d3e50 Author: lfzCarlosC Authored: Wed May 25 10:53:53 2016 -0700 Committer: Andrew Or Committed: Wed May 25 10:53:57 2016 -0700 -- mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala| 2 +- .../org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala | 2 +- .../spark/sql/catalyst/plans/logical/basicLogicalOperators.scala | 2 +- .../org/apache/spark/sql/execution/vectorized/ColumnVector.java | 2 +- .../spark/sql/execution/streaming/state/StateStoreSuite.scala| 2 +- sql/hive-thriftserver/if/TCLIService.thrift | 4 ++-- .../java/org/apache/hive/service/ServiceStateChangeListener.java | 2 +- .../java/org/apache/hive/service/cli/operation/SQLOperation.java | 2 +- .../apache/hive/service/cli/session/HiveSessionHookContext.java | 2 +- .../main/scala/org/apache/spark/sql/hive/HiveSessionState.scala | 2 +- .../apache/spark/streaming/util/WriteAheadLogRecordHandle.java | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/02c8072e/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index 9457c6e..bb4b37e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -204,7 +204,7 @@ private object IDFModel { * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector * * @param idf an IDF vector - * @param v a term frequence vector + * @param v a term frequency vector * @return a TF-IDF vector */ def transform(idf: Vector, v: Vector): Vector = { http://git-wip-us.apache.org/repos/asf/spark/blob/02c8072e/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala index 9748fbf..c3de5d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala @@ -45,7 +45,7 @@ import org.apache.spark.rdd.RDD * many elements are in each partition. Once these three values have been returned for every * partition, we can collect and operate locally. Locally, we can now adjust each distance by the * appropriate constant (the cumulative sum of number of elements in the prior partitions divided by - * thedata set size). Finally, we take the maximum absolute value, and this is the statistic. + * the data set size). Finally, we take the maximum absolute value, and this is the statistic. */ private[stat] object KolmogorovSmirnovTest extends Logging { http://git-wip-us.apache.org/repos/asf/spark/blob/02c8072e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 0a9250b..8b7e21b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -591,7 +591,7 @@ case class Expand( } // This operator can reuse attributes (for example making them null when doing a roll up) so - // the contraints of the child may no longer be valid. + // the constraints of the child may no longer be valid. override protected def validConstraints: Set[Expression] = Set.empty[Expression] } http://git-wip-us.apache.org/repos/asf/spark/blob/02c8072e/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java ---
spark git commit: [SPARK-15534][SPARK-15535][SQL] Truncate table fixes
Repository: spark Updated Branches: refs/heads/branch-2.0 0064a4dcb -> 347acc4ea [SPARK-15534][SPARK-15535][SQL] Truncate table fixes ## What changes were proposed in this pull request? Two changes: - When things fail, `TRUNCATE TABLE` just returns nothing. Instead, we should throw exceptions. - Remove `TRUNCATE TABLE ... COLUMN`, which was never supported by either Spark or Hive. ## How was this patch tested? Jenkins. Author: Andrew Or Closes #13302 from andrewor14/truncate-table. (cherry picked from commit ee682fe293b47988056b540ee46ca49861309982) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/347acc4e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/347acc4e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/347acc4e Branch: refs/heads/branch-2.0 Commit: 347acc4ea411cd430cfb1d8b8b596463f5b9dd3c Parents: 0064a4d Author: Andrew Or Authored: Wed May 25 15:08:39 2016 -0700 Committer: Andrew Or Committed: Wed May 25 15:08:59 2016 -0700 -- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 3 +-- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 7 +-- .../org/apache/spark/sql/execution/command/tables.scala | 7 --- .../spark/sql/hive/execution/HiveCommandSuite.scala | 12 4 files changed, 6 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/347acc4e/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 -- diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 403191a..b0e71c7 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -115,8 +115,7 @@ statement | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData -| TRUNCATE TABLE tableIdentifier partitionSpec? -(COLUMNS identifierList)? #truncateTable +| TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration http://git-wip-us.apache.org/repos/asf/spark/blob/347acc4e/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 57f534c..cfebfc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -368,17 +368,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * For example: * {{{ * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] - * [COLUMNS (col1, col2)] * }}} */ override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { -if (ctx.identifierList != null) { - throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) -} TruncateTableCommand( visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) -) + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/347acc4e/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 13e63a1..bef4c92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -278,7 +278,7 @@ case class LoadDataCommand( * * The syntax of this command is: * {{{ - * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} *
spark git commit: [SPARK-15534][SPARK-15535][SQL] Truncate table fixes
Repository: spark Updated Branches: refs/heads/master 589cce93c -> ee682fe29 [SPARK-15534][SPARK-15535][SQL] Truncate table fixes ## What changes were proposed in this pull request? Two changes: - When things fail, `TRUNCATE TABLE` just returns nothing. Instead, we should throw exceptions. - Remove `TRUNCATE TABLE ... COLUMN`, which was never supported by either Spark or Hive. ## How was this patch tested? Jenkins. Author: Andrew Or Closes #13302 from andrewor14/truncate-table. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ee682fe2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ee682fe2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ee682fe2 Branch: refs/heads/master Commit: ee682fe293b47988056b540ee46ca49861309982 Parents: 589cce9 Author: Andrew Or Authored: Wed May 25 15:08:39 2016 -0700 Committer: Andrew Or Committed: Wed May 25 15:08:39 2016 -0700 -- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 3 +-- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 7 +-- .../org/apache/spark/sql/execution/command/tables.scala | 7 --- .../spark/sql/hive/execution/HiveCommandSuite.scala | 12 4 files changed, 6 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ee682fe2/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 -- diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 403191a..b0e71c7 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -115,8 +115,7 @@ statement | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData -| TRUNCATE TABLE tableIdentifier partitionSpec? -(COLUMNS identifierList)? #truncateTable +| TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration http://git-wip-us.apache.org/repos/asf/spark/blob/ee682fe2/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 57f534c..cfebfc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -368,17 +368,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * For example: * {{{ * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] - * [COLUMNS (col1, col2)] * }}} */ override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { -if (ctx.identifierList != null) { - throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) -} TruncateTableCommand( visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) -) + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/ee682fe2/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 13e63a1..bef4c92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -278,7 +278,7 @@ case class LoadDataCommand( * * The syntax of this command is: * {{{ - * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ case class TruncateTableCommand( @@ -288,9 +288,10 @@ case class TruncateTableCommand( ove
spark git commit: [SPARK-15539][SQL] DROP TABLE throw exception if table doesn't exist
Repository: spark Updated Branches: refs/heads/master 01b350a4f -> 2b1ac6cea [SPARK-15539][SQL] DROP TABLE throw exception if table doesn't exist ## What changes were proposed in this pull request? Same as #13302, but for DROP TABLE. ## How was this patch tested? `DDLSuite` Author: Andrew Or Closes #13307 from andrewor14/drop-table. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2b1ac6ce Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2b1ac6ce Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2b1ac6ce Branch: refs/heads/master Commit: 2b1ac6cea882246ef0e655bb2c134ef1656a5068 Parents: 01b350a Author: Andrew Or Authored: Thu May 26 12:04:18 2016 -0700 Committer: Andrew Or Committed: Thu May 26 12:04:18 2016 -0700 -- .../spark/sql/execution/command/ddl.scala | 4 +- .../spark/sql/execution/command/DDLSuite.scala | 10 ++-- .../hive/execution/HiveCompatibilitySuite.scala | 52 ++-- .../HiveWindowFunctionQuerySuite.scala | 10 ++-- .../sql/hive/MetastoreDataSourcesSuite.scala| 2 +- .../spark/sql/hive/QueryPartitionSuite.scala| 4 +- 6 files changed, 42 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2b1ac6ce/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index dd3f17d..ffea628 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -185,7 +185,7 @@ case class DropTableCommand( if (!catalog.tableExists(tableName)) { if (!ifExists) { val objectName = if (isView) "View" else "Table" -logError(s"$objectName '${tableName.quotedString}' does not exist") +throw new AnalysisException(s"$objectName to drop '$tableName' does not exist") } } else { // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view @@ -202,7 +202,7 @@ case class DropTableCommand( try { sparkSession.cacheManager.tryUncacheQuery(sparkSession.table(tableName.quotedString)) } catch { -case NonFatal(e) => log.warn(s"${e.getMessage}", e) +case NonFatal(e) => log.warn(e.toString, e) } catalog.invalidateTable(tableName) catalog.dropTable(tableName, ifExists) http://git-wip-us.apache.org/repos/asf/spark/blob/2b1ac6ce/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 64f5a4a..bddd3f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -741,14 +741,12 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("DROP TABLE dbx.tab1") assert(catalog.listTables("dbx") == Nil) sql("DROP TABLE IF EXISTS dbx.tab1") -// no exception will be thrown -sql("DROP TABLE dbx.tab1") +intercept[AnalysisException] { + sql("DROP TABLE dbx.tab1") +} } - test("drop view in SQLContext") { -// SQLContext does not support create view. Log an error message, if tab1 does not exists -sql("DROP VIEW tab1") - + test("drop view") { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) createDatabase(catalog, "dbx") http://git-wip-us.apache.org/repos/asf/spark/blob/2b1ac6ce/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala -- diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index a8645f7..2d5a970 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -515,7 +515,33 @@ cla
spark git commit: [SPARK-15539][SQL] DROP TABLE throw exception if table doesn't exist
Repository: spark Updated Branches: refs/heads/branch-2.0 0cb69a918 -> 69b3e9cee [SPARK-15539][SQL] DROP TABLE throw exception if table doesn't exist ## What changes were proposed in this pull request? Same as #13302, but for DROP TABLE. ## How was this patch tested? `DDLSuite` Author: Andrew Or Closes #13307 from andrewor14/drop-table. (cherry picked from commit 2b1ac6cea882246ef0e655bb2c134ef1656a5068) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/69b3e9ce Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/69b3e9ce Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/69b3e9ce Branch: refs/heads/branch-2.0 Commit: 69b3e9ceeaba8b1b836e93c0164566ed2e25a84e Parents: 0cb69a9 Author: Andrew Or Authored: Thu May 26 12:04:18 2016 -0700 Committer: Andrew Or Committed: Thu May 26 12:04:26 2016 -0700 -- .../spark/sql/execution/command/ddl.scala | 4 +- .../spark/sql/execution/command/DDLSuite.scala | 10 ++-- .../hive/execution/HiveCompatibilitySuite.scala | 52 ++-- .../HiveWindowFunctionQuerySuite.scala | 10 ++-- .../sql/hive/MetastoreDataSourcesSuite.scala| 2 +- .../spark/sql/hive/QueryPartitionSuite.scala| 4 +- 6 files changed, 42 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/69b3e9ce/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index dd3f17d..ffea628 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -185,7 +185,7 @@ case class DropTableCommand( if (!catalog.tableExists(tableName)) { if (!ifExists) { val objectName = if (isView) "View" else "Table" -logError(s"$objectName '${tableName.quotedString}' does not exist") +throw new AnalysisException(s"$objectName to drop '$tableName' does not exist") } } else { // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view @@ -202,7 +202,7 @@ case class DropTableCommand( try { sparkSession.cacheManager.tryUncacheQuery(sparkSession.table(tableName.quotedString)) } catch { -case NonFatal(e) => log.warn(s"${e.getMessage}", e) +case NonFatal(e) => log.warn(e.toString, e) } catalog.invalidateTable(tableName) catalog.dropTable(tableName, ifExists) http://git-wip-us.apache.org/repos/asf/spark/blob/69b3e9ce/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 64f5a4a..bddd3f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -741,14 +741,12 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("DROP TABLE dbx.tab1") assert(catalog.listTables("dbx") == Nil) sql("DROP TABLE IF EXISTS dbx.tab1") -// no exception will be thrown -sql("DROP TABLE dbx.tab1") +intercept[AnalysisException] { + sql("DROP TABLE dbx.tab1") +} } - test("drop view in SQLContext") { -// SQLContext does not support create view. Log an error message, if tab1 does not exists -sql("DROP VIEW tab1") - + test("drop view") { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) createDatabase(catalog, "dbx") http://git-wip-us.apache.org/repos/asf/spark/blob/69b3e9ce/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala -- diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index a8645f7..2d5a970 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/
spark git commit: [SPARK-15520][SQL] Also set sparkContext confs when using SparkSession builder in pyspark
Repository: spark Updated Branches: refs/heads/master 2b1ac6cea -> 594a1bf20 [SPARK-15520][SQL] Also set sparkContext confs when using SparkSession builder in pyspark ## What changes were proposed in this pull request? Also sets confs in the underlying sc when using SparkSession.builder.getOrCreate(). This is a bug-fix from a post-merge comment in https://github.com/apache/spark/pull/13289 ## How was this patch tested? Python doc-tests. Author: Eric Liang Closes #13309 from ericl/spark-15520-1. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/594a1bf2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/594a1bf2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/594a1bf2 Branch: refs/heads/master Commit: 594a1bf200fea8d6bcf25839a49186f66f922bc8 Parents: 2b1ac6c Author: Eric Liang Authored: Thu May 26 12:05:47 2016 -0700 Committer: Andrew Or Committed: Thu May 26 12:05:47 2016 -0700 -- python/pyspark/sql/session.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/594a1bf2/python/pyspark/sql/session.py -- diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 52e7f3d..8f7dcb5 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -144,7 +144,7 @@ class SparkSession(object): default. >>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() ->>> s1.conf.get("k1") == "v1" +>>> s1.conf.get("k1") == s1.sparkContext.getConf().get("k1") == "v1" True In case an existing SparkSession is returned, the config options specified @@ -168,6 +168,8 @@ class SparkSession(object): session = SparkSession(sc) for key, value in self._options.items(): session.conf.set(key, value) +for key, value in self._options.items(): +session.sparkContext._conf.set(key, value) return session builder = Builder() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15520][SQL] Also set sparkContext confs when using SparkSession builder in pyspark
Repository: spark Updated Branches: refs/heads/branch-2.0 69b3e9cee -> eb0c49799 [SPARK-15520][SQL] Also set sparkContext confs when using SparkSession builder in pyspark ## What changes were proposed in this pull request? Also sets confs in the underlying sc when using SparkSession.builder.getOrCreate(). This is a bug-fix from a post-merge comment in https://github.com/apache/spark/pull/13289 ## How was this patch tested? Python doc-tests. Author: Eric Liang Closes #13309 from ericl/spark-15520-1. (cherry picked from commit 594a1bf200fea8d6bcf25839a49186f66f922bc8) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/eb0c4979 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/eb0c4979 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/eb0c4979 Branch: refs/heads/branch-2.0 Commit: eb0c49799880acf2c35b95984b17fcb0ad7b5eca Parents: 69b3e9c Author: Eric Liang Authored: Thu May 26 12:05:47 2016 -0700 Committer: Andrew Or Committed: Thu May 26 12:05:55 2016 -0700 -- python/pyspark/sql/session.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/eb0c4979/python/pyspark/sql/session.py -- diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 52e7f3d..8f7dcb5 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -144,7 +144,7 @@ class SparkSession(object): default. >>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() ->>> s1.conf.get("k1") == "v1" +>>> s1.conf.get("k1") == s1.sparkContext.getConf().get("k1") == "v1" True In case an existing SparkSession is returned, the config options specified @@ -168,6 +168,8 @@ class SparkSession(object): session = SparkSession(sc) for key, value in self._options.items(): session.conf.set(key, value) +for key, value in self._options.items(): +session.sparkContext._conf.set(key, value) return session builder = Builder() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15552][SQL] Remove unnecessary private[sql] methods in SparkSession
Repository: spark Updated Branches: refs/heads/branch-2.0 eb0c49799 -> 6eea33ec3 [SPARK-15552][SQL] Remove unnecessary private[sql] methods in SparkSession ## What changes were proposed in this pull request? SparkSession has a list of unnecessary private[sql] methods. These methods cause some trouble because private[sql] doesn't apply in Java. In the cases that they are easy to remove, we can simply remove them. This patch does that. As part of this pull request, I also replaced a bunch of protected[sql] with private[sql], to tighten up visibility. ## How was this patch tested? Updated test cases to reflect the changes. Author: Reynold Xin Closes #13319 from rxin/SPARK-15552. (cherry picked from commit 0f61d6efb45b9ee94fa663f67c4489fbdae2eded) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6eea33ec Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6eea33ec Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6eea33ec Branch: refs/heads/branch-2.0 Commit: 6eea33ec3ea805a16b812287141d22ce1aa659f9 Parents: eb0c497 Author: Reynold Xin Authored: Thu May 26 13:03:07 2016 -0700 Committer: Andrew Or Committed: Thu May 26 13:03:20 2016 -0700 -- .../org/apache/spark/sql/DataFrameWriter.scala | 4 +- .../scala/org/apache/spark/sql/Dataset.scala| 32 ++-- .../scala/org/apache/spark/sql/SQLContext.scala | 49 +++--- .../org/apache/spark/sql/SparkSession.scala | 54 .../spark/sql/execution/CacheManager.scala | 2 +- .../spark/sql/execution/QueryExecution.scala| 2 +- .../spark/sql/execution/command/commands.scala | 2 +- .../spark/sql/execution/command/ddl.scala | 3 +- .../spark/sql/execution/command/views.scala | 6 +-- .../sql/execution/datasources/DataSource.scala | 2 +- .../InsertIntoDataSourceCommand.scala | 2 +- .../apache/spark/sql/internal/CatalogImpl.scala | 24 - .../spark/sql/internal/SessionState.scala | 4 +- .../org/apache/spark/sql/CachedTableSuite.scala | 16 +++--- .../apache/spark/sql/DataFramePivotSuite.scala | 2 +- .../apache/spark/sql/DatasetCacheSuite.scala| 12 +++-- .../scala/org/apache/spark/sql/JoinSuite.scala | 8 +-- .../columnar/InMemoryColumnarQuerySuite.scala | 8 +-- .../sql/execution/metric/SQLMetricsSuite.scala | 18 --- .../sql/execution/ui/SQLListenerSuite.scala | 16 +++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../sql/hive/thriftserver/SparkSQLDriver.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 ++- .../execution/CreateTableAsSelectCommand.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala| 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/ShowCreateTableSuite.scala | 4 +- .../apache/spark/sql/hive/StatisticsSuite.scala | 2 +- .../hive/execution/ConcurrentHiveSuite.scala| 6 +-- 29 files changed, 129 insertions(+), 168 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6eea33ec/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 3aacce7..2e85e36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -402,7 +402,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { Project(inputDataCols ++ inputPartCols, df.logicalPlan) }.getOrElse(df.logicalPlan) -df.sparkSession.executePlan( +df.sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdent), partitions.getOrElse(Map.empty[String, Option[String]]), @@ -524,7 +524,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { mode, extraOptions.toMap, df.logicalPlan) -df.sparkSession.executePlan(cmd).toRdd +df.sparkSession.sessionState.executePlan(cmd).toRdd } } http://git-wip-us.apache.org/repos/asf/spark/blob/6eea33ec/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index e5140fc..961ae32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -58,7 +58,7 @@ private[sql] object Dataset { } def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPla
spark git commit: [SPARK-15552][SQL] Remove unnecessary private[sql] methods in SparkSession
Repository: spark Updated Branches: refs/heads/master 594a1bf20 -> 0f61d6efb [SPARK-15552][SQL] Remove unnecessary private[sql] methods in SparkSession ## What changes were proposed in this pull request? SparkSession has a list of unnecessary private[sql] methods. These methods cause some trouble because private[sql] doesn't apply in Java. In the cases that they are easy to remove, we can simply remove them. This patch does that. As part of this pull request, I also replaced a bunch of protected[sql] with private[sql], to tighten up visibility. ## How was this patch tested? Updated test cases to reflect the changes. Author: Reynold Xin Closes #13319 from rxin/SPARK-15552. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0f61d6ef Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0f61d6ef Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0f61d6ef Branch: refs/heads/master Commit: 0f61d6efb45b9ee94fa663f67c4489fbdae2eded Parents: 594a1bf Author: Reynold Xin Authored: Thu May 26 13:03:07 2016 -0700 Committer: Andrew Or Committed: Thu May 26 13:03:07 2016 -0700 -- .../org/apache/spark/sql/DataFrameWriter.scala | 4 +- .../scala/org/apache/spark/sql/Dataset.scala| 32 ++-- .../scala/org/apache/spark/sql/SQLContext.scala | 49 +++--- .../org/apache/spark/sql/SparkSession.scala | 54 .../spark/sql/execution/CacheManager.scala | 2 +- .../spark/sql/execution/QueryExecution.scala| 2 +- .../spark/sql/execution/command/commands.scala | 2 +- .../spark/sql/execution/command/ddl.scala | 3 +- .../spark/sql/execution/command/views.scala | 6 +-- .../sql/execution/datasources/DataSource.scala | 2 +- .../InsertIntoDataSourceCommand.scala | 2 +- .../apache/spark/sql/internal/CatalogImpl.scala | 24 - .../spark/sql/internal/SessionState.scala | 4 +- .../org/apache/spark/sql/CachedTableSuite.scala | 16 +++--- .../apache/spark/sql/DataFramePivotSuite.scala | 2 +- .../apache/spark/sql/DatasetCacheSuite.scala| 12 +++-- .../scala/org/apache/spark/sql/JoinSuite.scala | 8 +-- .../columnar/InMemoryColumnarQuerySuite.scala | 8 +-- .../sql/execution/metric/SQLMetricsSuite.scala | 18 --- .../sql/execution/ui/SQLListenerSuite.scala | 16 +++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../sql/hive/thriftserver/SparkSQLDriver.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 ++- .../execution/CreateTableAsSelectCommand.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala| 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/ShowCreateTableSuite.scala | 4 +- .../apache/spark/sql/hive/StatisticsSuite.scala | 2 +- .../hive/execution/ConcurrentHiveSuite.scala| 6 +-- 29 files changed, 129 insertions(+), 168 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0f61d6ef/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 3aacce7..2e85e36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -402,7 +402,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { Project(inputDataCols ++ inputPartCols, df.logicalPlan) }.getOrElse(df.logicalPlan) -df.sparkSession.executePlan( +df.sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdent), partitions.getOrElse(Map.empty[String, Option[String]]), @@ -524,7 +524,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { mode, extraOptions.toMap, df.logicalPlan) -df.sparkSession.executePlan(cmd).toRdd +df.sparkSession.sessionState.executePlan(cmd).toRdd } } http://git-wip-us.apache.org/repos/asf/spark/blob/0f61d6ef/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index e5140fc..961ae32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -58,7 +58,7 @@ private[sql] object Dataset { } def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = { -val qe = sparkSession.executePlan(logicalPlan) +val qe = sparkSession.sessio
spark git commit: [SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2
Repository: spark Updated Branches: refs/heads/master 3ac2363d7 -> 008a5377d [SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2 ## What changes were proposed in this pull request? Two more changes: (1) Fix truncate table for data source tables (only for cases without `PARTITION`) (2) Disallow truncating external tables or views ## How was this patch tested? `DDLSuite` Author: Andrew Or Closes #13315 from andrewor14/truncate-table. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/008a5377 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/008a5377 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/008a5377 Branch: refs/heads/master Commit: 008a5377d57ce6692eca4a41539fb27978b58e01 Parents: 3ac2363 Author: Andrew Or Authored: Thu May 26 19:01:41 2016 -0700 Committer: Andrew Or Committed: Thu May 26 19:01:41 2016 -0700 -- .../spark/sql/execution/command/tables.scala| 78 +--- .../spark/sql/execution/command/DDLSuite.scala | 34 + 2 files changed, 86 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/008a5377/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index bef4c92..e34beec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -285,41 +285,67 @@ case class TruncateTableCommand( tableName: TableIdentifier, partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand { - override def run(sparkSession: SparkSession): Seq[Row] = { -val catalog = sparkSession.sessionState.catalog + override def run(spark: SparkSession): Seq[Row] = { +val catalog = spark.sessionState.catalog if (!catalog.tableExists(tableName)) { throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does not exist.") -} else if (catalog.isTemporaryTable(tableName)) { +} +if (catalog.isTemporaryTable(tableName)) { throw new AnalysisException( s"Operation not allowed: TRUNCATE TABLE on temporary tables: '$tableName'") -} else { - val locations = if (partitionSpec.isDefined) { -catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) +} +val table = catalog.getTableMetadata(tableName) +if (table.tableType == CatalogTableType.EXTERNAL) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE on external tables: '$tableName'") +} +if (table.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE on views: '$tableName'") +} +val isDatasourceTable = DDLUtils.isDatasourceTable(table) +if (isDatasourceTable && partitionSpec.isDefined) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + +s"for tables created using the data sources API: '$tableName'") +} +if (table.partitionColumnNames.isEmpty && partitionSpec.isDefined) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + +s"for tables that are not partitioned: '$tableName'") +} +val locations = + if (isDatasourceTable || table.partitionColumnNames.isEmpty) { +Seq(table.storage.locationUri) } else { -val table = catalog.getTableMetadata(tableName) -if (table.partitionColumnNames.nonEmpty) { - catalog.listPartitions(tableName).map(_.storage.locationUri) -} else { - Seq(table.storage.locationUri) -} +catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) } - val hadoopConf = sparkSession.sessionState.newHadoopConf() - locations.foreach { location => -if (location.isDefined) { - val path = new Path(location.get) - try { -val fs = path.getFileSystem(hadoopConf) -fs.delete(path, true) -fs.mkdirs(path) - } catch { -case NonFatal(e) => - throw new AnalysisException( -s"Failed to truncate table '$tableName' when removing data of the path: $path " + - s"because of ${e.toS
spark git commit: [SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2
Repository: spark Updated Branches: refs/heads/branch-2.0 d3cd579d7 -> c99ad9215 [SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2 ## What changes were proposed in this pull request? Two more changes: (1) Fix truncate table for data source tables (only for cases without `PARTITION`) (2) Disallow truncating external tables or views ## How was this patch tested? `DDLSuite` Author: Andrew Or Closes #13315 from andrewor14/truncate-table. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c99ad921 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c99ad921 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c99ad921 Branch: refs/heads/branch-2.0 Commit: c99ad9215cea7f0f983ce06e9cbdbcfa7bd092bf Parents: d3cd579 Author: Andrew Or Authored: Thu May 26 19:01:41 2016 -0700 Committer: Andrew Or Committed: Thu May 26 19:02:28 2016 -0700 -- .../spark/sql/execution/command/tables.scala| 78 +--- .../spark/sql/execution/command/DDLSuite.scala | 34 + 2 files changed, 86 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c99ad921/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index bef4c92..e34beec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -285,41 +285,67 @@ case class TruncateTableCommand( tableName: TableIdentifier, partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand { - override def run(sparkSession: SparkSession): Seq[Row] = { -val catalog = sparkSession.sessionState.catalog + override def run(spark: SparkSession): Seq[Row] = { +val catalog = spark.sessionState.catalog if (!catalog.tableExists(tableName)) { throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does not exist.") -} else if (catalog.isTemporaryTable(tableName)) { +} +if (catalog.isTemporaryTable(tableName)) { throw new AnalysisException( s"Operation not allowed: TRUNCATE TABLE on temporary tables: '$tableName'") -} else { - val locations = if (partitionSpec.isDefined) { -catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) +} +val table = catalog.getTableMetadata(tableName) +if (table.tableType == CatalogTableType.EXTERNAL) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE on external tables: '$tableName'") +} +if (table.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE on views: '$tableName'") +} +val isDatasourceTable = DDLUtils.isDatasourceTable(table) +if (isDatasourceTable && partitionSpec.isDefined) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + +s"for tables created using the data sources API: '$tableName'") +} +if (table.partitionColumnNames.isEmpty && partitionSpec.isDefined) { + throw new AnalysisException( +s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + +s"for tables that are not partitioned: '$tableName'") +} +val locations = + if (isDatasourceTable || table.partitionColumnNames.isEmpty) { +Seq(table.storage.locationUri) } else { -val table = catalog.getTableMetadata(tableName) -if (table.partitionColumnNames.nonEmpty) { - catalog.listPartitions(tableName).map(_.storage.locationUri) -} else { - Seq(table.storage.locationUri) -} +catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) } - val hadoopConf = sparkSession.sessionState.newHadoopConf() - locations.foreach { location => -if (location.isDefined) { - val path = new Path(location.get) - try { -val fs = path.getFileSystem(hadoopConf) -fs.delete(path, true) -fs.mkdirs(path) - } catch { -case NonFatal(e) => - throw new AnalysisException( -s"Failed to truncate table '$tableName' when removing data of the path: $path " + - s"because of ${e.toS
spark git commit: [SPARK-15603][MLLIB] Replace SQLContext with SparkSession in ML/MLLib
Repository: spark Updated Branches: refs/heads/master c17272902 -> d24e25157 [SPARK-15603][MLLIB] Replace SQLContext with SparkSession in ML/MLLib ## What changes were proposed in this pull request? This PR replaces all deprecated `SQLContext` occurrences with `SparkSession` in `ML/MLLib` module except the following two classes. These two classes use `SQLContext` in their function signatures. - ReadWrite.scala - TreeModels.scala ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13352 from dongjoon-hyun/SPARK-15603. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d24e2515 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d24e2515 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d24e2515 Branch: refs/heads/master Commit: d24e251572d39a453293cabfe14e4aed25a55208 Parents: c172729 Author: Dongjoon Hyun Authored: Fri May 27 11:09:15 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:09:15 2016 -0700 -- .../spark/ml/clustering/GaussianMixture.scala | 7 ++-- .../spark/ml/feature/SQLTransformer.scala | 14 .../org/apache/spark/ml/feature/Word2Vec.scala | 16 - .../spark/mllib/api/python/PythonMLLibAPI.scala | 23 +++-- .../classification/LogisticRegression.scala | 19 +-- .../spark/mllib/classification/NaiveBayes.scala | 24 ++--- .../impl/GLMClassificationModel.scala | 18 +- .../mllib/clustering/BisectingKMeansModel.scala | 12 +++ .../mllib/clustering/GaussianMixtureModel.scala | 16 - .../spark/mllib/clustering/KMeansModel.scala| 13 --- .../spark/mllib/clustering/LDAModel.scala | 36 +--- .../clustering/PowerIterationClustering.scala | 12 +++ .../spark/mllib/feature/ChiSqSelector.scala | 13 --- .../apache/spark/mllib/feature/Word2Vec.scala | 13 +++ .../org/apache/spark/mllib/fpm/FPGrowth.scala | 10 +++--- .../org/apache/spark/mllib/fpm/PrefixSpan.scala | 10 +++--- .../MatrixFactorizationModel.scala | 12 +++ .../mllib/regression/IsotonicRegression.scala | 12 +++ .../regression/impl/GLMRegressionModel.scala| 18 +- .../mllib/tree/model/DecisionTreeModel.scala| 20 +-- .../mllib/tree/model/treeEnsembleModels.scala | 17 + .../ml/feature/QuantileDiscretizerSuite.scala | 14 .../mllib/util/MLlibTestSparkContext.scala | 6 ++-- 23 files changed, 160 insertions(+), 195 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d24e2515/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 88b6b27..773e50e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -20,7 +20,6 @@ package org.apache.spark.ml.clustering import breeze.linalg.{DenseVector => BDV} import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.impl.Utils.EPSILON @@ -33,7 +32,7 @@ import org.apache.spark.mllib.clustering.{GaussianMixture => MLlibGM} import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors, VectorUDT => OldVectorUDT} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -134,9 +133,7 @@ class GaussianMixtureModel private[ml] ( val modelGaussians = gaussians.map { gaussian => (OldVectors.fromML(gaussian.mean), OldMatrices.fromML(gaussian.cov)) } -val sc = SparkContext.getOrCreate() -val sqlContext = SQLContext.getOrCreate(sc) -sqlContext.createDataFrame(modelGaussians).toDF("mean", "cov") + SparkSession.builder().getOrCreate().createDataFrame(modelGaussians).toDF("mean", "cov") } /** http://git-wip-us.apache.org/repos/asf/spark/blob/d24e2515/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 2d4cac6..bd8f949 100644 --- a/
spark git commit: [SPARK-15603][MLLIB] Replace SQLContext with SparkSession in ML/MLLib
Repository: spark Updated Branches: refs/heads/branch-2.0 2cb84dd23 -> a355edeef [SPARK-15603][MLLIB] Replace SQLContext with SparkSession in ML/MLLib ## What changes were proposed in this pull request? This PR replaces all deprecated `SQLContext` occurrences with `SparkSession` in `ML/MLLib` module except the following two classes. These two classes use `SQLContext` in their function signatures. - ReadWrite.scala - TreeModels.scala ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13352 from dongjoon-hyun/SPARK-15603. (cherry picked from commit d24e251572d39a453293cabfe14e4aed25a55208) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a355edee Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a355edee Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a355edee Branch: refs/heads/branch-2.0 Commit: a355edeefa16988da8b05d2539a91277e75e823c Parents: 2cb84dd Author: Dongjoon Hyun Authored: Fri May 27 11:09:15 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:09:33 2016 -0700 -- .../spark/ml/clustering/GaussianMixture.scala | 7 ++-- .../spark/ml/feature/SQLTransformer.scala | 14 .../org/apache/spark/ml/feature/Word2Vec.scala | 16 - .../spark/mllib/api/python/PythonMLLibAPI.scala | 23 +++-- .../classification/LogisticRegression.scala | 19 +-- .../spark/mllib/classification/NaiveBayes.scala | 24 ++--- .../impl/GLMClassificationModel.scala | 18 +- .../mllib/clustering/BisectingKMeansModel.scala | 12 +++ .../mllib/clustering/GaussianMixtureModel.scala | 16 - .../spark/mllib/clustering/KMeansModel.scala| 13 --- .../spark/mllib/clustering/LDAModel.scala | 36 +--- .../clustering/PowerIterationClustering.scala | 12 +++ .../spark/mllib/feature/ChiSqSelector.scala | 13 --- .../apache/spark/mllib/feature/Word2Vec.scala | 13 +++ .../org/apache/spark/mllib/fpm/FPGrowth.scala | 10 +++--- .../org/apache/spark/mllib/fpm/PrefixSpan.scala | 10 +++--- .../MatrixFactorizationModel.scala | 12 +++ .../mllib/regression/IsotonicRegression.scala | 12 +++ .../regression/impl/GLMRegressionModel.scala| 18 +- .../mllib/tree/model/DecisionTreeModel.scala| 20 +-- .../mllib/tree/model/treeEnsembleModels.scala | 17 + .../ml/feature/QuantileDiscretizerSuite.scala | 14 .../mllib/util/MLlibTestSparkContext.scala | 6 ++-- 23 files changed, 160 insertions(+), 195 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a355edee/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 88b6b27..773e50e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -20,7 +20,6 @@ package org.apache.spark.ml.clustering import breeze.linalg.{DenseVector => BDV} import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.impl.Utils.EPSILON @@ -33,7 +32,7 @@ import org.apache.spark.mllib.clustering.{GaussianMixture => MLlibGM} import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors, VectorUDT => OldVectorUDT} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -134,9 +133,7 @@ class GaussianMixtureModel private[ml] ( val modelGaussians = gaussians.map { gaussian => (OldVectors.fromML(gaussian.mean), OldMatrices.fromML(gaussian.cov)) } -val sc = SparkContext.getOrCreate() -val sqlContext = SQLContext.getOrCreate(sc) -sqlContext.createDataFrame(modelGaussians).toDF("mean", "cov") + SparkSession.builder().getOrCreate().createDataFrame(modelGaussians).toDF("mean", "cov") } /** http://git-wip-us.apache.org/repos/asf/spark/blob/a355edee/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/ml
spark git commit: [SPARK-15584][SQL] Abstract duplicate code: `spark.sql.sources.` properties
Repository: spark Updated Branches: refs/heads/branch-2.0 a355edeef -> a14c88acc [SPARK-15584][SQL] Abstract duplicate code: `spark.sql.sources.` properties ## What changes were proposed in this pull request? This PR replaces `spark.sql.sources.` strings with `CreateDataSourceTableUtils.*` constant variables. ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13349 from dongjoon-hyun/SPARK-15584. (cherry picked from commit 4538443e276597530a27c6922e48503677b13956) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a14c88ac Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a14c88ac Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a14c88ac Branch: refs/heads/branch-2.0 Commit: a14c88acce0733f3db8b0508ae8b0417822e08d8 Parents: a355ede Author: Dongjoon Hyun Authored: Fri May 27 11:10:31 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:10:39 2016 -0700 -- .../spark/ml/source/libsvm/LibSVMRelation.scala | 3 +- .../command/createDataSourceTables.scala| 28 +- .../spark/sql/execution/command/ddl.scala | 19 +++ .../spark/sql/execution/command/tables.scala| 4 +- .../datasources/DataSourceStrategy.scala| 2 +- .../execution/datasources/WriterContainer.scala | 10 ++-- .../execution/datasources/csv/CSVRelation.scala | 3 +- .../datasources/json/JsonFileFormat.scala | 5 +- .../datasources/parquet/ParquetFileFormat.scala | 4 +- .../datasources/text/TextFileFormat.scala | 3 +- .../spark/sql/execution/command/DDLSuite.scala | 10 ++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 +++--- .../spark/sql/hive/orc/OrcFileFormat.scala | 3 +- .../sql/hive/MetastoreDataSourcesSuite.scala| 58 ++-- .../sql/hive/execution/HiveCommandSuite.scala | 16 +++--- .../spark/sql/sources/SimpleTextRelation.scala | 3 +- 16 files changed, 95 insertions(+), 94 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a14c88ac/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 64ebf0c..7629369 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -51,7 +52,7 @@ private[libsvm] class LibSVMOutputWriter( new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration -val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") +val uniqueWriteJobId = configuration.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") http://git-wip-us.apache.org/repos/asf/spark/blob/a14c88ac/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index deedb68..4b9aab6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -256,15 +256,15 @@ case class CreateDataSourceTableAsSelectCommand( object CreateDataSourceTableUtils extends Logging { - // TODO: Actually replace usages with these variables (SPARK-15584) - val DATASOURCE_PREFIX = "spark.sql.sources." val DATASOURCE_PROVIDER = DATASOURCE_PREFIX + "provider" val DATASOURCE_WRITEJOBUUID = DATASOURCE_PREFIX + "writeJobUUID" val DATASOURCE_OUTPUTPATH = DATASOURCE_PREFIX + "output.path" - val DATASOURCE_SCHEMA_PREFIX = D
spark git commit: [SPARK-15584][SQL] Abstract duplicate code: `spark.sql.sources.` properties
Repository: spark Updated Branches: refs/heads/master d24e25157 -> 4538443e2 [SPARK-15584][SQL] Abstract duplicate code: `spark.sql.sources.` properties ## What changes were proposed in this pull request? This PR replaces `spark.sql.sources.` strings with `CreateDataSourceTableUtils.*` constant variables. ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13349 from dongjoon-hyun/SPARK-15584. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4538443e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4538443e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4538443e Branch: refs/heads/master Commit: 4538443e276597530a27c6922e48503677b13956 Parents: d24e251 Author: Dongjoon Hyun Authored: Fri May 27 11:10:31 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:10:31 2016 -0700 -- .../spark/ml/source/libsvm/LibSVMRelation.scala | 3 +- .../command/createDataSourceTables.scala| 28 +- .../spark/sql/execution/command/ddl.scala | 19 +++ .../spark/sql/execution/command/tables.scala| 4 +- .../datasources/DataSourceStrategy.scala| 2 +- .../execution/datasources/WriterContainer.scala | 10 ++-- .../execution/datasources/csv/CSVRelation.scala | 3 +- .../datasources/json/JsonFileFormat.scala | 5 +- .../datasources/parquet/ParquetFileFormat.scala | 4 +- .../datasources/text/TextFileFormat.scala | 3 +- .../spark/sql/execution/command/DDLSuite.scala | 10 ++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 +++--- .../spark/sql/hive/orc/OrcFileFormat.scala | 3 +- .../sql/hive/MetastoreDataSourcesSuite.scala| 58 ++-- .../sql/hive/execution/HiveCommandSuite.scala | 16 +++--- .../spark/sql/sources/SimpleTextRelation.scala | 3 +- 16 files changed, 95 insertions(+), 94 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4538443e/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 64ebf0c..7629369 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -51,7 +52,7 @@ private[libsvm] class LibSVMOutputWriter( new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration -val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") +val uniqueWriteJobId = configuration.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") http://git-wip-us.apache.org/repos/asf/spark/blob/4538443e/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index deedb68..4b9aab6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -256,15 +256,15 @@ case class CreateDataSourceTableAsSelectCommand( object CreateDataSourceTableUtils extends Logging { - // TODO: Actually replace usages with these variables (SPARK-15584) - val DATASOURCE_PREFIX = "spark.sql.sources." val DATASOURCE_PROVIDER = DATASOURCE_PREFIX + "provider" val DATASOURCE_WRITEJOBUUID = DATASOURCE_PREFIX + "writeJobUUID" val DATASOURCE_OUTPUTPATH = DATASOURCE_PREFIX + "output.path" - val DATASOURCE_SCHEMA_PREFIX = DATASOURCE_PREFIX + "schema." + val DATASOURCE_SCHEMA = DATASOURCE_PREFIX + "schema" + val DATASOURCE_S
spark git commit: [SPARK-15599][SQL][DOCS] API docs for `createDataset` functions in SparkSession
Repository: spark Updated Branches: refs/heads/master 4538443e2 -> 635fb30f8 [SPARK-15599][SQL][DOCS] API docs for `createDataset` functions in SparkSession ## What changes were proposed in this pull request? Adds API docs and usage examples for the 3 `createDataset` calls in `SparkSession` ## How was this patch tested? N/A Author: Sameer Agarwal Closes #13345 from sameeragarwal/dataset-doc. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/635fb30f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/635fb30f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/635fb30f Branch: refs/heads/master Commit: 635fb30f83a66cc56f5fecfed5bff77873bf49a6 Parents: 4538443 Author: Sameer Agarwal Authored: Fri May 27 11:11:31 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:11:31 2016 -0700 -- .../org/apache/spark/sql/SparkSession.scala | 63 1 file changed, 63 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/635fb30f/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5dabe0e..aa60048 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -376,6 +376,40 @@ class SparkSession private( Dataset.ofRows(self, LogicalRelation(baseRelation)) } + /* --- * + | Methods for creating DataSets | + * --- */ + + /** + * :: Experimental :: + * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Example == + * + * {{{ + * + * import spark.implicits._ + * case class Person(name: String, age: Long) + * val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19)) + * val ds = spark.createDataset(data) + * + * ds.show() + * // +---+---+ + * // | name|age| + * // +---+---+ + * // |Michael| 29| + * // | Andy| 30| + * // | Justin| 19| + * // +---+---+ + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -384,6 +418,17 @@ class SparkSession private( Dataset[T](self, plan) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from an RDD of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -392,6 +437,24 @@ class SparkSession private( Dataset[T](self, plan) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from a [[java.util.List]] of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Java Example == + * + * {{{ + * List data = Arrays.asList("hello", "world"); + * Dataset ds = spark.createDataset(data, Encoders.STRING()); + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15599][SQL][DOCS] API docs for `createDataset` functions in SparkSession
Repository: spark Updated Branches: refs/heads/branch-2.0 a14c88acc -> e69639f43 [SPARK-15599][SQL][DOCS] API docs for `createDataset` functions in SparkSession ## What changes were proposed in this pull request? Adds API docs and usage examples for the 3 `createDataset` calls in `SparkSession` ## How was this patch tested? N/A Author: Sameer Agarwal Closes #13345 from sameeragarwal/dataset-doc. (cherry picked from commit 635fb30f83a66cc56f5fecfed5bff77873bf49a6) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e69639f4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e69639f4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e69639f4 Branch: refs/heads/branch-2.0 Commit: e69639f4334aae3ace5e50452603dd667467ea9a Parents: a14c88a Author: Sameer Agarwal Authored: Fri May 27 11:11:31 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:11:40 2016 -0700 -- .../org/apache/spark/sql/SparkSession.scala | 63 1 file changed, 63 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e69639f4/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5dabe0e..aa60048 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -376,6 +376,40 @@ class SparkSession private( Dataset.ofRows(self, LogicalRelation(baseRelation)) } + /* --- * + | Methods for creating DataSets | + * --- */ + + /** + * :: Experimental :: + * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Example == + * + * {{{ + * + * import spark.implicits._ + * case class Person(name: String, age: Long) + * val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19)) + * val ds = spark.createDataset(data) + * + * ds.show() + * // +---+---+ + * // | name|age| + * // +---+---+ + * // |Michael| 29| + * // | Andy| 30| + * // | Justin| 19| + * // +---+---+ + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -384,6 +418,17 @@ class SparkSession private( Dataset[T](self, plan) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from an RDD of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -392,6 +437,24 @@ class SparkSession private( Dataset[T](self, plan) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from a [[java.util.List]] of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Java Example == + * + * {{{ + * List data = Arrays.asList("hello", "world"); + * Dataset ds = spark.createDataset(data, Encoders.STRING()); + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15597][SQL] Add SparkSession.emptyDataset
Repository: spark Updated Branches: refs/heads/master 635fb30f8 -> a52e68133 [SPARK-15597][SQL] Add SparkSession.emptyDataset ## What changes were proposed in this pull request? This patch adds a new function emptyDataset to SparkSession, for creating an empty dataset. ## How was this patch tested? Added a test case. Author: Reynold Xin Closes #13344 from rxin/SPARK-15597. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a52e6813 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a52e6813 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a52e6813 Branch: refs/heads/master Commit: a52e6813392ba4bdb1b818694b7ced8f6caa6a2b Parents: 635fb30 Author: Reynold Xin Authored: Fri May 27 11:13:09 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:13:09 2016 -0700 -- .../main/scala/org/apache/spark/sql/SparkSession.scala | 12 .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 6 ++ 2 files changed, 18 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a52e6813/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index aa60048..c9276cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -224,6 +224,18 @@ class SparkSession private( /** * :: Experimental :: + * Creates a new [[Dataset]] of type T containing zero elements. + * + * @return 2.0.0 + */ + @Experimental + def emptyDataset[T: Encoder]: Dataset[T] = { +val encoder = implicitly[Encoder[T]] +new Dataset(self, LocalRelation(encoder.schema.toAttributes), encoder) + } + + /** + * :: Experimental :: * Creates a [[DataFrame]] from an RDD of Product (e.g. case classes, tuples). * * @group dataframes http://git-wip-us.apache.org/repos/asf/spark/blob/a52e6813/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 2a65916..e395007 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -46,6 +46,12 @@ class DatasetSuite extends QueryTest with SharedSQLContext { 1, 1, 1) } + test("emptyDataset") { +val ds = spark.emptyDataset[Int] +assert(ds.count() == 0L) +assert(ds.collect() sameElements Array.empty[Int]) + } + test("range") { assert(spark.range(10).map(_ + 1).reduce(_ + _) == 55) assert(spark.range(10).map{ case i: java.lang.Long => i + 1 }.reduce(_ + _) == 55) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15597][SQL] Add SparkSession.emptyDataset
Repository: spark Updated Branches: refs/heads/branch-2.0 e69639f43 -> f52a95248 [SPARK-15597][SQL] Add SparkSession.emptyDataset ## What changes were proposed in this pull request? This patch adds a new function emptyDataset to SparkSession, for creating an empty dataset. ## How was this patch tested? Added a test case. Author: Reynold Xin Closes #13344 from rxin/SPARK-15597. (cherry picked from commit a52e6813392ba4bdb1b818694b7ced8f6caa6a2b) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f52a9524 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f52a9524 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f52a9524 Branch: refs/heads/branch-2.0 Commit: f52a9524865b8c56058a65b29a1aaacffb709f69 Parents: e69639f Author: Reynold Xin Authored: Fri May 27 11:13:09 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:13:17 2016 -0700 -- .../main/scala/org/apache/spark/sql/SparkSession.scala | 12 .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 6 ++ 2 files changed, 18 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f52a9524/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index aa60048..c9276cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -224,6 +224,18 @@ class SparkSession private( /** * :: Experimental :: + * Creates a new [[Dataset]] of type T containing zero elements. + * + * @return 2.0.0 + */ + @Experimental + def emptyDataset[T: Encoder]: Dataset[T] = { +val encoder = implicitly[Encoder[T]] +new Dataset(self, LocalRelation(encoder.schema.toAttributes), encoder) + } + + /** + * :: Experimental :: * Creates a [[DataFrame]] from an RDD of Product (e.g. case classes, tuples). * * @group dataframes http://git-wip-us.apache.org/repos/asf/spark/blob/f52a9524/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 2a65916..e395007 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -46,6 +46,12 @@ class DatasetSuite extends QueryTest with SharedSQLContext { 1, 1, 1) } + test("emptyDataset") { +val ds = spark.emptyDataset[Int] +assert(ds.count() == 0L) +assert(ds.collect() sameElements Array.empty[Int]) + } + test("range") { assert(spark.range(10).map(_ + 1).reduce(_ + _) == 55) assert(spark.range(10).map{ case i: java.lang.Long => i + 1 }.reduce(_ + _) == 55) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR][DOCS] Typo fixes in Dataset scaladoc
Repository: spark Updated Branches: refs/heads/branch-2.0 f52a95248 -> 89fdb6972 [MINOR][DOCS] Typo fixes in Dataset scaladoc ## What changes were proposed in this pull request? Minor typo fixes in Dataset scaladoc * Corrected context type as SparkSession, not SQLContext. liancheng rxin andrewor14 ## How was this patch tested? Compiled locally Author: Xinh Huynh Closes #13330 from xinhhuynh/fix-dataset-typos. (cherry picked from commit 5bdbedf2201efa6c34392aa9eff709761f027e1d) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/89fdb697 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/89fdb697 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/89fdb697 Branch: refs/heads/branch-2.0 Commit: 89fdb6972d5410f250bc56f8a834c939ee6653d2 Parents: f52a952 Author: Xinh Huynh Authored: Fri May 27 11:13:53 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:14:01 2016 -0700 -- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/89fdb697/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 85f0cf8..abd16f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -121,7 +121,7 @@ private[sql] object Dataset { * * A more concrete example in Scala: * {{{ - * // To create Dataset[Row] using SQLContext + * // To create Dataset[Row] using SparkSession * val people = spark.read.parquet("...") * val department = spark.read.parquet("...") * @@ -133,7 +133,7 @@ private[sql] object Dataset { * * and in Java: * {{{ - * // To create Dataset using SQLContext + * // To create Dataset using SparkSession * Dataset people = spark.read().parquet("..."); * Dataset department = spark.read().parquet("..."); * - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR][DOCS] Typo fixes in Dataset scaladoc
Repository: spark Updated Branches: refs/heads/master a52e68133 -> 5bdbedf22 [MINOR][DOCS] Typo fixes in Dataset scaladoc ## What changes were proposed in this pull request? Minor typo fixes in Dataset scaladoc * Corrected context type as SparkSession, not SQLContext. liancheng rxin andrewor14 ## How was this patch tested? Compiled locally Author: Xinh Huynh Closes #13330 from xinhhuynh/fix-dataset-typos. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5bdbedf2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5bdbedf2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5bdbedf2 Branch: refs/heads/master Commit: 5bdbedf2201efa6c34392aa9eff709761f027e1d Parents: a52e681 Author: Xinh Huynh Authored: Fri May 27 11:13:53 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:13:53 2016 -0700 -- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5bdbedf2/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 85f0cf8..abd16f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -121,7 +121,7 @@ private[sql] object Dataset { * * A more concrete example in Scala: * {{{ - * // To create Dataset[Row] using SQLContext + * // To create Dataset[Row] using SparkSession * val people = spark.read.parquet("...") * val department = spark.read.parquet("...") * @@ -133,7 +133,7 @@ private[sql] object Dataset { * * and in Java: * {{{ - * // To create Dataset using SQLContext + * // To create Dataset using SparkSession * Dataset people = spark.read().parquet("..."); * Dataset department = spark.read().parquet("..."); * - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15569] Reduce frequency of updateBytesWritten function in Disk…
Repository: spark Updated Branches: refs/heads/branch-2.0 89fdb6972 -> 30e87b55b [SPARK-15569] Reduce frequency of updateBytesWritten function in Disk… ## What changes were proposed in this pull request? Profiling a Spark job spilling large amount of intermediate data we found that significant portion of time is being spent in DiskObjectWriter.updateBytesWritten function. Looking at the code, we see that the function is being called too frequently to update the number of bytes written to disk. We should reduce the frequency to avoid this. ## How was this patch tested? Tested by running the job on cluster and saw 20% CPU gain by this change. Author: Sital Kedia Closes #13332 from sitalkedia/DiskObjectWriter. (cherry picked from commit ce756daa4f012ebdc5a41bf5a89ff11b6dfdab8c) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/30e87b55 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/30e87b55 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/30e87b55 Branch: refs/heads/branch-2.0 Commit: 30e87b55b6f59ca029778087710effc768fafc35 Parents: 89fdb69 Author: Sital Kedia Authored: Fri May 27 11:22:39 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:22:48 2016 -0700 -- .../apache/spark/storage/DiskBlockObjectWriter.scala| 3 +-- .../spark/storage/DiskBlockObjectWriterSuite.scala | 12 ++-- 2 files changed, 7 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/30e87b55/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index ab97d2e..5b493f4 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -203,8 +203,7 @@ private[spark] class DiskBlockObjectWriter( numRecordsWritten += 1 writeMetrics.incRecordsWritten(1) -// TODO: call updateBytesWritten() less frequently. -if (numRecordsWritten % 32 == 0) { +if (numRecordsWritten % 16384 == 0) { updateBytesWritten() } } http://git-wip-us.apache.org/repos/asf/spark/blob/30e87b55/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index 8eff3c2..ec4ef4b 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -53,13 +53,13 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.bytesWritten == 0) -// After 32 writes, metrics should update -for (i <- 0 until 32) { +// After 16384 writes, metrics should update +for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.bytesWritten > 0) -assert(writeMetrics.recordsWritten === 33) +assert(writeMetrics.recordsWritten === 16385) writer.commitAndClose() assert(file.length() == writeMetrics.bytesWritten) } @@ -75,13 +75,13 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.bytesWritten == 0) -// After 32 writes, metrics should update -for (i <- 0 until 32) { +// After 16384 writes, metrics should update +for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.bytesWritten > 0) -assert(writeMetrics.recordsWritten === 33) +assert(writeMetrics.recordsWritten === 16385) writer.revertPartialWritesAndClose() assert(writeMetrics.bytesWritten == 0) assert(writeMetrics.recordsWritten == 0) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15569] Reduce frequency of updateBytesWritten function in Disk…
Repository: spark Updated Branches: refs/heads/master 5bdbedf22 -> ce756daa4 [SPARK-15569] Reduce frequency of updateBytesWritten function in Disk… ## What changes were proposed in this pull request? Profiling a Spark job spilling large amount of intermediate data we found that significant portion of time is being spent in DiskObjectWriter.updateBytesWritten function. Looking at the code, we see that the function is being called too frequently to update the number of bytes written to disk. We should reduce the frequency to avoid this. ## How was this patch tested? Tested by running the job on cluster and saw 20% CPU gain by this change. Author: Sital Kedia Closes #13332 from sitalkedia/DiskObjectWriter. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ce756daa Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ce756daa Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ce756daa Branch: refs/heads/master Commit: ce756daa4f012ebdc5a41bf5a89ff11b6dfdab8c Parents: 5bdbedf Author: Sital Kedia Authored: Fri May 27 11:22:39 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:22:39 2016 -0700 -- .../apache/spark/storage/DiskBlockObjectWriter.scala| 3 +-- .../spark/storage/DiskBlockObjectWriterSuite.scala | 12 ++-- 2 files changed, 7 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ce756daa/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index ab97d2e..5b493f4 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -203,8 +203,7 @@ private[spark] class DiskBlockObjectWriter( numRecordsWritten += 1 writeMetrics.incRecordsWritten(1) -// TODO: call updateBytesWritten() less frequently. -if (numRecordsWritten % 32 == 0) { +if (numRecordsWritten % 16384 == 0) { updateBytesWritten() } } http://git-wip-us.apache.org/repos/asf/spark/blob/ce756daa/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index 8eff3c2..ec4ef4b 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -53,13 +53,13 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.bytesWritten == 0) -// After 32 writes, metrics should update -for (i <- 0 until 32) { +// After 16384 writes, metrics should update +for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.bytesWritten > 0) -assert(writeMetrics.recordsWritten === 33) +assert(writeMetrics.recordsWritten === 16385) writer.commitAndClose() assert(file.length() == writeMetrics.bytesWritten) } @@ -75,13 +75,13 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.bytesWritten == 0) -// After 32 writes, metrics should update -for (i <- 0 until 32) { +// After 16384 writes, metrics should update +for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.bytesWritten > 0) -assert(writeMetrics.recordsWritten === 33) +assert(writeMetrics.recordsWritten === 16385) writer.revertPartialWritesAndClose() assert(writeMetrics.bytesWritten == 0) assert(writeMetrics.recordsWritten == 0) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [HOTFIX] Scala 2.10 compile GaussianMixtureModel
Repository: spark Updated Branches: refs/heads/master 1b98fa2e4 -> b376a4eab [HOTFIX] Scala 2.10 compile GaussianMixtureModel Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b376a4ea Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b376a4ea Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b376a4ea Branch: refs/heads/master Commit: b376a4eabc82d622ea26290345c01465af7a628d Parents: 1b98fa2 Author: Andrew Or Authored: Fri May 27 11:43:01 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:43:01 2016 -0700 -- .../org/apache/spark/mllib/clustering/GaussianMixtureModel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b376a4ea/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 4b06816..f470b0f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -170,7 +170,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { (weight, new MultivariateGaussian(mu, sigma)) }.unzip - new GaussianMixtureModel(weights, gaussians) + new GaussianMixtureModel(weights.toArray, gaussians.toArray) } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [HOTFIX] Scala 2.10 compile GaussianMixtureModel
Repository: spark Updated Branches: refs/heads/branch-2.0 17f43cc87 -> 5ea58898c [HOTFIX] Scala 2.10 compile GaussianMixtureModel Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5ea58898 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5ea58898 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5ea58898 Branch: refs/heads/branch-2.0 Commit: 5ea58898cc9413fd0b04b60db230c8894d8bb9ef Parents: 17f43cc Author: Andrew Or Authored: Fri May 27 11:43:01 2016 -0700 Committer: Andrew Or Committed: Fri May 27 11:43:50 2016 -0700 -- .../org/apache/spark/mllib/clustering/GaussianMixtureModel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5ea58898/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 4b06816..f470b0f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -170,7 +170,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { (weight, new MultivariateGaussian(mu, sigma)) }.unzip - new GaussianMixtureModel(weights, gaussians) + new GaussianMixtureModel(weights.toArray, gaussians.toArray) } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15662][SQL] Add since annotation for classes in sql.catalog
Repository: spark Updated Branches: refs/heads/master 695470429 -> 223f1d58c [SPARK-15662][SQL] Add since annotation for classes in sql.catalog ## What changes were proposed in this pull request? This patch does a few things: 1. Adds since version annotation to methods and classes in sql.catalog. 2. Fixed a typo in FilterFunction and a whitespace issue in spark/api/java/function/package.scala 3. Added "database" field to Function class. ## How was this patch tested? Updated unit test case for "database" field in Function class. Author: Reynold Xin Closes #13406 from rxin/SPARK-15662. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/223f1d58 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/223f1d58 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/223f1d58 Branch: refs/heads/master Commit: 223f1d58c4f4b6eb0f0037a118a0bb635ae20bb1 Parents: 6954704 Author: Reynold Xin Authored: Tue May 31 17:29:10 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:29:10 2016 -0700 -- .../spark/api/java/function/FilterFunction.java | 2 +- .../spark/api/java/function/package.scala | 2 +- .../org/apache/spark/sql/catalog/Catalog.scala | 2 + .../apache/spark/sql/catalog/interface.scala| 42 +++- .../apache/spark/sql/internal/CatalogImpl.scala | 1 + .../spark/sql/internal/CatalogSuite.scala | 22 +++--- 6 files changed, 62 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/223f1d58/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java -- diff --git a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java index e8d999d..462ca3f 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java @@ -22,7 +22,7 @@ import java.io.Serializable; /** * Base interface for a function used in Dataset's filter function. * - * If the function returns true, the element is discarded in the returned Dataset. + * If the function returns true, the element is included in the returned Dataset. */ public interface FilterFunction extends Serializable { boolean call(T value) throws Exception; http://git-wip-us.apache.org/repos/asf/spark/blob/223f1d58/core/src/main/java/org/apache/spark/api/java/function/package.scala -- diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala index 0f9bac7..e19f12f 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/package.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -22,4 +22,4 @@ package org.apache.spark.api.java * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's * Java programming guide for more details. */ -package object function +package object function http://git-wip-us.apache.org/repos/asf/spark/blob/223f1d58/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index a99bc3b..6ddb1a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -24,6 +24,8 @@ import org.apache.spark.sql.types.StructType /** * Catalog interface for Spark. To access this, use `SparkSession.catalog`. + * + * @since 2.0.0 */ abstract class Catalog { http://git-wip-us.apache.org/repos/asf/spark/blob/223f1d58/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala index 0f7feb8..33032f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala @@ -25,6 +25,14 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams // Note: all classes here are expected to be wrapped in Datasets and so must extend // DefinedByConstructorParams for the catalog to be able to create encoders for them. +/** + * A database in Spark, as returned by the `listDatabases` method defined in [[Catalog]].
spark git commit: [SPARK-15662][SQL] Add since annotation for classes in sql.catalog
Repository: spark Updated Branches: refs/heads/branch-2.0 1f4c4def9 -> 978f54e76 [SPARK-15662][SQL] Add since annotation for classes in sql.catalog ## What changes were proposed in this pull request? This patch does a few things: 1. Adds since version annotation to methods and classes in sql.catalog. 2. Fixed a typo in FilterFunction and a whitespace issue in spark/api/java/function/package.scala 3. Added "database" field to Function class. ## How was this patch tested? Updated unit test case for "database" field in Function class. Author: Reynold Xin Closes #13406 from rxin/SPARK-15662. (cherry picked from commit 223f1d58c4f4b6eb0f0037a118a0bb635ae20bb1) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/978f54e7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/978f54e7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/978f54e7 Branch: refs/heads/branch-2.0 Commit: 978f54e76bab2e03e034e2dfcc2c1ebaffefc5e0 Parents: 1f4c4de Author: Reynold Xin Authored: Tue May 31 17:29:10 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:29:38 2016 -0700 -- .../spark/api/java/function/FilterFunction.java | 2 +- .../spark/api/java/function/package.scala | 2 +- .../org/apache/spark/sql/catalog/Catalog.scala | 2 + .../apache/spark/sql/catalog/interface.scala| 42 +++- .../apache/spark/sql/internal/CatalogImpl.scala | 1 + .../spark/sql/internal/CatalogSuite.scala | 22 +++--- 6 files changed, 62 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/978f54e7/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java -- diff --git a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java index e8d999d..462ca3f 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java @@ -22,7 +22,7 @@ import java.io.Serializable; /** * Base interface for a function used in Dataset's filter function. * - * If the function returns true, the element is discarded in the returned Dataset. + * If the function returns true, the element is included in the returned Dataset. */ public interface FilterFunction extends Serializable { boolean call(T value) throws Exception; http://git-wip-us.apache.org/repos/asf/spark/blob/978f54e7/core/src/main/java/org/apache/spark/api/java/function/package.scala -- diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala index 0f9bac7..e19f12f 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/package.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -22,4 +22,4 @@ package org.apache.spark.api.java * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's * Java programming guide for more details. */ -package object function +package object function http://git-wip-us.apache.org/repos/asf/spark/blob/978f54e7/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index a99bc3b..6ddb1a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -24,6 +24,8 @@ import org.apache.spark.sql.types.StructType /** * Catalog interface for Spark. To access this, use `SparkSession.catalog`. + * + * @since 2.0.0 */ abstract class Catalog { http://git-wip-us.apache.org/repos/asf/spark/blob/978f54e7/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala index 0f7feb8..33032f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala @@ -25,6 +25,14 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams // Note: all classes here are expected to be wrapped in Datasets and so must extend // DefinedByConstructorParams for the catalog to be able to create encoders for
spark git commit: [CORE][DOC][MINOR] typos + links
Repository: spark Updated Branches: refs/heads/master 8ca01a6fe -> 0f2471346 [CORE][DOC][MINOR] typos + links ## What changes were proposed in this pull request? A very tiny change to javadoc (which I don't mind if gets merged with a bigger change). I've just found it annoying and couldn't resist proposing a pull request. Sorry srowen and rxin. ## How was this patch tested? Manual build Author: Jacek Laskowski Closes #13383 from jaceklaskowski/memory-consumer. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0f247134 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0f247134 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0f247134 Branch: refs/heads/master Commit: 0f24713468088fa7617d208572179d558e1f286b Parents: 8ca01a6 Author: Jacek Laskowski Authored: Tue May 31 17:32:37 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:32:37 2016 -0700 -- core/src/main/java/org/apache/spark/memory/MemoryConsumer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0f247134/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java -- diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 38a21a8..fc1f3a8 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -23,7 +23,7 @@ import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; /** - * An memory consumer of TaskMemoryManager, which support spilling. + * A memory consumer of {@link TaskMemoryManager} that supports spilling. * * Note: this only supports allocation / spilling of Tungsten memory. */ @@ -45,7 +45,7 @@ public abstract class MemoryConsumer { } /** - * Returns the memory mode, ON_HEAP or OFF_HEAP. + * Returns the memory mode, {@link MemoryMode#ON_HEAP} or {@link MemoryMode#OFF_HEAP}. */ public MemoryMode getMode() { return mode; - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [CORE][DOC][MINOR] typos + links
Repository: spark Updated Branches: refs/heads/branch-2.0 f0e8738c1 -> 00fca2a05 [CORE][DOC][MINOR] typos + links ## What changes were proposed in this pull request? A very tiny change to javadoc (which I don't mind if gets merged with a bigger change). I've just found it annoying and couldn't resist proposing a pull request. Sorry srowen and rxin. ## How was this patch tested? Manual build Author: Jacek Laskowski Closes #13383 from jaceklaskowski/memory-consumer. (cherry picked from commit 0f24713468088fa7617d208572179d558e1f286b) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/00fca2a0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/00fca2a0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/00fca2a0 Branch: refs/heads/branch-2.0 Commit: 00fca2a058d9803fe2b07d7c5827d51e821e523e Parents: f0e8738 Author: Jacek Laskowski Authored: Tue May 31 17:32:37 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:32:47 2016 -0700 -- core/src/main/java/org/apache/spark/memory/MemoryConsumer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/00fca2a0/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java -- diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 38a21a8..fc1f3a8 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -23,7 +23,7 @@ import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; /** - * An memory consumer of TaskMemoryManager, which support spilling. + * A memory consumer of {@link TaskMemoryManager} that supports spilling. * * Note: this only supports allocation / spilling of Tungsten memory. */ @@ -45,7 +45,7 @@ public abstract class MemoryConsumer { } /** - * Returns the memory mode, ON_HEAP or OFF_HEAP. + * Returns the memory mode, {@link MemoryMode#ON_HEAP} or {@link MemoryMode#OFF_HEAP}. */ public MemoryMode getMode() { return mode; - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15670][JAVA API][SPARK CORE] label_accumulator_deprecate_in_java_spark_context
Repository: spark Updated Branches: refs/heads/master 06514d689 -> dad5a6881 [SPARK-15670][JAVA API][SPARK CORE] label_accumulator_deprecate_in_java_spark_context ## What changes were proposed in this pull request? Add deprecate annotation for acumulator V1 interface in JavaSparkContext class ## How was this patch tested? N/A Author: WeichenXu Closes #13412 from WeichenXu123/label_accumulator_deprecate_in_java_spark_context. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dad5a688 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dad5a688 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dad5a688 Branch: refs/heads/master Commit: dad5a68818436eb7feaeb762b72433248eff298f Parents: 06514d6 Author: WeichenXu Authored: Tue May 31 17:34:34 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:34:34 2016 -0700 -- .../main/scala/org/apache/spark/api/java/JavaSparkContext.scala | 4 1 file changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/dad5a688/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index fb63234..bfb6a35 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -596,6 +596,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `add` method. Only the master can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) @@ -605,6 +606,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, name: String, accumulatorParam: AccumulatorParam[T]) : Accumulator[T] = sc.accumulator(initialValue, name)(accumulatorParam) @@ -613,6 +615,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks * can "add" values with `add`. Only the master can access the accumuable's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) @@ -622,6 +625,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[T, R](initialValue: T, name: String, param: AccumulableParam[T, R]) : Accumulable[T, R] = sc.accumulable(initialValue, name)(param) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15670][JAVA API][SPARK CORE] label_accumulator_deprecate_in_java_spark_context
Repository: spark Updated Branches: refs/heads/branch-2.0 00fca2a05 -> 7f240eaee [SPARK-15670][JAVA API][SPARK CORE] label_accumulator_deprecate_in_java_spark_context ## What changes were proposed in this pull request? Add deprecate annotation for acumulator V1 interface in JavaSparkContext class ## How was this patch tested? N/A Author: WeichenXu Closes #13412 from WeichenXu123/label_accumulator_deprecate_in_java_spark_context. (cherry picked from commit dad5a68818436eb7feaeb762b72433248eff298f) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7f240eae Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7f240eae Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7f240eae Branch: refs/heads/branch-2.0 Commit: 7f240eaeed78b42c23973678c5f643e486d4cfa3 Parents: 00fca2a Author: WeichenXu Authored: Tue May 31 17:34:34 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:34:43 2016 -0700 -- .../main/scala/org/apache/spark/api/java/JavaSparkContext.scala | 4 1 file changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7f240eae/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index fb63234..bfb6a35 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -596,6 +596,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `add` method. Only the master can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) @@ -605,6 +606,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, name: String, accumulatorParam: AccumulatorParam[T]) : Accumulator[T] = sc.accumulator(initialValue, name)(accumulatorParam) @@ -613,6 +615,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks * can "add" values with `add`. Only the master can access the accumuable's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) @@ -622,6 +625,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[T, R](initialValue: T, name: String, param: AccumulableParam[T, R]) : Accumulable[T, R] = sc.accumulable(initialValue, name)(param) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR][SQL][DOCS] Fix docs of Dataset.scala and SQLImplicits.scala.
Repository: spark Updated Branches: refs/heads/master dad5a6881 -> 196a0d827 [MINOR][SQL][DOCS] Fix docs of Dataset.scala and SQLImplicits.scala. This PR fixes a sample code, a description, and indentations in docs. Manual. Author: Dongjoon Hyun Closes #13420 from dongjoon-hyun/minor_fix_dataset_doc. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/196a0d82 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/196a0d82 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/196a0d82 Branch: refs/heads/master Commit: 196a0d82730e78b573a64a791a6ad873aa9ec74d Parents: dad5a68 Author: Dongjoon Hyun Authored: Tue May 31 17:36:24 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:37:33 2016 -0700 -- .../scala/org/apache/spark/sql/Dataset.scala| 36 ++-- .../org/apache/spark/sql/SQLImplicits.scala | 2 +- 2 files changed, 19 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/196a0d82/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 31000dc..7be49b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -*http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql @@ -93,14 +93,14 @@ private[sql] object Dataset { * to some files on storage systems, using the `read` function available on a `SparkSession`. * {{{ * val people = spark.read.parquet("...").as[Person] // Scala - * Dataset people = spark.read().parquet("...").as(Encoders.bean(Person.class) // Java + * Dataset people = spark.read().parquet("...").as(Encoders.bean(Person.class)); // Java * }}} * * Datasets can also be created through transformations available on existing Datasets. For example, * the following creates a new Dataset by applying a filter on the existing one: * {{{ * val names = people.map(_.name) // in Scala; names is a Dataset[String] - * Dataset names = people.map((Person p) -> p.name, Encoders.STRING)) // in Java 8 + * Dataset names = people.map((Person p) -> p.name, Encoders.STRING)); // in Java 8 * }}} * * Dataset operations can also be untyped, through various domain-specific-language (DSL) @@ -110,7 +110,7 @@ private[sql] object Dataset { * To select a column from the Dataset, use `apply` method in Scala and `col` in Java. * {{{ * val ageCol = people("age") // in Scala - * Column ageCol = people.col("age") // in Java + * Column ageCol = people.col("age"); // in Java * }}} * * Note that the [[Column]] type can also be manipulated through its various functions. http://git-wip-us.apache.org/repos/asf/spark/blob/196a0d82/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index f423e7d..b7ea2a
spark git commit: [MINOR][SQL][DOCS] Fix docs of Dataset.scala and SQLImplicits.scala.
Repository: spark Updated Branches: refs/heads/branch-2.0 b8de4ad7d -> 4b19c9776 [MINOR][SQL][DOCS] Fix docs of Dataset.scala and SQLImplicits.scala. This PR fixes a sample code, a description, and indentations in docs. Manual. Author: Dongjoon Hyun Closes #13420 from dongjoon-hyun/minor_fix_dataset_doc. (cherry picked from commit 196a0d82730e78b573a64a791a6ad873aa9ec74d) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4b19c977 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4b19c977 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4b19c977 Branch: refs/heads/branch-2.0 Commit: 4b19c97764489a48abccab75e1b132b469383f44 Parents: b8de4ad Author: Dongjoon Hyun Authored: Tue May 31 17:36:24 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:37:49 2016 -0700 -- .../scala/org/apache/spark/sql/Dataset.scala| 36 ++-- .../org/apache/spark/sql/SQLImplicits.scala | 2 +- 2 files changed, 19 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4b19c977/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 31000dc..7be49b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -*http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql @@ -93,14 +93,14 @@ private[sql] object Dataset { * to some files on storage systems, using the `read` function available on a `SparkSession`. * {{{ * val people = spark.read.parquet("...").as[Person] // Scala - * Dataset people = spark.read().parquet("...").as(Encoders.bean(Person.class) // Java + * Dataset people = spark.read().parquet("...").as(Encoders.bean(Person.class)); // Java * }}} * * Datasets can also be created through transformations available on existing Datasets. For example, * the following creates a new Dataset by applying a filter on the existing one: * {{{ * val names = people.map(_.name) // in Scala; names is a Dataset[String] - * Dataset names = people.map((Person p) -> p.name, Encoders.STRING)) // in Java 8 + * Dataset names = people.map((Person p) -> p.name, Encoders.STRING)); // in Java 8 * }}} * * Dataset operations can also be untyped, through various domain-specific-language (DSL) @@ -110,7 +110,7 @@ private[sql] object Dataset { * To select a column from the Dataset, use `apply` method in Scala and `col` in Java. * {{{ * val ageCol = people("age") // in Scala - * Column ageCol = people.col("age") // in Java + * Column ageCol = people.col("age"); // in Java * }}} * * Note that the [[Column]] type can also be manipulated through its various functions. http://git-wip-us.apache.org/repos/asf/spark/blob/4b19c977/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQL
spark git commit: [MINOR] Slightly better error message when attempting to query hive tables w/in-mem catalog
Repository: spark Updated Branches: refs/heads/master 196a0d827 -> 93e97147e [MINOR] Slightly better error message when attempting to query hive tables w/in-mem catalog andrewor14 Author: Eric Liang Closes #13427 from ericl/better-error-msg. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/93e97147 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/93e97147 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/93e97147 Branch: refs/heads/master Commit: 93e97147eb499dde1e54e07ba113eebcbe25508a Parents: 196a0d8 Author: Eric Liang Authored: Tue May 31 17:39:03 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:39:03 2016 -0700 -- .../org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/93e97147/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c5f221d..7b451ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -309,14 +309,14 @@ trait CheckAnalysis extends PredicateHelper { case s: SimpleCatalogRelation => failAnalysis( s""" - |Please enable Hive support when selecting the regular tables: + |Hive support is required to select over the following tables: |${s.catalogTable.identifier} """.stripMargin) case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => failAnalysis( s""" - |Please enable Hive support when inserting the regular tables: + |Hive support is required to insert into the following tables: |${s.catalogTable.identifier} """.stripMargin) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR] Slightly better error message when attempting to query hive tables w/in-mem catalog
Repository: spark Updated Branches: refs/heads/branch-2.0 4b19c9776 -> ac4cb1718 [MINOR] Slightly better error message when attempting to query hive tables w/in-mem catalog andrewor14 Author: Eric Liang Closes #13427 from ericl/better-error-msg. (cherry picked from commit 93e97147eb499dde1e54e07ba113eebcbe25508a) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ac4cb171 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ac4cb171 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ac4cb171 Branch: refs/heads/branch-2.0 Commit: ac4cb17182615dba56e7fb5978150e3eb69a113a Parents: 4b19c97 Author: Eric Liang Authored: Tue May 31 17:39:03 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:39:13 2016 -0700 -- .../org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ac4cb171/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c5f221d..7b451ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -309,14 +309,14 @@ trait CheckAnalysis extends PredicateHelper { case s: SimpleCatalogRelation => failAnalysis( s""" - |Please enable Hive support when selecting the regular tables: + |Hive support is required to select over the following tables: |${s.catalogTable.identifier} """.stripMargin) case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => failAnalysis( s""" - |Please enable Hive support when inserting the regular tables: + |Hive support is required to insert into the following tables: |${s.catalogTable.identifier} """.stripMargin) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-15618][SQL][MLLIB] Use SparkSession.builder.sparkContext if applicable.
Repository: spark Updated Branches: refs/heads/master 93e97147e -> 85d6b0db9 [SPARK-15618][SQL][MLLIB] Use SparkSession.builder.sparkContext if applicable. ## What changes were proposed in this pull request? This PR changes function `SparkSession.builder.sparkContext(..)` from **private[sql]** into **private[spark]**, and uses it if applicable like the followings. ``` - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() ``` ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13365 from dongjoon-hyun/SPARK-15618. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/85d6b0db Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/85d6b0db Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/85d6b0db Branch: refs/heads/master Commit: 85d6b0db9f5bd425c36482ffcb1c3b9fd0fcdb31 Parents: 93e9714 Author: Dongjoon Hyun Authored: Tue May 31 17:40:44 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:40:44 2016 -0700 -- .../apache/spark/examples/BroadcastTest.scala | 7 +--- .../spark/examples/mllib/LDAExample.scala | 1 + .../spark/examples/sql/hive/HiveFromSpark.scala | 5 +-- .../spark/mllib/api/python/PythonMLLibAPI.scala | 6 +-- .../classification/LogisticRegression.scala | 2 +- .../spark/mllib/classification/NaiveBayes.scala | 8 ++-- .../impl/GLMClassificationModel.scala | 4 +- .../mllib/clustering/BisectingKMeansModel.scala | 4 +- .../mllib/clustering/GaussianMixtureModel.scala | 4 +- .../spark/mllib/clustering/KMeansModel.scala| 4 +- .../spark/mllib/clustering/LDAModel.scala | 8 ++-- .../clustering/PowerIterationClustering.scala | 4 +- .../spark/mllib/feature/ChiSqSelector.scala | 4 +- .../apache/spark/mllib/feature/Word2Vec.scala | 4 +- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 4 +- .../org/apache/spark/mllib/fpm/PrefixSpan.scala | 4 +- .../MatrixFactorizationModel.scala | 4 +- .../mllib/regression/IsotonicRegression.scala | 4 +- .../regression/impl/GLMRegressionModel.scala| 4 +- .../mllib/tree/model/DecisionTreeModel.scala| 4 +- .../mllib/tree/model/treeEnsembleModels.scala | 4 +- .../spark/ml/feature/ChiSqSelectorSuite.scala | 8 +--- .../ml/feature/QuantileDiscretizerSuite.scala | 6 +-- .../spark/ml/recommendation/ALSSuite.scala | 3 +- .../apache/spark/ml/tree/impl/TreeTests.scala | 3 +- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../execution/joins/BroadcastJoinSuite.scala| 41 ++-- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 13 +++ 28 files changed, 79 insertions(+), 90 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/85d6b0db/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala -- diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index c50f25d..a68fd02 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -29,13 +29,10 @@ object BroadcastTest { val blockSize = if (args.length > 2) args(2) else "4096" -val sparkConf = new SparkConf() - .set("spark.broadcast.blockSize", blockSize) - val spark = SparkSession - .builder - .config(sparkConf) + .builder() .appName("Broadcast Test") + .config("spark.broadcast.blockSize", blockSize) .getOrCreate() val sc = spark.sparkContext http://git-wip-us.apache.org/repos/asf/spark/blob/85d6b0db/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala -- diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 7651aad..3fbf8e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -191,6 +191,7 @@ object LDAExample { val spark = SparkSession .builder + .sparkContext(sc) .getOrCreate() import spark.implicits._ http://git-wip-us.apache.org/repos/asf/spark/blob/85d6b0db/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala -- diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/ap
spark git commit: [SPARK-15618][SQL][MLLIB] Use SparkSession.builder.sparkContext if applicable.
Repository: spark Updated Branches: refs/heads/branch-2.0 ac4cb1718 -> 459fd34a0 [SPARK-15618][SQL][MLLIB] Use SparkSession.builder.sparkContext if applicable. This PR changes function `SparkSession.builder.sparkContext(..)` from **private[sql]** into **private[spark]**, and uses it if applicable like the followings. ``` - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() ``` Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13365 from dongjoon-hyun/SPARK-15618. (cherry picked from commit 85d6b0db9f5bd425c36482ffcb1c3b9fd0fcdb31) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/459fd34a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/459fd34a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/459fd34a Branch: refs/heads/branch-2.0 Commit: 459fd34a0b011589d924e318925c97657e71930d Parents: ac4cb17 Author: Dongjoon Hyun Authored: Tue May 31 17:40:44 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:41:28 2016 -0700 -- .../apache/spark/examples/BroadcastTest.scala | 7 +--- .../spark/examples/mllib/LDAExample.scala | 1 + .../spark/examples/sql/hive/HiveFromSpark.scala | 5 +-- .../spark/mllib/api/python/PythonMLLibAPI.scala | 6 +-- .../classification/LogisticRegression.scala | 2 +- .../spark/mllib/classification/NaiveBayes.scala | 8 ++-- .../impl/GLMClassificationModel.scala | 4 +- .../mllib/clustering/BisectingKMeansModel.scala | 4 +- .../mllib/clustering/GaussianMixtureModel.scala | 4 +- .../spark/mllib/clustering/KMeansModel.scala| 4 +- .../spark/mllib/clustering/LDAModel.scala | 8 ++-- .../clustering/PowerIterationClustering.scala | 4 +- .../spark/mllib/feature/ChiSqSelector.scala | 4 +- .../apache/spark/mllib/feature/Word2Vec.scala | 4 +- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 4 +- .../org/apache/spark/mllib/fpm/PrefixSpan.scala | 4 +- .../MatrixFactorizationModel.scala | 4 +- .../mllib/regression/IsotonicRegression.scala | 4 +- .../regression/impl/GLMRegressionModel.scala| 4 +- .../mllib/tree/model/DecisionTreeModel.scala| 4 +- .../mllib/tree/model/treeEnsembleModels.scala | 4 +- .../spark/ml/feature/ChiSqSelectorSuite.scala | 8 +--- .../ml/feature/QuantileDiscretizerSuite.scala | 6 +-- .../spark/ml/recommendation/ALSSuite.scala | 3 +- .../apache/spark/ml/tree/impl/TreeTests.scala | 3 +- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../execution/joins/BroadcastJoinSuite.scala| 39 ++-- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 13 +++ 28 files changed, 78 insertions(+), 89 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/459fd34a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala -- diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index c50f25d..a68fd02 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -29,13 +29,10 @@ object BroadcastTest { val blockSize = if (args.length > 2) args(2) else "4096" -val sparkConf = new SparkConf() - .set("spark.broadcast.blockSize", blockSize) - val spark = SparkSession - .builder - .config(sparkConf) + .builder() .appName("Broadcast Test") + .config("spark.broadcast.blockSize", blockSize) .getOrCreate() val sc = spark.sparkContext http://git-wip-us.apache.org/repos/asf/spark/blob/459fd34a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala -- diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 7651aad..3fbf8e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -191,6 +191,7 @@ object LDAExample { val spark = SparkSession .builder + .sparkContext(sc) .getOrCreate() import spark.implicits._ http://git-wip-us.apache.org/repos/asf/spark/blob/459fd34a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala -- diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/s
spark git commit: [SPARK-15236][SQL][SPARK SHELL] Add spark-defaults property to switch to use InMemoryCatalog
Repository: spark Updated Branches: refs/heads/master 85d6b0db9 -> 04f925ede [SPARK-15236][SQL][SPARK SHELL] Add spark-defaults property to switch to use InMemoryCatalog ## What changes were proposed in this pull request? This PR change REPL/Main to check this property `spark.sql.catalogImplementation` to decide if `enableHiveSupport `should be called. If `spark.sql.catalogImplementation` is set to `hive`, and hive classes are built, Spark will use Hive support. Other wise, Spark will create a SparkSession with in-memory catalog support. ## How was this patch tested? Run the REPL component test. Author: xin Wu Author: Xin Wu Closes #13088 from xwu0226/SPARK-15236. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/04f925ed Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/04f925ed Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/04f925ed Branch: refs/heads/master Commit: 04f925ede851fc77add9ef1cacb79fb3a617f650 Parents: 85d6b0d Author: xin Wu Authored: Tue May 31 17:42:47 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:42:47 2016 -0700 -- .../main/scala/org/apache/spark/repl/Main.scala | 20 ++-- .../scala/org/apache/spark/repl/ReplSuite.scala | 50 +++- 2 files changed, 66 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/04f925ed/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala -- diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 005edda..771670f 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -22,6 +22,7 @@ import java.io.File import scala.tools.nsc.GenericRunnerSettings import org.apache.spark._ +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils @@ -88,10 +89,23 @@ object Main extends Logging { } val builder = SparkSession.builder.config(conf) -if (SparkSession.hiveClassesArePresent) { - sparkSession = builder.enableHiveSupport().getOrCreate() - logInfo("Created Spark session with Hive support") +if (conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase == "hive") { + if (SparkSession.hiveClassesArePresent) { +// In the case that the property is not set at all, builder's config +// does not have this value set to 'hive' yet. The original default +// behavior is that when there are hive classes, we use hive catalog. +sparkSession = builder.enableHiveSupport().getOrCreate() +logInfo("Created Spark session with Hive support") + } else { +// Need to change it back to 'in-memory' if no hive classes are found +// in the case that the property is set to hive in spark-defaults.conf +builder.config(CATALOG_IMPLEMENTATION.key, "in-memory") +sparkSession = builder.getOrCreate() +logInfo("Created Spark session") + } } else { + // In the case that the property is set but not to 'hive', the internal + // default is 'in-memory'. So the sparkSession will use in-memory catalog. sparkSession = builder.getOrCreate() logInfo("Created Spark session") } http://git-wip-us.apache.org/repos/asf/spark/blob/04f925ed/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala -- diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index af82e7a..1256860 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -21,9 +21,11 @@ import java.io._ import java.net.URLClassLoader import scala.collection.mutable.ArrayBuffer - import org.apache.commons.lang3.StringEscapeUtils +import org.apache.log4j.{Level, LogManager} import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils class ReplSuite extends SparkFunSuite { @@ -99,6 +101,52 @@ class ReplSuite extends SparkFunSuite { System.clearProperty("spark.driver.port") } + test("SPARK-15236: use Hive catalog") { +// turn on the INFO log so that it is possible the code will dump INFO +// entry for using "HiveMetastore" +val rootLogger = LogManager.getRootLogger() +val logLevel = rootLogger.getLe
spark git commit: [SPARK-15236][SQL][SPARK SHELL] Add spark-defaults property to switch to use InMemoryCatalog
Repository: spark Updated Branches: refs/heads/branch-2.0 459fd34a0 -> db96f398b [SPARK-15236][SQL][SPARK SHELL] Add spark-defaults property to switch to use InMemoryCatalog ## What changes were proposed in this pull request? This PR change REPL/Main to check this property `spark.sql.catalogImplementation` to decide if `enableHiveSupport `should be called. If `spark.sql.catalogImplementation` is set to `hive`, and hive classes are built, Spark will use Hive support. Other wise, Spark will create a SparkSession with in-memory catalog support. ## How was this patch tested? Run the REPL component test. Author: xin Wu Author: Xin Wu Closes #13088 from xwu0226/SPARK-15236. (cherry picked from commit 04f925ede851fc77add9ef1cacb79fb3a617f650) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/db96f398 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/db96f398 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/db96f398 Branch: refs/heads/branch-2.0 Commit: db96f398be338a937ef369515615f7fe7bb439a5 Parents: 459fd34 Author: xin Wu Authored: Tue May 31 17:42:47 2016 -0700 Committer: Andrew Or Committed: Tue May 31 17:42:57 2016 -0700 -- .../main/scala/org/apache/spark/repl/Main.scala | 20 ++-- .../scala/org/apache/spark/repl/ReplSuite.scala | 50 +++- 2 files changed, 66 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/db96f398/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala -- diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 005edda..771670f 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -22,6 +22,7 @@ import java.io.File import scala.tools.nsc.GenericRunnerSettings import org.apache.spark._ +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils @@ -88,10 +89,23 @@ object Main extends Logging { } val builder = SparkSession.builder.config(conf) -if (SparkSession.hiveClassesArePresent) { - sparkSession = builder.enableHiveSupport().getOrCreate() - logInfo("Created Spark session with Hive support") +if (conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase == "hive") { + if (SparkSession.hiveClassesArePresent) { +// In the case that the property is not set at all, builder's config +// does not have this value set to 'hive' yet. The original default +// behavior is that when there are hive classes, we use hive catalog. +sparkSession = builder.enableHiveSupport().getOrCreate() +logInfo("Created Spark session with Hive support") + } else { +// Need to change it back to 'in-memory' if no hive classes are found +// in the case that the property is set to hive in spark-defaults.conf +builder.config(CATALOG_IMPLEMENTATION.key, "in-memory") +sparkSession = builder.getOrCreate() +logInfo("Created Spark session") + } } else { + // In the case that the property is set but not to 'hive', the internal + // default is 'in-memory'. So the sparkSession will use in-memory catalog. sparkSession = builder.getOrCreate() logInfo("Created Spark session") } http://git-wip-us.apache.org/repos/asf/spark/blob/db96f398/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala -- diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index af82e7a..1256860 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -21,9 +21,11 @@ import java.io._ import java.net.URLClassLoader import scala.collection.mutable.ArrayBuffer - import org.apache.commons.lang3.StringEscapeUtils +import org.apache.log4j.{Level, LogManager} import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils class ReplSuite extends SparkFunSuite { @@ -99,6 +101,52 @@ class ReplSuite extends SparkFunSuite { System.clearProperty("spark.driver.port") } + test("SPARK-15236: use Hive catalog") { +// turn on the INFO log so that it is possible the code will dump INFO +// entry for usi