From 25b71d8c15572f0f2b951c827c169f8c65f726ad Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Thu, 12 Mar 2015 00:55:26 -0700 Subject: [PATCH 01/62] [SPARK-6296] [SQL] Added equals to Column Author: Volodymyr Lyubinets Closes #4988 from vlyubin/columncomp and squashes the following commits: 92d7c8f [Volodymyr Lyubinets] Added equals to Column --- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 7 +++++++ .../scala/org/apache/spark/sql/ColumnExpressionSuite.scala | 5 +++++ 2 files changed, 12 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index a2cc9a9b93eb8..908c78a4d3f10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -59,6 +59,13 @@ class Column(protected[sql] val expr: Expression) { override def toString: String = expr.prettyString + override def equals(that: Any) = that match { + case that: Column => that.expr.equals(this.expr) + case _ => false + } + + override def hashCode: Int = this.expr.hashCode + /** * Unary minus, i.e. negate the expression. * {{{ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 37c02aaa5460b..3036fbc05d021 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -313,4 +313,9 @@ class ColumnExpressionSuite extends QueryTest { test("lift alias out of cast") { assert(col("1234").as("name").cast("int").expr === col("1234").cast("int").as("name").expr) } + + test("columns can be compared") { + assert('key.desc == 'key.desc) + assert('key.desc != 'key.asc) + } } From 712679a7b447346a365b38574d7a86d56a93f767 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 12 Mar 2015 01:34:38 -0700 Subject: [PATCH 02/62] [SPARK-6294] fix hang when call take() in JVM on PythonRDD The Thread.interrupt() can not terminate the thread in some cases, so we should not wait for the writerThread of PythonRDD. This PR also ignore some exception during clean up. cc JoshRosen mengxr Author: Davies Liu Closes #4987 from davies/fix_take and squashes the following commits: 4488f1a [Davies Liu] fix hang when call take() in JVM on PythonRDD --- .../scala/org/apache/spark/api/python/PythonRDD.scala | 9 ++++++--- python/pyspark/daemon.py | 5 ++++- python/pyspark/tests.py | 5 +++++ 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8d4a53b4ca9b0..4c71b69069eb3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -76,7 +76,6 @@ private[spark] class PythonRDD( context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() - writerThread.join() if (!reuse_worker || !released) { try { worker.close() @@ -248,13 +247,17 @@ private[spark] class PythonRDD( } catch { case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) - Utils.tryLog(worker.shutdownOutput()) + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } case e: Exception => // We must avoid throwing exceptions here, because the thread uncaught exception handler // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e - Utils.tryLog(worker.shutdownOutput()) + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } } finally { // Release memory used by this thread for shuffles env.shuffleMemoryManager.releaseMemoryForThisThread() diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index f09587f211708..93885985fe377 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -61,7 +61,10 @@ def worker(sock): except SystemExit as exc: exit_code = compute_real_exit_code(exc.code) finally: - outfile.flush() + try: + outfile.flush() + except Exception: + pass return exit_code diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 06ba2b461d53e..dd8d3b1c53733 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -782,6 +782,11 @@ def test_narrow_dependency_in_join(self): jobId = tracker.getJobIdsForGroup("test4")[0] self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) + # Regression test for SPARK-6294 + def test_take_on_jrdd(self): + rdd = self.sc.parallelize(range(1 << 20)).map(lambda x: str(x)) + rdd._jrdd.first() + class ProfilerTests(PySparkTestCase): From 0cba802adf15f5ab8da24dd1e8a5e7214cc4e148 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 12 Mar 2015 01:39:04 -0700 Subject: [PATCH 03/62] [SPARK-5814][MLLIB][GRAPHX] Remove JBLAS from runtime The issue is discussed in https://issues.apache.org/jira/browse/SPARK-5669. Replacing all JBLAS usage by netlib-java gives us a simpler dependency tree and less license issues to worry about. I didn't touch the test scope in this PR. The user guide is not modified to avoid merge conflicts with branch-1.3. srowen ankurdave pwendell Author: Xiangrui Meng Closes #4699 from mengxr/SPARK-5814 and squashes the following commits: 48635c6 [Xiangrui Meng] move netlib-java version to parent pom ca21c74 [Xiangrui Meng] remove jblas from ml-guide 5f7767a [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5814 c5c4183 [Xiangrui Meng] merge master 0f20cad [Xiangrui Meng] add mima excludes e53e9f4 [Xiangrui Meng] remove jblas from mllib runtime ceaa14d [Xiangrui Meng] replace jblas by netlib-java in graphx fa7c2ca [Xiangrui Meng] move jblas to test scope --- assembly/pom.xml | 10 -- docs/mllib-guide.md | 5 - graphx/pom.xml | 11 ++- .../apache/spark/graphx/lib/SVDPlusPlus.scala | 96 ++++++++++++------- .../spark/graphx/lib/SVDPlusPlusSuite.scala | 6 +- mllib/pom.xml | 3 +- .../apache/spark/ml/recommendation/ALS.scala | 14 ++- .../spark/mllib/optimization/NNLS.scala | 86 +++++++++-------- .../MatrixFactorizationModel.scala | 15 ++- .../mllib/util/LinearDataGenerator.scala | 9 +- .../spark/mllib/util/MFDataGenerator.scala | 26 ++--- .../spark/mllib/util/SVMDataGenerator.scala | 7 +- .../spark/mllib/optimization/NNLSSuite.scala | 6 +- .../spark/mllib/stat/KernelDensitySuite.scala | 4 +- pom.xml | 1 + project/MimaExcludes.scala | 28 ++++++ 16 files changed, 183 insertions(+), 144 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index cbf5b6c4aa8df..d3bb4bde0c412 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -114,16 +114,6 @@ META-INF/*.RSA - - - org.jblas:jblas - - - lib/static/Linux/i386/** - lib/static/Mac OS X/** - lib/static/Windows/** - - diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4c7a7d9115ca1..598374f66df5e 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -80,11 +80,6 @@ include `netlib-java`'s native proxies by default. To configure [netlib-java](https://github.com/fommil/netlib-java) documentation for your platform's additional installation instructions. -MLlib also uses [jblas](https://github.com/mikiobraun/jblas) which -will require you to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) -if it is not already present on your nodes. - To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. diff --git a/graphx/pom.xml b/graphx/pom.xml index 57e338c03ecf9..c0d534e185d7f 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -45,9 +45,14 @@ guava - org.jblas - jblas - ${jblas.version} + com.github.fommil.netlib + core + ${netlib.java.version} + + + net.sourceforge.f2j + arpack_combined_all + 0.1 org.scalacheck diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 3e4157a63fd1c..1a7178b82e3af 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -18,7 +18,9 @@ package org.apache.spark.graphx.lib import scala.util.Random -import org.jblas.DoubleMatrix + +import com.github.fommil.netlib.BLAS.{getInstance => blas} + import org.apache.spark.rdd._ import org.apache.spark.graphx._ @@ -53,7 +55,7 @@ object SVDPlusPlus { * a Multifaceted Collaborative Filtering Model", * available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. * - * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), + * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^^-0.5^^*sum(y)), * see the details on page 6. * * @param edges edges for constructing the graph @@ -66,13 +68,10 @@ object SVDPlusPlus { : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = { // Generate default vertex attribute - def defaultF(rank: Int): (DoubleMatrix, DoubleMatrix, Double, Double) = { - val v1 = new DoubleMatrix(rank) - val v2 = new DoubleMatrix(rank) - for (i <- 0 until rank) { - v1.put(i, Random.nextDouble()) - v2.put(i, Random.nextDouble()) - } + def defaultF(rank: Int): (Array[Double], Array[Double], Double, Double) = { + // TODO: use a fixed random seed + val v1 = Array.fill(rank)(Random.nextDouble()) + val v2 = Array.fill(rank)(Random.nextDouble()) (v1, v2, 0.0, 0.0) } @@ -92,7 +91,7 @@ object SVDPlusPlus { (g1, g2) => (g1._1 + g2._1, g1._2 + g2._2)) val gJoinT0 = g.outerJoinVertices(t0) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[(Long, Double)]) => (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) }.cache() @@ -102,24 +101,28 @@ object SVDPlusPlus { def sendMsgTrainF(conf: Conf, u: Double) (ctx: EdgeContext[ - (DoubleMatrix, DoubleMatrix, Double, Double), + (Array[Double], Array[Double], Double, Double), Double, - (DoubleMatrix, DoubleMatrix, Double)]) { + (Array[Double], Array[Double], Double)]) { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dot(usr._2) + val rank = p.length + var pred = u + usr._3 + itm._3 + blas.ddot(rank, q, 1, usr._2, 1) pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = ctx.attr - pred - val updateP = q.mul(err) - .subColumnVector(p.mul(conf.gamma7)) - .mul(conf.gamma2) - val updateQ = usr._2.mul(err) - .subColumnVector(q.mul(conf.gamma7)) - .mul(conf.gamma2) - val updateY = q.mul(err * usr._4) - .subColumnVector(itm._2.mul(conf.gamma7)) - .mul(conf.gamma2) + // updateP = (err * q - conf.gamma7 * p) * conf.gamma2 + val updateP = q.clone() + blas.dscal(rank, err * conf.gamma2, updateP, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, p, 1, updateP, 1) + // updateQ = (err * usr._2 - conf.gamma7 * q) * conf.gamma2 + val updateQ = usr._2.clone() + blas.dscal(rank, err * conf.gamma2, updateQ, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, q, 1, updateQ, 1) + // updateY = (err * usr._4 * q - conf.gamma7 * itm._2) * conf.gamma2 + val updateY = q.clone() + blas.dscal(rank, err * usr._4 * conf.gamma2, updateY, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, itm._2, 1, updateY, 1) ctx.sendToSrc((updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)) ctx.sendToDst((updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1)) } @@ -127,14 +130,23 @@ object SVDPlusPlus { for (i <- 0 until conf.maxIters) { // Phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes g.cache() - val t1 = g.aggregateMessages[DoubleMatrix]( + val t1 = g.aggregateMessages[Array[Double]]( ctx => ctx.sendToSrc(ctx.dstAttr._2), - (g1, g2) => g1.addColumnVector(g2)) + (g1, g2) => { + val out = g1.clone() + blas.daxpy(out.length, 1.0, g2, 1, out, 1) + out + }) val gJoinT1 = g.outerJoinVertices(t1) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), - msg: Option[DoubleMatrix]) => - if (msg.isDefined) (vd._1, vd._1 - .addColumnVector(msg.get.mul(vd._4)), vd._3, vd._4) else vd + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), + msg: Option[Array[Double]]) => + if (msg.isDefined) { + val out = vd._1.clone() + blas.daxpy(out.length, vd._4, msg.get, 1, out, 1) + (vd._1, out, vd._3, vd._4) + } else { + vd + } }.cache() materialize(gJoinT1) g.unpersist() @@ -144,14 +156,24 @@ object SVDPlusPlus { g.cache() val t2 = g.aggregateMessages( sendMsgTrainF(conf, u), - (g1: (DoubleMatrix, DoubleMatrix, Double), g2: (DoubleMatrix, DoubleMatrix, Double)) => - (g1._1.addColumnVector(g2._1), g1._2.addColumnVector(g2._2), g1._3 + g2._3)) + (g1: (Array[Double], Array[Double], Double), g2: (Array[Double], Array[Double], Double)) => + { + val out1 = g1._1.clone() + blas.daxpy(out1.length, 1.0, g2._1, 1, out1, 1) + val out2 = g2._2.clone() + blas.daxpy(out2.length, 1.0, g2._2, 1, out2, 1) + (out1, out2, g1._3 + g2._3) + }) val gJoinT2 = g.outerJoinVertices(t2) { (vid: VertexId, - vd: (DoubleMatrix, DoubleMatrix, Double, Double), - msg: Option[(DoubleMatrix, DoubleMatrix, Double)]) => - (vd._1.addColumnVector(msg.get._1), vd._2.addColumnVector(msg.get._2), - vd._3 + msg.get._3, vd._4) + vd: (Array[Double], Array[Double], Double, Double), + msg: Option[(Array[Double], Array[Double], Double)]) => { + val out1 = vd._1.clone() + blas.daxpy(out1.length, 1.0, msg.get._1, 1, out1, 1) + val out2 = vd._2.clone() + blas.daxpy(out2.length, 1.0, msg.get._2, 1, out2, 1) + (out1, out2, vd._3 + msg.get._3, vd._4) + } }.cache() materialize(gJoinT2) g.unpersist() @@ -160,10 +182,10 @@ object SVDPlusPlus { // calculate error on training set def sendMsgTestF(conf: Conf, u: Double) - (ctx: EdgeContext[(DoubleMatrix, DoubleMatrix, Double, Double), Double, Double]) { + (ctx: EdgeContext[(Array[Double], Array[Double], Double, Double), Double, Double]) { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dot(usr._2) + var pred = u + usr._3 + itm._3 + blas.ddot(q.length, q, 1, usr._2, 1) pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = (ctx.attr - pred) * (ctx.attr - pred) @@ -173,7 +195,7 @@ object SVDPlusPlus { g.cache() val t3 = g.aggregateMessages[Double](sendMsgTestF(conf, u), _ + _) val gJoinT3 = g.outerJoinVertices(t3) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[Double]) => + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd }.cache() materialize(gJoinT3) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala index 9987a4b1a3c25..7bd6b7f3c4ab2 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -32,11 +32,11 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations - var (graph, u) = SVDPlusPlus.runSVDPlusPlus(edges, conf) + val (graph, _) = SVDPlusPlus.run(edges, conf) graph.cache() - val err = graph.vertices.collect().map{ case (vid, vd) => + val err = graph.vertices.map { case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 - }.reduce(_ + _) / graph.triplets.collect().size + }.reduce(_ + _) / graph.numEdges assert(err <= svdppErr) } } diff --git a/mllib/pom.xml b/mllib/pom.xml index b5c949e155cfd..a76704a8c2c59 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -59,6 +59,7 @@ org.jblas jblas ${jblas.version} + test org.scalanlp @@ -116,7 +117,7 @@ com.github.fommil.netlib all - 1.1.2 + ${netlib.java.version} pom diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 7bb69df65362b..e3515ee81af3d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -26,7 +26,6 @@ import scala.util.hashing.byteswap64 import com.github.fommil.netlib.BLAS.{getInstance => blas} import com.github.fommil.netlib.LAPACK.{getInstance => lapack} -import org.jblas.DoubleMatrix import org.netlib.util.intW import org.apache.spark.{Logging, Partitioner} @@ -361,14 +360,14 @@ object ALS extends Logging { private[recommendation] class NNLSSolver extends LeastSquaresNESolver { private var rank: Int = -1 private var workspace: NNLS.Workspace = _ - private var ata: DoubleMatrix = _ + private var ata: Array[Double] = _ private var initialized: Boolean = false private def initialize(rank: Int): Unit = { if (!initialized) { this.rank = rank workspace = NNLS.createWorkspace(rank) - ata = new DoubleMatrix(rank, rank) + ata = new Array[Double](rank * rank) initialized = true } else { require(this.rank == rank) @@ -385,7 +384,7 @@ object ALS extends Logging { val rank = ne.k initialize(rank) fillAtA(ne.ata, lambda * ne.n) - val x = NNLS.solve(ata, new DoubleMatrix(rank, 1, ne.atb: _*), workspace) + val x = NNLS.solve(ata, ne.atb, workspace) ne.reset() x.map(x => x.toFloat) } @@ -398,17 +397,16 @@ object ALS extends Logging { var i = 0 var pos = 0 var a = 0.0 - val data = ata.data while (i < rank) { var j = 0 while (j <= i) { a = triAtA(pos) - data(i * rank + j) = a - data(j * rank + i) = a + ata(i * rank + j) = a + ata(j * rank + i) = a pos += 1 j += 1 } - data(i * rank + i) += lambda + ata(i * rank + i) += lambda i += 1 } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala index ccd93b318bc23..4766f7708295d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala @@ -17,7 +17,9 @@ package org.apache.spark.mllib.optimization -import org.jblas.{DoubleMatrix, SimpleBlas} +import java.{util => ju} + +import com.github.fommil.netlib.BLAS.{getInstance => blas} /** * Object used to solve nonnegative least squares problems using a modified @@ -25,20 +27,20 @@ import org.jblas.{DoubleMatrix, SimpleBlas} */ private[spark] object NNLS { class Workspace(val n: Int) { - val scratch = new DoubleMatrix(n, 1) - val grad = new DoubleMatrix(n, 1) - val x = new DoubleMatrix(n, 1) - val dir = new DoubleMatrix(n, 1) - val lastDir = new DoubleMatrix(n, 1) - val res = new DoubleMatrix(n, 1) - - def wipe() { - scratch.fill(0.0) - grad.fill(0.0) - x.fill(0.0) - dir.fill(0.0) - lastDir.fill(0.0) - res.fill(0.0) + val scratch = new Array[Double](n) + val grad = new Array[Double](n) + val x = new Array[Double](n) + val dir = new Array[Double](n) + val lastDir = new Array[Double](n) + val res = new Array[Double](n) + + def wipe(): Unit = { + ju.Arrays.fill(scratch, 0.0) + ju.Arrays.fill(grad, 0.0) + ju.Arrays.fill(x, 0.0) + ju.Arrays.fill(dir, 0.0) + ju.Arrays.fill(lastDir, 0.0) + ju.Arrays.fill(res, 0.0) } } @@ -60,18 +62,18 @@ private[spark] object NNLS { * direction, however, while this method only uses a conjugate gradient direction if the last * iteration did not cause a previously-inactive constraint to become active. */ - def solve(ata: DoubleMatrix, atb: DoubleMatrix, ws: Workspace): Array[Double] = { + def solve(ata: Array[Double], atb: Array[Double], ws: Workspace): Array[Double] = { ws.wipe() - val n = atb.rows + val n = atb.length val scratch = ws.scratch // find the optimal unconstrained step - def steplen(dir: DoubleMatrix, res: DoubleMatrix): Double = { - val top = SimpleBlas.dot(dir, res) - SimpleBlas.gemv(1.0, ata, dir, 0.0, scratch) + def steplen(dir: Array[Double], res: Array[Double]): Double = { + val top = blas.ddot(n, dir, 1, res, 1) + blas.dgemv("N", n, n, 1.0, ata, n, dir, 1, 0.0, scratch, 1) // Push the denominator upward very slightly to avoid infinities and silliness - top / (SimpleBlas.dot(scratch, dir) + 1e-20) + top / (blas.ddot(n, scratch, 1, dir, 1) + 1e-20) } // stopping condition @@ -96,52 +98,52 @@ private[spark] object NNLS { var i = 0 while (iterno < iterMax) { // find the residual - SimpleBlas.gemv(1.0, ata, x, 0.0, res) - SimpleBlas.axpy(-1.0, atb, res) - SimpleBlas.copy(res, grad) + blas.dgemv("N", n, n, 1.0, ata, n, x, 1, 0.0, res, 1) + blas.daxpy(n, -1.0, atb, 1, res, 1) + blas.dcopy(n, res, 1, grad, 1) // project the gradient i = 0 while (i < n) { - if (grad.data(i) > 0.0 && x.data(i) == 0.0) { - grad.data(i) = 0.0 + if (grad(i) > 0.0 && x(i) == 0.0) { + grad(i) = 0.0 } i = i + 1 } - val ngrad = SimpleBlas.dot(grad, grad) + val ngrad = blas.ddot(n, grad, 1, grad, 1) - SimpleBlas.copy(grad, dir) + blas.dcopy(n, grad, 1, dir, 1) // use a CG direction under certain conditions var step = steplen(grad, res) var ndir = 0.0 - val nx = SimpleBlas.dot(x, x) + val nx = blas.ddot(n, x, 1, x, 1) if (iterno > lastWall + 1) { val alpha = ngrad / lastNorm - SimpleBlas.axpy(alpha, lastDir, dir) + blas.daxpy(n, alpha, lastDir, 1, dir, 1) val dstep = steplen(dir, res) - ndir = SimpleBlas.dot(dir, dir) + ndir = blas.ddot(n, dir, 1, dir, 1) if (stop(dstep, ndir, nx)) { // reject the CG step if it could lead to premature termination - SimpleBlas.copy(grad, dir) - ndir = SimpleBlas.dot(dir, dir) + blas.dcopy(n, grad, 1, dir, 1) + ndir = blas.ddot(n, dir, 1, dir, 1) } else { step = dstep } } else { - ndir = SimpleBlas.dot(dir, dir) + ndir = blas.ddot(n, dir, 1, dir, 1) } // terminate? if (stop(step, ndir, nx)) { - return x.data.clone + return x.clone } // don't run through the walls i = 0 while (i < n) { - if (step * dir.data(i) > x.data(i)) { - step = x.data(i) / dir.data(i) + if (step * dir(i) > x(i)) { + step = x(i) / dir(i) } i = i + 1 } @@ -149,19 +151,19 @@ private[spark] object NNLS { // take the step i = 0 while (i < n) { - if (step * dir.data(i) > x.data(i) * (1 - 1e-14)) { - x.data(i) = 0 + if (step * dir(i) > x(i) * (1 - 1e-14)) { + x(i) = 0 lastWall = iterno } else { - x.data(i) -= step * dir.data(i) + x(i) -= step * dir(i) } i = i + 1 } iterno = iterno + 1 - SimpleBlas.copy(dir, lastDir) + blas.dcopy(n, dir, 1, lastDir, 1) lastNorm = ngrad } - x.data.clone + x.clone } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 5f5a996a87b81..36cbf060d9998 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -21,10 +21,10 @@ import java.io.IOException import java.lang.{Integer => JavaInteger} import org.apache.hadoop.fs.Path -import org.jblas.DoubleMatrix import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.{Logging, SparkContext} import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} @@ -70,9 +70,9 @@ class MatrixFactorizationModel( /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { - val userVector = new DoubleMatrix(userFeatures.lookup(user).head) - val productVector = new DoubleMatrix(productFeatures.lookup(product).head) - userVector.dot(productVector) + val userVector = userFeatures.lookup(user).head + val productVector = productFeatures.lookup(product).head + blas.ddot(userVector.length, userVector, 1, productVector, 1) } /** @@ -89,9 +89,7 @@ class MatrixFactorizationModel( } users.join(productFeatures).map { case (product, ((user, uFeatures), pFeatures)) => - val userVector = new DoubleMatrix(uFeatures) - val productVector = new DoubleMatrix(pFeatures) - Rating(user, product, userVector.dot(productVector)) + Rating(user, product, blas.ddot(uFeatures.length, uFeatures, 1, pFeatures, 1)) } } @@ -143,9 +141,8 @@ class MatrixFactorizationModel( recommendToFeatures: Array[Double], recommendableFeatures: RDD[(Int, Array[Double])], num: Int): Array[(Int, Double)] = { - val recommendToVector = new DoubleMatrix(recommendToFeatures) val scored = recommendableFeatures.map { case (id,features) => - (id, recommendToVector.dot(new DoubleMatrix(features))) + (id, blas.ddot(features.length, recommendToFeatures, 1, features, 1)) } scored.top(num)(Ordering.by(_._2)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 97f54aa62d31c..c9d33787b0bb5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.util import scala.collection.JavaConversions._ import scala.util.Random -import org.jblas.DoubleMatrix +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext @@ -72,11 +72,10 @@ object LinearDataGenerator { eps: Double = 0.1): Seq[LabeledPoint] = { val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)( Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0)) val y = x.map { xi => - new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + intercept + eps * rnd.nextGaussian() + blas.ddot(weights.length, xi, 1, weights, 1) + intercept + eps * rnd.nextGaussian() } y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } @@ -100,9 +99,9 @@ object LinearDataGenerator { eps: Double, nparts: Int = 2, intercept: Double = 0.0) : RDD[LabeledPoint] = { - org.jblas.util.Random.seed(42) + val random = new Random(42) // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + val w = Array.fill(nfeatures)(random.nextDouble() - 0.5) val data: RDD[LabeledPoint] = sc.parallelize(0 until nparts, nparts).flatMap { p => val seed = 42 + p diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index b76fbe89c3681..0c5b4f9d04a74 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -17,13 +17,14 @@ package org.apache.spark.mllib.util +import java.{util => ju} + import scala.language.postfixOps import scala.util.Random -import org.jblas.DoubleMatrix - -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix} import org.apache.spark.rdd.RDD /** @@ -72,24 +73,25 @@ object MFDataGenerator { val sc = new SparkContext(sparkMaster, "MFDataGenerator") - val A = DoubleMatrix.randn(m, rank) - val B = DoubleMatrix.randn(rank, n) - val z = 1 / scala.math.sqrt(scala.math.sqrt(rank)) - A.mmuli(z) - B.mmuli(z) - val fullData = A.mmul(B) + val random = new ju.Random(42L) + + val A = DenseMatrix.randn(m, rank, random) + val B = DenseMatrix.randn(rank, n, random) + val z = 1 / math.sqrt(rank) + val fullData = DenseMatrix.zeros(m, n) + BLAS.gemm(z, A, B, 1.0, fullData) val df = rank * (m + n - rank) val sampSize = scala.math.min(scala.math.round(trainSampFact * df), scala.math.round(.99 * m * n)).toInt val rand = new Random() val mn = m * n - val shuffled = rand.shuffle(1 to mn toList) + val shuffled = rand.shuffle((0 until mn).toList) val omega = shuffled.slice(0, sampSize) val ordered = omega.sortWith(_ < _).toArray val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + .map(x => (x % m, x / m, fullData.values(x))) // optionally add gaussian noise if (noise) { @@ -105,7 +107,7 @@ object MFDataGenerator { val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) val testOrdered = testOmega.sortWith(_ < _).toArray val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + .map(x => (x % m, x / m, fullData.values(x))) testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 7db97e6bac688..a8e30cc9d730c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.jblas.DoubleMatrix +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext @@ -51,8 +51,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures + 1, - Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()):_*) + val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -60,7 +59,7 @@ object SVMDataGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val yD = new DoubleMatrix(1, x.length, x: _*).dot(trueWeights) + rnd.nextGaussian() * 0.1 + val yD = blas.ddot(trueWeights.length, x, 1, trueWeights, 1) + rnd.nextGaussian() * 0.1 val y = if (yD < 0) 0.0 else 1.0 LabeledPoint(y, Vectors.dense(x)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index 82c327bd49fcd..22855e4e8f247 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -55,7 +55,7 @@ class NNLSSuite extends FunSuite { for (k <- 0 until 100) { val (ata, atb) = genOnesData(n, rand) - val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) assert(x.length === n) val answer = DoubleMatrix.ones(n, 1) SimpleBlas.axpy(-1.0, answer, x) @@ -79,7 +79,7 @@ class NNLSSuite extends FunSuite { val goodx = Array(0.13025, 0.54506, 0.2874, 0.0, 0.028628) val ws = NNLS.createWorkspace(n) - val x = NNLS.solve(ata, atb, ws) + val x = NNLS.solve(ata.data, atb.data, ws) for (i <- 0 until n) { assert(x(i) ~== goodx(i) absTol 1E-3) assert(x(i) >= 0) @@ -104,7 +104,7 @@ class NNLSSuite extends FunSuite { val ws = NNLS.createWorkspace(n) - val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) val obj = computeObjectiveValue(ata, atb, x) assert(obj < refObj + 1E-5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala index f6a1e19f50296..16ecae23dd9d4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala @@ -21,9 +21,9 @@ import org.scalatest.FunSuite import org.apache.commons.math3.distribution.NormalDistribution -import org.apache.spark.mllib.util.LocalClusterSparkContext +import org.apache.spark.mllib.util.MLlibTestSparkContext -class KernelDensitySuite extends FunSuite with LocalClusterSparkContext { +class KernelDensitySuite extends FunSuite with MLlibTestSparkContext { test("kernel density single sample") { val rdd = sc.parallelize(Array(5.0)) val evaluationPoints = Array(5.0, 6.0) diff --git a/pom.xml b/pom.xml index a19da73cf45b3..6fc56a86d44ac 100644 --- a/pom.xml +++ b/pom.xml @@ -157,6 +157,7 @@ 1.8.8 2.4.4 1.1.1.6 + 1.1.2 [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5001) Author: Cheng Lian Closes #5001 from liancheng/parquet-doc and squashes the following commits: 89ad3db [Cheng Lian] Addresses @rxin's comments 7eb6955 [Cheng Lian] Docs for the new Parquet data source 415eefb [Cheng Lian] Some minor formatting improvements --- docs/sql-programming-guide.md | 237 ++++++++++++++++++++++++++-------- 1 file changed, 180 insertions(+), 57 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 76aa1a533d56e..11c29e20632ae 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -21,14 +21,14 @@ The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark. All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell` or the `pyspark` shell. -## Starting Point: SQLContext +## Starting Point: `SQLContext`
The entry point into all functionality in Spark SQL is the -[SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/scala/index.html#org.apache.spark.sql.`SQLContext`) class, or one of its +descendants. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. @@ -43,8 +43,8 @@ import sqlContext.implicits._
The entry point into all functionality in Spark SQL is the -[SQLContext](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its +descendants. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight java %} JavaSparkContext sc = ...; // An existing JavaSparkContext. @@ -56,8 +56,8 @@ SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
The entry point into all relational functionality in Spark is the -[SQLContext](api/python/pyspark.sql.SQLContext-class.html) class, or one -of its decedents. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/python/pyspark.sql.SQLContext-class.html) class, or one +of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight python %} from pyspark.sql import SQLContext @@ -67,20 +67,20 @@ sqlContext = SQLContext(sc)
-In addition to the basic SQLContext, you can also create a HiveContext, which provides a -superset of the functionality provided by the basic SQLContext. Additional features include +In addition to the basic `SQLContext`, you can also create a `HiveContext`, which provides a +superset of the functionality provided by the basic `SQLContext`. Additional features include the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the -ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing Hive setup, and all of the data sources available to a SQLContext are still available. -HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default -Spark build. If these dependencies are not a problem for your application then using HiveContext -is recommended for the 1.3 release of Spark. Future releases will focus on bringing SQLContext up -to feature parity with a HiveContext. +ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an +existing Hive setup, and all of the data sources available to a `SQLContext` are still available. +`HiveContext` is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using `HiveContext` +is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up +to feature parity with a `HiveContext`. The specific variant of SQL that is used to parse queries can also be selected using the `spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on -a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect -available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the +a `SQLContext` or by using a `SET key=value` command in SQL. For a `SQLContext`, the only dialect +available is "sql" which uses a simple SQL parser provided by Spark SQL. In a `HiveContext`, the default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete, this is recommended for most use cases. @@ -100,7 +100,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) val df = sqlContext.jsonFile("examples/src/main/resources/people.json") // Displays the content of the DataFrame to stdout -df.show() +df.show() {% endhighlight %}
@@ -151,10 +151,10 @@ val df = sqlContext.jsonFile("examples/src/main/resources/people.json") // Show the content of the DataFrame df.show() -// age name +// age name // null Michael -// 30 Andy -// 19 Justin +// 30 Andy +// 19 Justin // Print the schema in a tree format df.printSchema() @@ -164,17 +164,17 @@ df.printSchema() // Select only the "name" column df.select("name").show() -// name +// name // Michael -// Andy -// Justin +// Andy +// Justin // Select everybody, but increment the age by 1 df.select("name", df("age") + 1).show() // name (age + 1) -// Michael null -// Andy 31 -// Justin 20 +// Michael null +// Andy 31 +// Justin 20 // Select people older than 21 df.filter(df("name") > 21).show() @@ -201,10 +201,10 @@ DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); // Show the content of the DataFrame df.show(); -// age name +// age name // null Michael -// 30 Andy -// 19 Justin +// 30 Andy +// 19 Justin // Print the schema in a tree format df.printSchema(); @@ -214,17 +214,17 @@ df.printSchema(); // Select only the "name" column df.select("name").show(); -// name +// name // Michael -// Andy -// Justin +// Andy +// Justin // Select everybody, but increment the age by 1 df.select("name", df.col("age").plus(1)).show(); // name (age + 1) -// Michael null -// Andy 31 -// Justin 20 +// Michael null +// Andy 31 +// Justin 20 // Select people older than 21 df.filter(df("name") > 21).show(); @@ -251,10 +251,10 @@ df = sqlContext.jsonFile("examples/src/main/resources/people.json") # Show the content of the DataFrame df.show() -## age name +## age name ## null Michael -## 30 Andy -## 19 Justin +## 30 Andy +## 19 Justin # Print the schema in a tree format df.printSchema() @@ -264,17 +264,17 @@ df.printSchema() # Select only the "name" column df.select("name").show() -## name +## name ## Michael -## Andy -## Justin +## Andy +## Justin # Select everybody, but increment the age by 1 df.select("name", df.age + 1).show() ## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 +## Michael null +## Andy 31 +## Justin 20 # Select people older than 21 df.filter(df.name > 21).show() @@ -797,7 +797,7 @@ When working with a `HiveContext`, `DataFrames` can also be saved as persistent contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables will still exist even after your Spark program has restarted, as long as you maintain your connection to the same metastore. A DataFrame for a persistent table can be created by calling the `table` -method on a SQLContext with the name of the table. +method on a `SQLContext` with the name of the table. By default `saveAsTable` will create a "managed table", meaning that the location of the data will be controlled by the metastore. Managed tables will also have their data deleted automatically @@ -907,9 +907,132 @@ SELECT * FROM parquetTable
+### Partition discovery + +Table partitioning is a common optimization approach used in systems like Hive. In a partitioned +table, data are usually stored in different directories, with partitioning column values encoded in +the path of each partition directory. The Parquet data source is now able to discover and infer +partitioning information automatically. For exmaple, we can store all our previously used +population data into a partitioned table using the following directory structure, with two extra +columns, `gender` and `country` as partitioning columns: + +{% highlight text %} + +path +└── to + └── table + ├── gender=male + │   ├── ... + │   │ + │   ├── country=US + │   │   └── data.parquet + │   ├── country=CN + │   │   └── data.parquet + │   └── ... + └── gender=female +    ├── ... +    │ +    ├── country=US +    │   └── data.parquet +    ├── country=CN +    │   └── data.parquet +    └── ... + +{% endhighlight %} + +By passing `path/to/table` to either `SQLContext.parquetFile` or `SQLContext.load`, Spark SQL will +automatically extract the partitioning information from the paths. Now the schema of the returned +DataFrame becomes: + +{% highlight text %} + +root +|-- name: string (nullable = true) +|-- age: long (nullable = true) +|-- gender: string (nullable = true) +|-- country: string (nullable = true) + +{% endhighlight %} + +Notice that the data types of the partitioning columns are automatically inferred. Currently, +numeric data types and string type are supported. + +### Schema merging + +Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple Parquet files with different but mutually compatible schemas. The Parquet data +source is now able to automatically detect this case and merge schemas of all these files. + +
+ +
+ +{% highlight scala %} +// sqlContext from the previous example is used in this example. +// This is used to implicitly convert an RDD to a DataFrame. +import sqlContext.implicits._ + +// Create a simple DataFrame, stored into a partition directory +val df1 = sparkContext.makeRDD(1 to 5).map(i => (i, i * 2)).toDF("single", "double") +df1.saveAsParquetFile("data/test_table/key=1") + +// Create another DataFrame in a new partition directory, +// adding a new column and dropping an existing column +val df2 = sparkContext.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple") +df2.saveAsParquetFile("data/test_table/key=2") + +// Read the partitioned table +val df3 = sqlContext.parquetFile("data/test_table") +df3.printSchema() + +// The final schema consists of all 3 columns in the Parquet files together +// with the partiioning column appeared in the partition directory paths. +// root +// |-- single: int (nullable = true) +// |-- double: int (nullable = true) +// |-- triple: int (nullable = true) +// |-- key : int (nullable = true) +{% endhighlight %} + +
+ +
+ +{% highlight python %} +# sqlContext from the previous example is used in this example. + +# Create a simple DataFrame, stored into a partition directory +df1 = sqlContext.createDataFrame(sc.parallelize(range(1, 6))\ + .map(lambda i: Row(single=i, double=i * 2))) +df1.save("data/test_table/key=1", "parquet") + +# Create another DataFrame in a new partition directory, +# adding a new column and dropping an existing column +df2 = sqlContext.createDataFrame(sc.parallelize(range(6, 11)) + .map(lambda i: Row(single=i, triple=i * 3))) +df2.save("data/test_table/key=2", "parquet") + +# Read the partitioned table +df3 = sqlContext.parquetFile("data/test_table") +df3.printSchema() + +# The final schema consists of all 3 columns in the Parquet files together +# with the partiioning column appeared in the partition directory paths. +# root +# |-- single: int (nullable = true) +# |-- double: int (nullable = true) +# |-- triple: int (nullable = true) +# |-- key : int (nullable = true) +{% endhighlight %} + +
+ +
+ ### Configuration -Configuration of Parquet can be done using the `setConf` method on SQLContext or by running +Configuration of Parquet can be done using the `setConf` method on `SQLContext` or by running `SET key=value` commands using SQL. @@ -972,7 +1095,7 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or
Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext: +This conversion can be done using one of two methods in a `SQLContext`: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1014,7 +1137,7 @@ val anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD)
Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext : +This conversion can be done using one of two methods in a `SQLContext` : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1056,7 +1179,7 @@ DataFrame anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD);
Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext: +This conversion can be done using one of two methods in a `SQLContext`: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1085,7 +1208,7 @@ people.printSchema() # Register this DataFrame as a table. people.registerTempTable("people") -# SQL statements can be run by using the sql methods provided by sqlContext. +# SQL statements can be run by using the sql methods provided by `sqlContext`. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Alternatively, a DataFrame can be created for a JSON dataset represented by @@ -1131,7 +1254,7 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can still create a HiveContext. When not configured by the +not have an existing Hive deployment can still create a `HiveContext`. When not configured by the hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current directory. @@ -1318,7 +1441,7 @@ Spark SQL can cache tables using an in-memory columnar format by calling `sqlCon Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running +Configuration of in-memory caching can be done using the `setConf` method on `SQLContext` or by running `SET key=value` commands using SQL.
@@ -1429,10 +1552,10 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script that comes with Hive. -Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. -Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: - hive.server2.transport.mode - Set this to value: http + hive.server2.transport.mode - Set this to value: http hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001 hive.server2.http.endpoint - HTTP endpoint; default is cliservice @@ -1506,7 +1629,7 @@ When using function inside of the DSL (now replaced with the `DataFrame` API) us Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users should instead import the classes in `org.apache.spark.sql.types` -#### UDF Registration Moved to sqlContext.udf (Java & Scala) +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been moved into the udf object in `SQLContext`. From 9048e8102e3f564842fa0dc6e82edce70b7dd3d7 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 13 Mar 2015 13:59:54 +0000 Subject: [PATCH 13/62] [SPARK-6197][CORE] handle json exception when hisotry file not finished writing For details, please refer to [SPARK-6197](https://issues.apache.org/jira/browse/SPARK-6197) Author: Zhang, Liye Closes #4927 from liyezhang556520/jsonParseError and squashes the following commits: 5cbdc82 [Zhang, Liye] without unnecessary wrap 2b48831 [Zhang, Liye] small changes with sean owen's comments 2973024 [Zhang, Liye] handle json exception when file not finished writing --- .../apache/spark/deploy/master/Master.scala | 3 ++- .../spark/scheduler/ReplayListenerBus.scala | 25 ++++++++++++++++--- 2 files changed, 23 insertions(+), 5 deletions(-) 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 15814293227ab..22935c9b1d394 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 @@ -764,8 +764,9 @@ private[spark] class Master( val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS) try { - replayBus.replay(logInput, eventLogFile) + replayBus.replay(logInput, eventLogFile, maybeTruncated) } finally { logInput.close() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 95273c716b3e2..86f357abb8723 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -21,6 +21,7 @@ import java.io.{InputStream, IOException} import scala.io.Source +import com.fasterxml.jackson.core.JsonParseException import org.json4s.jackson.JsonMethods._ import org.apache.spark.Logging @@ -40,15 +41,31 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * * @param logData Stream containing event log data. * @param sourceName Filename (or other source identifier) from whence @logData is being read + * @param maybeTruncated Indicate whether log file might be truncated (some abnormal situations + * encountered, log file might not finished writing) or not */ - def replay(logData: InputStream, sourceName: String): Unit = { + def replay( + logData: InputStream, + sourceName: String, + maybeTruncated: Boolean = false): Unit = { var currentLine: String = null var lineNumber: Int = 1 try { val lines = Source.fromInputStream(logData).getLines() - lines.foreach { line => - currentLine = line - postToAll(JsonProtocol.sparkEventFromJson(parse(line))) + while (lines.hasNext) { + currentLine = lines.next() + try { + postToAll(JsonProtocol.sparkEventFromJson(parse(currentLine))) + } catch { + case jpe: JsonParseException => + // We can only ignore exception from last line of the file that might be truncated + if (!maybeTruncated || lines.hasNext) { + throw jpe + } else { + logWarning(s"Got JsonParseException from log file $sourceName" + + s" at line $lineNumber, the file might not have finished writing cleanly.") + } + } lineNumber += 1 } } catch { From ea3d2eed9b0a94b34543d9a9df87dc63a279deb1 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 13 Mar 2015 14:08:56 +0000 Subject: [PATCH 14/62] [CORE][minor] remove unnecessary ClassTag in `DAGScheduler` This existed at the very beginning, but became unnecessary after [this commit](https://github.com/apache/spark/commit/37d8f37a8ec110416fba0d51d8ba70370ac380c1#diff-6a9ff7fb74fd490a50462d45db2d5e11L272). I think we should remove it if we don't plan to use it in the future. Author: Wenchen Fan Closes #4992 from cloud-fan/small and squashes the following commits: e857f2e [Wenchen Fan] remove unnecessary ClassTag --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 bc84e2351ad74..e4170a55b7981 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -26,7 +26,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps -import scala.reflect.ClassTag import scala.util.control.NonFatal import akka.pattern.ask @@ -497,7 +496,7 @@ class DAGScheduler( waiter } - def runJob[T, U: ClassTag]( + def runJob[T, U]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], From dc4abd4dc40deacab39bfa9572b06bf0ea6daa6d Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 13 Mar 2015 10:26:09 -0700 Subject: [PATCH 15/62] [SPARK-6252] [mllib] Added getLambda to Scala NaiveBayes Note: not relevant for Python API since it only has a static train method Author: Joseph K. Bradley Author: Joseph K. Bradley Closes #4969 from jkbradley/SPARK-6252 and squashes the following commits: a471d90 [Joseph K. Bradley] small edits from review 63eff48 [Joseph K. Bradley] Added getLambda to Scala NaiveBayes --- .../apache/spark/mllib/classification/NaiveBayes.scala | 3 +++ .../spark/mllib/classification/NaiveBayesSuite.scala | 8 ++++++++ 2 files changed, 11 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index b11fd4f128c56..2ebc7fa5d4234 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -166,6 +166,9 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with this } + /** Get the smoothing parameter. Default: 1.0. */ + def getLambda: Double = lambda + /** * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. * diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 64dcc0fb9f82c..5a27c7d2309c5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -85,6 +85,14 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { assert(numOfPredictions < input.length / 5) } + test("get, set params") { + val nb = new NaiveBayes() + nb.setLambda(2.0) + assert(nb.getLambda === 2.0) + nb.setLambda(3.0) + assert(nb.getLambda === 3.0) + } + test("Naive Bayes") { val nPoints = 10000 From 7f13434a5c52b815c584ec773ab0e5df1a35ea86 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 13 Mar 2015 10:27:28 -0700 Subject: [PATCH 16/62] [SPARK-6278][MLLIB] Mention the change of objective in linear regression As discussed in the RC3 vote thread, we should mention the change of objective in linear regression in the migration guide. srowen Author: Xiangrui Meng Closes #4978 from mengxr/SPARK-6278 and squashes the following commits: fb3bbe6 [Xiangrui Meng] mention regularization parameter bfd6cff [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-6278 375fd09 [Xiangrui Meng] address Sean's comments f87ae71 [Xiangrui Meng] mention step size change --- docs/mllib-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 598374f66df5e..f8e879496c135 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -102,6 +102,8 @@ In the `spark.mllib` package, there were several breaking changes. The first ch * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. * `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. +* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. + So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. ## Previous Spark Versions From b943f5d907df0607ecffb729f2bccfa436438d7e Mon Sep 17 00:00:00 2001 From: Brennon York Date: Fri, 13 Mar 2015 18:48:31 +0000 Subject: [PATCH 17/62] [SPARK-4600][GraphX]: org.apache.spark.graphx.VertexRDD.diff does not work Turns out, per the [convo on the JIRA](https://issues.apache.org/jira/browse/SPARK-4600), `diff` is acting exactly as should. It became a large misconception as I thought it meant set difference, when in fact it does not. To that extent I merely updated the `diff` documentation to, hopefully, better reflect its true intentions moving forward. Author: Brennon York Closes #5015 from brennonyork/SPARK-4600 and squashes the following commits: 1e1d1e5 [Brennon York] reverted internal diff docs 92288f7 [Brennon York] reverted both the test suite and the diff function back to its origin functionality f428623 [Brennon York] updated diff documentation to better represent its function cc16d65 [Brennon York] Merge remote-tracking branch 'upstream/master' into SPARK-4600 66818b9 [Brennon York] added small secondary diff test 99ad412 [Brennon York] Merge remote-tracking branch 'upstream/master' into SPARK-4600 74b8c95 [Brennon York] corrected method by leveraging bitmask operations to correctly return only the portions of that are different from the calling VertexRDD 9717120 [Brennon York] updated diff impl to cause fewer objects to be created 710a21c [Brennon York] working diff given test case aa57f83 [Brennon York] updated to set ShortestPaths to run 'forward' rather than 'backward' --- .../src/main/scala/org/apache/spark/graphx/VertexRDD.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 09ae3f9f6c09b..40ecff7107109 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -122,8 +122,11 @@ abstract class VertexRDD[VD]( def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] /** - * Hides vertices that are the same between `this` and `other`; for vertices that are different, - * keeps the values from `other`. + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is + * only guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other the other VertexRDD with which to diff against. */ def diff(other: VertexRDD[VD]): VertexRDD[VD] From cdc34ed9108688fea32ad170b1ba344fe047716b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 14 Mar 2015 07:09:53 +0800 Subject: [PATCH 18/62] [SPARK-6285] [SQL] Removes unused ParquetTestData and duplicated TestGroupWriteSupport All the contents in this file are not referenced anywhere and should have been removed in #4116 when I tried to get rid of the old Parquet test suites. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5010) Author: Cheng Lian Closes #5010 from liancheng/spark-6285 and squashes the following commits: 06ed057 [Cheng Lian] Removes unused ParquetTestData and duplicated TestGroupWriteSupport --- .../spark/sql/parquet/ParquetTestData.scala | 466 ------------------ 1 file changed, 466 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala deleted file mode 100644 index e4a10aa2ae6c3..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ /dev/null @@ -1,466 +0,0 @@ -/* - * 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.parquet - -import java.io.File - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.mapreduce.Job -import org.apache.spark.sql.test.TestSQLContext - -import parquet.example.data.{GroupWriter, Group} -import parquet.example.data.simple.{NanoTime, SimpleGroup} -import parquet.hadoop.{ParquetReader, ParquetFileReader, ParquetWriter} -import parquet.hadoop.api.WriteSupport -import parquet.hadoop.api.WriteSupport.WriteContext -import parquet.hadoop.example.GroupReadSupport -import parquet.hadoop.util.ContextUtil -import parquet.io.api.RecordConsumer -import parquet.schema.{MessageType, MessageTypeParser} - -import org.apache.spark.util.Utils - -// Write support class for nested groups: ParquetWriter initializes GroupWriteSupport -// with an empty configuration (it is after all not intended to be used in this way?) -// and members are private so we need to make our own in order to pass the schema -// to the writer. -private class TestGroupWriteSupport(schema: MessageType) extends WriteSupport[Group] { - var groupWriter: GroupWriter = null - override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { - groupWriter = new GroupWriter(recordConsumer, schema) - } - override def init(configuration: Configuration): WriteContext = { - new WriteContext(schema, new java.util.HashMap[String, String]()) - } - override def write(record: Group) { - groupWriter.write(record) - } -} - -private[sql] object ParquetTestData { - - val testSchema = - """message myrecord { - optional boolean myboolean; - optional int32 myint; - optional binary mystring (UTF8); - optional int64 mylong; - optional float myfloat; - optional double mydouble; - optional int96 mytimestamp; - }""" - - // field names for test assertion error messages - val testSchemaFieldNames = Seq( - "myboolean:Boolean", - "myint:Int", - "mystring:String", - "mylong:Long", - "myfloat:Float", - "mydouble:Double", - "mytimestamp:Timestamp" - ) - - val subTestSchema = - """ - message myrecord { - optional boolean myboolean; - optional int64 mylong; - } - """ - - val testFilterSchema = - """ - message myrecord { - required boolean myboolean; - required int32 myint; - required binary mystring (UTF8); - required int64 mylong; - required float myfloat; - required double mydouble; - optional boolean myoptboolean; - optional int32 myoptint; - optional binary myoptstring (UTF8); - optional int64 myoptlong; - optional float myoptfloat; - optional double myoptdouble; - optional int96 mytimestamp; - } - """ - - // field names for test assertion error messages - val subTestSchemaFieldNames = Seq( - "myboolean:Boolean", - "mylong:Long" - ) - - val testDir = Utils.createTempDir() - val testFilterDir = Utils.createTempDir() - - lazy val testData = new ParquetRelation(testDir.toURI.toString, None, TestSQLContext) - - val testNestedSchema1 = - // based on blogpost example, source: - // https://blog.twitter.com/2013/dremel-made-simple-with-parquet - // note: instead of string we have to use binary (?) otherwise - // Parquet gives us: - // IllegalArgumentException: expected one of [INT64, INT32, BOOLEAN, - // BINARY, FLOAT, DOUBLE, INT96, FIXED_LEN_BYTE_ARRAY] - // Also repeated primitives seem tricky to convert (AvroParquet - // only uses them in arrays?) so only use at most one in each group - // and nothing else in that group (-> is mapped to array)! - // The "values" inside ownerPhoneNumbers is a keyword currently - // so that array types can be translated correctly. - """ - message AddressBook { - required binary owner (UTF8); - optional group ownerPhoneNumbers { - repeated binary array (UTF8); - } - optional group contacts { - repeated group array { - required binary name (UTF8); - optional binary phoneNumber (UTF8); - } - } - } - """ - - - val testNestedSchema2 = - """ - message TestNested2 { - required int32 firstInt; - optional int32 secondInt; - optional group longs { - repeated int64 array; - } - required group entries { - repeated group array { - required double value; - optional boolean truth; - } - } - optional group outerouter { - repeated group array { - repeated group array { - repeated int32 array; - } - } - } - } - """ - - val testNestedSchema3 = - """ - message TestNested3 { - required int32 x; - optional group booleanNumberPairs { - repeated group array { - required int32 key; - optional group value { - repeated group array { - required double nestedValue; - optional boolean truth; - } - } - } - } - } - """ - - val testNestedSchema4 = - """ - message TestNested4 { - required int32 x; - optional group data1 { - repeated group map { - required binary key (UTF8); - required int32 value; - } - } - required group data2 { - repeated group map { - required binary key (UTF8); - required group value { - required int64 payload1; - optional binary payload2 (UTF8); - } - } - } - } - """ - - val testNestedDir1 = Utils.createTempDir() - val testNestedDir2 = Utils.createTempDir() - val testNestedDir3 = Utils.createTempDir() - val testNestedDir4 = Utils.createTempDir() - - lazy val testNestedData1 = - new ParquetRelation(testNestedDir1.toURI.toString, None, TestSQLContext) - lazy val testNestedData2 = - new ParquetRelation(testNestedDir2.toURI.toString, None, TestSQLContext) - - def writeFile() = { - testDir.delete() - val path: Path = new Path(new Path(testDir.toURI), new Path("part-r-0.parquet")) - val job = new Job() - val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - - for(i <- 0 until 15) { - val record = new SimpleGroup(schema) - if (i % 3 == 0) { - record.add(0, true) - } else { - record.add(0, false) - } - if (i % 5 == 0) { - record.add(1, 5) - } - record.add(2, "abc") - record.add(3, i.toLong << 33) - record.add(4, 2.5F) - record.add(5, 4.5D) - record.add(6, new NanoTime(1,2)) - writer.write(record) - } - writer.close() - } - - def writeFilterFile(records: Int = 200) = { - // for microbenchmark use: records = 300000000 - testFilterDir.delete - val path: Path = new Path(new Path(testFilterDir.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testFilterSchema) - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - - for(i <- 0 to records) { - val record = new SimpleGroup(schema) - if (i % 4 == 0) { - record.add(0, true) - } else { - record.add(0, false) - } - record.add(1, i) - record.add(2, i.toString) - record.add(3, i.toLong) - record.add(4, i.toFloat + 0.5f) - record.add(5, i.toDouble + 0.5d) - if (i % 2 == 0) { - if (i % 3 == 0) { - record.add(6, true) - } else { - record.add(6, false) - } - record.add(7, i) - record.add(8, i.toString) - record.add(9, i.toLong) - record.add(10, i.toFloat + 0.5f) - record.add(11, i.toDouble + 0.5d) - } - - writer.write(record) - } - writer.close() - } - - def writeNestedFile1() { - // example data from https://blog.twitter.com/2013/dremel-made-simple-with-parquet - testNestedDir1.delete() - val path: Path = new Path(new Path(testNestedDir1.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema1) - - val r1 = new SimpleGroup(schema) - r1.add(0, "Julien Le Dem") - r1.addGroup(1) - .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "555 123 4567") - .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "555 666 1337") - .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "XXX XXX XXXX") - val contacts = r1.addGroup(2) - contacts.addGroup(0) - .append("name", "Dmitriy Ryaboy") - .append("phoneNumber", "555 987 6543") - contacts.addGroup(0) - .append("name", "Chris Aniszczyk") - - val r2 = new SimpleGroup(schema) - r2.add(0, "A. Nonymous") - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.write(r2) - writer.close() - } - - def writeNestedFile2() { - testNestedDir2.delete() - val path: Path = new Path(new Path(testNestedDir2.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema2) - - val r1 = new SimpleGroup(schema) - r1.add(0, 1) - r1.add(1, 7) - val longs = r1.addGroup(2) - longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME , 1.toLong << 32) - longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 33) - longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 34) - val booleanNumberPair = r1.addGroup(3).addGroup(0) - booleanNumberPair.add("value", 2.5) - booleanNumberPair.add("truth", false) - val top_level = r1.addGroup(4) - val second_level_a = top_level.addGroup(0) - val second_level_b = top_level.addGroup(0) - val third_level_aa = second_level_a.addGroup(0) - val third_level_ab = second_level_a.addGroup(0) - val third_level_c = second_level_b.addGroup(0) - third_level_aa.add( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - 7) - third_level_ab.add( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - 8) - third_level_c.add( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - 9) - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.close() - } - - def writeNestedFile3() { - testNestedDir3.delete() - val path: Path = new Path(new Path(testNestedDir3.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema3) - - val r1 = new SimpleGroup(schema) - r1.add(0, 1) - val booleanNumberPairs = r1.addGroup(1) - val g1 = booleanNumberPairs.addGroup(0) - g1.add(0, 1) - val nested1 = g1.addGroup(1) - val ng1 = nested1.addGroup(0) - ng1.add(0, 1.5) - ng1.add(1, false) - val ng2 = nested1.addGroup(0) - ng2.add(0, 2.5) - ng2.add(1, true) - val g2 = booleanNumberPairs.addGroup(0) - g2.add(0, 2) - val ng3 = g2.addGroup(1) - .addGroup(0) - ng3.add(0, 3.5) - ng3.add(1, false) - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.close() - } - - def writeNestedFile4() { - testNestedDir4.delete() - val path: Path = new Path(new Path(testNestedDir4.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema4) - - val r1 = new SimpleGroup(schema) - r1.add(0, 7) - val map1 = r1.addGroup(1) - val keyValue1 = map1.addGroup(0) - keyValue1.add(0, "key1") - keyValue1.add(1, 1) - val keyValue2 = map1.addGroup(0) - keyValue2.add(0, "key2") - keyValue2.add(1, 2) - val map2 = r1.addGroup(2) - val keyValue3 = map2.addGroup(0) - // TODO: currently only string key type supported - keyValue3.add(0, "seven") - val valueGroup1 = keyValue3.addGroup(1) - valueGroup1.add(0, 42.toLong) - valueGroup1.add(1, "the answer") - val keyValue4 = map2.addGroup(0) - // TODO: currently only string key type supported - keyValue4.add(0, "eight") - val valueGroup2 = keyValue4.addGroup(1) - valueGroup2.add(0, 49.toLong) - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.close() - } - - // TODO: this is not actually used anywhere but useful for debugging - /* def readNestedFile(file: File, schemaString: String): Unit = { - val configuration = new Configuration() - val path = new Path(new Path(file.toURI), new Path("part-r-0.parquet")) - val fs: FileSystem = path.getFileSystem(configuration) - val schema: MessageType = MessageTypeParser.parseMessageType(schemaString) - assert(schema != null) - val outputStatus: FileStatus = fs.getFileStatus(new Path(path.toString)) - val footers = ParquetFileReader.readFooter(configuration, outputStatus) - assert(footers != null) - val reader = new ParquetReader(new Path(path.toString), new GroupReadSupport()) - val first = reader.read() - assert(first != null) - } */ - - // to test golb pattern (wild card pattern matching for parquetFile input - val testGlobDir = Utils.createTempDir() - val testGlobSubDir1 = Utils.createTempDir(testGlobDir.getPath) - val testGlobSubDir2 = Utils.createTempDir(testGlobDir.getPath) - val testGlobSubDir3 = Utils.createTempDir(testGlobDir.getPath) - - def writeGlobFiles() = { - val subDirs = Array(testGlobSubDir1, testGlobSubDir2, testGlobSubDir3) - - subDirs.foreach { dir => - val path: Path = new Path(new Path(dir.toURI), new Path("part-r-0.parquet")) - val job = new Job() - val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - - for(i <- 0 until 15) { - val record = new SimpleGroup(schema) - if(i % 3 == 0) { - record.add(0, true) - } else { - record.add(0, false) - } - if(i % 5 == 0) { - record.add(1, 5) - } - record.add(2, "abc") - record.add(3, i.toLong << 33) - record.add(4, 2.5F) - record.add(5, 4.5D) - writer.write(record) - } - writer.close() - } - } -} - From e360d5e4adf287444c10e72f8e4d57548839bf6e Mon Sep 17 00:00:00 2001 From: vinodkc Date: Sat, 14 Mar 2015 07:17:54 +0800 Subject: [PATCH 19/62] [SPARK-6317][SQL]Fixed HIVE console startup issue Author: vinodkc Author: Vinod K C Closes #5011 from vinodkc/HIVE_console_startupError and squashes the following commits: b43925f [vinodkc] Changed order of import b4f5453 [Vinod K C] Fixed HIVE console startup issue --- project/SparkBuild.scala | 4 ++-- sql/README.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4a06b9821bb98..f4c74c4051014 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -269,8 +269,8 @@ object SQL { |import org.apache.spark.sql.catalyst.plans.logical._ |import org.apache.spark.sql.catalyst.rules._ |import org.apache.spark.sql.catalyst.util._ - |import org.apache.spark.sql.Dsl._ |import org.apache.spark.sql.execution + |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.test.TestSQLContext._ |import org.apache.spark.sql.types._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, @@ -300,8 +300,8 @@ object Hive { |import org.apache.spark.sql.catalyst.plans.logical._ |import org.apache.spark.sql.catalyst.rules._ |import org.apache.spark.sql.catalyst.util._ - |import org.apache.spark.sql.Dsl._ |import org.apache.spark.sql.execution + |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ |import org.apache.spark.sql.types._ diff --git a/sql/README.md b/sql/README.md index a79249965ee67..48f83340e37b3 100644 --- a/sql/README.md +++ b/sql/README.md @@ -36,8 +36,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.execution +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ From b38e073fee794188d5267f1812b095e51874839e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 14 Mar 2015 00:43:33 -0700 Subject: [PATCH 20/62] [SPARK-6210] [SQL] use prettyString as column name in agg() use prettyString instead of toString() (which include id of expression) as column name in agg() Author: Davies Liu Closes #5006 from davies/prettystring and squashes the following commits: cb1fdcf [Davies Liu] use prettyString as column name in agg() --- python/pyspark/sql/dataframe.py | 32 +++++++++---------- .../spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../org/apache/spark/sql/GroupedData.scala | 8 ++--- 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index e8ce4547455a5..94001aec3774b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -631,11 +631,11 @@ def groupBy(self, *cols): for all the available aggregate functions. >>> df.groupBy().avg().collect() - [Row(AVG(age#0)=3.5)] + [Row(AVG(age)=3.5)] >>> df.groupBy('name').agg({'age': 'mean'}).collect() - [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] + [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] >>> df.groupBy(df.name).avg().collect() - [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] + [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] """ jcols = ListConverter().convert([_to_java_column(c) for c in cols], self._sc._gateway._gateway_client) @@ -647,10 +647,10 @@ def agg(self, *exprs): (shorthand for df.groupBy.agg()). >>> df.agg({"age": "max"}).collect() - [Row(MAX(age#0)=5)] + [Row(MAX(age)=5)] >>> from pyspark.sql import functions as F >>> df.agg(F.min(df.age)).collect() - [Row(MIN(age#0)=2)] + [Row(MIN(age)=2)] """ return self.groupBy().agg(*exprs) @@ -766,7 +766,7 @@ def agg(self, *exprs): >>> from pyspark.sql import functions as F >>> gdf.agg(F.min(df.age)).collect() - [Row(MIN(age#0)=5), Row(MIN(age#0)=2)] + [Row(MIN(age)=5), Row(MIN(age)=2)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): @@ -795,9 +795,9 @@ def mean(self, *cols): for each group. This is an alias for `avg`. >>> df.groupBy().mean('age').collect() - [Row(AVG(age#0)=3.5)] + [Row(AVG(age)=3.5)] >>> df3.groupBy().mean('age', 'height').collect() - [Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)] + [Row(AVG(age)=3.5, AVG(height)=82.5)] """ @df_varargs_api @@ -806,9 +806,9 @@ def avg(self, *cols): for each group. >>> df.groupBy().avg('age').collect() - [Row(AVG(age#0)=3.5)] + [Row(AVG(age)=3.5)] >>> df3.groupBy().avg('age', 'height').collect() - [Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)] + [Row(AVG(age)=3.5, AVG(height)=82.5)] """ @df_varargs_api @@ -817,9 +817,9 @@ def max(self, *cols): each group. >>> df.groupBy().max('age').collect() - [Row(MAX(age#0)=5)] + [Row(MAX(age)=5)] >>> df3.groupBy().max('age', 'height').collect() - [Row(MAX(age#4L)=5, MAX(height#5L)=85)] + [Row(MAX(age)=5, MAX(height)=85)] """ @df_varargs_api @@ -828,9 +828,9 @@ def min(self, *cols): each group. >>> df.groupBy().min('age').collect() - [Row(MIN(age#0)=2)] + [Row(MIN(age)=2)] >>> df3.groupBy().min('age', 'height').collect() - [Row(MIN(age#4L)=2, MIN(height#5L)=80)] + [Row(MIN(age)=2, MIN(height)=80)] """ @df_varargs_api @@ -839,9 +839,9 @@ def sum(self, *cols): group. >>> df.groupBy().sum('age').collect() - [Row(SUM(age#0)=7)] + [Row(SUM(age)=7)] >>> df3.groupBy().sum('age', 'height').collect() - [Row(SUM(age#4L)=7, SUM(height#5L)=165)] + [Row(SUM(age)=7, SUM(height)=165)] """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 7e191ad0315a5..f84ffe4e176cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -343,7 +343,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { }.mkString(", ") /** String representation of this node without any children */ - def simpleString = s"$nodeName $argString" + def simpleString = s"$nodeName $argString".trim override def toString: String = treeString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index d00175265924c..45a63ae26ed71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -37,7 +37,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.toString)() + case expr: Expression => Alias(expr, expr.prettyString)() } DataFrame( df.sqlContext, Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) @@ -63,7 +63,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) } columnExprs.map { c => val a = f(c) - Alias(a, a.toString)() + Alias(a, a.prettyString)() } } @@ -115,7 +115,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) def agg(exprs: Map[String, String]): DataFrame = { exprs.map { case (colName, expr) => val a = strToExpr(expr)(df(colName).expr) - Alias(a, a.toString)() + Alias(a, a.prettyString)() }.toSeq } @@ -159,7 +159,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) def agg(expr: Column, exprs: Column*): DataFrame = { val aggExprs = (expr +: exprs).map(_.expr).map { case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.toString)() + case expr: Expression => Alias(expr, expr.prettyString)() } DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) } From ee15404a2b0009fc70119ac7af69137b54890d48 Mon Sep 17 00:00:00 2001 From: ArcherShao Date: Sat, 14 Mar 2015 08:27:18 +0000 Subject: [PATCH 21/62] [SQL]Delete some dupliate code in HiveThriftServer2 Author: ArcherShao Author: ArcherShao Closes #5007 from ArcherShao/20150313 and squashes the following commits: ae422ae [ArcherShao] Updated 459efbd [ArcherShao] [SQL]Delete some dupliate code in HiveThriftServer2 --- .../sql/hive/thriftserver/HiveThriftServer2.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 6e07df18b0e15..c3a3f8c0f41df 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -98,16 +98,14 @@ private[hive] class HiveThriftServer2(hiveContext: HiveContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) - if (isHTTPTransportMode(hiveConf)) { - val thriftCliService = new ThriftHttpCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) + val thriftCliService = if (isHTTPTransportMode(hiveConf)) { + new ThriftHttpCLIService(sparkSqlCliService) } else { - val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) + new ThriftBinaryCLIService(sparkSqlCliService) } + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) initCompositeService(hiveConf) } From 5be6b0e4f48aca12fcd47c1b77c4675ad651c332 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 14 Mar 2015 19:53:54 +0800 Subject: [PATCH 22/62] [SPARK-6195] [SQL] Adds in-memory column type for fixed-precision decimals This PR adds a specialized in-memory column type for fixed-precision decimals. For all other column types, a single integer column type ID is enough to determine which column type to use. However, this doesn't apply to fixed-precision decimal types with different precision and scale parameters. Moreover, according to the previous design, there seems no trivial way to encode precision and scale information into the columnar byte buffer. On the other hand, considering we always know the data type of the column to be built / scanned ahead of time. This PR no longer use column type ID to construct `ColumnBuilder`s and `ColumnAccessor`s, but resorts to the actual column data type. In this way, we can pass precision / scale information along the way. The column type ID is now not used anymore and can be removed in a future PR. ### Micro benchmark result The following micro benchmark builds a simple table with 2 million decimals (precision = 10, scale = 0), cache it in memory, then count all the rows. Code (simply paste it into Spark shell): ```scala import sc._ import sqlContext._ import sqlContext.implicits._ import org.apache.spark.sql.types._ import com.google.common.base.Stopwatch def benchmark(n: Int)(f: => Long) { val stopwatch = new Stopwatch() def run() = { stopwatch.reset() stopwatch.start() f stopwatch.stop() stopwatch.elapsedMillis() } val records = (0 until n).map(_ => run()) (0 until n).foreach(i => println(s"Round $i: ${records(i)} ms")) println(s"Average: ${records.sum / n.toDouble} ms") } // Explicit casting is required because ScalaReflection can't inspect decimal precision parallelize(1 to 2000000) .map(i => Tuple1(Decimal(i, 10, 0))) .toDF("dec") .select($"dec" cast DecimalType(10, 0)) .registerTempTable("dec") sql("CACHE TABLE dec") val df = table("dec") // Warm up df.count() df.count() benchmark(5) { df.count() } ``` With `FIXED_DECIMAL` column type: - Round 0: 75 ms - Round 1: 97 ms - Round 2: 75 ms - Round 3: 70 ms - Round 4: 72 ms - Average: 77.8 ms Without `FIXED_DECIMAL` column type: - Round 0: 1233 ms - Round 1: 1170 ms - Round 2: 1171 ms - Round 3: 1141 ms - Round 4: 1141 ms - Average: 1171.2 ms [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4938) Author: Cheng Lian Closes #4938 from liancheng/decimal-column-type and squashes the following commits: fef5338 [Cheng Lian] Updates fixed decimal column type related test cases e08ab5b [Cheng Lian] Only resorts to FIXED_DECIMAL when the value can be held in a long 4db713d [Cheng Lian] Adds in-memory column type for fixed-precision decimals --- .../spark/sql/columnar/ColumnAccessor.scala | 43 +++++++++------ .../spark/sql/columnar/ColumnBuilder.scala | 39 +++++++------ .../spark/sql/columnar/ColumnStats.scala | 17 ++++++ .../spark/sql/columnar/ColumnType.scala | 55 ++++++++++++++----- .../columnar/InMemoryColumnarTableScan.scala | 8 ++- .../spark/sql/columnar/ColumnStatsSuite.scala | 1 + .../spark/sql/columnar/ColumnTypeSuite.scala | 46 ++++++++++++---- .../sql/columnar/ColumnarTestUtils.scala | 23 ++++---- .../columnar/InMemoryColumnarQuerySuite.scala | 17 +++++- .../NullableColumnAccessorSuite.scala | 3 +- .../columnar/NullableColumnBuilderSuite.scala | 3 +- 11 files changed, 179 insertions(+), 76 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 91c4c105b14e6..b615eaa0dca0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -21,7 +21,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor -import org.apache.spark.sql.types.{BinaryType, DataType, NativeType} +import org.apache.spark.sql.types._ /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is @@ -89,6 +89,9 @@ private[sql] class DoubleColumnAccessor(buffer: ByteBuffer) private[sql] class FloatColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, FLOAT) +private[sql] class FixedDecimalColumnAccessor(buffer: ByteBuffer, precision: Int, scale: Int) + extends NativeColumnAccessor(buffer, FIXED_DECIMAL(precision, scale)) + private[sql] class StringColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, STRING) @@ -107,24 +110,28 @@ private[sql] class GenericColumnAccessor(buffer: ByteBuffer) with NullableColumnAccessor private[sql] object ColumnAccessor { - def apply(buffer: ByteBuffer): ColumnAccessor = { + def apply(dataType: DataType, buffer: ByteBuffer): ColumnAccessor = { val dup = buffer.duplicate().order(ByteOrder.nativeOrder) - // The first 4 bytes in the buffer indicate the column type. - val columnTypeId = dup.getInt() - - columnTypeId match { - case INT.typeId => new IntColumnAccessor(dup) - case LONG.typeId => new LongColumnAccessor(dup) - case FLOAT.typeId => new FloatColumnAccessor(dup) - case DOUBLE.typeId => new DoubleColumnAccessor(dup) - case BOOLEAN.typeId => new BooleanColumnAccessor(dup) - case BYTE.typeId => new ByteColumnAccessor(dup) - case SHORT.typeId => new ShortColumnAccessor(dup) - case STRING.typeId => new StringColumnAccessor(dup) - case DATE.typeId => new DateColumnAccessor(dup) - case TIMESTAMP.typeId => new TimestampColumnAccessor(dup) - case BINARY.typeId => new BinaryColumnAccessor(dup) - case GENERIC.typeId => new GenericColumnAccessor(dup) + + // The first 4 bytes in the buffer indicate the column type. This field is not used now, + // because we always know the data type of the column ahead of time. + dup.getInt() + + dataType match { + case IntegerType => new IntColumnAccessor(dup) + case LongType => new LongColumnAccessor(dup) + case FloatType => new FloatColumnAccessor(dup) + case DoubleType => new DoubleColumnAccessor(dup) + case BooleanType => new BooleanColumnAccessor(dup) + case ByteType => new ByteColumnAccessor(dup) + case ShortType => new ShortColumnAccessor(dup) + case StringType => new StringColumnAccessor(dup) + case BinaryType => new BinaryColumnAccessor(dup) + case DateType => new DateColumnAccessor(dup) + case TimestampType => new TimestampColumnAccessor(dup) + case DecimalType.Fixed(precision, scale) if precision < 19 => + new FixedDecimalColumnAccessor(dup, precision, scale) + case _ => new GenericColumnAccessor(dup) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 3a4977b836af7..d8d24a577347c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -106,6 +106,13 @@ private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleCol private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT) +private[sql] class FixedDecimalColumnBuilder( + precision: Int, + scale: Int) + extends NativeColumnBuilder( + new FixedDecimalColumnStats, + FIXED_DECIMAL(precision, scale)) + private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) private[sql] class DateColumnBuilder extends NativeColumnBuilder(new DateColumnStats, DATE) @@ -139,25 +146,25 @@ private[sql] object ColumnBuilder { } def apply( - typeId: Int, + dataType: DataType, initialSize: Int = 0, columnName: String = "", useCompression: Boolean = false): ColumnBuilder = { - - val builder = (typeId match { - case INT.typeId => new IntColumnBuilder - case LONG.typeId => new LongColumnBuilder - case FLOAT.typeId => new FloatColumnBuilder - case DOUBLE.typeId => new DoubleColumnBuilder - case BOOLEAN.typeId => new BooleanColumnBuilder - case BYTE.typeId => new ByteColumnBuilder - case SHORT.typeId => new ShortColumnBuilder - case STRING.typeId => new StringColumnBuilder - case BINARY.typeId => new BinaryColumnBuilder - case GENERIC.typeId => new GenericColumnBuilder - case DATE.typeId => new DateColumnBuilder - case TIMESTAMP.typeId => new TimestampColumnBuilder - }).asInstanceOf[ColumnBuilder] + val builder: ColumnBuilder = dataType match { + case IntegerType => new IntColumnBuilder + case LongType => new LongColumnBuilder + case DoubleType => new DoubleColumnBuilder + case BooleanType => new BooleanColumnBuilder + case ByteType => new ByteColumnBuilder + case ShortType => new ShortColumnBuilder + case StringType => new StringColumnBuilder + case BinaryType => new BinaryColumnBuilder + case DateType => new DateColumnBuilder + case TimestampType => new TimestampColumnBuilder + case DecimalType.Fixed(precision, scale) if precision < 19 => + new FixedDecimalColumnBuilder(precision, scale) + case _ => new GenericColumnBuilder + } builder.initialize(initialSize, columnName, useCompression) builder diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index cad0667b46435..04047b9c062be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -181,6 +181,23 @@ private[sql] class FloatColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } +private[sql] class FixedDecimalColumnStats extends ColumnStats { + protected var upper: Decimal = null + protected var lower: Decimal = null + + override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + val value = row(ordinal).asInstanceOf[Decimal] + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + sizeInBytes += FIXED_DECIMAL.defaultSize + } + } + + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) +} + private[sql] class IntColumnStats extends ColumnStats { protected var upper = Int.MinValue protected var lower = Int.MaxValue diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index db5bc0de363c7..36ea1c77e0470 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -373,6 +373,33 @@ private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 9, 12) { } } +private[sql] case class FIXED_DECIMAL(precision: Int, scale: Int) + extends NativeColumnType( + DecimalType(Some(PrecisionInfo(precision, scale))), + 10, + FIXED_DECIMAL.defaultSize) { + + override def extract(buffer: ByteBuffer): Decimal = { + Decimal(buffer.getLong(), precision, scale) + } + + override def append(v: Decimal, buffer: ByteBuffer): Unit = { + buffer.putLong(v.toUnscaledLong) + } + + override def getField(row: Row, ordinal: Int): Decimal = { + row(ordinal).asInstanceOf[Decimal] + } + + override def setField(row: MutableRow, ordinal: Int, value: Decimal): Unit = { + row(ordinal) = value + } +} + +private[sql] object FIXED_DECIMAL { + val defaultSize = 8 +} + private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( typeId: Int, defaultSize: Int) @@ -394,7 +421,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](10, 16) { +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](11, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = value } @@ -405,7 +432,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](10, 16) // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](11, 16) { +private[sql] object GENERIC extends ByteArrayColumnType[DataType](12, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } @@ -416,18 +443,20 @@ private[sql] object GENERIC extends ByteArrayColumnType[DataType](11, 16) { private[sql] object ColumnType { def apply(dataType: DataType): ColumnType[_, _] = { dataType match { - case IntegerType => INT - case LongType => LONG - case FloatType => FLOAT - case DoubleType => DOUBLE - case BooleanType => BOOLEAN - case ByteType => BYTE - case ShortType => SHORT - case StringType => STRING - case BinaryType => BINARY - case DateType => DATE + case IntegerType => INT + case LongType => LONG + case FloatType => FLOAT + case DoubleType => DOUBLE + case BooleanType => BOOLEAN + case ByteType => BYTE + case ShortType => SHORT + case StringType => STRING + case BinaryType => BINARY + case DateType => DATE case TimestampType => TIMESTAMP - case _ => GENERIC + case DecimalType.Fixed(precision, scale) if precision < 19 => + FIXED_DECIMAL(precision, scale) + case _ => GENERIC } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 8944a32bc3887..387faee12b3cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -113,7 +113,7 @@ private[sql] case class InMemoryRelation( val columnBuilders = output.map { attribute => val columnType = ColumnType(attribute.dataType) val initialBufferSize = columnType.defaultSize * batchSize - ColumnBuilder(columnType.typeId, initialBufferSize, attribute.name, useCompression) + ColumnBuilder(attribute.dataType, initialBufferSize, attribute.name, useCompression) }.toArray var rowCount = 0 @@ -274,8 +274,10 @@ private[sql] case class InMemoryColumnarTableScan( def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]) = { val rows = cacheBatches.flatMap { cachedBatch => // Build column accessors - val columnAccessors = requestedColumnIndices.map { batch => - ColumnAccessor(ByteBuffer.wrap(cachedBatch.buffers(batch))) + val columnAccessors = requestedColumnIndices.map { batchColumnIndex => + ColumnAccessor( + relation.output(batchColumnIndex).dataType, + ByteBuffer.wrap(cachedBatch.buffers(batchColumnIndex))) } // Extract rows via column accessors diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 581fccf8ee613..fec487f1d2c82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -29,6 +29,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[LongColumnStats], LONG, Row(Long.MaxValue, Long.MinValue, 0)) testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) + testColumnStats(classOf[FixedDecimalColumnStats], FIXED_DECIMAL(15, 10), Row(null, null, 0)) testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 9ce845912f1c7..5f08834f73c6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -33,8 +33,9 @@ class ColumnTypeSuite extends FunSuite with Logging { test("defaultSize") { val checks = Map( - INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, BOOLEAN -> 1, - STRING -> 8, DATE -> 4, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) + INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, + FIXED_DECIMAL(15, 10) -> 8, BOOLEAN -> 1, STRING -> 8, DATE -> 4, TIMESTAMP -> 12, + BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -56,15 +57,16 @@ class ColumnTypeSuite extends FunSuite with Logging { } } - checkActualSize(INT, Int.MaxValue, 4) - checkActualSize(SHORT, Short.MaxValue, 2) - checkActualSize(LONG, Long.MaxValue, 8) - checkActualSize(BYTE, Byte.MaxValue, 1) - checkActualSize(DOUBLE, Double.MaxValue, 8) - checkActualSize(FLOAT, Float.MaxValue, 4) - checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) - checkActualSize(DATE, 0, 4) + checkActualSize(INT, Int.MaxValue, 4) + checkActualSize(SHORT, Short.MaxValue, 2) + checkActualSize(LONG, Long.MaxValue, 8) + checkActualSize(BYTE, Byte.MaxValue, 1) + checkActualSize(DOUBLE, Double.MaxValue, 8) + checkActualSize(FLOAT, Float.MaxValue, 4) + checkActualSize(FIXED_DECIMAL(15, 10), Decimal(0, 15, 10), 8) + checkActualSize(BOOLEAN, true, 1) + checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: Byte) @@ -93,12 +95,20 @@ class ColumnTypeSuite extends FunSuite with Logging { testNativeColumnType[DoubleType.type](DOUBLE, _.putDouble(_), _.getDouble) + testNativeColumnType[DecimalType]( + FIXED_DECIMAL(15, 10), + (buffer: ByteBuffer, decimal: Decimal) => { + buffer.putLong(decimal.toUnscaledLong) + }, + (buffer: ByteBuffer) => { + Decimal(buffer.getLong(), 15, 10) + }) + testNativeColumnType[FloatType.type](FLOAT, _.putFloat(_), _.getFloat) testNativeColumnType[StringType.type]( STRING, (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes("utf-8") buffer.putInt(bytes.length) buffer.put(bytes) @@ -206,4 +216,16 @@ class ColumnTypeSuite extends FunSuite with Logging { if (sb.nonEmpty) sb.setLength(sb.length - 1) sb.toString() } + + test("column type for decimal types with different precision") { + (1 to 18).foreach { i => + assertResult(FIXED_DECIMAL(i, 0)) { + ColumnType(DecimalType(i, 0)) + } + } + + assertResult(GENERIC) { + ColumnType(DecimalType(19, 0)) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index 60ed28cc97bf1..c7a40845db16c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.types.{DataType, NativeType} +import org.apache.spark.sql.types.{Decimal, DataType, NativeType} object ColumnarTestUtils { def makeNullRow(length: Int) = { @@ -41,16 +41,17 @@ object ColumnarTestUtils { } (columnType match { - case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte - case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort - case INT => Random.nextInt() - case LONG => Random.nextLong() - case FLOAT => Random.nextFloat() - case DOUBLE => Random.nextDouble() - case STRING => Random.nextString(Random.nextInt(32)) - case BOOLEAN => Random.nextBoolean() - case BINARY => randomBytes(Random.nextInt(32)) - case DATE => Random.nextInt() + case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte + case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort + case INT => Random.nextInt() + case LONG => Random.nextLong() + case FLOAT => Random.nextFloat() + case DOUBLE => Random.nextDouble() + case FIXED_DECIMAL(precision, scale) => Decimal(Random.nextLong() % 100, precision, scale) + case STRING => Random.nextString(Random.nextInt(32)) + case BOOLEAN => Random.nextBoolean() + case BINARY => randomBytes(Random.nextInt(32)) + case DATE => Random.nextInt() case TIMESTAMP => val timestamp = new Timestamp(Random.nextLong()) timestamp.setNanos(Random.nextInt(999999999)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 38b0f666ab90b..27dfabca90217 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar -import org.apache.spark.sql.functions._ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.types.{DecimalType, Decimal} import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY @@ -117,4 +117,19 @@ class InMemoryColumnarQuerySuite extends QueryTest { complexData.count() complexData.unpersist() } + + test("decimal type") { + // Casting is required here because ScalaReflection can't capture decimal precision information. + val df = (1 to 10) + .map(i => Tuple1(Decimal(i, 15, 10))) + .toDF("dec") + .select($"dec" cast DecimalType(15, 10)) + + assert(df.schema.head.dataType === DecimalType(15, 10)) + + df.cache().registerTempTable("test_fixed_decimal") + checkAnswer( + sql("SELECT * FROM test_fixed_decimal"), + (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index f95c895587f3f..bb305355276bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -42,7 +42,8 @@ class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestUtils._ Seq( - INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, FIXED_DECIMAL(15, 10), BINARY, GENERIC, + DATE, TIMESTAMP ).foreach { testNullableColumnAccessor(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 80bd5c94570cb..75a47498683f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -38,7 +38,8 @@ class NullableColumnBuilderSuite extends FunSuite { import ColumnarTestUtils._ Seq( - INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, FIXED_DECIMAL(15, 10), BINARY, GENERIC, + DATE, TIMESTAMP ).foreach { testNullableColumnBuilder(_) } From 127268bc3999201ec1c0a040a29c7fa9ac25476b Mon Sep 17 00:00:00 2001 From: Brennon York Date: Sat, 14 Mar 2015 17:28:13 +0000 Subject: [PATCH 23/62] [SPARK-6329][Docs]: Minor doc changes for Mesos and TOC Updated the configuration docs from the minor items that Reynold had left over from SPARK-1182; specifically I updated the `running-on-mesos` link to point directly to `running-on-mesos#configuration` and upgraded the `yarn`, `mesos`, etc. bullets to `
` tags in hopes that they'll get pushed into the TOC. Author: Brennon York Closes #5022 from brennonyork/SPARK-6329 and squashes the following commits: 42a10a9 [Brennon York] minor doc fixes --- docs/configuration.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index a7116fbece9bb..63fc99e7d3e29 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1391,9 +1391,11 @@ Apart from these, the following properties are also available, and may be useful Each cluster manager in Spark has additional configuration options. Configurations can be found on the pages for each mode: - * [YARN](running-on-yarn.html#configuration) - * [Mesos](running-on-mesos.html) - * [Standalone Mode](spark-standalone.html#cluster-launch-scripts) +##### [YARN](running-on-yarn.html#configuration) + +##### [Mesos](running-on-mesos.html#configuration) + +##### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) # Environment Variables From c49d156624624a719c0d1262a58933ea3e346963 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Sat, 14 Mar 2015 17:38:12 +0000 Subject: [PATCH 24/62] [SPARK-5790][GraphX]: VertexRDD's won't zip properly for `diff` capability (added tests) Added tests that maropu [created](https://github.com/maropu/spark/blob/1f64794b2ce33e64f340e383d4e8a60639a7eb4b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala) for vertices with differing partition counts. Wanted to make sure his work got captured /merged as its not in the master branch and I don't believe there's a PR out already for it. Author: Brennon York Closes #5023 from brennonyork/SPARK-5790 and squashes the following commits: 83bbd29 [Brennon York] added maropu's tests for vertices with differing partition counts --- .../apache/spark/graphx/VertexRDDSuite.scala | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 131959cea3ef7..97533dd3aa6ce 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx import org.scalatest.FunSuite -import org.apache.spark.SparkContext +import org.apache.spark.{HashPartitioner, SparkContext} import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { @@ -58,6 +58,16 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("diff vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) + val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.diff(vertexB) + assert(vertexC.map(_._1).collect.toSet === (8 until 16).toSet) + } + } + test("leftJoin") { withSpark { sc => val n = 100 @@ -73,6 +83,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("leftJoin vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = VertexRDD( + vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.leftJoin(vertexB) { (vid, old, newOpt) => + old - newOpt.getOrElse(0) + } + assert(vertexC.filter(v => v._2 != 0).map(_._1).collect.toSet == (1 to 99 by 2).toSet) + } + } + test("innerJoin") { withSpark { sc => val n = 100 @@ -87,6 +110,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { (0 to n by 2).map(x => (x.toLong, 0)).toSet) } } + test("innerJoin vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = VertexRDD( + vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.innerJoin(vertexB) { (vid, old, newVal) => + old - newVal + } + assert(vertexC.filter(v => v._2 == 0).map(_._1).collect.toSet == (0 to 98 by 2).toSet) + } + } + test("aggregateUsingIndex") { withSpark { sc => val n = 100 From 62ede5383f64b69570a66d46939638f4bf38d1b1 Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Sun, 15 Mar 2015 20:44:45 +0800 Subject: [PATCH 25/62] [SPARK-6285][SQL]Remove ParquetTestData in SparkBuild.scala and in README.md This is a following clean up PR for #5010 This will resolve issues when launching `hive/console` like below: ``` :20: error: object ParquetTestData is not a member of package org.apache.spark.sql.parquet import org.apache.spark.sql.parquet.ParquetTestData ``` Author: OopsOutOfMemory Closes #5032 from OopsOutOfMemory/SPARK-6285 and squashes the following commits: 2996aeb [OopsOutOfMemory] remove ParquetTestData --- project/SparkBuild.scala | 6 ++---- sql/README.md | 1 - 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f4c74c4051014..ac37c605de4b6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -272,8 +272,7 @@ object SQL { |import org.apache.spark.sql.execution |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.test.TestSQLContext._ - |import org.apache.spark.sql.types._ - |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + |import org.apache.spark.sql.types._""".stripMargin, cleanupCommands in console := "sparkContext.stop()" ) } @@ -304,8 +303,7 @@ object Hive { |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ - |import org.apache.spark.sql.types._ - |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + |import org.apache.spark.sql.types._""".stripMargin, cleanupCommands in console := "sparkContext.stop()", // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce // in order to generate golden files. This is only required for developers who are adding new diff --git a/sql/README.md b/sql/README.md index 48f83340e37b3..fbb3200a3a4b4 100644 --- a/sql/README.md +++ b/sql/README.md @@ -41,7 +41,6 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.parquet.ParquetTestData Type in expressions to have them evaluated. Type :help for more information. From aa6536fa3c2ed1cac47abc79fc22e273f0814858 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Sun, 15 Mar 2015 15:46:55 +0000 Subject: [PATCH 26/62] [SPARK-3619] Part 2. Upgrade to Mesos 0.21 to work around MESOS-1688 - MESOS_NATIVE_LIBRARY become deprecated - Chagned MESOS_NATIVE_LIBRARY to MESOS_NATIVE_JAVA_LIBRARY Author: Jongyoul Lee Closes #4361 from jongyoul/SPARK-3619-1 and squashes the following commits: f1ea91f [Jongyoul Lee] Merge branch 'SPARK-3619-1' of https://github.com/jongyoul/spark into SPARK-3619-1 a6a00c2 [Jongyoul Lee] [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688 - Removed 'Known issues' section 2e15a21 [Jongyoul Lee] [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688 - MESOS_NATIVE_LIBRARY become deprecated - Chagned MESOS_NATIVE_LIBRARY to MESOS_NATIVE_JAVA_LIBRARY 0dace7b [Jongyoul Lee] [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688 - MESOS_NATIVE_LIBRARY become deprecated - Chagned MESOS_NATIVE_LIBRARY to MESOS_NATIVE_JAVA_LIBRARY --- conf/spark-env.sh.template | 2 +- docs/running-on-mesos.md | 5 +---- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 2 +- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 2 +- 4 files changed, 4 insertions(+), 7 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 0886b0276fb90..67f81d33361e1 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -15,7 +15,7 @@ # - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program # - SPARK_CLASSPATH, default classpath entries to append # - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data -# - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos +# - MESOS_NATIVE_JAVA_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode # - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index e509e4bf37396..59a3e9d25baf1 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -110,7 +110,7 @@ cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooK The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: 1. In `spark-env.sh` set some environment variables: - * `export MESOS_NATIVE_LIBRARY=`. This path is typically + * `export MESOS_NATIVE_JAVA_LIBRARY=`. This path is typically `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`. @@ -167,9 +167,6 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). -# Known issues -- When using the "fine-grained" mode, make sure that your executors always leave 32 MB free on the slaves. Otherwise it can happen that your Spark job does not proceed anymore. Currently, Apache Mesos only offers resources if there are at least 32 MB memory allocatable. But as Spark allocates memory only for the executor and cpu only for tasks, it can happen on high slave memory usage that no new tasks will be started anymore. More details can be found in [MESOS-1688](https://issues.apache.org/jira/browse/MESOS-1688). Alternatively use the "coarse-gained" mode, which is not affected by this issue. - # Running Alongside Hadoop You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 529914a2b6141..249f438459300 100644 --- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -281,7 +281,7 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } - if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { + if (System.getenv("MESOS_NATIVE_JAVA_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", """ 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 ed9b207a86a0b..b3bd135548124 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 @@ -289,7 +289,7 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } - if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { + if (System.getenv("MESOS_NATIVE_JAVA_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", """ From 45f4c66122c57011e74c694a424756812ab77d99 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 16 Mar 2015 01:06:26 -0700 Subject: [PATCH 27/62] [SPARK-5922][GraphX]: Add diff(other: RDD[VertexId, VD]) in VertexRDD Changed method invocation of 'diff' to match that of 'innerJoin' and 'leftJoin' from VertexRDD[VD] to RDD[(VertexId, VD)]. This change maintains backwards compatibility and better unifies the VertexRDD methods to match each other. Author: Brennon York Closes #4733 from brennonyork/SPARK-5922 and squashes the following commits: e800f08 [Brennon York] fixed merge conflicts b9274af [Brennon York] fixed merge conflicts f86375c [Brennon York] fixed minor include line 398ddb4 [Brennon York] fixed merge conflicts aac1810 [Brennon York] updated to aggregateUsingIndex and added test to ensure that method works properly 2af0b88 [Brennon York] removed deprecation line 753c963 [Brennon York] fixed merge conflicts and set preference to use the diff(other: VertexRDD[VD]) method 2c678c6 [Brennon York] added mima exclude to exclude new public diff method from VertexRDD 93186f3 [Brennon York] added back the original diff method to sustain binary compatibility f18356e [Brennon York] changed method invocation of 'diff' to match that of 'innerJoin' and 'leftJoin' from VertexRDD[VD] to RDD[(VertexId, VD)] --- .../scala/org/apache/spark/graphx/VertexRDD.scala | 9 +++++++++ .../apache/spark/graphx/impl/VertexRDDImpl.scala | 4 ++++ .../org/apache/spark/graphx/VertexRDDSuite.scala | 13 +++++++++++++ project/MimaExcludes.scala | 3 +++ 4 files changed, 29 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 40ecff7107109..ad4bfe077293a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -121,6 +121,15 @@ abstract class VertexRDD[VD]( */ def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] + /** + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is + * only guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other the other RDD[(VertexId, VD)] with which to diff against. + */ + def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] + /** * For each vertex present in both `this` and `other`, `diff` returns only those vertices with * differing values; for values that are different, keeps the values from `other`. This is diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 904be213147dc..125692ddaad83 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -103,6 +103,10 @@ class VertexRDDImpl[VD] private[graphx] ( override def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) + override def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { + diff(this.aggregateUsingIndex(other, (a: VD, b: VD) => a)) + } + override def diff(other: VertexRDD[VD]): VertexRDD[VD] = { val otherPartition = other match { case other: VertexRDD[_] if this.partitioner == other.partitioner => diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 97533dd3aa6ce..4f7a442ab503d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import org.scalatest.FunSuite import org.apache.spark.{HashPartitioner, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { @@ -58,6 +59,18 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("diff with RDD[(VertexId, VD)]") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n).cache() + val flipEvens: RDD[(VertexId, Int)] = + sc.parallelize(0L to 100L) + .map(id => if (id % 2 == 0) (id, -id.toInt) else (id, id.toInt)).cache() + // diff should keep only the changed vertices + assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) + } + } + test("diff vertices with the non-equal number of partitions") { withSpark { sc => val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 627b2cea4d020..a6b07fa7cddec 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -181,6 +181,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.RealClock"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Clock"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.TestClock") + ) ++ Seq( + // SPARK-5922 Adding a generalized diff(other: RDD[(VertexId, VD)]) to VertexRDD + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.diff") ) case v if v.startsWith("1.2") => From 00e730b94cba1202a73af1e2476ff5a44af4b6b2 Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Mon, 16 Mar 2015 12:27:15 +0000 Subject: [PATCH 28/62] [SPARK-6300][Spark Core] sc.addFile(path) does not support the relative path. when i run cmd like that sc.addFile("../test.txt"), it did not work and throwed an exception: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:../test.txt at org.apache.hadoop.fs.Path.initialize(Path.java:206) at org.apache.hadoop.fs.Path.(Path.java:172) ........ ....... Caused by: java.net.URISyntaxException: Relative path in absolute URI: file:../test.txt at java.net.URI.checkPath(URI.java:1804) at java.net.URI.(URI.java:752) at org.apache.hadoop.fs.Path.initialize(Path.java:203) Author: DoingDone9 <799203320@qq.com> Closes #4993 from DoingDone9/relativePath and squashes the following commits: ee375cd [DoingDone9] Update SparkContextSuite.scala d594e16 [DoingDone9] Update SparkContext.scala 0ff3fa8 [DoingDone9] test for add file dced8eb [DoingDone9] Update SparkContext.scala e4a13fe [DoingDone9] getCanonicalPath 161cae3 [DoingDone9] Merge pull request #4 from apache/master c87e8b6 [DoingDone9] Merge pull request #3 from apache/master cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 51 ++++++++++++++----- 2 files changed, 38 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8121aab3b0b34..4457f40286fda 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1093,7 +1093,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def addFile(path: String, recursive: Boolean): Unit = { val uri = new URI(path) val schemeCorrectedPath = uri.getScheme match { - case null | "local" => "file:" + uri.getPath + case null | "local" => new File(path).getCanonicalFile.toURI.toString case _ => path } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 50f347f1954de..b8e3e83b5a47b 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -79,26 +79,49 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { val byteArray2 = converter.convert(bytesWritable) assert(byteArray2.length === 0) } - + test("addFile works") { - val file = File.createTempFile("someprefix", "somesuffix") - val absolutePath = file.getAbsolutePath + val file1 = File.createTempFile("someprefix1", "somesuffix1") + val absolutePath1 = file1.getAbsolutePath + + val pluto = Utils.createTempDir() + val file2 = File.createTempFile("someprefix2", "somesuffix2", pluto) + val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + "/" + file2.getName + val absolutePath2 = file2.getAbsolutePath + try { - Files.write("somewords", file, UTF_8) - val length = file.length() + Files.write("somewords1", file1, UTF_8) + Files.write("somewords2", file2, UTF_8) + val length1 = file1.length() + val length2 = file2.length() + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addFile(file.getAbsolutePath) + sc.addFile(file1.getAbsolutePath) + sc.addFile(relativePath) sc.parallelize(Array(1), 1).map(x => { - val gotten = new File(SparkFiles.get(file.getName)) - if (!gotten.exists()) { - throw new SparkException("file doesn't exist") + val gotten1 = new File(SparkFiles.get(file1.getName)) + val gotten2 = new File(SparkFiles.get(file2.getName)) + if (!gotten1.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath1) + } + if (!gotten2.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath2) } - if (length != gotten.length()) { + + if (length1 != gotten1.length()) { + throw new SparkException( + s"file has different length $length1 than added file ${gotten1.length()} : " + absolutePath1) + } + if (length2 != gotten2.length()) { throw new SparkException( - s"file has different length $length than added file ${gotten.length()}") + s"file has different length $length2 than added file ${gotten2.length()} : " + absolutePath2) } - if (absolutePath == gotten.getAbsolutePath) { - throw new SparkException("file should have been copied") + + if (absolutePath1 == gotten1.getAbsolutePath) { + throw new SparkException("file should have been copied :" + absolutePath1) + } + if (absolutePath2 == gotten2.getAbsolutePath) { + throw new SparkException("file should have been copied : " + absolutePath2) } x }).count() @@ -106,7 +129,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { sc.stop() } } - + test("addFile recursive works") { val pluto = Utils.createTempDir() val neptune = Utils.createTempDir(pluto.getAbsolutePath) From 12a345adcbaee359199ddfed4f41bf0e19d66d48 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 17 Mar 2015 01:09:27 +0800 Subject: [PATCH 29/62] [SPARK-2087] [SQL] Multiple thriftserver sessions with single HiveContext instance Still, we keep only a single HiveContext within ThriftServer, and we also create a object called `SQLSession` for isolating the different user states. Developers can obtain/release a new user session via `openSession` and `closeSession`, and `SQLContext` and `HiveContext` will also provide a default session if no `openSession` called, for backward-compatibility. Author: Cheng Hao Closes #4885 from chenghao-intel/multisessions_singlecontext and squashes the following commits: 1c47b2a [Cheng Hao] rename the tss => tlSession 815b27a [Cheng Hao] code style issue 57e3fa0 [Cheng Hao] openSession is not compatible between Hive0.12 & 0.13.1 4665b0d [Cheng Hao] thriftservice with single context --- .../org/apache/spark/sql/SQLContext.scala | 43 ++++- .../spark/sql/test/TestSQLContext.scala | 17 +- .../thriftserver/SparkSQLSessionManager.scala | 56 ------ .../HiveThriftServer2Suites.scala | 161 +++++++++++++++++- .../spark/sql/hive/thriftserver/Shim12.scala | 48 +++++- .../spark/sql/hive/thriftserver/Shim13.scala | 49 +++++- .../apache/spark/sql/hive/HiveContext.scala | 70 ++++---- .../apache/spark/sql/hive/test/TestHive.scala | 14 +- 8 files changed, 353 insertions(+), 105 deletions(-) delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 9c49e84bf9680..297d0d644a423 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -63,8 +63,10 @@ class SQLContext(@transient val sparkContext: SparkContext) def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) - // Note that this is a lazy val so we can override the default value in subclasses. - protected[sql] lazy val conf: SQLConf = new SQLConf + /** + * @return Spark SQL configuration + */ + protected[sql] def conf = tlSession.get().conf /** * Set Spark SQL configuration properties. @@ -103,9 +105,11 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def getAllConfs: immutable.Map[String, String] = conf.getAllConfs + // TODO how to handle the temp table per user session? @transient protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) + // TODO how to handle the temp function per user session? @transient protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry(true) @@ -138,6 +142,14 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] def executePlan(plan: LogicalPlan) = new this.QueryExecution(plan) + @transient + protected[sql] val tlSession = new ThreadLocal[SQLSession]() { + override def initialValue = defaultSession + } + + @transient + protected[sql] val defaultSession = createSession() + sparkContext.getConf.getAll.foreach { case (key, value) if key.startsWith("spark.sql") => setConf(key, value) case _ => @@ -194,6 +206,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * }}} * * @group basic + * TODO move to SQLSession? */ @transient val udf: UDFRegistration = new UDFRegistration(this) @@ -1059,6 +1072,32 @@ class SQLContext(@transient val sparkContext: SparkContext) ) } + + protected[sql] def openSession(): SQLSession = { + detachSession() + val session = createSession() + tlSession.set(session) + + session + } + + protected[sql] def currentSession(): SQLSession = { + tlSession.get() + } + + protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[sql] def detachSession(): Unit = { + tlSession.remove() + } + + protected[sql] class SQLSession { + // Note that this is a lazy val so we can override the default value in subclasses. + protected[sql] lazy val conf: SQLConf = new SQLConf + } + /** * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 4e1ec38bd0158..356a6100d2cf5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -24,16 +24,22 @@ import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /** A SQLContext that can be used for local testing. */ -object TestSQLContext +class LocalSQLContext extends SQLContext( new SparkContext( "local[2]", "TestSQLContext", new SparkConf().set("spark.sql.testkey", "true"))) { - /** Fewer partitions to speed up testing. */ - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def numShufflePartitions: Int = this.getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + override protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[sql] class SQLSession extends super.SQLSession { + protected[sql] override lazy val conf: SQLConf = new SQLConf { + /** Fewer partitions to speed up testing. */ + override def numShufflePartitions: Int = this.getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + } } /** @@ -45,3 +51,6 @@ object TestSQLContext } } + +object TestSQLContext extends LocalSQLContext + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala deleted file mode 100644 index 89e9ede7261c9..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.hive.thriftserver - -import java.util.concurrent.Executors - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.session.SessionManager - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager -import org.apache.hive.service.cli.SessionHandle - -private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) - extends SessionManager - with ReflectedCompositeService { - - private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) - setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) - getAncestorField[Log](this, 3, "LOG").info( - s"HiveServer2: Async execution pool size $backgroundPoolSize") - - setSuperField(this, "operationManager", sparkSqlOperationManager) - addService(sparkSqlOperationManager) - - initCompositeService(hiveConf) - } - - override def closeSession(sessionHandle: SessionHandle) { - super.closeSession(sessionHandle) - sparkSqlOperationManager.sessionToActivePool -= sessionHandle - } -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index d783d487b5c60..aff96e21a5373 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -195,6 +195,146 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } } } + + test("test multiple session") { + import org.apache.spark.sql.SQLConf + var defaultV1: String = null + var defaultV2: String = null + + withMultipleConnectionJdbcStatement( + // create table + { statement => + + val queries = Seq( + "DROP TABLE IF EXISTS test_map", + "CREATE TABLE test_map(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map", + "CACHE TABLE test_table AS SELECT key FROM test_map ORDER BY key DESC") + + queries.foreach(statement.execute) + + val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") + val buf1 = new collection.mutable.ArrayBuffer[Int]() + while (rs1.next()) { + buf1 += rs1.getInt(1) + } + rs1.close() + + val rs2 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf2 = new collection.mutable.ArrayBuffer[Int]() + while (rs2.next()) { + buf2 += rs2.getInt(1) + } + rs2.close() + + assert(buf1 === buf2) + }, + + // first session, we get the default value of the session status + { statement => + + val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}") + rs1.next() + defaultV1 = rs1.getString(1) + assert(defaultV1 != "200") + rs1.close() + + val rs2 = statement.executeQuery("SET hive.cli.print.header") + rs2.next() + + defaultV2 = rs2.getString(1) + assert(defaultV1 != "true") + rs2.close() + }, + + // second session, we update the session status + { statement => + + val queries = Seq( + s"SET ${SQLConf.SHUFFLE_PARTITIONS}=291", + "SET hive.cli.print.header=true" + ) + + queries.map(statement.execute) + val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}") + rs1.next() + assert("spark.sql.shuffle.partitions=291" === rs1.getString(1)) + rs1.close() + + val rs2 = statement.executeQuery("SET hive.cli.print.header") + rs2.next() + assert("hive.cli.print.header=true" === rs2.getString(1)) + rs2.close() + }, + + // third session, we get the latest session status, supposed to be the + // default value + { statement => + + val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}") + rs1.next() + assert(defaultV1 === rs1.getString(1)) + rs1.close() + + val rs2 = statement.executeQuery("SET hive.cli.print.header") + rs2.next() + assert(defaultV2 === rs2.getString(1)) + rs2.close() + }, + + // accessing the cached data in another session + { statement => + + val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") + val buf1 = new collection.mutable.ArrayBuffer[Int]() + while (rs1.next()) { + buf1 += rs1.getInt(1) + } + rs1.close() + + val rs2 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf2 = new collection.mutable.ArrayBuffer[Int]() + while (rs2.next()) { + buf2 += rs2.getInt(1) + } + rs2.close() + + assert(buf1 === buf2) + statement.executeQuery("UNCACHE TABLE test_table") + + // TODO need to figure out how to determine if the data loaded from cache + val rs3 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf3 = new collection.mutable.ArrayBuffer[Int]() + while (rs3.next()) { + buf3 += rs3.getInt(1) + } + rs3.close() + + assert(buf1 === buf3) + }, + + // accessing the uncached table + { statement => + + // TODO need to figure out how to determine if the data loaded from cache + val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") + val buf1 = new collection.mutable.ArrayBuffer[Int]() + while (rs1.next()) { + buf1 += rs1.getInt(1) + } + rs1.close() + + val rs2 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf2 = new collection.mutable.ArrayBuffer[Int]() + while (rs2.next()) { + buf2 += rs2.getInt(1) + } + rs2.close() + + assert(buf1 === buf2) + } + ) + } } class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { @@ -245,15 +385,22 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { s"jdbc:hive2://localhost:$serverPort/" } - protected def withJdbcStatement(f: Statement => Unit): Unit = { - val connection = DriverManager.getConnection(jdbcUri, user, "") - val statement = connection.createStatement() - - try f(statement) finally { - statement.close() - connection.close() + def withMultipleConnectionJdbcStatement(fs: (Statement => Unit)*) { + val user = System.getProperty("user.name") + val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).map { case (s, f) => f(s) } + } finally { + statements.map(_.close()) + connections.map(_.close()) } } + + def withJdbcStatement(f: Statement => Unit) { + withMultipleConnectionJdbcStatement(f) + } } abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll with Logging { diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index 13116b40bb259..95a6e86d0546d 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -18,8 +18,15 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{Date, Timestamp} +import java.util.concurrent.Executors import java.util.{ArrayList => JArrayList, Map => JMap} +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.thrift.TProtocolVersion +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, Map => SMap} @@ -29,7 +36,7 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation -import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.cli.session.{SessionManager, HiveSession} import org.apache.spark.Logging import org.apache.spark.sql.{DataFrame, SQLConf, Row => SparkRow} @@ -220,3 +227,42 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.FINISHED) } } + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } + + override def openSession( + username: String, + passwd: String, + sessionConf: java.util.Map[String, String], + withImpersonation: Boolean, + delegationToken: String): SessionHandle = { + hiveContext.openSession() + + super.openSession(username, passwd, sessionConf, withImpersonation, delegationToken) + } + + override def closeSession(sessionHandle: SessionHandle) { + super.closeSession(sessionHandle) + sparkSqlOperationManager.sessionToActivePool -= sessionHandle + + hiveContext.detachSession() + } +} diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 9b8faeff94eab..178eb1af7cdcd 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -18,8 +18,15 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{Date, Timestamp} +import java.util.concurrent.Executors import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.thrift.TProtocolVersion +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, Map => SMap} @@ -27,7 +34,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation -import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.cli.session.{SessionManager, HiveSession} import org.apache.spark.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf} @@ -191,3 +198,43 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.FINISHED) } } + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } + + override def openSession( + protocol: TProtocolVersion, + username: String, + passwd: String, + sessionConf: java.util.Map[String, String], + withImpersonation: Boolean, + delegationToken: String): SessionHandle = { + hiveContext.openSession() + + super.openSession(protocol, username, passwd, sessionConf, withImpersonation, delegationToken) + } + + override def closeSession(sessionHandle: SessionHandle) { + super.closeSession(sessionHandle) + sparkSqlOperationManager.sessionToActivePool -= sessionHandle + + hiveContext.detachSession() + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index c439dfe0a71f8..a5c435fdfa778 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -49,10 +49,6 @@ import org.apache.spark.sql.types._ class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") - } - /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive @@ -214,33 +210,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - /** - * SQLConf and HiveConf contracts: - * - * 1. reuse existing started SessionState if any - * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the - * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be - * set in the SQLConf *as well as* in the HiveConf. - */ - @transient protected[hive] lazy val sessionState: SessionState = { - var state = SessionState.get() - if (state == null) { - state = new SessionState(new HiveConf(classOf[SessionState])) - SessionState.start(state) - } - if (state.out == null) { - state.out = new PrintStream(outputBuffer, true, "UTF-8") - } - if (state.err == null) { - state.err = new PrintStream(outputBuffer, true, "UTF-8") - } - state - } + protected[hive] def sessionState = tlSession.get().asInstanceOf[this.SQLSession].sessionState - @transient protected[hive] lazy val hiveconf: HiveConf = { - setConf(sessionState.getConf.getAllProperties) - sessionState.getConf - } + protected[hive] def hiveconf = tlSession.get().asInstanceOf[this.SQLSession].hiveconf override def setConf(key: String, value: String): Unit = { super.setConf(key, value) @@ -272,6 +244,44 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Nil } + override protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[hive] class SQLSession extends super.SQLSession { + protected[sql] override lazy val conf: SQLConf = new SQLConf { + override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + } + + protected[hive] lazy val hiveconf: HiveConf = { + setConf(sessionState.getConf.getAllProperties) + sessionState.getConf + } + + /** + * SQLConf and HiveConf contracts: + * + * 1. reuse existing started SessionState if any + * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the + * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be + * set in the SQLConf *as well as* in the HiveConf. + */ + protected[hive] lazy val sessionState: SessionState = { + var state = SessionState.get() + if (state == null) { + state = new SessionState(new HiveConf(classOf[SessionState])) + SessionState.start(state) + } + if (state.out == null) { + state.out = new PrintStream(outputBuffer, true, "UTF-8") + } + if (state.err == null) { + state.err = new PrintStream(outputBuffer, true, "UTF-8") + } + state + } + } + /** * Runs the specified SQL query using Hive. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index a2d99f1f4b28d..4859991e2351a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -102,10 +102,16 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) - /** Fewer partitions to speed up testing. */ - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt - override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + override protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[hive] class SQLSession extends super.SQLSession { + /** Fewer partitions to speed up testing. */ + protected[sql] override lazy val conf: SQLConf = new SQLConf { + override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + } } /** From d19efeddc0cb710c9496af11e447d39e1ad61b31 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Mon, 16 Mar 2015 12:13:18 -0700 Subject: [PATCH 30/62] [SPARK-6330] Fix filesystem bug in newParquet relation If I'm running this locally and my path points to S3, this would currently error out because of incorrect FS. I tested this in a scenario that previously didn't work, this change seemed to fix the issue. Author: Volodymyr Lyubinets Closes #5020 from vlyubin/parquertbug and squashes the following commits: a645ad5 [Volodymyr Lyubinets] Fix filesystem bug in newParquet relation --- .../main/scala/org/apache/spark/sql/parquet/newParquet.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 234e6bb8443af..c38b6e8c61d8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.parquet import java.io.IOException import java.lang.{Double => JDouble, Float => JFloat, Long => JLong} import java.math.{BigDecimal => JBigDecimal} +import java.net.URI import java.text.SimpleDateFormat import java.util.{Date, List => JList} @@ -244,11 +245,10 @@ private[sql] case class ParquetRelation2( * Refreshes `FileStatus`es, footers, partition spec, and table schema. */ def refresh(): Unit = { - val fs = FileSystem.get(sparkContext.hadoopConfiguration) - // Support either reading a collection of raw Parquet part-files, or a collection of folders // containing Parquet files (e.g. partitioned Parquet table). val baseStatuses = paths.distinct.map { p => + val fs = FileSystem.get(URI.create(p), sparkContext.hadoopConfiguration) val qualified = fs.makeQualified(new Path(p)) if (!fs.exists(qualified) && maybeSchema.isDefined) { @@ -262,6 +262,7 @@ private[sql] case class ParquetRelation2( // Lists `FileStatus`es of all leaf nodes (files) under all base directories. val leaves = baseStatuses.flatMap { f => + val fs = FileSystem.get(f.getPath.toUri, sparkContext.hadoopConfiguration) SparkHadoopUtil.get.listLeafStatuses(fs, f.getPath).filter { f => isSummaryFile(f.getPath) || !(f.getPath.getName.startsWith("_") || f.getPath.getName.startsWith(".")) From f149b8b5e542af44650923d0156f037121b45a20 Mon Sep 17 00:00:00 2001 From: lisurprise Date: Mon, 16 Mar 2015 13:10:32 -0700 Subject: [PATCH 31/62] [SPARK-6077] Remove streaming tab while stopping StreamingContext Currently we would create a new streaming tab for each streamingContext even if there's already one on the same sparkContext which would cause duplicate StreamingTab created and none of them is taking effect. snapshot: https://www.dropbox.com/s/t4gd6hqyqo0nivz/bad%20multiple%20streamings.png?dl=0 How to reproduce: 1) import org.apache.spark.SparkConf import org.apache.spark.streaming. {Seconds, StreamingContext} import org.apache.spark.storage.StorageLevel val ssc = new StreamingContext(sc, Seconds(1)) val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() ..... 2) ssc.stop(false) val ssc = new StreamingContext(sc, Seconds(1)) val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() Author: lisurprise Closes #4828 from zhichao-li/master and squashes the following commits: c329806 [lisurprise] add test for attaching/detaching streaming tab 51e6c7f [lisurprise] move detach method into StreamingTab 31a44fa [lisurprise] add unit test for attaching and detaching new tab db25ed2 [lisurprise] clean code 8281bcb [lisurprise] clean code 193c542 [lisurprise] remove streaming tab while closing streaming context --- .../scala/org/apache/spark/ui/WebUI.scala | 28 +++++- .../org/apache/spark/ui/UISeleniumSuite.scala | 50 +++++++++- .../scala/org/apache/spark/ui/UISuite.scala | 38 +------- streaming/pom.xml | 5 + .../spark/streaming/StreamingContext.scala | 1 + .../spark/streaming/ui/StreamingPage.scala | 4 +- .../spark/streaming/ui/StreamingTab.scala | 4 + .../spark/streaming/UISeleniumSuite.scala | 95 +++++++++++++++++++ .../org/apache/spark/streaming/UISuite.scala | 55 ----------- 9 files changed, 179 insertions(+), 101 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala delete mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index ec68837a1516c..ea548f23120d9 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -20,14 +20,15 @@ package org.apache.spark.ui import javax.servlet.http.HttpServletRequest import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler import org.json4s.JsonAST.{JNothing, JValue} -import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SecurityManager, SparkConf} /** * The top level component of the UI hierarchy that contains the server. @@ -45,6 +46,7 @@ private[spark] abstract class WebUI( protected val tabs = ArrayBuffer[WebUITab]() protected val handlers = ArrayBuffer[ServletContextHandler]() + protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]] protected var serverInfo: Option[ServerInfo] = None protected val localHostName = Utils.localHostName() protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) @@ -60,14 +62,30 @@ private[spark] abstract class WebUI( tab.pages.foreach(attachPage) tabs += tab } + + def detachTab(tab: WebUITab) { + tab.pages.foreach(detachPage) + tabs -= tab + } + + def detachPage(page: WebUIPage) { + pageToHandlers.remove(page).foreach(_.foreach(detachHandler)) + } /** Attach a page to this UI. */ def attachPage(page: WebUIPage) { val pagePath = "/" + page.prefix - attachHandler(createServletHandler(pagePath, - (request: HttpServletRequest) => page.render(request), securityManager, basePath)) - attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", - (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) + val renderHandler = createServletHandler(pagePath, + (request: HttpServletRequest) => page.render(request), securityManager, basePath) + val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath) + attachHandler(renderHandler) + attachHandler(renderJsonHandler) + pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) + .append(renderHandler) + pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) + .append(renderJsonHandler) + } /** Attach a handler to this UI. */ diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 6a972381faf14..0d155982a8c54 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,20 +17,24 @@ package org.apache.spark.ui +import javax.servlet.http.HttpServletRequest + import scala.collection.JavaConversions._ +import scala.xml.Node -import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.openqa.selenium.{By, WebDriver} import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ -import org.apache.spark._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException + /** * Selenium tests for the Spark Web UI. */ @@ -310,4 +314,46 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } } + + test("attaching and detaching a new tab") { + withSpark(newSparkContext()) { sc => + val sparkUI = sc.ui.get + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/")) + find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) + find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) + find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) + find(cssSelector("""ul li a[href*="environment"]""")) should not be(None) + find(cssSelector("""ul li a[href*="foo"]""")) should not be(None) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check whether new page exists + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + find(cssSelector("b")).get.text should include ("html magic") + } + sparkUI.detachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/")) + find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) + find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) + find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) + find(cssSelector("""ul li a[href*="environment"]""")) should not be(None) + find(cssSelector("""ul li a[href*="foo"]""")) should be(None) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check new page not exist + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + find(cssSelector("b")) should be(None) + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 92a21f82f3c21..77a038dc1720d 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.ui import java.net.ServerSocket -import javax.servlet.http.HttpServletRequest import scala.io.Source import scala.util.{Failure, Success, Try} @@ -28,9 +27,8 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.LocalSparkContext._ -import scala.xml.Node +import org.apache.spark.{SparkConf, SparkContext} class UISuite extends FunSuite { @@ -72,40 +70,6 @@ class UISuite extends FunSuite { } } - ignore("attaching a new tab") { - withSpark(newSparkContext()) { sc => - val sparkUI = sc.ui.get - - val newTab = new WebUITab(sparkUI, "foo") { - attachPage(new WebUIPage("") { - def render(request: HttpServletRequest): Seq[Node] = { - "html magic" - } - }) - } - sparkUI.attachTab(newTab) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sparkUI.appUIAddress).mkString - assert(!html.contains("random data that should not be present")) - - // check whether new page exists - assert(html.toLowerCase.contains("foo")) - - // check whether other pages still exist - assert(html.toLowerCase.contains("stages")) - assert(html.toLowerCase.contains("storage")) - assert(html.toLowerCase.contains("environment")) - assert(html.toLowerCase.contains("executors")) - } - - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sparkUI.appUIAddress.stripSuffix("/") + "/foo").mkString - // check whether new page exists - assert(html.contains("magic")) - } - } - } - test("jetty selects different port under contention") { val server = new ServerSocket(0) val startPort = server.getLocalPort diff --git a/streaming/pom.xml b/streaming/pom.xml index 0370b0e9e1aa3..96508d83f4049 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -82,6 +82,11 @@ junit test + + org.seleniumhq.selenium + selenium-java + test + com.novocode junit-interface diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ba3f23434f24c..b5b6770a8a150 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -578,6 +578,7 @@ class StreamingContext private[streaming] ( // Even if we have already stopped, we still need to attempt to stop the SparkContext because // a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true). if (stopSparkContext) sc.stop() + uiTab.foreach(_.detach()) // The state should always be Stopped after calling `stop()`, even if we haven't started yet: state = Stopped } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 98e9a2e639e25..bfe8086fcf8fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -32,7 +32,7 @@ private[ui] class StreamingPage(parent: StreamingTab) extends WebUIPage("") with Logging { private val listener = parent.listener - private val startTime = Calendar.getInstance().getTime() + private val startTime = System.currentTimeMillis() private val emptyCell = "-" /** Render the page */ @@ -47,7 +47,7 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Generate basic stats of the streaming program */ private def generateBasicStats(): Seq[Node] = { - val timeSinceStart = System.currentTimeMillis() - startTime.getTime + val timeSinceStart = System.currentTimeMillis() - startTime
  • Started at: {startTime.toString} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index d9d04cd706a04..9a860ea4a6c68 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -36,6 +36,10 @@ private[spark] class StreamingTab(ssc: StreamingContext) ssc.addStreamingListener(listener) attachPage(new StreamingPage(this)) parent.attachTab(this) + + def detach() { + getSparkUI(ssc).detachTab(this) + } } private object StreamingTab { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala new file mode 100644 index 0000000000000..87a0395efbf2a --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.streaming + +import org.openqa.selenium.WebDriver +import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.scalatest._ +import org.scalatest.concurrent.Eventually._ +import org.scalatest.selenium.WebBrowser +import org.scalatest.time.SpanSugar._ + +import org.apache.spark._ + + + + +/** + * Selenium tests for the Spark Web UI. + */ +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll with TestSuiteBase { + + implicit var webDriver: WebDriver = _ + + override def beforeAll(): Unit = { + webDriver = new HtmlUnitDriver + } + + override def afterAll(): Unit = { + if (webDriver != null) { + webDriver.quit() + } + } + + /** + * Create a test SparkStreamingContext with the SparkUI enabled. + */ + private def newSparkStreamingContext(): StreamingContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val ssc = new StreamingContext(conf, Seconds(1)) + assert(ssc.sc.ui.isDefined, "Spark UI is not started!") + ssc + } + + test("attaching and detaching a Streaming tab") { + withStreamingContext(newSparkStreamingContext()) { ssc => + val sparkUI = ssc.sparkContext.ui.get + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sparkUI.appUIAddress.stripSuffix("/")) + find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check whether streaming page exists + go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq + statisticText should contain("Network receivers:") + statisticText should contain("Batch interval:") + } + + ssc.stop(false) + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sparkUI.appUIAddress.stripSuffix("/")) + find(cssSelector( """ul li a[href*="streaming"]""")) should be(None) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq + statisticText should not contain ("Network receivers:") + statisticText should not contain ("Batch interval:") + } + } + } +} + diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala deleted file mode 100644 index 8e30118266855..0000000000000 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.streaming - -import scala.io.Source - -import org.scalatest.FunSuite -import org.scalatest.concurrent.Eventually._ -import org.scalatest.time.SpanSugar._ - -import org.apache.spark.SparkConf - -class UISuite extends FunSuite { - - // Ignored: See SPARK-1530 - ignore("streaming tab in spark UI") { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set("spark.ui.enabled", "true") - val ssc = new StreamingContext(conf, Seconds(1)) - assert(ssc.sc.ui.isDefined, "Spark UI is not started!") - val ui = ssc.sc.ui.get - - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(ui.appUIAddress).mkString - assert(!html.contains("random data that should not be present")) - // test if streaming tab exist - assert(html.toLowerCase.contains("streaming")) - // test if other Spark tabs still exist - assert(html.toLowerCase.contains("stages")) - } - - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(ui.appUIAddress.stripSuffix("/") + "/streaming").mkString - assert(html.toLowerCase.contains("batch")) - assert(html.toLowerCase.contains("network")) - } - } -} From e3f315ac358dfe4f5b9705c3eac76e8b1e24f82a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 16 Mar 2015 16:26:55 -0700 Subject: [PATCH 32/62] [SPARK-6327] [PySpark] fix launch spark-submit from python SparkSubmit should be launched without setting PYSPARK_SUBMIT_ARGS cc JoshRosen , this mode is actually used by python unit test, so I will not add more test for it. Author: Davies Liu Closes #5019 from davies/fix_submit and squashes the following commits: 2c20b0c [Davies Liu] fix launch spark-submit from python --- bin/pyspark | 1 - python/pyspark/java_gateway.py | 6 ++---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index e7f6a1a072c2a..776b28dc41099 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -89,7 +89,6 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR - export PYSPARK_SUBMIT_ARGS=pyspark-shell if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 43d2cf5171880..0a16cbd8bff62 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -38,10 +38,8 @@ def launch_gateway(): # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" script = "./bin/spark-submit.cmd" if on_windows else "./bin/spark-submit" - submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS") - submit_args = submit_args if submit_args is not None else "" - submit_args = shlex.split(submit_args) - command = [os.path.join(SPARK_HOME, script)] + submit_args + submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell") + command = [os.path.join(SPARK_HOME, script)] + shlex.split(submit_args) # Start a socket that will be used by PythonGatewayServer to communicate its port to us callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) From 9667b9f9c3239f814a0b1120355d9e7bd7a89158 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 17 Mar 2015 12:29:15 +0800 Subject: [PATCH 33/62] [SPARK-5712] [SQL] fix comment with semicolon at end ---- comment; Author: Daoyuan Wang Closes #4500 from adrian-wang/semicolon and squashes the following commits: 70b8abb [Daoyuan Wang] use mkstring instead of reduce 2d49738 [Daoyuan Wang] remove outdated golden file 317346e [Daoyuan Wang] only skip comment with semicolon at end of line, to avoid golden file outdated d3ae01e [Daoyuan Wang] fix error a11602d [Daoyuan Wang] fix comment with semicolon at end --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 25 ++++++++++--------- .../execution/HiveCompatibilitySuite.scala | 1 + ...micolon-0-f104632770dc96b81f00ccdac51fe5a8 | 1 + .../hive/execution/HiveComparisonTest.scala | 5 +++- 4 files changed, 19 insertions(+), 13 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 401e97b162dea..895688ab2ec2e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -202,20 +202,21 @@ private[hive] object SparkSQLCLIDriver { var line = reader.readLine(currentPrompt + "> ") while (line != null) { - if (prefix.nonEmpty) { - prefix += '\n' - } + if (!line.startsWith("--")) { + if (prefix.nonEmpty) { + prefix += '\n' + } - if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { - line = prefix + line - ret = cli.processLine(line, true) - prefix = "" - currentPrompt = promptWithCurrentDB - } else { - prefix = prefix + line - currentPrompt = continuedPromptWithDBSpaces + if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { + line = prefix + line + ret = cli.processLine(line, true) + prefix = "" + currentPrompt = promptWithCurrentDB + } else { + prefix = prefix + line + currentPrompt = continuedPromptWithDBSpaces + } } - line = reader.readLine(currentPrompt + "> ") } 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 6126ce7130426..68cb34d698ef3 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 @@ -726,6 +726,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "select_unquote_and", "select_unquote_not", "select_unquote_or", + "semicolon", "semijoin", "serde_regex", "serde_reported_schema", diff --git a/sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 b/sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 @@ -0,0 +1 @@ +500 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a90bd1e257ade..8f3285242091c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -241,7 +241,10 @@ abstract class HiveComparisonTest // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) - val allQueries = sql.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + val sqlWithoutComment = + sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") + val allQueries = + sqlWithoutComment.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq // TODO: DOCUMENT UNSUPPORTED val queryList = From f0edeae7f9ab7eae02c227be9162ec69d22c92bd Mon Sep 17 00:00:00 2001 From: "Kevin (Sangwoo) Kim" Date: Mon, 16 Mar 2015 23:49:23 -0700 Subject: [PATCH 34/62] [SPARK-6299][CORE] ClassNotFoundException in standalone mode when running groupByKey with class defined in REPL ``` case class ClassA(value: String) val rdd = sc.parallelize(List(("k1", ClassA("v1")), ("k1", ClassA("v2")) )) rdd.groupByKey.collect ``` This code used to be throw exception in spark-shell, because while shuffling ```JavaSerializer```uses ```defaultClassLoader``` which was defined like ```env.serializer.setDefaultClassLoader(urlClassLoader)```. It should be ```env.serializer.setDefaultClassLoader(replClassLoader)```, like ``` override def run() { val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) ``` in TaskRunner. When ```replClassLoader``` cannot be defined, it's identical with ```urlClassLoader``` Author: Kevin (Sangwoo) Kim Closes #5046 from swkimme/master and squashes the following commits: fa2b9ee [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() ) 6e9620b [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() ) d23e4e2 [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() ) a4a3c8a [Kevin (Sangwoo) Kim] add 'class defined in repl - shuffle' test to ReplSuite bd00da5 [Kevin (Sangwoo) Kim] add 'class defined in repl - shuffle' test to ReplSuite c1b1fc7 [Kevin (Sangwoo) Kim] use REPL class loader for executor's serializer --- .../org/apache/spark/executor/Executor.scala | 2 +- .../org/apache/spark/repl/ReplSuite.scala | 50 ++++++++++++------- .../org/apache/spark/repl/ReplSuite.scala | 50 ++++++++++++------- 3 files changed, 63 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index a897e532184ac..6196f7b165049 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -103,7 +103,7 @@ private[spark] class Executor( private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) // Set the classloader for serializer - env.serializer.setDefaultClassLoader(urlClassLoader) + env.serializer.setDefaultClassLoader(replClassLoader) // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 249f438459300..934daaeaafca1 100644 --- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -121,9 +121,9 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |var v = 7 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -137,7 +137,7 @@ class ReplSuite extends FunSuite { |class C { |def foo = 5 |} - |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => (new C).foo).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -148,7 +148,7 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |def double(x: Int) = x + x - |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => double(x)).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -160,9 +160,9 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -178,9 +178,9 @@ class ReplSuite extends FunSuite { """ |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -216,14 +216,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -262,7 +262,7 @@ class ReplSuite extends FunSuite { |val sqlContext = new org.apache.spark.sql.SQLContext(sc) |import sqlContext.implicits._ |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF.collect() + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF().collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -275,7 +275,7 @@ class ReplSuite extends FunSuite { |val t = new TestClass |import t.testMethod |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -287,14 +287,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -309,10 +309,22 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local[2]", """ |case class Foo(i: Int) - |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect + |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("ret: Array[Foo] = Array(Foo(1),", output) } + + test("collecting objects of class defined in repl - shuffling") { + val output = runInterpreter("local-cluster[1,1,512]", + """ + |case class Foo(i: Int) + |val list = List((1, Foo(1)), (1, Foo(2))) + |val ret = sc.parallelize(list).groupByKey().collect() + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("ret: Array[(Int, Iterable[Foo])] = Array((1,", output) + } } 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 b3bd135548124..fbef5b25ba688 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 @@ -128,9 +128,9 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |var v = 7 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -144,7 +144,7 @@ class ReplSuite extends FunSuite { |class C { |def foo = 5 |} - |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => (new C).foo).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -155,7 +155,7 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |def double(x: Int) = x + x - |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => double(x)).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -167,9 +167,9 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -185,9 +185,9 @@ class ReplSuite extends FunSuite { """ |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -224,14 +224,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -270,7 +270,7 @@ class ReplSuite extends FunSuite { |val sqlContext = new org.apache.spark.sql.SQLContext(sc) |import sqlContext.implicits._ |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF.collect + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF().collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -283,7 +283,7 @@ class ReplSuite extends FunSuite { |val t = new TestClass |import t.testMethod |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -295,14 +295,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -317,10 +317,22 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local[2]", """ |case class Foo(i: Int) - |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect + |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("ret: Array[Foo] = Array(Foo(1),", output) } + + test("collecting objects of class defined in repl - shuffling") { + val output = runInterpreter("local-cluster[1,1,512]", + """ + |case class Foo(i: Int) + |val list = List((1, Foo(1)), (1, Foo(2))) + |val ret = sc.parallelize(list).groupByKey().collect() + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("ret: Array[(Int, Iterable[Foo])] = Array((1,", output) + } } From 68707225f1a4081aadbf0fd7e6221293a190529b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lomig=20Me=CC=81gard?= Date: Mon, 16 Mar 2015 23:52:42 -0700 Subject: [PATCH 35/62] [SQL][docs][minor] Fixed sample code in SQLContext scaladoc MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Error in the code sample of the `implicits` object in `SQLContext`. Author: Lomig Mégard Closes #5051 from tarfaa/simple and squashes the following commits: 5a88acc [Lomig Mégard] [docs][minor] Fixed sample code in SQLContext scaladoc --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 297d0d644a423..6de46a50db20e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -242,8 +242,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * common Scala objects into [[DataFrame]]s. * * {{{ - * val sqlContext = new SQLContext - * import sqlContext._ + * val sqlContext = new SQLContext(sc) + * import sqlContext.implicits._ * }}} * * @group basic From b3e6eca81f79ba3c9205211797fa825b199bac83 Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Mon, 16 Mar 2015 23:54:54 -0700 Subject: [PATCH 36/62] [SPARK-6357][GraphX] Add unapply in EdgeContext This extractor is mainly used for Graph#aggregateMessages*. Author: Takeshi YAMAMURO Closes #5047 from maropu/AddUnapplyInEdgeContext and squashes the following commits: 87e04df [Takeshi YAMAMURO] Add unapply in EdgeContext --- .../org/apache/spark/graphx/EdgeContext.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala index f70715fca6eea..d8be02e2023d5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala @@ -49,3 +49,20 @@ abstract class EdgeContext[VD, ED, A] { et } } + +object EdgeContext { + + /** + * Extractor mainly used for Graph#aggregateMessages*. + * Example: + * {{{ + * val messages = graph.aggregateMessages( + * case ctx @ EdgeContext(_, _, _, _, attr) => + * ctx.sendToDst(attr) + * , _ + _) + * }}} + */ + def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]) = + Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr) +} + From b2d8c02224892192b1aa314b4265fe50845932f9 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 16 Mar 2015 23:58:52 -0700 Subject: [PATCH 37/62] SPARK-6044 [CORE] RDD.aggregate() should not use the closure serializer on the zero value Use configured serializer in RDD.aggregate, to match PairRDDFunctions.aggregateByKey, instead of closure serializer. Compare with https://github.com/apache/spark/blob/e60ad2f4c47b011be7a3198689ac2b82ee317d96/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala#L127 Author: Sean Owen Closes #5028 from srowen/SPARK-6044 and squashes the following commits: a4040a7 [Sean Owen] Use configured serializer in RDD.aggregate, to match PairRDDFunctions.aggregateByKey, instead of closure serializer --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index cf0433010aa03..a139780d967e9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -960,7 +960,7 @@ abstract class RDD[T: ClassTag]( */ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { // Clone the zero value since we will also be serializing it as part of tasks - var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + var jobResult = Utils.clone(zeroValue, sc.env.serializer.newInstance()) val cleanSeqOp = sc.clean(seqOp) val cleanCombOp = sc.clean(combOp) val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) From 25f35806e307c9635e63b8b12698446a14bdd29d Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 17 Mar 2015 11:18:27 +0000 Subject: [PATCH 38/62] [SPARK-4011] tighten the visibility of the members in Master/Worker class https://issues.apache.org/jira/browse/SPARK-4011 Currently, most of the members in Master/Worker are with public accessibility. We might wish to tighten the accessibility of them a bit more discussion is here: https://github.com/apache/spark/pull/2828 Author: CodingCat Closes #4844 from CodingCat/SPARK-4011 and squashes the following commits: 1a64175 [CodingCat] fix compilation issue e7fd375 [CodingCat] Sean is right.... f5034a4 [CodingCat] fix rebase mistake 8d5b0c0 [CodingCat] loose more fields 0072f96 [CodingCat] lose some restrictions based on the possible design intention de77286 [CodingCat] tighten accessibility of deploy package 12b4fd3 [CodingCat] tighten accessibility of deploy.worker 1243bc7 [CodingCat] tighten accessibility of deploy.rest c5f622c [CodingCat] tighten the accessibility of deploy.history d441e20 [CodingCat] tighten accessibility of deploy.client 4e0ce4a [CodingCat] tighten the accessibility of the members of classes in master 23cddbb [CodingCat] stylistic fix 9a3a340 [CodingCat] tighten the access of worker class 67a0559 [CodingCat] tighten the access permission in Master --- .../apache/spark/deploy/ClientArguments.scala | 12 +- .../spark/deploy/DriverDescription.scala | 2 +- .../spark/deploy/ExecutorDescription.scala | 2 +- .../apache/spark/deploy/ExecutorState.scala | 2 +- .../spark/deploy/FaultToleranceTest.scala | 60 +++++----- .../apache/spark/deploy/JsonProtocol.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 38 +++--- .../spark/deploy/SparkSubmitArguments.scala | 2 +- .../spark/deploy/client/AppClient.scala | 18 +-- .../spark/deploy/client/TestClient.scala | 2 +- .../history/ApplicationHistoryProvider.scala | 4 +- .../spark/deploy/history/HistoryPage.scala | 2 +- .../history/HistoryServerArguments.scala | 3 +- .../spark/deploy/master/ApplicationInfo.scala | 19 +-- .../deploy/master/ApplicationSource.scala | 2 +- .../deploy/master/ApplicationState.scala | 2 +- .../spark/deploy/master/DriverInfo.scala | 2 +- .../spark/deploy/master/DriverState.scala | 2 +- .../spark/deploy/master/ExecutorDesc.scala | 2 +- .../master/FileSystemPersistenceEngine.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 110 +++++++++--------- .../spark/deploy/master/MasterArguments.scala | 6 +- .../deploy/master/PersistenceEngine.scala | 2 +- .../deploy/master/RecoveryModeFactory.scala | 4 +- .../spark/deploy/master/RecoveryState.scala | 2 +- .../deploy/master/SparkCuratorUtil.scala | 10 +- .../spark/deploy/master/WorkerState.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 6 +- .../master/ZooKeeperPersistenceEngine.scala | 12 +- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../master/ui/HistoryNotFoundPage.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 2 +- .../spark/deploy/master/ui/MasterWebUI.scala | 6 +- .../deploy/rest/StandaloneRestClient.scala | 14 +-- .../deploy/rest/StandaloneRestServer.scala | 2 +- .../rest/SubmitRestProtocolException.scala | 6 +- .../rest/SubmitRestProtocolMessage.scala | 2 +- .../rest/SubmitRestProtocolRequest.scala | 4 +- .../rest/SubmitRestProtocolResponse.scala | 10 +- .../spark/deploy/worker/CommandUtils.scala | 2 +- .../spark/deploy/worker/DriverRunner.scala | 24 ++-- .../spark/deploy/worker/ExecutorRunner.scala | 24 ++-- .../apache/spark/deploy/worker/Worker.scala | 85 +++++++------- .../spark/deploy/worker/WorkerArguments.scala | 4 +- .../spark/deploy/worker/WorkerSource.scala | 2 +- .../spark/deploy/worker/WorkerWatcher.scala | 2 +- .../spark/deploy/worker/ui/LogPage.scala | 2 +- .../spark/deploy/worker/ui/WorkerPage.scala | 7 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 6 +- 49 files changed, 277 insertions(+), 265 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 415bd50591692..53bc62aff7395 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.{IntParam, MemoryParam} /** * Command-line parser for the driver client. */ -private[spark] class ClientArguments(args: Array[String]) { +private[deploy] class ClientArguments(args: Array[String]) { import ClientArguments._ var cmd: String = "" // 'launch' or 'kill' @@ -96,7 +96,7 @@ private[spark] class ClientArguments(args: Array[String]) { /** * Print usage and exit JVM with the given exit code. */ - def printUsageAndExit(exitCode: Int) { + private def printUsageAndExit(exitCode: Int) { // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars // separately similar to in the YARN client. val usage = @@ -116,10 +116,10 @@ private[spark] class ClientArguments(args: Array[String]) { } } -object ClientArguments { - private[spark] val DEFAULT_CORES = 1 - private[spark] val DEFAULT_MEMORY = 512 // MB - private[spark] val DEFAULT_SUPERVISE = false +private[deploy] object ClientArguments { + val DEFAULT_CORES = 1 + val DEFAULT_MEMORY = 512 // MB + val DEFAULT_SUPERVISE = false def isValidJarUrl(s: String): Boolean = { try { diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala index b056a19ce6598..659fb434a80f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[spark] class DriverDescription( +private[deploy] class DriverDescription( val jarUrl: String, val mem: Int, val cores: Int, diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala index 2abf0b69dddb3..ec23371b52f93 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala @@ -22,7 +22,7 @@ package org.apache.spark.deploy * This state is sufficient for the Master to reconstruct its internal data structures during * failover. */ -private[spark] class ExecutorDescription( +private[deploy] class ExecutorDescription( val appId: String, val execId: Int, val cores: Int, diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 9f34d01e6db48..efa88c62e1f5d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[spark] object ExecutorState extends Enumeration { +private[deploy] object ExecutorState extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST, EXITED = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 47dbcd87c35b5..4e58aa0ed4c7e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -55,29 +55,29 @@ import org.apache.spark.deploy.master.{RecoveryState, SparkCuratorUtil} * - The docker images tagged spark-test-master and spark-test-worker are built from the * docker/ directory. Run 'docker/spark-test/build' to generate these. */ -private[spark] object FaultToleranceTest extends App with Logging { +private object FaultToleranceTest extends App with Logging { - val conf = new SparkConf() - val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + private val conf = new SparkConf() + private val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") - val masters = ListBuffer[TestMasterInfo]() - val workers = ListBuffer[TestWorkerInfo]() - var sc: SparkContext = _ + private val masters = ListBuffer[TestMasterInfo]() + private val workers = ListBuffer[TestWorkerInfo]() + private var sc: SparkContext = _ - val zk = SparkCuratorUtil.newClient(conf) + private val zk = SparkCuratorUtil.newClient(conf) - var numPassed = 0 - var numFailed = 0 + private var numPassed = 0 + private var numFailed = 0 - val sparkHome = System.getenv("SPARK_HOME") + private val sparkHome = System.getenv("SPARK_HOME") assertTrue(sparkHome != null, "Run with a valid SPARK_HOME") - val containerSparkHome = "/opt/spark" - val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) + private val containerSparkHome = "/opt/spark" + private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip - def afterEach() { + private def afterEach() { if (sc != null) { sc.stop() sc = null @@ -179,7 +179,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - def test(name: String)(fn: => Unit) { + private def test(name: String)(fn: => Unit) { try { fn numPassed += 1 @@ -197,19 +197,19 @@ private[spark] object FaultToleranceTest extends App with Logging { afterEach() } - def addMasters(num: Int) { + private def addMasters(num: Int) { logInfo(s">>>>> ADD MASTERS $num <<<<<") (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) } } - def addWorkers(num: Int) { + private def addWorkers(num: Int) { logInfo(s">>>>> ADD WORKERS $num <<<<<") val masterUrls = getMasterUrls(masters) (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } } /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ - def createClient() = { + private def createClient() = { logInfo(">>>>> CREATE CLIENT <<<<<") if (sc != null) { sc.stop() } // Counter-hack: Because of a hack in SparkEnv#create() that changes this @@ -218,17 +218,17 @@ private[spark] object FaultToleranceTest extends App with Logging { sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) } - def getMasterUrls(masters: Seq[TestMasterInfo]): String = { + private def getMasterUrls(masters: Seq[TestMasterInfo]): String = { "spark://" + masters.map(master => master.ip + ":7077").mkString(",") } - def getLeader: TestMasterInfo = { + private def getLeader: TestMasterInfo = { val leaders = masters.filter(_.state == RecoveryState.ALIVE) assertTrue(leaders.size == 1) leaders(0) } - def killLeader(): Unit = { + private def killLeader(): Unit = { logInfo(">>>>> KILL LEADER <<<<<") masters.foreach(_.readState()) val leader = getLeader @@ -236,9 +236,9 @@ private[spark] object FaultToleranceTest extends App with Logging { leader.kill() } - def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) + private def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) - def terminateCluster() { + private def terminateCluster() { logInfo(">>>>> TERMINATE CLUSTER <<<<<") masters.foreach(_.kill()) workers.foreach(_.kill()) @@ -247,7 +247,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } /** This includes Client retry logic, so it may take a while if the cluster is recovering. */ - def assertUsable() = { + private def assertUsable() = { val f = future { try { val res = sc.parallelize(0 until 10).collect() @@ -269,7 +269,7 @@ private[spark] object FaultToleranceTest extends App with Logging { * Asserts that the cluster is usable and that the expected masters and workers * are all alive in a proper configuration (e.g., only one leader). */ - def assertValidClusterState() = { + private def assertValidClusterState() = { logInfo(">>>>> ASSERT VALID CLUSTER STATE <<<<<") assertUsable() var numAlive = 0 @@ -325,7 +325,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - def assertTrue(bool: Boolean, message: String = "") { + private def assertTrue(bool: Boolean, message: String = "") { if (!bool) { throw new IllegalStateException("Assertion failed: " + message) } @@ -335,7 +335,7 @@ private[spark] object FaultToleranceTest extends App with Logging { numFailed)) } -private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) +private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { implicit val formats = org.json4s.DefaultFormats @@ -377,7 +377,7 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val format(ip, dockerId.id, logFile.getAbsolutePath, state) } -private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) +private class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { implicit val formats = org.json4s.DefaultFormats @@ -390,7 +390,7 @@ private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath) } -private[spark] object SparkDocker { +private object SparkDocker { def startMaster(mountDir: String): TestMasterInfo = { val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir) val (ip, id, outFile) = startNode(cmd) @@ -425,11 +425,11 @@ private[spark] object SparkDocker { } } -private[spark] class DockerId(val id: String) { +private class DockerId(val id: String) { override def toString = id } -private[spark] object Docker extends Logging { +private object Docker extends Logging { def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = { val mountCmd = if (mountDir != "") { " -v " + mountDir } else "" diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 696f32a6f5730..458a7c3a455de 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -23,7 +23,7 @@ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateR import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.worker.ExecutorRunner -private[spark] object JsonProtocol { +private[deploy] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { ("id" -> obj.id) ~ ("host" -> obj.host) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4a74641f4e1fa..4f506be63fe59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, U * Whether to submit, kill, or request the status of an application. * The latter two operations are currently supported only for standalone cluster mode. */ -private[spark] object SparkSubmitAction extends Enumeration { +private[deploy] object SparkSubmitAction extends Enumeration { type SparkSubmitAction = Value val SUBMIT, KILL, REQUEST_STATUS = Value } @@ -137,7 +137,7 @@ object SparkSubmit { * Second, we use this launch environment to invoke the main method of the child * main class. */ - private[spark] def submit(args: SparkSubmitArguments): Unit = { + private def submit(args: SparkSubmitArguments): Unit = { val (childArgs, childClasspath, sysProps, childMainClass) = prepareSubmitEnvironment(args) def doRunMain(): Unit = { @@ -199,7 +199,7 @@ object SparkSubmit { * (4) the main class for the child * Exposed for testing. */ - private[spark] def prepareSubmitEnvironment(args: SparkSubmitArguments) + private[deploy] def prepareSubmitEnvironment(args: SparkSubmitArguments) : (Seq[String], Seq[String], Map[String, String], String) = { // Return values val childArgs = new ArrayBuffer[String]() @@ -598,32 +598,32 @@ object SparkSubmit { /** * Return whether the given primary resource represents a shell. */ - private[spark] def isShell(primaryResource: String): Boolean = { + private[deploy] def isShell(primaryResource: String): Boolean = { primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL } /** * Return whether the given main class represents a sql shell. */ - private[spark] def isSqlShell(mainClass: String): Boolean = { + private def isSqlShell(mainClass: String): Boolean = { mainClass == "org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" } /** * Return whether the given main class represents a thrift server. */ - private[spark] def isThriftServer(mainClass: String): Boolean = { + private def isThriftServer(mainClass: String): Boolean = { mainClass == "org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" } /** * Return whether the given primary resource requires running python. */ - private[spark] def isPython(primaryResource: String): Boolean = { + private[deploy] def isPython(primaryResource: String): Boolean = { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } - private[spark] def isInternal(primaryResource: String): Boolean = { + private[deploy] def isInternal(primaryResource: String): Boolean = { primaryResource == SPARK_INTERNAL } @@ -631,7 +631,7 @@ object SparkSubmit { * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. */ - private[spark] def mergeFileLists(lists: String*): String = { + private def mergeFileLists(lists: String*): String = { val merged = lists.filter(_ != null) .flatMap(_.split(",")) .mkString(",") @@ -640,10 +640,10 @@ object SparkSubmit { } /** Provides utility functions to be used inside SparkSubmit. */ -private[spark] object SparkSubmitUtils { +private[deploy] object SparkSubmitUtils { // Exposed for testing - private[spark] var printStream = SparkSubmit.printStream + var printStream = SparkSubmit.printStream /** * Represents a Maven Coordinate @@ -651,7 +651,7 @@ private[spark] object SparkSubmitUtils { * @param artifactId the artifactId of the coordinate * @param version the version of the coordinate */ - private[spark] case class MavenCoordinate(groupId: String, artifactId: String, version: String) + private[deploy] case class MavenCoordinate(groupId: String, artifactId: String, version: String) /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided @@ -659,7 +659,7 @@ private[spark] object SparkSubmitUtils { * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ - private[spark] def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = { + def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = { coordinates.split(",").map { p => val splits = p.replace("/", ":").split(":") require(splits.length == 3, s"Provided Maven Coordinates must be in the form " + @@ -679,7 +679,7 @@ private[spark] object SparkSubmitUtils { * @param remoteRepos Comma-delimited string of remote repositories * @return A ChainResolver used by Ivy to search for and resolve dependencies. */ - private[spark] def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = { + def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = { // We need a chain resolver if we want to check multiple repositories val cr = new ChainResolver cr.setName("list") @@ -722,7 +722,7 @@ private[spark] object SparkSubmitUtils { * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies */ - private[spark] def resolveDependencyPaths( + def resolveDependencyPaths( artifacts: Array[AnyRef], cacheDirectory: File): String = { artifacts.map { artifactInfo => @@ -734,7 +734,7 @@ private[spark] object SparkSubmitUtils { } /** Adds the given maven coordinates to Ivy's module descriptor. */ - private[spark] def addDependenciesToIvy( + def addDependenciesToIvy( md: DefaultModuleDescriptor, artifacts: Seq[MavenCoordinate], ivyConfName: String): Unit = { @@ -748,7 +748,7 @@ private[spark] object SparkSubmitUtils { } /** Add exclusion rules for dependencies already included in the spark-assembly */ - private[spark] def addExclusionRules( + def addExclusionRules( ivySettings: IvySettings, ivyConfName: String, md: DefaultModuleDescriptor): Unit = { @@ -777,7 +777,7 @@ private[spark] object SparkSubmitUtils { } /** A nice function to use in tests as well. Values are dummy strings. */ - private[spark] def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( + def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0")) /** @@ -788,7 +788,7 @@ private[spark] object SparkSubmitUtils { * @return The comma-delimited path to the jars of the given maven artifacts including their * transitive dependencies */ - private[spark] def resolveMavenCoordinates( + def resolveMavenCoordinates( coordinates: String, remoteRepos: Option[String], ivyPath: Option[String], diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 94e4bdbfb7d7b..2250d5a28e4ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils * Parses and encapsulates arguments from the spark-submit script. * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) +private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) extends SparkSubmitArgumentsParser { var master: String = null var deployMode: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index ffe940fbda2fb..2d24083a77b73 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -47,18 +47,18 @@ private[spark] class AppClient( conf: SparkConf) extends Logging { - val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) + private val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) - val REGISTRATION_TIMEOUT = 20.seconds - val REGISTRATION_RETRIES = 3 + private val REGISTRATION_TIMEOUT = 20.seconds + private val REGISTRATION_RETRIES = 3 - var masterAddress: Address = null - var actor: ActorRef = null - var appId: String = null - var registered = false - var activeMasterUrl: String = null + private var masterAddress: Address = null + private var actor: ActorRef = null + private var appId: String = null + private var registered = false + private var activeMasterUrl: String = null - class ClientActor extends Actor with ActorLogReceive with Logging { + private class ClientActor extends Actor with ActorLogReceive with Logging { var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 88a0862b96afe..c1c4812f17fbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -23,7 +23,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] object TestClient { - class TestListener extends AppClientListener with Logging { + private class TestListener extends AppClientListener with Logging { def connected(id: String) { logInfo("Connected to master, got app ID " + id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 553bf3cb945ab..ea6c85ee511d5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI -private[spark] case class ApplicationHistoryInfo( +private[history] case class ApplicationHistoryInfo( id: String, name: String, startTime: Long, @@ -28,7 +28,7 @@ private[spark] case class ApplicationHistoryInfo( sparkUser: String, completed: Boolean = false) -private[spark] abstract class ApplicationHistoryProvider { +private[history] abstract class ApplicationHistoryProvider { /** * Returns a list of applications available for the history server to show. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 26ebc75971c66..6e432d63c6b5a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -23,7 +23,7 @@ import scala.xml.Node import org.apache.spark.ui.{WebUIPage, UIUtils} -private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { +private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { private val pageSize = 20 private val plusOrMinus = 2 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index b1270ade9f750..a2a97a7877ce7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -23,7 +23,8 @@ import org.apache.spark.util.Utils /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) extends Logging { +private[history] class HistoryServerArguments(conf: SparkConf, args: Array[String]) + extends Logging { private var propertiesFile: String = null parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index a962dc4af2f6c..536aedb6f9fe9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.util.Utils -private[spark] class ApplicationInfo( +private[deploy] class ApplicationInfo( val startTime: Long, val id: String, val desc: ApplicationDescription, @@ -75,14 +75,15 @@ private[spark] class ApplicationInfo( } } - def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorDesc = { + private[master] def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): + ExecutorDesc = { val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) executors(exec.id) = exec coresGranted += cores exec } - def removeExecutor(exec: ExecutorDesc) { + private[master] def removeExecutor(exec: ExecutorDesc) { if (executors.contains(exec.id)) { removedExecutors += executors(exec.id) executors -= exec.id @@ -90,22 +91,22 @@ private[spark] class ApplicationInfo( } } - val requestedCores = desc.maxCores.getOrElse(defaultCores) + private[master] val requestedCores = desc.maxCores.getOrElse(defaultCores) - def coresLeft: Int = requestedCores - coresGranted + private[master] def coresLeft: Int = requestedCores - coresGranted private var _retryCount = 0 - def retryCount = _retryCount + private[master] def retryCount = _retryCount - def incrementRetryCount() = { + private[master] def incrementRetryCount() = { _retryCount += 1 _retryCount } - def resetRetryCount() = _retryCount = 0 + private[master] def resetRetryCount() = _retryCount = 0 - def markFinished(endState: ApplicationState.Value) { + private[master] def markFinished(endState: ApplicationState.Value) { state = endState endTime = System.currentTimeMillis() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala index 38db02cd2421b..017e8b55cbe7f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala @@ -21,7 +21,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -class ApplicationSource(val application: ApplicationInfo) extends Source { +private[master] class ApplicationSource(val application: ApplicationInfo) extends Source { override val metricRegistry = new MetricRegistry() override val sourceName = "%s.%s.%s".format("application", application.desc.name, System.currentTimeMillis()) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index f5b946329ae9b..37bfcdfdf4777 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object ApplicationState extends Enumeration { +private[master] object ApplicationState extends Enumeration { type ApplicationState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 9d3d7938c6ccb..b197dbcbfe294 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.DriverDescription import org.apache.spark.util.Utils -private[spark] class DriverInfo( +private[deploy] class DriverInfo( val startTime: Long, val id: String, val desc: DriverDescription, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala index 26a68bade3c60..35ff33a61653c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object DriverState extends Enumeration { +private[deploy] object DriverState extends Enumeration { type DriverState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala index 5d620dfcabad5..fc62b094def67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} -private[spark] class ExecutorDesc( +private[master] class ExecutorDesc( val id: Int, val application: ApplicationInfo, val worker: WorkerInfo, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 36a2e2c6a6349..d2d30bfd7fcba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -33,7 +33,7 @@ import org.apache.spark.Logging * @param dir Directory to store files. Created if non-existent (but not recursively). * @param serialization Used to serialize our objects. */ -private[spark] class FileSystemPersistenceEngine( +private[master] class FileSystemPersistenceEngine( val dir: String, val serialization: Serialization) extends PersistenceEngine with Logging { 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 22935c9b1d394..1b42121c8db05 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 @@ -49,7 +49,7 @@ import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} -private[spark] class Master( +private[master] class Master( host: String, port: Int, webUiPort: Int, @@ -59,65 +59,68 @@ private[spark] class Master( import context.dispatcher // to use Akka's scheduler.schedule() - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 - val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) - val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) - val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + + private val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 + private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) + private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) + private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) + private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") val workers = new HashSet[WorkerInfo] - val idToWorker = new HashMap[String, WorkerInfo] - val addressToWorker = new HashMap[Address, WorkerInfo] - - val apps = new HashSet[ApplicationInfo] val idToApp = new HashMap[String, ApplicationInfo] - val actorToApp = new HashMap[ActorRef, ApplicationInfo] - val addressToApp = new HashMap[Address, ApplicationInfo] val waitingApps = new ArrayBuffer[ApplicationInfo] - val completedApps = new ArrayBuffer[ApplicationInfo] - var nextAppNumber = 0 - val appIdToUI = new HashMap[String, SparkUI] + val apps = new HashSet[ApplicationInfo] + + private val idToWorker = new HashMap[String, WorkerInfo] + private val addressToWorker = new HashMap[Address, WorkerInfo] + + private val actorToApp = new HashMap[ActorRef, ApplicationInfo] + private val addressToApp = new HashMap[Address, ApplicationInfo] + private val completedApps = new ArrayBuffer[ApplicationInfo] + private var nextAppNumber = 0 + private val appIdToUI = new HashMap[String, SparkUI] - val drivers = new HashSet[DriverInfo] - val completedDrivers = new ArrayBuffer[DriverInfo] - val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling - var nextDriverNumber = 0 + private val drivers = new HashSet[DriverInfo] + private val completedDrivers = new ArrayBuffer[DriverInfo] + // Drivers currently spooled for scheduling + private val waitingDrivers = new ArrayBuffer[DriverInfo] + private var nextDriverNumber = 0 Utils.checkHost(host, "Expected hostname") - val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) - val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, + private val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) + private val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, securityMgr) - val masterSource = new MasterSource(this) + private val masterSource = new MasterSource(this) - val webUi = new MasterWebUI(this, webUiPort) + private val webUi = new MasterWebUI(this, webUiPort) - val masterPublicAddress = { + private val masterPublicAddress = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val masterUrl = "spark://" + host + ":" + port - var masterWebUiUrl: String = _ + private val masterUrl = "spark://" + host + ":" + port + private var masterWebUiUrl: String = _ - var state = RecoveryState.STANDBY + private var state = RecoveryState.STANDBY - var persistenceEngine: PersistenceEngine = _ + private var persistenceEngine: PersistenceEngine = _ - var leaderElectionAgent: LeaderElectionAgent = _ + private var leaderElectionAgent: LeaderElectionAgent = _ private var recoveryCompletionTask: Cancellable = _ // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + private val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) - val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) + private val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) if (defaultCores < 1) { throw new SparkException("spark.deploy.defaultCores must be positive") } @@ -449,11 +452,11 @@ private[spark] class Master( } } - def canCompleteRecovery = + private def canCompleteRecovery = workers.count(_.state == WorkerState.UNKNOWN) == 0 && apps.count(_.state == ApplicationState.UNKNOWN) == 0 - def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo], + private def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo], storedWorkers: Seq[WorkerInfo]) { for (app <- storedApps) { logInfo("Trying to recover app: " + app.id) @@ -484,7 +487,7 @@ private[spark] class Master( } } - def completeRecovery() { + private def completeRecovery() { // Ensure "only-once" recovery semantics using a short synchronization period. synchronized { if (state != RecoveryState.RECOVERING) { return } @@ -517,7 +520,7 @@ private[spark] class Master( * launched an executor for the app on it (right now the standalone backend doesn't like having * two executors on the same worker). */ - def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { + private def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) } @@ -596,7 +599,7 @@ private[spark] class Master( } } - def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { + private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, @@ -605,7 +608,7 @@ private[spark] class Master( exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) } - def registerWorker(worker: WorkerInfo): Boolean = { + private def registerWorker(worker: WorkerInfo): Boolean = { // There may be one or more refs to dead workers on this same node (w/ different ID's), // remove them. workers.filter { w => @@ -633,7 +636,7 @@ private[spark] class Master( true } - def removeWorker(worker: WorkerInfo) { + private def removeWorker(worker: WorkerInfo) { logInfo("Removing worker " + worker.id + " on " + worker.host + ":" + worker.port) worker.setState(WorkerState.DEAD) idToWorker -= worker.id @@ -656,20 +659,20 @@ private[spark] class Master( persistenceEngine.removeWorker(worker) } - def relaunchDriver(driver: DriverInfo) { + private def relaunchDriver(driver: DriverInfo) { driver.worker = None driver.state = DriverState.RELAUNCHING waitingDrivers += driver schedule() } - def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { + private def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) new ApplicationInfo(now, newApplicationId(date), desc, date, driver, defaultCores) } - def registerApplication(app: ApplicationInfo): Unit = { + private def registerApplication(app: ApplicationInfo): Unit = { val appAddress = app.driver.path.address if (addressToApp.contains(appAddress)) { logInfo("Attempted to re-register application at same address: " + appAddress) @@ -684,7 +687,7 @@ private[spark] class Master( waitingApps += app } - def finishApplication(app: ApplicationInfo) { + private def finishApplication(app: ApplicationInfo) { removeApplication(app, ApplicationState.FINISHED) } @@ -732,7 +735,7 @@ private[spark] class Master( * Rebuild a new SparkUI from the given application's event logs. * Return whether this is successful. */ - def rebuildSparkUI(app: ApplicationInfo): Boolean = { + private def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { @@ -798,14 +801,14 @@ private[spark] class Master( } /** Generate a new app ID given a app's submission date */ - def newApplicationId(submitDate: Date): String = { + private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 appId } /** Check for, and remove, any timed-out workers */ - def timeOutDeadWorkers() { + private def timeOutDeadWorkers() { // Copy the workers into an array so we don't modify the hashset while iterating through it val currentTime = System.currentTimeMillis() val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT).toArray @@ -822,19 +825,19 @@ private[spark] class Master( } } - def newDriverId(submitDate: Date): String = { + private def newDriverId(submitDate: Date): String = { val appId = "driver-%s-%04d".format(createDateFormat.format(submitDate), nextDriverNumber) nextDriverNumber += 1 appId } - def createDriver(desc: DriverDescription): DriverInfo = { + private def createDriver(desc: DriverDescription): DriverInfo = { val now = System.currentTimeMillis() val date = new Date(now) new DriverInfo(now, newDriverId(date), desc, date) } - def launchDriver(worker: WorkerInfo, driver: DriverInfo) { + private def launchDriver(worker: WorkerInfo, driver: DriverInfo) { logInfo("Launching driver " + driver.id + " on worker " + worker.id) worker.addDriver(driver) driver.worker = Some(worker) @@ -842,7 +845,10 @@ private[spark] class Master( driver.state = DriverState.RUNNING } - def removeDriver(driverId: String, finalState: DriverState, exception: Option[Exception]) { + private def removeDriver( + driverId: String, + finalState: DriverState, + exception: Option[Exception]) { drivers.find(d => d.id == driverId) match { case Some(driver) => logInfo(s"Removing driver: $driverId") @@ -863,7 +869,7 @@ private[spark] class Master( } } -private[spark] object Master extends Logging { +private[deploy] object Master extends Logging { val systemName = "sparkMaster" private val actorName = "Master" diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index e34bee7854292..435b9b12f83b8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -23,7 +23,7 @@ import org.apache.spark.util.{IntParam, Utils} /** * Command-line parser for the master. */ -private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { +private[master] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 @@ -49,7 +49,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { webUiPort = conf.get("spark.master.ui.port").toInt } - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) host = value @@ -84,7 +84,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { /** * Print usage and exit JVM with the given exit code. */ - def printUsageAndExit(exitCode: Int) { + private def printUsageAndExit(exitCode: Int) { System.err.println( "Usage: Master [options]\n" + "\n" + diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala index 2e0e1e7036ac8..da5060778edeb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -87,7 +87,7 @@ trait PersistenceEngine { def close() {} } -private[spark] class BlackHolePersistenceEngine extends PersistenceEngine { +private[master] class BlackHolePersistenceEngine extends PersistenceEngine { override def persist(name: String, obj: Object): Unit = {} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index 1096eb0368357..1583bf1f60032 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -49,7 +49,7 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial * LeaderAgent in this case is a no-op. Since leader is forever leader as the actual * recovery is made by restoring from filesystem. */ -private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) +private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") @@ -61,7 +61,7 @@ private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) } -private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) +private[master] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) { def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala index 256a5a7c28e47..aa0f02fa625cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object RecoveryState extends Enumeration { +private[deploy] object RecoveryState extends Enumeration { type MasterState = Value val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala index 4781a80d470e1..5b22481ea8c5f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala @@ -25,12 +25,12 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.{Logging, SparkConf} -object SparkCuratorUtil extends Logging { +private[deploy] object SparkCuratorUtil extends Logging { - val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 - val ZK_SESSION_TIMEOUT_MILLIS = 60000 - val RETRY_WAIT_MILLIS = 5000 - val MAX_RECONNECT_ATTEMPTS = 3 + private val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 + private val ZK_SESSION_TIMEOUT_MILLIS = 60000 + private val RETRY_WAIT_MILLIS = 5000 + private val MAX_RECONNECT_ATTEMPTS = 3 def newClient(conf: SparkConf): CuratorFramework = { val ZK_URL = conf.get("spark.deploy.zookeeper.url") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala index 0b36ef60051fc..b60baaadfb4bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object WorkerState extends Enumeration { +private[master] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED, UNKNOWN = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 8eaa0ad948519..4823fd7cac0cb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -24,7 +24,7 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} -private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, +private[master] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" @@ -35,7 +35,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectab start() - def start() { + private def start() { logInfo("Starting ZooKeeper LeaderElection agent") zk = SparkCuratorUtil.newClient(conf) leaderLatch = new LeaderLatch(zk, WORKING_DIR) @@ -72,7 +72,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectab } } - def updateLeadershipStatus(isLeader: Boolean) { + private def updateLeadershipStatus(isLeader: Boolean) { if (isLeader && status == LeadershipStatus.NOT_LEADER) { status = LeadershipStatus.LEADER masterActor.electedLeader() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index e11ac031fb9c6..1ac6677ad2b6d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -28,12 +28,12 @@ import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} -private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) +private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) extends PersistenceEngine - with Logging -{ - val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) + with Logging { + + private val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + private val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) SparkCuratorUtil.mkdir(zk, WORKING_DIR) @@ -61,7 +61,7 @@ private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializati zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } - def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { + private def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 76fc40e17d9a8..761aa8f7b1ef6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { +private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { private val master = parent.masterActorRef private val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala index d8daff3e7fb9c..e021f1eef794f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.apache.spark.ui.{UIUtils, WebUIPage} -private[spark] class HistoryNotFoundPage(parent: MasterWebUI) +private[ui] class HistoryNotFoundPage(parent: MasterWebUI) extends WebUIPage("history/not-found") { /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index c086cadca2c7d..dee2e4a447c6e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.master._ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { +private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterActorRef private val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 170f90a00ad2a..1b670418ab1ff 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone master. */ -private[spark] +private[master] class MasterWebUI(val master: Master, requestedPort: Int) extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { @@ -62,6 +62,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) } } -private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR +private[master] object MasterWebUI { + private val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala index c4be1f19e8e9f..420442f7564cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala @@ -52,7 +52,7 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} * implementation of this client can use that information to retry using the version specified * by the server. */ -private[spark] class StandaloneRestClient extends Logging { +private[deploy] class StandaloneRestClient extends Logging { import StandaloneRestClient._ /** @@ -61,7 +61,7 @@ private[spark] class StandaloneRestClient extends Logging { * If the submission was successful, poll the status of the submission and report * it to the user. Otherwise, report the error message provided by the server. */ - def createSubmission( + private[rest] def createSubmission( master: String, request: CreateSubmissionRequest): SubmitRestProtocolResponse = { logInfo(s"Submitting a request to launch an application in $master.") @@ -106,7 +106,7 @@ private[spark] class StandaloneRestClient extends Logging { } /** Construct a message that captures the specified parameters for submitting an application. */ - def constructSubmitRequest( + private[rest] def constructSubmitRequest( appResource: String, mainClass: String, appArgs: Array[String], @@ -291,16 +291,16 @@ private[spark] class StandaloneRestClient extends Logging { } } -private[spark] object StandaloneRestClient { - val REPORT_DRIVER_STATUS_INTERVAL = 1000 - val REPORT_DRIVER_STATUS_MAX_TRIES = 10 +private[rest] object StandaloneRestClient { + private val REPORT_DRIVER_STATUS_INTERVAL = 1000 + private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" /** * Submit an application, assuming Spark parameters are specified through the given config. * This is abstracted to its own method for testing purposes. */ - private[rest] def run( + def run( appResource: String, mainClass: String, appArgs: Array[String], diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index f9e0478e4f874..4f19af59f409f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -58,7 +58,7 @@ import org.apache.spark.deploy.ClientArguments._ * @param masterUrl the URL of the Master new drivers will attempt to connect to * @param masterConf the conf used by the Master */ -private[spark] class StandaloneRestServer( +private[deploy] class StandaloneRestServer( host: String, requestedPort: Int, masterActor: ActorRef, diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala index d7a0bdbe10778..b97921ec934a0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala @@ -20,17 +20,17 @@ package org.apache.spark.deploy.rest /** * An exception thrown in the REST application submission protocol. */ -private[spark] class SubmitRestProtocolException(message: String, cause: Throwable = null) +private[rest] class SubmitRestProtocolException(message: String, cause: Throwable = null) extends Exception(message, cause) /** * An exception thrown if a field is missing from a [[SubmitRestProtocolMessage]]. */ -private[spark] class SubmitRestMissingFieldException(message: String) +private[rest] class SubmitRestMissingFieldException(message: String) extends SubmitRestProtocolException(message) /** * An exception thrown if the REST client cannot reach the REST server. */ -private[spark] class SubmitRestConnectionException(message: String, cause: Throwable) +private[deploy] class SubmitRestConnectionException(message: String, cause: Throwable) extends SubmitRestProtocolException(message, cause) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala index 8f36635674a28..e6615a3174ce1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala @@ -39,7 +39,7 @@ import org.apache.spark.util.Utils @JsonInclude(Include.NON_NULL) @JsonAutoDetect(getterVisibility = Visibility.ANY, setterVisibility = Visibility.ANY) @JsonPropertyOrder(alphabetic = true) -private[spark] abstract class SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolMessage { @JsonIgnore val messageType = Utils.getFormattedClassName(this) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index 9e1fd8c40cabd..d80abdf15fb34 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.Utils /** * An abstract request sent from the client in the REST application submission protocol. */ -private[spark] abstract class SubmitRestProtocolRequest extends SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolRequest extends SubmitRestProtocolMessage { var clientSparkVersion: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -35,7 +35,7 @@ private[spark] abstract class SubmitRestProtocolRequest extends SubmitRestProtoc /** * A request to launch a new application in the REST application submission protocol. */ -private[spark] class CreateSubmissionRequest extends SubmitRestProtocolRequest { +private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { var appResource: String = null var mainClass: String = null var appArgs: Array[String] = null diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala index 16dfe041d4bea..8fde8c142a4c1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala @@ -22,7 +22,7 @@ import java.lang.Boolean /** * An abstract response sent from the server in the REST application submission protocol. */ -private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage { var serverSparkVersion: String = null var success: Boolean = null var unknownFields: Array[String] = null @@ -35,7 +35,7 @@ private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProto /** * A response to a [[CreateSubmissionRequest]] in the REST application submission protocol. */ -private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse { +private[rest] class CreateSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -46,7 +46,7 @@ private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse /** * A response to a kill request in the REST application submission protocol. */ -private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { +private[rest] class KillSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -58,7 +58,7 @@ private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { /** * A response to a status request in the REST application submission protocol. */ -private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse { +private[rest] class SubmissionStatusResponse extends SubmitRestProtocolResponse { var submissionId: String = null var driverState: String = null var workerId: String = null @@ -74,7 +74,7 @@ private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse /** * An error response message used in the REST application submission protocol. */ -private[spark] class ErrorResponse extends SubmitRestProtocolResponse { +private[rest] class ErrorResponse extends SubmitRestProtocolResponse { // The highest protocol version that the server knows about // This is set when the client specifies an unknown version var highestProtocolVersion: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 83f78cf47306c..0a1d60f58bc58 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils /** ** Utilities for running commands with the spark classpath. */ -private[spark] +private[deploy] object CommandUtils extends Logging { /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e16bccb24d2c4..27a9eabb1ede7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -37,8 +37,8 @@ import org.apache.spark.util.{Clock, SystemClock} * Manages the execution of one driver, including automatically restarting the driver on failure. * This is currently only used in standalone cluster deploy mode. */ -private[spark] class DriverRunner( - val conf: SparkConf, +private[deploy] class DriverRunner( + conf: SparkConf, val driverId: String, val workDir: File, val sparkHome: File, @@ -47,24 +47,24 @@ private[spark] class DriverRunner( val workerUrl: String) extends Logging { - @volatile var process: Option[Process] = None - @volatile var killed = false + @volatile private var process: Option[Process] = None + @volatile private var killed = false // Populated once finished - var finalState: Option[DriverState] = None - var finalException: Option[Exception] = None - var finalExitCode: Option[Int] = None + private[worker] var finalState: Option[DriverState] = None + private[worker] var finalException: Option[Exception] = None + private var finalExitCode: Option[Int] = None // Decoupled for testing - private[deploy] def setClock(_clock: Clock) = clock = _clock - private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper + def setClock(_clock: Clock) = clock = _clock + def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper private var clock: Clock = new SystemClock() private var sleeper = new Sleeper { def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) } /** Starts a thread to run and manage the driver. */ - def start() = { + private[worker] def start() = { new Thread("DriverRunner for " + driverId) { override def run() { try { @@ -106,7 +106,7 @@ private[spark] class DriverRunner( } /** Terminate this driver (or prevent it from ever starting if not yet started) */ - def kill() { + private[worker] def kill() { synchronized { process.foreach(p => p.destroy()) killed = true @@ -169,7 +169,7 @@ private[spark] class DriverRunner( runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) } - private[deploy] def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, + def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean) { // Time to wait between submission retries. var waitSeconds = 1 diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 023f3c6269062..83e24a7a1f80c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.logging.FileAppender * Manages the execution of one executor process. * This is currently only used in standalone mode. */ -private[spark] class ExecutorRunner( +private[deploy] class ExecutorRunner( val appId: String, val execId: Int, val appDesc: ApplicationDescription, @@ -48,22 +48,22 @@ private[spark] class ExecutorRunner( val sparkHome: File, val executorDir: File, val workerUrl: String, - val conf: SparkConf, + conf: SparkConf, val appLocalDirs: Seq[String], var state: ExecutorState.Value) extends Logging { - val fullId = appId + "/" + execId - var workerThread: Thread = null - var process: Process = null - var stdoutAppender: FileAppender = null - var stderrAppender: FileAppender = null + private val fullId = appId + "/" + execId + private var workerThread: Thread = null + private var process: Process = null + private var stdoutAppender: FileAppender = null + private var stderrAppender: FileAppender = null // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. - var shutdownHook: Thread = null + private var shutdownHook: Thread = null - def start() { + private[worker] def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } } @@ -99,7 +99,7 @@ private[spark] class ExecutorRunner( } /** Stop this executor runner, including killing the process it launched */ - def kill() { + private[worker] def kill() { if (workerThread != null) { // the workerThread will kill the child process when interrupted workerThread.interrupt() @@ -114,7 +114,7 @@ private[spark] class ExecutorRunner( } /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */ - def substituteVariables(argument: String): String = argument match { + private[worker] def substituteVariables(argument: String): String = argument match { case "{{WORKER_URL}}" => workerUrl case "{{EXECUTOR_ID}}" => execId.toString case "{{HOSTNAME}}" => host @@ -126,7 +126,7 @@ private[spark] class ExecutorRunner( /** * Download and run the executor described in our ApplicationDescription */ - def fetchAndRunExecutor() { + private def fetchAndRunExecutor() { try { // Launch the process val builder = CommandUtils.buildProcessBuilder(appDesc.command, memory, diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f2e7418f4bf15..c1b0a295f9f74 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} /** * @param masterAkkaUrls Each url should be a valid akka url. */ -private[spark] class Worker( +private[worker] class Worker( host: String, port: Int, webUiPort: Int, @@ -60,85 +60,90 @@ private[spark] class Worker( Utils.checkHost(host, "Expected hostname") assert (port > 0) - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs + // For worker and executor IDs + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 + private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 // Model retries to connect to the master, after Hadoop's model. // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) // Afterwards, the next 10 attempts are between 30 and 90 seconds. // A bit of randomness is introduced so that not all of the workers attempt to reconnect at // the same time. - val INITIAL_REGISTRATION_RETRIES = 6 - val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 - val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 - val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { + private val INITIAL_REGISTRATION_RETRIES = 6 + private val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 + private val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 + private val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { val randomNumberGenerator = new Random(UUID.randomUUID.getMostSignificantBits) randomNumberGenerator.nextDouble + FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND } - val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * + private val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds - val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 + private val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds - val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) + private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders - val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + private val CLEANUP_INTERVAL_MILLIS = + conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) - - val testing: Boolean = sys.props.contains("spark.testing") - var master: ActorSelection = null - var masterAddress: Address = null - var activeMasterUrl: String = "" - var activeMasterWebUiUrl : String = "" - val akkaUrl = AkkaUtils.address( + private val APP_DATA_RETENTION_SECS = + conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + + private val testing: Boolean = sys.props.contains("spark.testing") + private var master: ActorSelection = null + private var masterAddress: Address = null + private var activeMasterUrl: String = "" + private[worker] var activeMasterWebUiUrl : String = "" + private val akkaUrl = AkkaUtils.address( AkkaUtils.protocol(context.system), actorSystemName, host, port, actorName) - @volatile var registered = false - @volatile var connected = false - val workerId = generateWorkerId() - val sparkHome = + @volatile private var registered = false + @volatile private var connected = false + private val workerId = generateWorkerId() + private val sparkHome = if (testing) { assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!") new File(sys.props("spark.test.home")) } else { new File(sys.env.get("SPARK_HOME").getOrElse(".")) } + var workDir: File = null - val executors = new HashMap[String, ExecutorRunner] val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] + val executors = new HashMap[String, ExecutorRunner] val finishedDrivers = new HashMap[String, DriverRunner] val appDirectories = new HashMap[String, Seq[String]] val finishedApps = new HashSet[String] // The shuffle service is not actually started unless configured. - val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) + private val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) - val publicAddress = { + private val publicAddress = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - var webUi: WorkerWebUI = null + private var webUi: WorkerWebUI = null - var coresUsed = 0 - var memoryUsed = 0 - var connectionAttemptCount = 0 + private var connectionAttemptCount = 0 - val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) - val workerSource = new WorkerSource(this) + private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) + private val workerSource = new WorkerSource(this) + + private var registrationRetryTimer: Option[Cancellable] = None - var registrationRetryTimer: Option[Cancellable] = None + var coresUsed = 0 + var memoryUsed = 0 def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed - def createWorkDir() { + private def createWorkDir() { workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) try { // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() @@ -175,7 +180,7 @@ private[spark] class Worker( metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } - def changeMaster(url: String, uiUrl: String) { + private def changeMaster(url: String, uiUrl: String) { // activeMasterUrl it's a valid Spark url since we receive it from master. activeMasterUrl = url activeMasterWebUiUrl = uiUrl @@ -252,7 +257,7 @@ private[spark] class Worker( } } - def registerWithMaster() { + private def registerWithMaster() { // DisassociatedEvent may be triggered multiple times, so don't attempt registration // if there are outstanding registration attempts scheduled. registrationRetryTimer match { @@ -506,7 +511,7 @@ private[spark] class Worker( } } - def generateWorkerId(): String = { + private def generateWorkerId(): String = { "worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port) } @@ -521,7 +526,7 @@ private[spark] class Worker( } } -private[spark] object Worker extends Logging { +private[deploy] object Worker extends Logging { def main(argStrings: Array[String]) { SignalLogger.register(log) val conf = new SparkConf @@ -554,7 +559,7 @@ private[spark] object Worker extends Logging { (actorSystem, boundPort) } - private[spark] def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { + def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { val pattern = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r val result = cmd.javaOpts.collectFirst { case pattern(_result) => _result.toBoolean @@ -562,7 +567,7 @@ private[spark] object Worker extends Logging { result.getOrElse(false) } - private[spark] def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { + def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { val prefix = "spark.ssl." val useNLC = "spark.ssl.useNodeLocalConf" if (isUseLocalNodeSSLConfig(cmd)) { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 019cd70f2a229..88f9d880ac209 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -25,7 +25,7 @@ import org.apache.spark.SparkConf /** * Command-line parser for the worker. */ -private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { +private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 0 var webUiPort = 8081 @@ -63,7 +63,7 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { checkWorkerMemory() - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) host = value diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala index df1e01b23b932..b36023bc40c3d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala @@ -21,7 +21,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -private[spark] class WorkerSource(val worker: Worker) extends Source { +private[worker] class WorkerSource(val worker: Worker) extends Source { override val sourceName = "worker" override val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 63a8ac817b618..09d866fb0cd90 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -48,7 +48,7 @@ private[spark] class WorkerWatcher(workerUrl: String) private val expectedHostPort = AddressFromURIString(workerUrl).hostPort private def isWorker(address: Address) = address.hostPort == expectedHostPort - def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) + private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) override def receiveWithLogging = { case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index ecb358c399819..88170d4df3053 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils import org.apache.spark.Logging import org.apache.spark.util.logging.RollingFileAppender -private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { +private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { private val worker = parent.worker private val workDir = parent.workDir diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 720f13bfa829b..9f9f27d71e1ae 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -31,10 +31,9 @@ import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { - val workerActor = parent.worker.self - val worker = parent.worker - val timeout = parent.timeout +private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { + private val workerActor = parent.worker.self + private val timeout = parent.timeout override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 7ac81a2d87efd..de6423beb543e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ -private[spark] +private[worker] class WorkerWebUI( val worker: Worker, val workDir: File, @@ -38,7 +38,7 @@ class WorkerWebUI( extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") with Logging { - val timeout = AkkaUtils.askTimeout(worker.conf) + private[ui] val timeout = AkkaUtils.askTimeout(worker.conf) initialize() @@ -53,6 +53,6 @@ class WorkerWebUI( } } -private[spark] object WorkerWebUI { +private[ui] object WorkerWebUI { val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR } From 005d1c5f290decc606a0be59fb191136dafc0c9d Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 17 Mar 2015 11:20:20 +0000 Subject: [PATCH 39/62] [SPARK-6269] [CORE] Use ScalaRunTime's array methods instead of java.lang.reflect.Array in size estimation This patch switches the usage of java.lang.reflect.Array in Size estimation to using scala's RunTime array-getter methods. The notes on https://bugs.openjdk.java.net/browse/JDK-8051447 tipped me off to the fact that using java.lang.reflect.Array was not ideal. At first, I used the code from that ticket, but it turns out that ScalaRunTime's array-related methods avoid the bottleneck of invoking native code anyways, so that was sufficient to boost performance in size estimation. The idea is to use pure Java code in implementing the methods there, as opposed to relying on native C code which ends up being ill-performing. This improves the performance of estimating the size of arrays when we are checking for spilling in Spark. Here's the benchmark discussion from the ticket: I did two tests. The first, less convincing, take-with-a-block-of-salt test I did was do a simple groupByKey operation to collect objects in a 4.0 GB text file RDD into 30,000 buckets. I ran 1 Master and 4 Spark Worker JVMs on my mac, fetching the RDD from a text file simply stored on disk, and saving it out to another file located on local disk. The wall clock times I got back before and after the change were: Before: 352.195s, 343.871s, 359.080s After (using code directly from the JDK ticket, not the scala code in this PR): 342.929583s, 329.456623s, 326.151481s So, there is a bit of an improvement after the change. I also did some YourKit profiling of the executors to get an idea of how much time was spent in size estimation before and after the change. I roughly saw that size estimation took up less of the time after my change, but YourKit's profiling can be inconsistent and who knows if I was profiling the executors that had the same data between runs? The more convincing test I did was to run the size-estimation logic itself in an isolated unit test. I ran the following code: ``` val bigArray = Array.fill(1000)(Array.fill(1000)(java.util.UUID.randomUUID().toString())) test("String arrays only perf testing") { val startTime = System.currentTimeMillis() for (i <- 1 to 50000) { SizeEstimator.estimate(bigArray) } println("Runtime: " + (System.currentTimeMillis() - startTime) / 1000.0000) } ``` I wanted to use a 2D array specifically because I wanted to measure the performance of repeatedly calling Array.getLength. I used UUID-Strings to ensure that the strings were randomized (so String object re-use doesn't happen), but that they would all be the same size. The results were as follows: Before PR: 222.681 s, 218.34 s, 211.739s After latest change: 170.715 s, 176.775 s, 180.298 s . Author: mcheah Author: Justin Uang Closes #4972 from mccheah/feature/spark-6269-reflect-array and squashes the following commits: 8527852 [mcheah] Respect CamelCase for numElementsDrawn 18d4b50 [mcheah] Addressing style comments - while loops instead of for loops 16ce534 [mcheah] Organizing imports properly db890ea [mcheah] Removing CastedArray and just using ScalaRunTime. cb67ce2 [mcheah] Fixing a scalastyle error - line too long 5d53c4c [mcheah] Removing unused parameter in visitArray. 6467759 [mcheah] Including primitive size information inside CastedArray. 93f4b05 [mcheah] Using Scala instead of Java for the array-reflection implementation. a557ab8 [mcheah] Using a wrapper around arrays to do casting only once ca063fc [mcheah] Fixing a compiler error made while refactoring style 1fe09de [Justin Uang] [SPARK-6269] Use a different implementation of java.lang.reflect.Array --- .../org/apache/spark/util/SizeEstimator.scala | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index bce3b3afe9aba..26ffbf9350388 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -18,18 +18,16 @@ package org.apache.spark.util import java.lang.management.ManagementFactory -import java.lang.reflect.{Array => JArray} -import java.lang.reflect.Field -import java.lang.reflect.Modifier -import java.util.IdentityHashMap -import java.util.Random +import java.lang.reflect.{Field, Modifier} +import java.util.{IdentityHashMap, Random} import java.util.concurrent.ConcurrentHashMap - import scala.collection.mutable.ArrayBuffer +import scala.runtime.ScalaRunTime import org.apache.spark.Logging import org.apache.spark.util.collection.OpenHashSet + /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in * memory-aware caches. @@ -184,9 +182,9 @@ private[spark] object SizeEstimator extends Logging { private val ARRAY_SIZE_FOR_SAMPLING = 200 private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING - private def visitArray(array: AnyRef, cls: Class[_], state: SearchState) { - val length = JArray.getLength(array) - val elementClass = cls.getComponentType + private def visitArray(array: AnyRef, arrayClass: Class[_], state: SearchState) { + val length = ScalaRunTime.array_length(array) + val elementClass = arrayClass.getComponentType() // Arrays have object header and length field which is an integer var arrSize: Long = alignSize(objectSize + INT_SIZE) @@ -199,22 +197,26 @@ private[spark] object SizeEstimator extends Logging { state.size += arrSize if (length <= ARRAY_SIZE_FOR_SAMPLING) { - for (i <- 0 until length) { - state.enqueue(JArray.get(array, i)) + var arrayIndex = 0 + while (arrayIndex < length) { + state.enqueue(ScalaRunTime.array_apply(array, arrayIndex).asInstanceOf[AnyRef]) + arrayIndex += 1 } } else { // Estimate the size of a large array by sampling elements without replacement. var size = 0.0 val rand = new Random(42) val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) - for (i <- 0 until ARRAY_SAMPLE_SIZE) { + var numElementsDrawn = 0 + while (numElementsDrawn < ARRAY_SAMPLE_SIZE) { var index = 0 do { index = rand.nextInt(length) } while (drawn.contains(index)) drawn.add(index) - val elem = JArray.get(array, index) + val elem = ScalaRunTime.array_apply(array, index).asInstanceOf[AnyRef] size += SizeEstimator.estimate(elem, state.visited) + numElementsDrawn += 1 } state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong } From e26db9be47b295c4ec9e651f4cce321ba1fddfcd Mon Sep 17 00:00:00 2001 From: Theodore Vasiloudis Date: Tue, 17 Mar 2015 11:25:01 +0000 Subject: [PATCH 40/62] [docs] [SPARK-4820] Spark build encounters "File name too long" on some encrypted filesystems Added a note instructing users how to build Spark in an encrypted file system. Author: Theodore Vasiloudis Closes #5041 from thvasilo/patch-2 and squashes the following commits: 09d890b [Theodore Vasiloudis] Workaroung for buiding in an encrypted filesystem --- docs/building-spark.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/building-spark.md b/docs/building-spark.md index 57d0ca834f460..ea79c5bc276d3 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -23,6 +23,18 @@ build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package Other build examples can be found below. +**Note:** When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: + + -Xmax-classfile-name + 128 + +and in `project/SparkBuild.scala` add: + + scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), + +to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. + + # Setting up Maven's Memory Usage You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: From c928796ade54f68e26bc55734a9867a046d2e3fe Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 17 Mar 2015 05:31:27 -0700 Subject: [PATCH 41/62] [SPARK-6331] Load new master URL if present when recovering streaming context from checkpoint In streaming driver recovery, when the SparkConf is reconstructed based on the checkpointed configuration, it recovers the old master URL. This okay if the cluster on which the streaming application is relaunched is the same cluster as it was running before. But if that cluster changes, there is no way to inject the new master URL of the new cluster. As a result, the restarted app tries to connect to the non-existent old cluster and fails. The solution is to check whether a master URL is set in the System properties (by Spark submit) before recreating the SparkConf. If a new master url is set in the properties, then use it as that is obviously the most relevant one. Otherwise load the old one (to maintain existing behavior). Author: Tathagata Das Closes #5024 from tdas/SPARK-6331 and squashes the following commits: 392fd44 [Tathagata Das] Fixed naming issue. c7c0b99 [Tathagata Das] Addressed comments. 6a0857c [Tathagata Das] Updated testsuites. 222485d [Tathagata Das] Load new master URL if present when recovering streaming context from checkpoint --- .../apache/spark/streaming/Checkpoint.scala | 7 +++++-- .../spark/streaming/StreamingContext.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 21 ++++++++++++++++--- .../streaming/StreamingContextSuite.scala | 2 +- 4 files changed, 25 insertions(+), 7 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index f88a8a0151550..cb4c94fb9d5a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -43,10 +43,13 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf) val sparkConfPairs = ssc.conf.getAll - def sparkConf = { - new SparkConf(false).setAll(sparkConfPairs) + def createSparkConf(): SparkConf = { + val newSparkConf = new SparkConf(loadDefaults = false).setAll(sparkConfPairs) .remove("spark.driver.host") .remove("spark.driver.port") + val newMasterOption = new SparkConf(loadDefaults = true).getOption("spark.master") + newMasterOption.foreach { newMaster => newSparkConf.setMaster(newMaster) } + newSparkConf } def validate() { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index b5b6770a8a150..543224d4b07bc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -116,7 +116,7 @@ class StreamingContext private[streaming] ( private[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.sparkConf) + new SparkContext(cp_.createSparkConf()) } else { sc_ } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 03c448f1df5f1..8ea91eca683cf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -146,7 +146,7 @@ class CheckpointSuite extends TestSuiteBase { // This tests whether spark conf persists through checkpoints, and certain // configs gets scrubbed - test("persistence of conf through checkpoints") { + test("recovery of conf through checkpoints") { val key = "spark.mykey" val value = "myvalue" System.setProperty(key, value) @@ -154,7 +154,7 @@ class CheckpointSuite extends TestSuiteBase { val originalConf = ssc.conf val cp = new Checkpoint(ssc, Time(1000)) - val cpConf = cp.sparkConf + val cpConf = cp.createSparkConf() assert(cpConf.get("spark.master") === originalConf.get("spark.master")) assert(cpConf.get("spark.app.name") === originalConf.get("spark.app.name")) assert(cpConf.get(key) === value) @@ -163,7 +163,8 @@ class CheckpointSuite extends TestSuiteBase { // Serialize/deserialize to simulate write to storage and reading it back val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - val newCpConf = newCp.sparkConf + // Verify new SparkConf has all the previous properties + val newCpConf = newCp.createSparkConf() assert(newCpConf.get("spark.master") === originalConf.get("spark.master")) assert(newCpConf.get("spark.app.name") === originalConf.get("spark.app.name")) assert(newCpConf.get(key) === value) @@ -174,6 +175,20 @@ class CheckpointSuite extends TestSuiteBase { ssc = new StreamingContext(null, newCp, null) val restoredConf = ssc.conf assert(restoredConf.get(key) === value) + ssc.stop() + + // Verify new SparkConf picks up new master url if it is set in the properties. See SPARK-6331. + try { + val newMaster = "local[100]" + System.setProperty("spark.master", newMaster) + val newCpConf = newCp.createSparkConf() + assert(newCpConf.get("spark.master") === newMaster) + assert(newCpConf.get("spark.app.name") === originalConf.get("spark.app.name")) + ssc = new StreamingContext(null, newCp, null) + assert(ssc.sparkContext.master === newMaster) + } finally { + System.clearProperty("spark.master") + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 6a7cd97aa3222..2e5005ef6ff14 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -100,7 +100,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.sparkConf.getInt("spark.cleaner.ttl", -1) === 10) + assert(newCp.createSparkConf().getInt("spark.cleaner.ttl", -1) === 10) ssc = new StreamingContext(null, newCp, null) assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } From e9f22c61290348c58af54c0ae3c6226105126a8d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 17 Mar 2015 09:41:06 -0500 Subject: [PATCH 42/62] [SPARK-6365] jetty-security also needed for SPARK_PREPEND_CLASSES to work https://issues.apache.org/jira/browse/SPARK-6365 thanks vanzin for helping me figure this out Author: Imran Rashid Closes #5052 from squito/fix_prepend_classes and squashes the following commits: 09d334c [Imran Rashid] jetty-security also needed for SPARK_PREPEND_CLASSES to work --- core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 4164a3a7208d4..81f8cba711df6 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -414,7 +414,7 @@ true true - guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server + guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security true From 0f673c21f68ee3d5df3c01ae405709d3c1f4909b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 17 Mar 2015 09:18:57 -0700 Subject: [PATCH 43/62] [SPARK-3266] Use intermediate abstract classes to fix type erasure issues in Java APIs This PR addresses a Scala compiler bug ([SI-8905](https://issues.scala-lang.org/browse/SI-8905)) that was breaking some of the Spark Java APIs. In a nutshell, it seems that methods whose implementations are inherited from generic traits sometimes have their type parameters erased to Object. This was causing methods like `DoubleRDD.min()` to throw confusing NoSuchMethodErrors at runtime. The fix implemented here is to introduce an intermediate layer of abstract classes and inherit from those instead of directly extends the `Java*Like` traits. This should not break binary compatibility. I also improved the test coverage of the Java API, adding several new tests for methods that failed at runtime due to this bug. Author: Josh Rosen Closes #5050 from JoshRosen/javardd-si-8905-fix and squashes the following commits: 2feb068 [Josh Rosen] Use intermediate abstract classes to work around SPARK-3266 d5f3e5d [Josh Rosen] Add failing regression tests for SPARK-3266 --- .../apache/spark/api/java/JavaDoubleRDD.scala | 3 +- .../apache/spark/api/java/JavaPairRDD.scala | 2 +- .../org/apache/spark/api/java/JavaRDD.scala | 2 +- .../apache/spark/api/java/JavaRDDLike.scala | 8 ++ .../java/org/apache/spark/JavaAPISuite.java | 129 ++++++++++++++++++ .../streaming/api/java/JavaDStream.scala | 2 +- .../streaming/api/java/JavaDStreamLike.scala | 9 ++ .../streaming/api/java/JavaPairDStream.scala | 2 +- 8 files changed, 152 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 8e8f7f6c4fda2..79e4ebf2db578 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -32,7 +32,8 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.StatCounter import org.apache.spark.util.Utils -class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { +class JavaDoubleRDD(val srdd: RDD[scala.Double]) + extends AbstractJavaRDDLike[JDouble, JavaDoubleRDD] { override val classTag: ClassTag[JDouble] = implicitly[ClassTag[JDouble]] diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 7af3538262fd6..4eadc9a85613e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.Utils class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) - extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { + extends AbstractJavaRDDLike[(K, V), JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 86fb374bef1e3..645dc3bfb6b06 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -30,7 +30,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) - extends JavaRDDLike[T, JavaRDD[T]] { + extends AbstractJavaRDDLike[T, JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 0f91c942ecd50..8da42934a7d96 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -38,6 +38,14 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils +/** + * As a workaround for https://issues.scala-lang.org/browse/SI-8905, implementations + * of JavaRDDLike should extend this dummy abstract class instead of directly inheriting + * from the trait. See SPARK-3266 for additional details. + */ +private[spark] abstract class AbstractJavaRDDLike[T, This <: JavaRDDLike[T, This]] + extends JavaRDDLike[T, This] + /** * Defines operations common to several Java RDD implementations. * Note that this trait is not intended to be implemented by user code. diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 74e88c767ee07..8ec54360ca42a 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -267,6 +267,22 @@ public void call(String s) throws IOException { Assert.assertEquals(2, accum.value().intValue()); } + @Test + public void foreachPartition() { + final Accumulator accum = sc.accumulator(0); + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreachPartition(new VoidFunction>() { + @Override + public void call(Iterator iter) throws IOException { + while (iter.hasNext()) { + iter.next(); + accum.add(1); + } + } + }); + Assert.assertEquals(2, accum.value().intValue()); + } + @Test public void toLocalIterator() { List correct = Arrays.asList(1, 2, 3, 4); @@ -657,6 +673,13 @@ public Boolean call(Integer i) { }).isEmpty()); } + @Test + public void toArray() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3)); + List list = rdd.toArray(); + Assert.assertEquals(Arrays.asList(1, 2, 3), list); + } + @Test public void cartesian() { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); @@ -714,6 +737,80 @@ public void javaDoubleRDDHistoGram() { sc.parallelizeDoubles(new ArrayList(0), 1).histogram(new double[]{0.0, 1.0})); } + private static class DoubleComparator implements Comparator, Serializable { + public int compare(Double o1, Double o2) { + return o1.compareTo(o2); + } + } + + @Test + public void max() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(new DoubleComparator()); + Assert.assertEquals(4.0, max, 0.001); + } + + @Test + public void min() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(new DoubleComparator()); + Assert.assertEquals(1.0, max, 0.001); + } + + @Test + public void takeOrdered() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator())); + Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2)); + } + + @Test + public void top() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + List top2 = rdd.top(2); + Assert.assertEquals(Arrays.asList(4, 3), top2); + } + + private static class AddInts implements Function2 { + @Override + public Integer call(Integer a, Integer b) { + return a + b; + } + } + + @Test + public void reduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.reduce(new AddInts()); + Assert.assertEquals(10, sum); + } + + @Test + public void reduceOnJavaDoubleRDD() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double sum = rdd.reduce(new Function2() { + @Override + public Double call(Double v1, Double v2) throws Exception { + return v1 + v2; + } + }); + Assert.assertEquals(10.0, sum, 0.001); + } + + @Test + public void fold() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.fold(0, new AddInts()); + Assert.assertEquals(10, sum); + } + + @Test + public void aggregate() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.aggregate(0, new AddInts(), new AddInts()); + Assert.assertEquals(10, sum); + } + @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); @@ -830,6 +927,25 @@ public Iterable call(Iterator iter) { Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); } + + @Test + public void mapPartitionsWithIndex() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSums = rdd.mapPartitionsWithIndex( + new Function2, Iterator>() { + @Override + public Iterator call(Integer index, Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(sum).iterator(); + } + }, false); + Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); + } + + @Test public void repartition() { // Shrinking number of partitions @@ -1516,6 +1632,19 @@ public void collectAsync() throws Exception { Assert.assertEquals(1, future.jobIds().size()); } + @Test + public void takeAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction> future = rdd.takeAsync(1); + List result = future.get(); + Assert.assertEquals(1, result.size()); + Assert.assertEquals((Integer) 1, result.get(0)); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + @Test public void foreachAsync() throws Exception { List data = Arrays.asList(1, 2, 3, 4, 5); diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 505e4431e4350..01cdcb0574040 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -36,7 +36,7 @@ import org.apache.spark.streaming.dstream.DStream * [[org.apache.spark.streaming.api.java.JavaPairDStream]]. */ class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T]) - extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] { + extends AbstractJavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index c382a12f4d099..2eabdd9387913 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -34,6 +34,15 @@ import org.apache.spark.streaming._ import org.apache.spark.streaming.api.java.JavaDStream._ import org.apache.spark.streaming.dstream.DStream +/** + * As a workaround for https://issues.scala-lang.org/browse/SI-8905, implementations + * of JavaDStreamLike should extend this dummy abstract class instead of directly inheriting + * from the trait. See SPARK-3266 for additional details. + */ +private[streaming] +abstract class AbstractJavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], + R <: JavaRDDLike[T, R]] extends JavaDStreamLike[T, This, R] + trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] extends Serializable { implicit val classTag: ClassTag[T] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index bd01789b611a4..f94f2d0e8bd31 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -45,7 +45,7 @@ import org.apache.spark.streaming.dstream.DStream class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifest: ClassTag[K], implicit val vManifest: ClassTag[V]) - extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] { + extends AbstractJavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) From 4cca3917dc30ee907e6cbd6a569b6ac58af963f7 Mon Sep 17 00:00:00 2001 From: nemccarthy Date: Tue, 17 Mar 2015 09:33:11 -0700 Subject: [PATCH 44/62] [SPARK-6313] Add config option to disable file locks/fetchFile cache to ... ...support NFS mounts. This is a work around for now with the goal to find a more permanent solution. https://issues.apache.org/jira/browse/SPARK-6313 Author: nemccarthy Closes #5036 from nemccarthy/master and squashes the following commits: 2eaaf42 [nemccarthy] [SPARK-6313] Update config wording doc for spark.files.useFetchCache 5de7eb4 [nemccarthy] [SPARK-6313] Add config option to disable file locks/fetchFile cache to support NFS mounts --- .../src/main/scala/org/apache/spark/util/Utils.scala | 3 ++- docs/configuration.md | 12 ++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d3dc1d09cb7b4..af8a24553a461 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -403,7 +403,8 @@ private[spark] object Utils extends Logging { useCache: Boolean) { val fileName = url.split("/").last val targetFile = new File(targetDir, fileName) - if (useCache) { + val fetchCacheEnabled = conf.getBoolean("spark.files.useFetchCache", defaultValue = true) + if (useCache && fetchCacheEnabled) { val cachedFileName = s"${url.hashCode}${timestamp}_cache" val lockFileName = s"${url.hashCode}${timestamp}_lock" val localDir = new File(getLocalDir(conf)) diff --git a/docs/configuration.md b/docs/configuration.md index 63fc99e7d3e29..7fe11475212b3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -745,6 +745,18 @@ Apart from these, the following properties are also available, and may be useful the driver, in seconds. +
+ + + + From d9f3e01688ad0a8d5fc2419a948a682ad7d957c9 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Tue, 17 Mar 2015 12:11:57 -0700 Subject: [PATCH 45/62] [SPARK-6336] LBFGS should document what convergenceTol means LBFGS uses convergence tolerance. This value should be written in document as an argument. Author: lewuathe Closes #5033 from Lewuathe/SPARK-6336 and squashes the following commits: e738b33 [lewuathe] Modify text to be more natural ac03c3a [lewuathe] Modify documentations 6ccb304 [lewuathe] [SPARK-6336] LBFGS should document what convergenceTol means --- docs/mllib-optimization.md | 4 ++++ .../scala/org/apache/spark/mllib/optimization/LBFGS.scala | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 4d101afca2c97..6cabc1610a151 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -203,6 +203,10 @@ regularization, as well as L2 regularizer. recommended. * `maxNumIterations` is the maximal number of iterations that L-BFGS can be run. * `regParam` is the regularization parameter when using regularization. +* `convergenceTol` controls how much relative change is still allowed when L-BFGS +is considered to converge. This must be nonnegative. Lower values are less tolerant and +therefore generally cause more iterations to be run. This value looks at both average +improvement and the norm of gradient inside [Breeze LBFGS](https://github.com/scalanlp/breeze/blob/master/math/src/main/scala/breeze/optimize/LBFGS.scala). The `return` is a tuple containing two elements. The first element is a column matrix containing weights for every feature, and the second element is an array containing diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index d5e4f4ccbff10..ef6eccd90711a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -60,6 +60,8 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) /** * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. * Smaller value will lead to higher accuracy with the cost of more iterations. + * This value must be nonnegative. Lower convergence values are less tolerant + * and therefore generally cause more iterations to be run. */ def setConvergenceTol(tolerance: Double): this.type = { this.convergenceTol = tolerance @@ -142,7 +144,9 @@ object LBFGS extends Logging { * one single data example) * @param updater - Updater function to actually perform a gradient step in a given direction. * @param numCorrections - The number of corrections used in the L-BFGS update. - * @param convergenceTol - The convergence tolerance of iterations for L-BFGS + * @param convergenceTol - The convergence tolerance of iterations for L-BFGS which is must be + * nonnegative. Lower values are less tolerant and therefore generally + * cause more iterations to be run. * @param maxNumIterations - Maximal number of iterations that L-BFGS can be run. * @param regParam - Regularization parameter * From c94d0626471e209ab7ebfc588f9a2992946b7ed5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 17 Mar 2015 12:14:40 -0700 Subject: [PATCH 46/62] [SPARK-6226][MLLIB] add save/load in PySpark's KMeansModel Use `_py2java` and `_java2py` to convert Python model to/from Java model. yinxusen Author: Xiangrui Meng Closes #5049 from mengxr/SPARK-6226-mengxr and squashes the following commits: 570ba81 [Xiangrui Meng] fix python style b10b911 [Xiangrui Meng] add save/load in PySpark's KMeansModel --- .../spark/mllib/clustering/KMeansModel.scala | 5 ++++ python/pyspark/mllib/clustering.py | 28 +++++++++++++++++-- python/pyspark/mllib/common.py | 4 +-- 3 files changed, 32 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 707da537d238f..e4e411a3c8b42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.clustering +import scala.collection.JavaConverters._ + import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ @@ -34,6 +36,9 @@ import org.apache.spark.sql.Row */ class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable { + /** A Java-friendly constructor that takes an Iterable of Vectors. */ + def this(centers: java.lang.Iterable[Vector]) = this(centers.asScala.toArray) + /** Total number of clusters. */ def k: Int = clusterCenters.length diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 949db5705abd7..464f49aeee3cd 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -19,14 +19,16 @@ from pyspark import RDD from pyspark import SparkContext -from pyspark.mllib.common import callMLlibFunc, callJavaFunc -from pyspark.mllib.linalg import DenseVector, SparseVector, _convert_to_vector +from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _py2java, _java2py +from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.stat.distribution import MultivariateGaussian +from pyspark.mllib.util import Saveable, Loader, inherit_doc __all__ = ['KMeansModel', 'KMeans', 'GaussianMixtureModel', 'GaussianMixture'] -class KMeansModel(object): +@inherit_doc +class KMeansModel(Saveable, Loader): """A clustering model derived from the k-means method. @@ -55,6 +57,16 @@ class KMeansModel(object): True >>> type(model.clusterCenters) + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> model.save(sc, path) + >>> sameModel = KMeansModel.load(sc, path) + >>> sameModel.predict(sparse_data[0]) == model.predict(sparse_data[0]) + True + >>> try: + ... os.removedirs(path) + ... except OSError: + ... pass """ def __init__(self, centers): @@ -77,6 +89,16 @@ def predict(self, x): best_distance = distance return best + def save(self, sc, path): + java_centers = _py2java(sc, map(_convert_to_vector, self.centers)) + java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel(java_centers) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel.load(sc._jsc.sc(), path) + return KMeansModel(_java2py(sc, java_model.clusterCenters())) + class KMeans(object): diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index 621591c26b77f..a539d2f2846f9 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -70,8 +70,8 @@ def _py2java(sc, obj): obj = _to_java_object_rdd(obj) elif isinstance(obj, SparkContext): obj = obj._jsc - elif isinstance(obj, list) and (obj or isinstance(obj[0], JavaObject)): - obj = ListConverter().convert(obj, sc._gateway._gateway_client) + elif isinstance(obj, list): + obj = ListConverter().convert([_py2java(sc, x) for x in obj], sc._gateway._gateway_client) elif isinstance(obj, JavaObject): pass elif isinstance(obj, (int, long, float, bool, basestring)): From 4633a87b86a6ef01fa724d31763dcb97cb5bc746 Mon Sep 17 00:00:00 2001 From: Pei-Lun Lee Date: Wed, 18 Mar 2015 08:34:46 +0800 Subject: [PATCH 47/62] [SPARK-6330] [SQL] Add a test case for SPARK-6330 When getting file statuses, create file system from each path instead of a single one from hadoop configuration. Author: Pei-Lun Lee Closes #5039 from ypcat/spark-6351 and squashes the following commits: a19a3fe [Pei-Lun Lee] [SPARK-6330] [SQL] fix test 506f5a0 [Pei-Lun Lee] [SPARK-6351] [SQL] fix test fa2290e [Pei-Lun Lee] [SPARK-6330] [SQL] Rename test case and add comment 606c967 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6351 896e80a [Pei-Lun Lee] [SPARK-6351] [SQL] Add test case 2ae0916 [Pei-Lun Lee] [SPARK-6351] [SQL] ParquetRelation2 supporting multiple file systems --- .../apache/spark/sql/parquet/ParquetIOSuite.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 36f3406a7825f..a70b3c7ce48d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -329,6 +329,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { checkAnswer(parquetFile(file), (data ++ newData).map(Row.fromTuple)) } } + } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { @@ -341,6 +342,18 @@ class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterA override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6330 regression test") { + // In 1.3.0, save to fs other than file: without configuring core-site.xml would get: + // IllegalArgumentException: Wrong FS: hdfs://..., expected: file:/// + intercept[java.io.FileNotFoundException] { + sqlContext.parquetFile("file:///nonexistent") + } + val errorMessage = intercept[Throwable] { + sqlContext.parquetFile("hdfs://nonexistent") + }.toString + assert(errorMessage.contains("UnknownHostException")) + } } class ParquetDataSourceOffIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From dc9c9196d63aa465e86ac52f0e86e10c12472100 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 18 Mar 2015 09:41:06 +0800 Subject: [PATCH 48/62] [SPARK-6366][SQL] In Python API, the default save mode for save and saveAsTable should be "error" instead of "append". https://issues.apache.org/jira/browse/SPARK-6366 Author: Yin Huai Closes #5053 from yhuai/SPARK-6366 and squashes the following commits: fc81897 [Yin Huai] Use error as the default save mode for save/saveAsTable. --- python/pyspark/sql/dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 94001aec3774b..5cb89da7a8ed5 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -162,7 +162,7 @@ def _java_save_mode(self, mode): "Only 'append', 'overwrite', 'ignore', and 'error' are acceptable save mode.") return jmode - def saveAsTable(self, tableName, source=None, mode="append", **options): + def saveAsTable(self, tableName, source=None, mode="error", **options): """Saves the contents of the :class:`DataFrame` to a data source as a table. The data source is specified by the `source` and a set of `options`. @@ -188,7 +188,7 @@ def saveAsTable(self, tableName, source=None, mode="append", **options): self.sql_ctx._sc._gateway._gateway_client) self._jdf.saveAsTable(tableName, source, jmode, joptions) - def save(self, path=None, source=None, mode="append", **options): + def save(self, path=None, source=None, mode="error", **options): """Saves the contents of the :class:`DataFrame` to a data source. The data source is specified by the `source` and a set of `options`. From a012e08635dc2d643715e11680fd6a3fb3afe44a Mon Sep 17 00:00:00 2001 From: Tijo Thomas Date: Tue, 17 Mar 2015 18:50:19 -0700 Subject: [PATCH 49/62] [SPARK-6383][SQL]Fixed compiler and errors in Dataframe examples Author: Tijo Thomas Closes #5068 from tijoparacka/fix_sql_dataframe_example and squashes the following commits: 6953ac1 [Tijo Thomas] Handled Java and Python example sections 0751a74 [Tijo Thomas] Fixed compiler and errors in Dataframe examples --- docs/sql-programming-guide.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 11c29e20632ae..2cbb4c967eb81 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -170,14 +170,14 @@ df.select("name").show() // Justin // Select everybody, but increment the age by 1 -df.select("name", df("age") + 1).show() +df.select(df("name"), df("age") + 1).show() // name (age + 1) // Michael null // Andy 31 // Justin 20 // Select people older than 21 -df.filter(df("name") > 21).show() +df.filter(df("age") > 21).show() // age name // 30 Andy @@ -220,14 +220,14 @@ df.select("name").show(); // Justin // Select everybody, but increment the age by 1 -df.select("name", df.col("age").plus(1)).show(); +df.select(df.col("name"), df.col("age").plus(1)).show(); // name (age + 1) // Michael null // Andy 31 // Justin 20 // Select people older than 21 -df.filter(df("name") > 21).show(); +df.filter(df.col("age").gt(21)).show(); // age name // 30 Andy @@ -270,14 +270,14 @@ df.select("name").show() ## Justin # Select everybody, but increment the age by 1 -df.select("name", df.age + 1).show() +df.select(df.name, df.age + 1).show() ## name (age + 1) ## Michael null ## Andy 31 ## Justin 20 # Select people older than 21 -df.filter(df.name > 21).show() +df.filter(df.age > 21).show() ## age name ## 30 Andy From 5c80643d137008ce8a0ac7467b31d8d52383c105 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 17 Mar 2015 18:58:52 -0700 Subject: [PATCH 50/62] [SPARK-5908][SQL] Resolve UdtfsAlias when only single Alias is used `ResolveUdtfsAlias` in `hiveUdfs` only considers the `HiveGenericUdtf` with multiple alias. When only single alias is used with `HiveGenericUdtf`, the alias is not working. Author: Liang-Chi Hsieh Closes #4692 from viirya/udft_alias and squashes the following commits: 8a3bae4 [Liang-Chi Hsieh] No need to test selected column from DataFrame since DataFrame API is updated. 160a379 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into udft_alias e6531cc [Liang-Chi Hsieh] Selected column from DataFrame should not re-analyze logical plan. a45cc2a [Liang-Chi Hsieh] Resolve UdtfsAlias when only single Alias is used. --- .../main/scala/org/apache/spark/sql/hive/hiveUdfs.scala | 2 ++ .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 34c21c11761ae..4a702d96563d5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -333,6 +333,8 @@ private[spark] object ResolveUdtfsAlias extends Rule[LogicalPlan] { if projectList.exists(_.isInstanceOf[MultiAlias]) && projectList.size != 1 => throw new TreeNodeException(p, "only single Generator supported for SELECT clause") + case Project(Seq(Alias(udtf @ HiveGenericUdtf(_, _, _), name)), child) => + Generate(udtf.copy(aliasNames = Seq(name)), join = false, outer = false, None, child) case Project(Seq(MultiAlias(udtf @ HiveGenericUdtf(_, _, _), names)), child) => Generate(udtf.copy(aliasNames = names), join = false, outer = false, None, child) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 22ea19bd82f86..1187228f4c3db 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -397,6 +397,13 @@ class SQLQuerySuite extends QueryTest { dropTempTable("data") } + test("resolve udtf with single alias") { + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + jsonRDD(rdd).registerTempTable("data") + val df = sql("SELECT explode(a) AS val FROM data") + val col = df("val") + } + test("logical.Project should not be resolved if it contains aggregates or generators") { // This test is used to test the fix of SPARK-5875. // The original issue was that Project's resolved will be true when it contains From 78cb08a5db7b3e1b61ffb28bc95d0b23e8db5c40 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 17 Mar 2015 19:32:38 -0700 Subject: [PATCH 51/62] [SPARK-5404] [SQL] Update the default statistic number By default, the statistic for logical plan with multiple children is quite aggressive, and those statistic are quite critical for the join optimization, hence we need to estimate the statistics as accurate as possible. For `Union`, which has 2 children, and overwrite the default implementation by `adding` its children `byteInSize` instead of `multiplying`. For `Expand`, which only has a single child, but it will grows the size, and we need to multiply its inflating factor. Author: Cheng Hao Closes #4914 from chenghao-intel/statistic and squashes the following commits: d466bbc [Cheng Hao] Update the default statistic --- .../sql/catalyst/plans/logical/basicOperators.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 20cc8e90a71a3..624912dab4652 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -81,6 +81,11 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved = childrenResolved && !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } + + override def statistics: Statistics = { + val sizeInBytes = left.statistics.sizeInBytes + right.statistics.sizeInBytes + Statistics(sizeInBytes = sizeInBytes) + } } case class Join( @@ -174,7 +179,12 @@ case class Aggregate( case class Expand( projections: Seq[GroupExpression], output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode + child: LogicalPlan) extends UnaryNode { + override def statistics: Statistics = { + val sizeInBytes = child.statistics.sizeInBytes * projections.length + Statistics(sizeInBytes = sizeInBytes) + } +} trait GroupingAnalytics extends UnaryNode { self: Product => From a6ee2f7940b9a64a81667615586ae597da837974 Mon Sep 17 00:00:00 2001 From: watermen Date: Tue, 17 Mar 2015 19:35:18 -0700 Subject: [PATCH 52/62] [SPARK-5651][SQL] Add input64 in blacklist and add test suit for create table within backticks Now spark version is only support ```create table table_in_database_creation.test1 as select * from src limit 1;``` in HiveContext. This patch is used to support ```create table `table_in_database_creation.test2` as select * from src limit 1;``` in HiveContext. Author: watermen Author: q00251598 Closes #4427 from watermen/SPARK-5651 and squashes the following commits: c5c8ed1 [watermen] add the generated golden files 1f0e42e [q00251598] add input64 in blacklist and add test suit --- .../execution/HiveCompatibilitySuite.scala | 6 +- ...ckticks-0-a253b1ed35dbf503d1b8902dacbe23ac | 0 ...ckticks-1-61dc640dfeaff471f3d2b730f9cbf959 | 0 ...ckticks-2-ce780d068b8d24786e639e361101a0c7 | 500 ++++++++++++++++++ ...ckticks-3-afd6e46b6a289c3c24a8eec75a94043c | 0 .../sql/hive/execution/HiveQuerySuite.scala | 8 + 6 files changed, 513 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-0-a253b1ed35dbf503d1b8902dacbe23ac create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-1-61dc640dfeaff471f3d2b730f9cbf959 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-3-afd6e46b6a289c3c24a8eec75a94043c 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 68cb34d698ef3..5180a7f09d80f 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 @@ -236,7 +236,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // timestamp in array, the output format of Hive contains double quotes, while // Spark SQL doesn't - "udf_sort_array" + "udf_sort_array", + + // It has a bug and it has been fixed by + // https://issues.apache.org/jira/browse/HIVE-7673 (in Hive 0.14 and trunk). + "input46" ) ++ HiveShim.compatibilityBlackList /** diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-0-a253b1ed35dbf503d1b8902dacbe23ac b/sql/hive/src/test/resources/golden/create table as with db name within backticks-0-a253b1ed35dbf503d1b8902dacbe23ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-1-61dc640dfeaff471f3d2b730f9cbf959 b/sql/hive/src/test/resources/golden/create table as with db name within backticks-1-61dc640dfeaff471f3d2b730f9cbf959 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 b/sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-3-afd6e46b6a289c3c24a8eec75a94043c b/sql/hive/src/test/resources/golden/create table as with db name within backticks-3-afd6e46b6a289c3c24a8eec75a94043c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index c0d21bc9a89da..de140fc72a2c3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -334,6 +334,14 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |DROP DATABASE IF EXISTS testdb CASCADE """.stripMargin) + createQueryTest("create table as with db name within backticks", + """ + |CREATE DATABASE IF NOT EXISTS testdb; + |CREATE TABLE `testdb`.`createdtable` AS SELECT * FROM default.src; + |SELECT * FROM testdb.createdtable; + |DROP DATABASE IF EXISTS testdb CASCADE + """.stripMargin) + createQueryTest("insert table with db name", """ |CREATE DATABASE IF NOT EXISTS testdb; From 3579003115fa3217cff6aa400729d96b0c7b257b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 17 Mar 2015 19:47:51 -0700 Subject: [PATCH 53/62] [SPARK-6247][SQL] Fix resolution of ambiguous joins caused by new aliases We need to handle ambiguous `exprId`s that are produced by new aliases as well as those caused by leaf nodes (`MultiInstanceRelation`). Attempting to fix this revealed a bug in `equals` for `Alias` as these objects were comparing equal even when the expression ids did not match. Additionally, `LocalRelation` did not correctly provide statistics, and some tests in `catalyst` and `hive` were not using the helper functions for comparing plans. Based on #4991 by chenghao-intel Author: Michael Armbrust Closes #5062 from marmbrus/selfJoins and squashes the following commits: 8e9b84b [Michael Armbrust] check qualifier too 8038a36 [Michael Armbrust] handle aggs too 0b9c687 [Michael Armbrust] fix more tests c3c574b [Michael Armbrust] revert change. 725f1ab [Michael Armbrust] add statistics a925d08 [Michael Armbrust] check for conflicting attributes in join resolution b022ef7 [Michael Armbrust] Handle project aliases. d8caa40 [Michael Armbrust] test case: SPARK-6247 f9c67c2 [Michael Armbrust] Check for duplicate attributes in join resolution. 898af73 [Michael Armbrust] Fix Alias equality. --- .../sql/catalyst/analysis/Analyzer.scala | 30 +++++++++++++++--- .../expressions/namedExpressions.scala | 6 ++++ .../plans/logical/LocalRelation.scala | 3 ++ .../plans/logical/basicOperators.scala | 7 +++++ .../analysis/HiveTypeCoercionSuite.scala | 10 +++--- .../spark/sql/catalyst/plans/PlanTest.scala | 11 +++++-- .../spark/sql/ColumnExpressionSuite.scala | 6 +++- .../org/apache/spark/sql/SQLQuerySuite.scala | 31 +++++++++++++++++++ .../spark/sql/catalyst/plans/PlanTest.scala | 4 ++- 9 files changed, 96 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7753331748d7b..92d3db077c5e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -237,22 +237,33 @@ class Analyzer(catalog: Catalog, // Special handling for cases when self-join introduce duplicate expression ids. case j @ Join(left, right, _, _) if left.outputSet.intersect(right.outputSet).nonEmpty => val conflictingAttributes = left.outputSet.intersect(right.outputSet) + logDebug(s"Conflicting attributes ${conflictingAttributes.mkString(",")} in $j") - val (oldRelation, newRelation, attributeRewrites) = right.collect { + val (oldRelation, newRelation) = right.collect { + // Handle base relations that might appear more than once. case oldVersion: MultiInstanceRelation if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => val newVersion = oldVersion.newInstance() - val newAttributes = AttributeMap(oldVersion.output.zip(newVersion.output)) - (oldVersion, newVersion, newAttributes) + (oldVersion, newVersion) + + // Handle projects that create conflicting aliases. + case oldVersion @ Project(projectList, _) + if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => + (oldVersion, oldVersion.copy(projectList = newAliases(projectList))) + + case oldVersion @ Aggregate(_, aggregateExpressions, _) + if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => + (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) }.head // Only handle first case found, others will be fixed on the next pass. + val attributeRewrites = AttributeMap(oldRelation.output.zip(newRelation.output)) val newRight = right transformUp { case r if r == oldRelation => newRelation + } transformUp { case other => other transformExpressions { case a: Attribute => attributeRewrites.get(a).getOrElse(a) } } - j.copy(right = newRight) case q: LogicalPlan => @@ -272,6 +283,17 @@ class Analyzer(catalog: Catalog, } } + def newAliases(expressions: Seq[NamedExpression]): Seq[NamedExpression] = { + expressions.map { + case a: Alias => Alias(a.child, a.name)() + case other => other + } + } + + def findAliases(projectList: Seq[NamedExpression]): AttributeSet = { + AttributeSet(projectList.collect { case a: Alias => a.toAttribute }) + } + /** * Returns true if `exprs` contains a [[Star]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 62c062be6d820..17f7f9fe51376 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -124,6 +124,12 @@ case class Alias(child: Expression, name: String) override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs = exprId :: qualifiers :: Nil + + override def equals(other: Any): Boolean = other match { + case a: Alias => + name == a.name && exprId == a.exprId && child == a.child && qualifiers == a.qualifiers + case _ => false + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index 92bd057c6f4b6..bb79dc340553b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -54,4 +54,7 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[Row] = Nil) otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data case _ => false } + + override lazy val statistics = + Statistics(sizeInBytes = output.map(_.dataType.defaultSize).sum * data.length) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 624912dab4652..1e7b449d75b80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -108,6 +108,13 @@ case class Join( left.output ++ right.output } } + + def selfJoinResolved = left.outputSet.intersect(right.outputSet).isEmpty + + // Joins are only resolved if they don't introduce ambiguious expression ids. + override lazy val resolved: Boolean = { + childrenResolved && !expressions.exists(!_.resolved) && selfJoinResolved + } } case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 85798d0871fda..ecbb54218d457 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.catalyst.analysis -import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.types._ -class HiveTypeCoercionSuite extends FunSuite { +class HiveTypeCoercionSuite extends PlanTest { test("tightest common bound for types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { @@ -106,7 +106,8 @@ class HiveTypeCoercionSuite extends FunSuite { val booleanCasts = new HiveTypeCoercion { }.BooleanCasts def ruleTest(initial: Expression, transformed: Expression) { val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) - assert(booleanCasts(Project(Seq(Alias(initial, "a")()), testRelation)) == + comparePlans( + booleanCasts(Project(Seq(Alias(initial, "a")()), testRelation)), Project(Seq(Alias(transformed, "a")()), testRelation)) } // Remove superflous boolean -> boolean casts. @@ -119,7 +120,8 @@ class HiveTypeCoercionSuite extends FunSuite { val fac = new HiveTypeCoercion { }.FunctionArgumentConversion def ruleTest(initial: Expression, transformed: Expression) { val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) - assert(fac(Project(Seq(Alias(initial, "a")()), testRelation)) == + comparePlans( + fac(Project(Seq(Alias(initial, "a")()), testRelation)), Project(Seq(Alias(transformed, "a")()), testRelation)) } ruleTest( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 7d609b91389c6..48884040bfce7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.plans import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{NoRelation, Filter, LogicalPlan} import org.apache.spark.sql.catalyst.util._ /** @@ -36,6 +36,8 @@ class PlanTest extends FunSuite { plan transformAllExpressions { case a: AttributeReference => AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) + case a: Alias => + Alias(a.child, a.name)(exprId = ExprId(0)) } } @@ -50,4 +52,9 @@ class PlanTest extends FunSuite { |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} """.stripMargin) } + + /** Fails the test if the two expressions do not match */ + protected def compareExpressions(e1: Expression, e2: Expression): Unit = { + comparePlans(Filter(e1, NoRelation), Filter(e2, NoRelation)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 3036fbc05d021..a53ae97d6243a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.spark.sql.catalyst.plans.logical.{Project, NoRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -311,7 +313,9 @@ class ColumnExpressionSuite extends QueryTest { } test("lift alias out of cast") { - assert(col("1234").as("name").cast("int").expr === col("1234").cast("int").as("name").expr) + compareExpressions( + col("1234").as("name").cast("int").expr, + col("1234").cast("int").as("name").expr) } test("columns can be compared") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4dedcd365f6cc..a3c0076e16d6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -36,6 +36,37 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { import org.apache.spark.sql.test.TestSQLContext.implicits._ val sqlCtx = TestSQLContext + test("self join with aliases") { + Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str").registerTempTable("df") + + checkAnswer( + sql( + """ + |SELECT x.str, COUNT(*) + |FROM df x JOIN df y ON x.str = y.str + |GROUP BY x.str + """.stripMargin), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + + test("self join with alias in agg") { + Seq(1,2,3) + .map(i => (i, i.toString)) + .toDF("int", "str") + .groupBy("str") + .agg($"str", count("str").as("strCount")) + .registerTempTable("df") + + checkAnswer( + sql( + """ + |SELECT x.str, SUM(x.strCount) + |FROM df x JOIN df y ON x.str = y.str + |GROUP BY x.str + """.stripMargin), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { checkAnswer( sql("SELECT a FROM testData2 SORT BY a"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 44ee5ab5975fb..98f1c0e69e29d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.scalatest.FunSuite @@ -38,6 +38,8 @@ class PlanTest extends FunSuite { plan transformAllExpressions { case a: AttributeReference => AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) + case a: Alias => + Alias(a.child, a.name)(exprId = ExprId(0)) } } From 6205a255aea0652dddadf953771f5405065b5bec Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Mar 2015 09:06:57 -0400 Subject: [PATCH 54/62] [SPARK-6372] [core] Propagate --conf to child processes. And add unit test. Author: Marcelo Vanzin Closes #5057 from vanzin/SPARK-6372 and squashes the following commits: b33728b [Marcelo Vanzin] [SPARK-6372] [core] Propagate --conf to child processes. --- .../spark/launcher/SparkSubmitCommandBuilder.java | 10 +--------- .../spark/launcher/SparkSubmitCommandBuilderSuite.java | 4 ++++ 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 6ffdff63d3c78..91dcf70f105db 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -253,12 +253,6 @@ private boolean isClientMode(Properties userProps) { private class OptionParser extends SparkSubmitOptionParser { - private final List driverJvmKeys = Arrays.asList( - SparkLauncher.DRIVER_EXTRA_CLASSPATH, - SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, - SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, - SparkLauncher.DRIVER_MEMORY); - @Override protected boolean handle(String opt, String value) { if (opt.equals(MASTER)) { @@ -278,9 +272,7 @@ protected boolean handle(String opt, String value) { } else if (opt.equals(CONF)) { String[] setConf = value.split("=", 2); checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); - if (driverJvmKeys.contains(setConf[0])) { - conf.put(setConf[0], setConf[1]); - } + conf.put(setConf[0], setConf[1]); } else if (opt.equals(CLASS)) { // The special classes require some special command line handling, since they allow // mixing spark-submit arguments with arguments that should be propagated to the shell diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 815edc4e4971f..626116a9e7477 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -68,6 +68,8 @@ public void testCliParser() throws Exception { parser.DRIVER_JAVA_OPTIONS, "extraJavaOpt", parser.CONF, + "spark.randomOption=foo", + parser.CONF, SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath"); Map env = new HashMap(); List cmd = buildCommand(sparkSubmitArgs, env); @@ -77,6 +79,8 @@ public void testCliParser() throws Exception { assertTrue(findInStringList(findArgValue(cmd, "-cp"), File.pathSeparator, "/driverCp")); assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms42g")); assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx42g")); + assertTrue("Command should contain user-defined conf.", + Collections.indexOfSubList(cmd, Arrays.asList(parser.CONF, "spark.randomOption=foo")) > 0); } @Test From e09c852d6b83b9b112685d113f2792daec8785a3 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 18 Mar 2015 09:09:32 -0400 Subject: [PATCH 55/62] SPARK-6389 YARN app diagnostics report doesn't report NPEs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Trivial patch to implicitly call `Exception.toString()` over `Exception.getMessage()` —this defaults to including the exception class & any non-null message; some subclasses include more. No test. Author: Steve Loughran Closes #5070 from steveloughran/stevel/patches/SPARK-6389-NPE-reporting and squashes the following commits: 8239d85 [Steve Loughran] SPARK-6389 cull use of getMessage over toString in the container launcher 6fbaf6a [Steve Loughran] SPARK-6389 YARN app diagnostics report doesn't report NPEs --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index e966bfba7bb7d..056b8c0257cfe 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -151,7 +151,7 @@ private[spark] class ApplicationMaster( logError("Uncaught exception: ", e) finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, - "Uncaught exception: " + e.getMessage()) + "Uncaught exception: " + e) } exitCode } @@ -486,10 +486,10 @@ private[spark] class ApplicationMaster( case _: InterruptedException => // Reporter thread can interrupt to stop user class case cause: Throwable => - logError("User class threw exception: " + cause.getMessage, cause) + logError("User class threw exception: " + cause, cause) finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_EXCEPTION_USER_CLASS, - "User class threw exception: " + cause.getMessage) + "User class threw exception: " + cause) } } } From 9d112a958ee2facad179344dd367a6d1ccbc9614 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Wed, 18 Mar 2015 09:15:33 -0400 Subject: [PATCH 56/62] [SPARK-6286][minor] Handle missing Mesos case TASK_ERROR. Author: Iulian Dragos Closes #5000 from dragos/issue/task-error-case and squashes the following commits: e063627 [Iulian Dragos] Handle TASK_ERROR in Mesos scheduler backends. ac17cf0 [Iulian Dragos] Handle missing Mesos case TASK_ERROR. --- core/src/main/scala/org/apache/spark/TaskState.scala | 1 + .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 12 ++---------- .../cluster/mesos/MesosSchedulerBackend.scala | 10 +--------- 3 files changed, 4 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala index 0bf1e4a5e2ccd..d85a6d683427d 100644 --- a/core/src/main/scala/org/apache/spark/TaskState.scala +++ b/core/src/main/scala/org/apache/spark/TaskState.scala @@ -46,5 +46,6 @@ private[spark] object TaskState extends Enumeration { case MesosTaskState.TASK_FAILED => FAILED case MesosTaskState.TASK_KILLED => KILLED case MesosTaskState.TASK_LOST => LOST + case MesosTaskState.TASK_ERROR => LOST } } 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 90dfe14352a8e..fc92b9c35c3a3 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 @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.{Utils, AkkaUtils} @@ -262,20 +262,12 @@ private[spark] class CoarseMesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - private def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { val taskId = status.getTaskId.getValue.toInt val state = status.getState logInfo("Mesos task " + taskId + " is now " + state) synchronized { - if (isFinished(state)) { + if (TaskState.isFinished(TaskState.fromMesos(state))) { val slaveId = taskIdToSlaveId(taskId) slaveIdsWithExecutors -= slaveId taskIdToSlaveId -= taskId 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 cfb6592e14aa8..df8f4306b88a8 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 @@ -313,14 +313,6 @@ private[spark] class MesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { inClassLoader() { val tid = status.getTaskId.getValue.toLong @@ -330,7 +322,7 @@ private[spark] class MesosSchedulerBackend( // We lost the executor on this slave, so remember that it's gone removeExecutor(taskIdToSlaveId(tid), "Lost executor") } - if (isFinished(status.getState)) { + if (TaskState.isFinished(state)) { taskIdToSlaveId.remove(tid) } } From 981fbafa2a878e86abeefe1d77cca01fd848f9f6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Mar 2015 09:18:28 -0400 Subject: [PATCH 57/62] [SPARK-6325] [core,yarn] Do not change target executor count when killing executors. The dynamic execution code has two ways to reduce the number of executors: one where it reduces the total number of executors it wants, by asking for an absolute number of executors that is lower than the previous one. The second is by explicitly killing idle executors. YarnAllocator was mixing those up and lowering the target number of executors when a kill was issued. Instead, trust the frontend knows what it's doing, and kill executors without messing with other accounting. That means that if the frontend kills an executor without lowering the target, it will get a new executor shortly. The one situation where both actions (lower the target and kill executor) need to happen together is when user code explicitly calls `SparkContext.killExecutors`. In that case, issue two calls to the backend to achieve the goal. I also did some minor cleanup in related code: - avoid sending a request for executors when target is unchanged, to avoid log spam in the AM - avoid printing misleading log messages in the AM when there are no requests to cancel - fix a slow memory leak plus misleading error message on the driver caused by failing to completely unregister the executor. Author: Marcelo Vanzin Closes #5018 from vanzin/SPARK-6325 and squashes the following commits: 2e782a3 [Marcelo Vanzin] Avoid redundant logging on the AM side. a3567cd [Marcelo Vanzin] Add parentheses. a363926 [Marcelo Vanzin] Update logic. a158101 [Marcelo Vanzin] [SPARK-6325] [core,yarn] Disallow reducing executor count past running count. --- .../CoarseGrainedSchedulerBackend.scala | 7 ++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 1 - .../spark/deploy/yarn/YarnAllocator.scala | 13 ++++++----- .../deploy/yarn/YarnAllocatorSuite.scala | 22 +++++++++++++++++++ 4 files changed, 37 insertions(+), 6 deletions(-) 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 6f77fa32ce37b..87ebf31139ce9 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 @@ -211,6 +211,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { + addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingToRemove -= executorId } @@ -371,6 +372,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste logWarning(s"Executor to kill $id does not exist!") } } + // Killing executors means effectively that we want less executors than before, so also update + // the target number of executors to avoid having the backend allocate new ones. + val newTotal = (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size + - filteredExecutorIds.size) + doRequestTotalExecutors(newTotal) + executorsPendingToRemove ++= filteredExecutorIds doKillExecutors(filteredExecutorIds) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 056b8c0257cfe..3d18690cd9cbf 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -534,7 +534,6 @@ private[spark] class ApplicationMaster( driver ! x case RequestExecutors(requestedTotal) => - logInfo(s"Driver requested a total number of $requestedTotal executor(s).") Option(allocator) match { case Some(a) => a.requestTotalExecutors(requestedTotal) case None => logWarning("Container allocator is not ready to request executors yet.") diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 55bfbcd9cb84b..c98763e15b58f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -86,7 +86,8 @@ private[yarn] class YarnAllocator( @volatile private var targetNumExecutors = args.numExecutors // Keep track of which container is running which executor to remove the executors later - private val executorIdToContainer = new HashMap[String, Container] + // Visible for testing. + private[yarn] val executorIdToContainer = new HashMap[String, Container] // Executor memory in MB. protected val executorMemory = args.executorMemory @@ -137,7 +138,10 @@ private[yarn] class YarnAllocator( * be killed. */ def requestTotalExecutors(requestedTotal: Int): Unit = synchronized { - targetNumExecutors = requestedTotal + if (requestedTotal != targetNumExecutors) { + logInfo(s"Driver requested a total number of $requestedTotal executor(s).") + targetNumExecutors = requestedTotal + } } /** @@ -148,8 +152,6 @@ private[yarn] class YarnAllocator( val container = executorIdToContainer.remove(executorId).get internalReleaseContainer(container) numExecutorsRunning -= 1 - targetNumExecutors -= 1 - assert(targetNumExecutors >= 0, "Allocator killed more executors than are allocated!") } else { logWarning(s"Attempted to kill unknown executor $executorId!") } @@ -351,7 +353,8 @@ private[yarn] class YarnAllocator( } } - private def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = { + // Visible for testing. + private[yarn] def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = { for (completedContainer <- completedContainers) { val containerId = completedContainer.getContainerId diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 3c224f148802e..c09b01bafce37 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -206,6 +206,28 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach handler.getNumExecutorsRunning should be (2) } + test("kill executors") { + val handler = createAllocator(4) + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getNumPendingAllocate should be (4) + + val container1 = createContainer("host1") + val container2 = createContainer("host2") + handler.handleAllocatedContainers(Array(container1, container2)) + + handler.requestTotalExecutors(1) + handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) } + + val statuses = Seq(container1, container2).map { c => + ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Finished", 0) + } + handler.updateResourceRequests() + handler.processCompletedContainers(statuses.toSeq) + handler.getNumExecutorsRunning should be (0) + handler.getNumPendingAllocate should be (1) + } + test("memory exceeded diagnostic regexes") { val diagnostics = "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " + From a95ee242b0a9644c912fc54ed68b4301c9558bc9 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Wed, 18 Mar 2015 13:44:37 -0400 Subject: [PATCH 58/62] [SPARK-6374] [MLlib] add get for GeneralizedLinearAlgo I find it's better to have getter for NumFeatures and addIntercept within GeneralizedLinearAlgorithm during actual usage, otherwise I 'll have to get the value through debug. Author: Yuhao Yang Closes #5058 from hhbyyh/addGetLinear and squashes the following commits: 9dc90e8 [Yuhao Yang] add get for GeneralizedLinearAlgo --- .../mllib/regression/GeneralizedLinearAlgorithm.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 7c66e8cdebdbe..b262bec904525 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -123,6 +123,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ private var useFeatureScaling = false + /** + * The dimension of training features. + */ + def getNumFeatures: Int = this.numFeatures + /** * The dimension of training features. */ @@ -141,6 +146,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ protected def createModel(weights: Vector, intercept: Double): M + /** + * Get if the algorithm uses addIntercept + */ + def isAddIntercept: Boolean = this.addIntercept + /** * Set if the algorithm should add an intercept. Default false. * We set the default to false because adding the intercept will cause memory allocation. From 3db13874250ded267d7455898e4048a70a47fdcd Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Wed, 18 Mar 2015 20:54:22 -0400 Subject: [PATCH 59/62] SPARK-6085 Part. 2 Increase default value for memory overhead - fixed a description of spark.mesos.executor.memoryOverhead from 7% to 10% - This is a second part of SPARK-6085 Author: Jongyoul Lee Closes #5065 from jongyoul/SPARK-6085-1 and squashes the following commits: c5af84c [Jongyoul Lee] SPARK-6085 Part. 2 Increase default value for memory overhead - Changed "MiB" to "MB" dbac1c0 [Jongyoul Lee] SPARK-6085 Part. 2 Increase default value for memory overhead - fixed a description of spark.mesos.executor.memoryOverhead from 7% to 10% --- docs/running-on-mesos.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 59a3e9d25baf1..6a9d304501dc0 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -224,11 +224,11 @@ See the [configuration page](configuration.html) for information on Spark config
spark.files.useFetchCachetrue + If set to true (default), file fetching will use a local cache that is shared by executors + that belong to the same application, which can improve task launching performance when + running many executors on the same host. If set to false, these caching optimizations will + be disabled and all executors will fetch their own copies of files. This optimization may be + disabled in order to use Spark local directories that reside on NFS filesystems (see + SPARK-6313 for more details). +
spark.files.overwrite falsespark.mesos.executor.memoryOverhead executor memory * 0.10, with minimum of 384 - This value is an additive for spark.executor.memory, specified in MiB, + This value is an additive for spark.executor.memory, specified in MB, which is used to calculate the total Mesos task memory. A value of 384 - implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum + implies a 384MB overhead. Additionally, there is a hard-coded 10% minimum overhead. The final overhead will be the larger of either - `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`. + `spark.mesos.executor.memoryOverhead` or 10% of `spark.executor.memory`.
From 540b2a4eabe0bad2455f5140c4ad6a315e37cc3f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 Mar 2015 19:43:04 -0700 Subject: [PATCH 60/62] [SPARK-6394][Core] cleanup BlockManager companion object and improve the getCacheLocs method in DAGScheduler The current implementation include searching a HashMap many times, we can avoid this. Actually if you look into `BlockManager.blockIdsToBlockManagers`, the core function call is [this](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/BlockManager.scala#L1258), so we can call `blockManagerMaster.getLocations` directly and avoid building a HashMap. Author: Wenchen Fan Closes #5043 from cloud-fan/small and squashes the following commits: e959d12 [Wenchen Fan] fix style 203c493 [Wenchen Fan] some cleanup in BlockManager companion object d409099 [Wenchen Fan] address rxin's comment faec999 [Wenchen Fan] add regression test 2fb57aa [Wenchen Fan] imporve the getCacheLocs method --- .../apache/spark/scheduler/DAGScheduler.scala | 11 +++++----- .../apache/spark/storage/BlockManager.scala | 22 ++++--------------- .../spark/scheduler/DAGSchedulerSuite.scala | 12 ++++++++++ 3 files changed, 22 insertions(+), 23 deletions(-) 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 e4170a55b7981..1021172e6afb4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -104,7 +104,7 @@ class DAGScheduler( * * All accesses to this map should be guarded by synchronizing on it (see SPARK-4454). */ - private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] + private val cacheLocs = new HashMap[Int, Seq[Seq[TaskLocation]]] // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with // every task. When we detect a node failing, we note the current epoch number and failed @@ -188,14 +188,15 @@ class DAGScheduler( eventProcessLoop.post(TaskSetFailed(taskSet, reason)) } - private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = cacheLocs.synchronized { + private[scheduler] + def getCacheLocs(rdd: RDD[_]): Seq[Seq[TaskLocation]] = cacheLocs.synchronized { // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] - val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) - cacheLocs(rdd.id) = blockIds.map { id => - locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) + val locs: Seq[Seq[TaskLocation]] = blockManagerMaster.getLocations(blockIds).map { bms => + bms.map(bm => TaskLocation(bm.host, bm.executorId)) } + cacheLocs(rdd.id) = locs } cacheLocs(rdd.id) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c8b7763f03fb7..80d66e59132da 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1245,10 +1245,10 @@ private[spark] object BlockManager extends Logging { } } - def blockIdsToBlockManagers( + def blockIdsToHosts( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { // blockManagerMaster != null is used in tests assert(env != null || blockManagerMaster != null) @@ -1258,24 +1258,10 @@ private[spark] object BlockManager extends Logging { blockManagerMaster.getLocations(blockIds) } - val blockManagers = new HashMap[BlockId, Seq[BlockManagerId]] + val blockManagers = new HashMap[BlockId, Seq[String]] for (i <- 0 until blockIds.length) { - blockManagers(blockIds(i)) = blockLocations(i) + blockManagers(blockIds(i)) = blockLocations(i).map(_.host) } blockManagers.toMap } - - def blockIdsToExecutorIds( - blockIds: Array[BlockId], - env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { - blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId)) - } - - def blockIdsToHosts( - blockIds: Array[BlockId], - env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { - blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host)) - } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 30119ce5d4eec..63360a0f189a3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -322,6 +322,18 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assertDataStructuresEmpty } + test("regression test for getCacheLocs") { + val rdd = new MyRDD(sc, 3, Nil) + cacheLocations(rdd.id -> 0) = + Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) + cacheLocations(rdd.id -> 1) = + Seq(makeBlockManagerId("hostB"), makeBlockManagerId("hostC")) + cacheLocations(rdd.id -> 2) = + Seq(makeBlockManagerId("hostC"), makeBlockManagerId("hostD")) + val locs = scheduler.getCacheLocs(rdd).map(_.map(_.host)) + assert(locs === Seq(Seq("hostA", "hostB"), Seq("hostB", "hostC"), Seq("hostC", "hostD"))) + } + test("avoid exponential blowup when getting preferred locs list") { // Build up a complex dependency graph with repeated zip operations, without preferred locations. var rdd: RDD[_] = new MyRDD(sc, 1, Nil) From 645cf3fcc21987417b2946bdeeeb60af3edf667e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 19 Mar 2015 02:15:50 -0400 Subject: [PATCH 61/62] [SPARK-6222][Streaming] Dont delete checkpoint data when doing pre-batch-start checkpoint This is another alternative approach to https://github.com/apache/spark/pull/4964/ I think this is a simpler fix that can be backported easily to other branches (1.2 and 1.3). All it does it introduce a flag so that the pre-batch-start checkpoint does not call clear checkpoint. There is not unit test yet. I will add it when this approach is commented upon. Not sure if this is testable easily. Author: Tathagata Das Closes #5008 from tdas/SPARK-6222 and squashes the following commits: 7315bc2 [Tathagata Das] Removed empty line. c438de4 [Tathagata Das] Revert unnecessary change. 5e98374 [Tathagata Das] Added unit test 50cb60b [Tathagata Das] Fixed style issue 295ca5c [Tathagata Das] Fixing SPARK-6222 --- .../apache/spark/streaming/Checkpoint.scala | 12 +- .../streaming/scheduler/JobGenerator.scala | 20 +-- .../scheduler/JobGeneratorSuite.scala | 133 ++++++++++++++++++ 3 files changed, 153 insertions(+), 12 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index cb4c94fb9d5a6..db64e11e16304 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -119,7 +119,10 @@ class CheckpointWriter( private var stopped = false private var fs_ : FileSystem = _ - class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable { + class CheckpointWriteHandler( + checkpointTime: Time, + bytes: Array[Byte], + clearCheckpointDataLater: Boolean) extends Runnable { def run() { var attempts = 0 val startTime = System.currentTimeMillis() @@ -166,7 +169,7 @@ class CheckpointWriter( val finishTime = System.currentTimeMillis() logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") - jobGenerator.onCheckpointCompletion(checkpointTime) + jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { case ioe: IOException => @@ -180,7 +183,7 @@ class CheckpointWriter( } } - def write(checkpoint: Checkpoint) { + def write(checkpoint: Checkpoint, clearCheckpointDataLater: Boolean) { val bos = new ByteArrayOutputStream() val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) @@ -188,7 +191,8 @@ class CheckpointWriter( oos.close() bos.close() try { - executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + executor.execute(new CheckpointWriteHandler( + checkpoint.checkpointTime, bos.toByteArray, clearCheckpointDataLater)) logDebug("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") } catch { case rej: RejectedExecutionException => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index ac92774a38273..59488dfb0f8c6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -30,7 +30,8 @@ import org.apache.spark.util.{Clock, ManualClock} private[scheduler] sealed trait JobGeneratorEvent private[scheduler] case class GenerateJobs(time: Time) extends JobGeneratorEvent private[scheduler] case class ClearMetadata(time: Time) extends JobGeneratorEvent -private[scheduler] case class DoCheckpoint(time: Time) extends JobGeneratorEvent +private[scheduler] case class DoCheckpoint( + time: Time, clearCheckpointDataLater: Boolean) extends JobGeneratorEvent private[scheduler] case class ClearCheckpointData(time: Time) extends JobGeneratorEvent /** @@ -163,8 +164,10 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** * Callback called when the checkpoint of a batch has been written. */ - def onCheckpointCompletion(time: Time) { - eventActor ! ClearCheckpointData(time) + def onCheckpointCompletion(time: Time, clearCheckpointDataLater: Boolean) { + if (clearCheckpointDataLater) { + eventActor ! ClearCheckpointData(time) + } } /** Processes all events */ @@ -173,7 +176,8 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { event match { case GenerateJobs(time) => generateJobs(time) case ClearMetadata(time) => clearMetadata(time) - case DoCheckpoint(time) => doCheckpoint(time) + case DoCheckpoint(time, clearCheckpointDataLater) => + doCheckpoint(time, clearCheckpointDataLater) case ClearCheckpointData(time) => clearCheckpointData(time) } } @@ -245,7 +249,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } - eventActor ! DoCheckpoint(time) + eventActor ! DoCheckpoint(time, clearCheckpointDataLater = false) } /** Clear DStream metadata for the given `time`. */ @@ -255,7 +259,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // If checkpointing is enabled, then checkpoint, // else mark batch to be fully processed if (shouldCheckpoint) { - eventActor ! DoCheckpoint(time) + eventActor ! DoCheckpoint(time, clearCheckpointDataLater = true) } else { // If checkpointing is not enabled, then delete metadata information about // received blocks (block data not saved in any case). Otherwise, wait for @@ -278,11 +282,11 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } /** Perform checkpoint for the give `time`. */ - private def doCheckpoint(time: Time) { + private def doCheckpoint(time: Time, clearCheckpointDataLater: Boolean) { if (shouldCheckpoint && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { logInfo("Checkpointing graph for time " + time) ssc.graph.updateCheckpointData(time) - checkpointWriter.write(new Checkpoint(ssc, time)) + checkpointWriter.write(new Checkpoint(ssc, time), clearCheckpointDataLater) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala new file mode 100644 index 0000000000000..4150b60635ed6 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala @@ -0,0 +1,133 @@ +/* + * 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.streaming.scheduler + +import java.util.concurrent.CountDownLatch + +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming._ +import org.apache.spark.util.{ManualClock, Utils} + +class JobGeneratorSuite extends TestSuiteBase { + + // SPARK-6222 is a tricky regression bug which causes received block metadata + // to be deleted before the corresponding batch has completed. This occurs when + // the following conditions are met. + // 1. streaming checkpointing is enabled by setting streamingContext.checkpoint(dir) + // 2. input data is received through a receiver as blocks + // 3. a batch processing a set of blocks takes a long time, such that a few subsequent + // batches have been generated and submitted for processing. + // + // The JobGenerator (as of Mar 16, 2015) checkpoints twice per batch, once after generation + // of a batch, and another time after the completion of a batch. The cleanup of + // checkpoint data (including block metadata, etc.) from DStream must be done only after the + // 2nd checkpoint has completed, that is, after the batch has been completely processed. + // However, the issue is that the checkpoint data and along with it received block data is + // cleaned even in the case of the 1st checkpoint, causing pre-mature deletion of received block + // data. For example, if the 3rd batch is still being process, the 7th batch may get generated, + // and the corresponding "1st checkpoint" will delete received block metadata of batch older + // than 6th batch. That, is 3rd batch's block metadata gets deleted even before 3rd batch has + // been completely processed. + // + // This test tries to create that scenario by the following. + // 1. enable checkpointing + // 2. generate batches with received blocks + // 3. make the 3rd batch never complete + // 4. allow subsequent batches to be generated (to allow premature deletion of 3rd batch metadata) + // 5. verify whether 3rd batch's block metadata still exists + // + test("SPARK-6222: Do not clear received block data too soon") { + import JobGeneratorSuite._ + val checkpointDir = Utils.createTempDir() + val testConf = conf + testConf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + testConf.set("spark.streaming.receiver.writeAheadLog.rollingInterval", "1") + + withStreamingContext(new StreamingContext(testConf, batchDuration)) { ssc => + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val numBatches = 10 + val longBatchNumber = 3 // 3rd batch will take a long time + val longBatchTime = longBatchNumber * batchDuration.milliseconds + + val testTimeout = timeout(10 seconds) + val inputStream = ssc.receiverStream(new TestReceiver) + + inputStream.foreachRDD((rdd: RDD[Int], time: Time) => { + if (time.milliseconds == longBatchTime) { + while (waitLatch.getCount() > 0) { + waitLatch.await() + println("Await over") + } + } + }) + + val batchCounter = new BatchCounter(ssc) + ssc.checkpoint(checkpointDir.getAbsolutePath) + ssc.start() + + // Make sure the only 1 batch of information is to be remembered + assert(inputStream.rememberDuration === batchDuration) + val receiverTracker = ssc.scheduler.receiverTracker + + // Get the blocks belonging to a batch + def getBlocksOfBatch(batchTime: Long) = { + receiverTracker.getBlocksOfBatchAndStream(Time(batchTime), inputStream.id) + } + + // Wait for new blocks to be received + def waitForNewReceivedBlocks() { + eventually(testTimeout) { + assert(receiverTracker.hasUnallocatedBlocks) + } + } + + // Wait for received blocks to be allocated to a batch + def waitForBlocksToBeAllocatedToBatch(batchTime: Long) { + eventually(testTimeout) { + assert(getBlocksOfBatch(batchTime).nonEmpty) + } + } + + // Generate a large number of batches with blocks in them + for (batchNum <- 1 to numBatches) { + waitForNewReceivedBlocks() + clock.advance(batchDuration.milliseconds) + waitForBlocksToBeAllocatedToBatch(clock.getTimeMillis()) + } + + // Wait for 3rd batch to start + eventually(testTimeout) { + ssc.scheduler.getPendingTimes().contains(Time(numBatches * batchDuration.milliseconds)) + } + + // Verify that the 3rd batch's block data is still present while the 3rd batch is incomplete + assert(getBlocksOfBatch(longBatchTime).nonEmpty, "blocks of incomplete batch already deleted") + assert(batchCounter.getNumCompletedBatches < longBatchNumber) + waitLatch.countDown() + } + } +} + +object JobGeneratorSuite { + val waitLatch = new CountDownLatch(1) +} From 2c3f83c34bb8d2c1bf13b33633d8c5a8089545d1 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 18 Mar 2015 23:48:45 -0700 Subject: [PATCH 62/62] [SPARK-4012] stop SparkContext when the exception is thrown from an infinite loop https://issues.apache.org/jira/browse/SPARK-4012 This patch is a resubmission for https://github.com/apache/spark/pull/2864 What I am proposing in this patch is that ***when the exception is thrown from an infinite loop, we should stop the SparkContext, instead of let JVM throws exception forever*** So, in the infinite loops where we originally wrapped with a ` logUncaughtExceptions`, I changed to `tryOrStopSparkContext`, so that the Spark component is stopped Early stopped JVM process is helpful for HA scheme design, for example, The user has a script checking the existence of the pid of the Spark Streaming driver for monitoring the availability; with the code before this patch, the JVM process is still available but not functional when the exceptions are thrown andrewor14, srowen , mind taking further consideration about the change? Author: CodingCat Closes #5004 from CodingCat/SPARK-4012-1 and squashes the following commits: 589276a [CodingCat] throw fatal error again 3c72cd8 [CodingCat] address the comments 6087864 [CodingCat] revise comments 6ad3eb0 [CodingCat] stop SparkContext instead of quit the JVM process 6322959 [CodingCat] exit JVM process when the exception is thrown from an infinite loop --- .../org/apache/spark/ContextCleaner.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../deploy/history/FsHistoryProvider.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/util/AsynchronousListenerBus.scala | 10 +++++-- .../scala/org/apache/spark/util/Utils.scala | 28 +++++++++++++++++++ .../scheduler/EventLoggingListenerSuite.scala | 9 +++--- .../spark/scheduler/SparkListenerSuite.scala | 10 +++---- .../streaming/scheduler/JobScheduler.scala | 2 +- 9 files changed, 51 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 0c59a61e81393..9b05c9623b704 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -145,7 +145,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Keep cleaning RDD, shuffle, and broadcast state. */ - private def keepCleaning(): Unit = Utils.logUncaughtExceptions { + private def keepCleaning(): Unit = Utils.tryOrStopSparkContext(sc) { while (!stopped) { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4457f40286fda..228ff715fe7cb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1736,7 +1736,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } - listenerBus.start() + listenerBus.start(this) } /** Post the application start event */ 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 16d88c17d1a76..7fde02040927d 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 @@ -93,7 +93,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def getRunner(operateFun: () => Unit): Runnable = { new Runnable() { - override def run() = Utils.logUncaughtExceptions { + override def run() = Utils.tryOrExit { operateFun() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 7a9cf1c2e7f30..f33fd4450b2a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -145,7 +145,7 @@ private[spark] class TaskSchedulerImpl( import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { - Utils.tryOrExit { checkSpeculatableTasks() } + Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } } } } diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala index 18c627e8c7a15..ce7887b76ff96 100644 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala @@ -21,6 +21,7 @@ import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean import com.google.common.annotations.VisibleForTesting +import org.apache.spark.SparkContext /** * Asynchronously passes events to registered listeners. @@ -38,6 +39,8 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri self => + private var sparkContext: SparkContext = null + /* Cap the capacity of the event queue so we get an explicit error (rather than * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ private val EVENT_QUEUE_CAPACITY = 10000 @@ -57,7 +60,7 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri private val listenerThread = new Thread(name) { setDaemon(true) - override def run(): Unit = Utils.logUncaughtExceptions { + override def run(): Unit = Utils.tryOrStopSparkContext(sparkContext) { while (true) { eventLock.acquire() self.synchronized { @@ -89,9 +92,12 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri * This first sends out all buffered events posted before this listener bus has started, then * listens for any additional events asynchronously while the listener bus is still running. * This should only be called once. + * + * @param sc Used to stop the SparkContext in case the listener thread dies. */ - def start() { + def start(sc: SparkContext) { if (started.compareAndSet(false, true)) { + sparkContext = sc listenerThread.start() } else { throw new IllegalStateException(s"$name already started!") diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index af8a24553a461..91aa70870ab20 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1146,6 +1146,8 @@ private[spark] object Utils extends Logging { /** * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the * default UncaughtExceptionHandler + * + * NOTE: This method is to be called by the spark-started JVM process. */ def tryOrExit(block: => Unit) { try { @@ -1156,6 +1158,32 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * exception + * + * NOTE: This method is to be called by the driver-side components to avoid stopping the + * user-started JVM process completely; in contrast, tryOrExit is to be called in the + * spark-started JVM process . + */ + def tryOrStopSparkContext(sc: SparkContext)(block: => Unit) { + try { + block + } catch { + case e: ControlThrowable => throw e + case t: Throwable => + val currentThreadName = Thread.currentThread().getName + if (sc != null) { + logError(s"uncaught error in thread $currentThreadName, stopping SparkContext", t) + sc.stop() + } + if (!NonFatal(t)) { + logError(s"throw uncaught fatal error in thread $currentThreadName", t) + throw t + } + } + } + /** * Execute a block of code that evaluates to Unit, re-throwing any non-fatal uncaught * exceptions as IOException. This is used when implementing Externalizable and Serializable's diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 992dde66f982f..448258a754153 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -25,9 +25,9 @@ import scala.io.Source import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.{FunSuiteLike, BeforeAndAfter, FunSuite} -import org.apache.spark.{Logging, SparkConf, SparkContext, SPARK_VERSION} +import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -39,7 +39,8 @@ import org.apache.spark.util.{JsonProtocol, Utils} * logging events, whether the parsing of the file names is correct, and whether the logged events * can be read and deserialized into actual SparkListenerEvents. */ -class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Logging { +class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter + with Logging { import EventLoggingListenerSuite._ private val fileSystem = Utils.getHadoopFileSystem("/", @@ -144,7 +145,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite eventLogger.start() - listenerBus.start() + listenerBus.start(sc) listenerBus.addListener(eventLogger) listenerBus.postToAll(applicationStart) listenerBus.postToAll(applicationEnd) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 3a41ee8d4ae0c..627c9a4ddfffc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -46,7 +46,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(counter.count === 0) // Starting listener bus should flush all buffered events - bus.start() + bus.start(sc) assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(counter.count === 5) @@ -58,8 +58,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers // Listener bus must not be started twice intercept[IllegalStateException] { val bus = new LiveListenerBus - bus.start() - bus.start() + bus.start(sc) + bus.start(sc) } // ... or stopped before starting @@ -96,7 +96,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val blockingListener = new BlockingListener bus.addListener(blockingListener) - bus.start() + bus.start(sc) bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) listenerStarted.acquire() @@ -347,7 +347,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.addListener(badListener) bus.addListener(jobCounter1) bus.addListener(jobCounter2) - bus.start() + bus.start(sc) // Post events to all listeners, and wait until the queue is drained (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index b3ffc71904c76..60bc099b27a4c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -61,7 +61,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } }), "JobScheduler") - listenerBus.start() + listenerBus.start(ssc.sparkContext) receiverTracker = new ReceiverTracker(ssc) receiverTracker.start() jobGenerator.start()