From c5aca0c27be31e94ffdb01ef2eb29d3b373d7f4c Mon Sep 17 00:00:00 2001 From: Glenn Weidner Date: Sun, 10 May 2015 19:18:32 -0700 Subject: [PATCH 01/59] [SPARK-7427] [PYSPARK] Make sharedParams match in Scala, Python Modified 2 files: python/pyspark/ml/param/_shared_params_code_gen.py python/pyspark/ml/param/shared.py Generated shared.py on Linux using Python 2.6.6 on Redhat Enterprise Linux Server 6.6. python _shared_params_code_gen.py > shared.py Only changed maxIter, regParam, rawPredictionCol based on strings from SharedParamsCodeGen.scala. Note warning was displayed when committing shared.py: warning: LF will be replaced by CRLF in python/pyspark/ml/param/shared.py. Author: Glenn Weidner Closes #6023 from gweidner/br-7427 and squashes the following commits: db72e32 [Glenn Weidner] [SPARK-7427] [PySpark] Make sharedParams match in Scala, Python 825e4a9 [Glenn Weidner] [SPARK-7427] [PySpark] Make sharedParams match in Scala, Python e6a865e [Glenn Weidner] [SPARK-7427] [PySpark] Make sharedParams match in Scala, Python 1eee702 [Glenn Weidner] Merge remote-tracking branch 'upstream/master' 1ac10e5 [Glenn Weidner] Merge remote-tracking branch 'upstream/master' cafd104 [Glenn Weidner] Merge remote-tracking branch 'upstream/master' 9bea1eb [Glenn Weidner] Merge remote-tracking branch 'upstream/master' 4a35c20 [Glenn Weidner] Merge remote-tracking branch 'upstream/master' 9790cbe [Glenn Weidner] Merge remote-tracking branch 'upstream/master' d9c30f4 [Glenn Weidner] [SPARK-7275] [SQL] [WIP] Make LogicalRelation public --- .../ml/param/_shared_params_code_gen.py | 6 ++-- python/pyspark/ml/param/shared.py | 30 +++++++++---------- python/pyspark/ml/tests.py | 4 +-- tox.ini | 2 +- 4 files changed, 20 insertions(+), 22 deletions(-) diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index ed3171b6976d3..3be0979b92013 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -88,12 +88,12 @@ def get$Name(self): print("\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n") print("from pyspark.ml.param import Param, Params\n\n") shared = [ - ("maxIter", "max number of iterations", None), - ("regParam", "regularization constant", None), + ("maxIter", "max number of iterations (>= 0)", None), + ("regParam", "regularization parameter (>= 0)", None), ("featuresCol", "features column name", "'features'"), ("labelCol", "label column name", "'label'"), ("predictionCol", "prediction column name", "'prediction'"), - ("rawPredictionCol", "raw prediction column name", "'rawPrediction'"), + ("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", "'rawPrediction'"), ("inputCol", "input column name", None), ("inputCols", "input column names", None), ("outputCol", "output column name", None), diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index d0bcadee22347..4b22322b895b4 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -22,16 +22,16 @@ class HasMaxIter(Params): """ - Mixin for param maxIter: max number of iterations. + Mixin for param maxIter: max number of iterations (>= 0). """ # a placeholder to make it appear in the generated doc - maxIter = Param(Params._dummy(), "maxIter", "max number of iterations") + maxIter = Param(Params._dummy(), "maxIter", "max number of iterations (>= 0)") def __init__(self): super(HasMaxIter, self).__init__() - #: param for max number of iterations - self.maxIter = Param(self, "maxIter", "max number of iterations") + #: param for max number of iterations (>= 0) + self.maxIter = Param(self, "maxIter", "max number of iterations (>= 0)") if None is not None: self._setDefault(maxIter=None) @@ -51,16 +51,16 @@ def getMaxIter(self): class HasRegParam(Params): """ - Mixin for param regParam: regularization constant. + Mixin for param regParam: regularization parameter (>= 0). """ # a placeholder to make it appear in the generated doc - regParam = Param(Params._dummy(), "regParam", "regularization constant") + regParam = Param(Params._dummy(), "regParam", "regularization parameter (>= 0)") def __init__(self): super(HasRegParam, self).__init__() - #: param for regularization constant - self.regParam = Param(self, "regParam", "regularization constant") + #: param for regularization parameter (>= 0) + self.regParam = Param(self, "regParam", "regularization parameter (>= 0)") if None is not None: self._setDefault(regParam=None) @@ -167,16 +167,16 @@ def getPredictionCol(self): class HasRawPredictionCol(Params): """ - Mixin for param rawPredictionCol: raw prediction column name. + Mixin for param rawPredictionCol: raw prediction (a.k.a. confidence) column name. """ # a placeholder to make it appear in the generated doc - rawPredictionCol = Param(Params._dummy(), "rawPredictionCol", "raw prediction column name") + rawPredictionCol = Param(Params._dummy(), "rawPredictionCol", "raw prediction (a.k.a. confidence) column name") def __init__(self): super(HasRawPredictionCol, self).__init__() - #: param for raw prediction column name - self.rawPredictionCol = Param(self, "rawPredictionCol", "raw prediction column name") + #: param for raw prediction (a.k.a. confidence) column name + self.rawPredictionCol = Param(self, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name") if 'rawPrediction' is not None: self._setDefault(rawPredictionCol='rawPrediction') @@ -403,14 +403,12 @@ class HasStepSize(Params): """ # a placeholder to make it appear in the generated doc - stepSize = Param(Params._dummy(), "stepSize", - "Step size to be used for each iteration of optimization.") + stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization.") def __init__(self): super(HasStepSize, self).__init__() #: param for Step size to be used for each iteration of optimization. - self.stepSize = Param(self, "stepSize", - "Step size to be used for each iteration of optimization.") + self.stepSize = Param(self, "stepSize", "Step size to be used for each iteration of optimization.") if None is not None: self._setDefault(stepSize=None) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 75bb5d749ca87..ba6478dcd58a9 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -128,7 +128,7 @@ def test_param(self): testParams = TestParams() maxIter = testParams.maxIter self.assertEqual(maxIter.name, "maxIter") - self.assertEqual(maxIter.doc, "max number of iterations") + self.assertEqual(maxIter.doc, "max number of iterations (>= 0)") self.assertTrue(maxIter.parent is testParams) def test_params(self): @@ -156,7 +156,7 @@ def test_params(self): self.assertEquals( testParams.explainParams(), "\n".join(["inputCol: input column name (undefined)", - "maxIter: max number of iterations (default: 10, current: 100)"])) + "maxIter: max number of iterations (>= 0) (default: 10, current: 100)"])) if __name__ == "__main__": diff --git a/tox.ini b/tox.ini index b568029a204cc..76e3f42cde62d 100644 --- a/tox.ini +++ b/tox.ini @@ -15,4 +15,4 @@ [pep8] max-line-length=100 -exclude=cloudpickle.py,heapq3.py +exclude=cloudpickle.py,heapq3.py,shared.py From 0835f1edd4c9c05439df85c248faf6787d45f7b7 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 10 May 2015 19:49:42 -0700 Subject: [PATCH 02/59] [SPARK-7512] [SPARKR] Fix RDD's show method to use getJRDD Since the RDD object might be a Pipelined RDD we should use `getJRDD` to get the right handle to the Java object. Fixes the bug reported at http://stackoverflow.com/questions/30057702/sparkr-filterrdd-and-flatmap-not-working cc concretevitamin Author: Shivaram Venkataraman Closes #6035 from shivaram/sparkr-show-bug and squashes the following commits: d70145c [Shivaram Venkataraman] Fix RDD's show method to use getJRDD Fixes the bug reported at http://stackoverflow.com/questions/30057702/sparkr-filterrdd-and-flatmap-not-working --- R/pkg/R/RDD.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 73999a6737032..9138629cac9c0 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -67,8 +67,8 @@ setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, }) setMethod("show", "RDD", - function(.Object) { - cat(paste(callJMethod(.Object@jrdd, "toString"), "\n", sep="")) + function(object) { + cat(paste(callJMethod(getJRDD(object), "toString"), "\n", sep="")) }) setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) { From 2242ab31e99227a102b0918d73db67e99899fd24 Mon Sep 17 00:00:00 2001 From: tianyi Date: Mon, 11 May 2015 14:08:15 +0800 Subject: [PATCH 03/59] [SPARK-7519] [SQL] fix minor bugs in thrift server UI Bugs description: 1. There are extra commas on the top of session list. 2. The format of time in "Start at:" part is not the same as others. 3. The total number of online sessions is wrong. Author: tianyi Closes #6048 from tianyi/SPARK-7519 and squashes the following commits: ed366b7 [tianyi] fix bug --- .../sql/hive/thriftserver/HiveThriftServer2.scala | 4 +++- .../sql/hive/thriftserver/ui/ThriftServerPage.scala | 10 +++++----- 2 files changed, 8 insertions(+), 6 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 0be5a92c2546c..3458b04bfba0f 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 @@ -147,7 +147,7 @@ object HiveThriftServer2 extends Logging { override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { server.stop() } - + var onlineSessionNum: Int = 0 val sessionList = new mutable.LinkedHashMap[String, SessionInfo] val executionList = new mutable.LinkedHashMap[String, ExecutionInfo] val retainedStatements = @@ -170,11 +170,13 @@ object HiveThriftServer2 extends Logging { def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { val info = new SessionInfo(sessionId, System.currentTimeMillis, ip, userName) sessionList.put(sessionId, info) + onlineSessionNum += 1 trimSessionIfNecessary() } def onSessionClosed(sessionId: String): Unit = { sessionList(sessionId).finishTimestamp = System.currentTimeMillis + onlineSessionNum -= 1 } def onStatementStart( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 71b16b6bebffb..6a2be4a58e5cb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -29,7 +29,7 @@ import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui._ -/** Page for Spark Web UI that shows statistics of a streaming job */ +/** Page for Spark Web UI that shows statistics of a thrift server */ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { private val listener = parent.listener @@ -42,7 +42,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" generateBasicStats() ++
++

- {listener.sessionList.size} session(s) are online, + {listener.onlineSessionNum} session(s) are online, running {listener.totalRunning} SQL statement(s)

++ generateSessionStatsTable() ++ @@ -50,12 +50,12 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" UIUtils.headerSparkPage("ThriftServer", content, parent, Some(5000)) } - /** Generate basic stats of the streaming program */ + /** Generate basic stats of the thrift server program */ private def generateBasicStats(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime.getTime
  • - Started at: {startTime.toString} + Started at: {formatDate(startTime)}
  • Time since start: {formatDurationVerbose(timeSinceStart)} @@ -148,7 +148,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {session.userName} {session.ip} - {session.sessionId} , + {session.sessionId} {formatDate(session.startTimestamp)} {if(session.finishTimestamp > 0) formatDate(session.finishTimestamp)} {formatDurationOption(Some(session.totalTime))} From d70a076892e0677acceccaba665908cdf664f1b4 Mon Sep 17 00:00:00 2001 From: Wesley Miao Date: Mon, 11 May 2015 12:20:06 +0100 Subject: [PATCH 04/59] [SPARK-7326] [STREAMING] Performing window() on a WindowedDStream doesn't work all the time tdas https://issues.apache.org/jira/browse/SPARK-7326 The problem most likely resides in DStream.slice() implementation, as shown below. def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = { if (!isInitialized) { throw new SparkException(this + " has not been initialized") } if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) { logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")") } if (!(toTime - zeroTime).isMultipleOf(slideDuration)) { logWarning("toTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")") } val alignedToTime = toTime.floor(slideDuration, zeroTime) val alignedFromTime = fromTime.floor(slideDuration, zeroTime) logInfo("Slicing from " + fromTime + " to " + toTime + " (aligned to " + alignedFromTime + " and " + alignedToTime + ")") alignedFromTime.to(alignedToTime, slideDuration).flatMap(time => { if (time >= zeroTime) getOrCompute(time) else None }) } Here after performing floor() on both fromTime and toTime, the result (alignedFromTime - zeroTime) and (alignedToTime - zeroTime) may no longer be multiple of the slidingDuration, thus making isTimeValid() check failed for all the remaining computation. The fix is to add a new floor() function in Time.scala to respect the zeroTime while performing the floor : def floor(that: Duration, zeroTime: Time): Time = { val t = that.milliseconds new Time(((this.millis - zeroTime.milliseconds) / t) * t + zeroTime.milliseconds) } And then change the DStream.slice to call this new floor function by passing in its zeroTime. val alignedToTime = toTime.floor(slideDuration, zeroTime) val alignedFromTime = fromTime.floor(slideDuration, zeroTime) This way the alignedToTime and alignedFromTime are *really* aligned in respect to zeroTime whose value is not really a 0. Author: Wesley Miao Author: Wesley Closes #5871 from wesleymiao/spark-7326 and squashes the following commits: 82a4d8c [Wesley Miao] [SPARK-7326] [STREAMING] Performing window() on a WindowedDStream dosen't work all the time 48b4dc0 [Wesley] [SPARK-7326] [STREAMING] Performing window() on a WindowedDStream doesn't work all the time 6ade399 [Wesley] [SPARK-7326] [STREAMING] Performing window() on a WindowedDStream doesn't work all the time 2611745 [Wesley Miao] [SPARK-7326] [STREAMING] Performing window() on a WindowedDStream doesn't work all the time --- .../org/apache/spark/streaming/Time.scala | 5 +++++ .../spark/streaming/dstream/DStream.scala | 22 ++++++++++++------- .../apache/spark/streaming/TimeSuite.scala | 3 +++ 3 files changed, 22 insertions(+), 8 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 42c49678d24f0..92cfd7d40338c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -63,6 +63,11 @@ case class Time(private val millis: Long) { new Time((this.millis / t) * t) } + def floor(that: Duration, zeroTime: Time): Time = { + val t = that.milliseconds + new Time(((this.millis - zeroTime.milliseconds) / t) * t + zeroTime.milliseconds) + } + def isMultipleOf(that: Duration): Boolean = (this.millis % that.milliseconds == 0) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index f1f8a70655996..7092a3d3f0b86 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -763,16 +763,22 @@ abstract class DStream[T: ClassTag] ( if (!isInitialized) { throw new SparkException(this + " has not been initialized") } - if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) { - logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" - + slideDuration + ")") + + val alignedToTime = if ((toTime - zeroTime).isMultipleOf(slideDuration)) { + toTime + } else { + logWarning("toTime (" + toTime + ") is not a multiple of slideDuration (" + + slideDuration + ")") + toTime.floor(slideDuration, zeroTime) } - if (!(toTime - zeroTime).isMultipleOf(slideDuration)) { - logWarning("toTime (" + fromTime + ") is not a multiple of slideDuration (" - + slideDuration + ")") + + val alignedFromTime = if ((fromTime - zeroTime).isMultipleOf(slideDuration)) { + fromTime + } else { + logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + + slideDuration + ")") + fromTime.floor(slideDuration, zeroTime) } - val alignedToTime = toTime.floor(slideDuration) - val alignedFromTime = fromTime.floor(slideDuration) logInfo("Slicing from " + fromTime + " to " + toTime + " (aligned to " + alignedFromTime + " and " + alignedToTime + ")") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala index 5579ac364346c..e6a01656f479d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala @@ -69,6 +69,9 @@ class TimeSuite extends TestSuiteBase { assert(new Time(1200).floor(new Duration(200)) == new Time(1200)) assert(new Time(199).floor(new Duration(200)) == new Time(0)) assert(new Time(1).floor(new Duration(1)) == new Time(1)) + assert(new Time(1350).floor(new Duration(200), new Time(50)) == new Time(1250)) + assert(new Time(1350).floor(new Duration(200), new Time(150)) == new Time(1350)) + assert(new Time(1350).floor(new Duration(200), new Time(200)) == new Time(1200)) } test("isMultipleOf") { From 042dda3c5c25b5ecb6ae4fd37c85b211b01c187b Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 11 May 2015 09:14:20 -0700 Subject: [PATCH 05/59] [SPARK-6092] [MLLIB] Add RankingMetrics in PySpark/MLlib Author: Yanbo Liang Closes #6044 from yanboliang/spark-6092 and squashes the following commits: 726a9b1 [Yanbo Liang] add newRankingMetrics 33f649c [Yanbo Liang] Add RankingMetrics in PySpark/MLlib --- .../mllib/api/python/PythonMLLibAPI.scala | 10 +++ python/pyspark/mllib/evaluation.py | 78 ++++++++++++++++++- 2 files changed, 86 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 8c30ad4b391ae..f4c477596557f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -32,6 +32,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ +import org.apache.spark.mllib.evaluation.RankingMetrics import org.apache.spark.mllib.feature._ import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} import org.apache.spark.mllib.linalg._ @@ -50,6 +51,7 @@ import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTree import org.apache.spark.mllib.tree.{DecisionTree, GradientBoostedTrees, RandomForest} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -923,6 +925,14 @@ private[python] class PythonMLLibAPI extends Serializable { RG.gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols, parts, s) } + /** + * Java stub for the constructor of Python mllib RankingMetrics + */ + def newRankingMetrics(predictionAndLabels: DataFrame): RankingMetrics[Any] = { + new RankingMetrics(predictionAndLabels.map( + r => (r.getSeq(0).toArray[Any], r.getSeq(1).toArray[Any]))) + } + } diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 36914597de228..4c777f2180dc9 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -15,9 +15,12 @@ # limitations under the License. # -from pyspark.mllib.common import JavaModelWrapper +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc from pyspark.sql import SQLContext -from pyspark.sql.types import StructField, StructType, DoubleType +from pyspark.sql.types import StructField, StructType, DoubleType, IntegerType, ArrayType + +__all__ = ['BinaryClassificationMetrics', 'RegressionMetrics', + 'MulticlassMetrics', 'RankingMetrics'] class BinaryClassificationMetrics(JavaModelWrapper): @@ -270,6 +273,77 @@ def weightedFMeasure(self, beta=None): return self.call("weightedFMeasure", beta) +class RankingMetrics(JavaModelWrapper): + """ + Evaluator for ranking algorithms. + + >>> predictionAndLabels = sc.parallelize([ + ... ([1, 6, 2, 7, 8, 3, 9, 10, 4, 5], [1, 2, 3, 4, 5]), + ... ([4, 1, 5, 6, 2, 7, 3, 8, 9, 10], [1, 2, 3]), + ... ([1, 2, 3, 4, 5], [])]) + >>> metrics = RankingMetrics(predictionAndLabels) + >>> metrics.precisionAt(1) + 0.33... + >>> metrics.precisionAt(5) + 0.26... + >>> metrics.precisionAt(15) + 0.17... + >>> metrics.meanAveragePrecision + 0.35... + >>> metrics.ndcgAt(3) + 0.33... + >>> metrics.ndcgAt(10) + 0.48... + + """ + + def __init__(self, predictionAndLabels): + """ + :param predictionAndLabels: an RDD of (predicted ranking, ground truth set) pairs. + """ + sc = predictionAndLabels.ctx + sql_ctx = SQLContext(sc) + df = sql_ctx.createDataFrame(predictionAndLabels, + schema=sql_ctx._inferSchema(predictionAndLabels)) + java_model = callMLlibFunc("newRankingMetrics", df._jdf) + super(RankingMetrics, self).__init__(java_model) + + def precisionAt(self, k): + """ + Compute the average precision of all the queries, truncated at ranking position k. + + If for a query, the ranking algorithm returns n (n < k) results, the precision value + will be computed as #(relevant items retrieved) / k. This formula also applies when + the size of the ground truth set is less than k. + + If a query has an empty ground truth set, zero will be used as precision together + with a log warning. + """ + return self.call("precisionAt", int(k)) + + @property + def meanAveragePrecision(self): + """ + Returns the mean average precision (MAP) of all the queries. + If a query has an empty ground truth set, the average precision will be zero and + a log warining is generated. + """ + return self.call("meanAveragePrecision") + + def ndcgAt(self, k): + """ + Compute the average NDCG value of all the queries, truncated at ranking position k. + The discounted cumulative gain at position k is computed as: + sum,,i=1,,^k^ (2^{relevance of ''i''th item}^ - 1) / log(i + 1), + and the NDCG is obtained by dividing the DCG value on the ground truth set. + In the current implementation, the relevance value is binary. + + If a query has an empty ground truth set, zero will be used as ndcg together with + a log warning. + """ + return self.call("ndcgAt", int(k)) + + def _test(): import doctest from pyspark import SparkContext From 4f8a15519267ac205424270155254382cc2d3690 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Mon, 11 May 2015 09:23:47 -0700 Subject: [PATCH 06/59] [SPARK-7522] [EXAMPLES] Removed angle brackets from dataFormat option As is, to specify this option on command line, you have to escape the angle brackets. Author: Bryan Cutler Closes #6049 from BryanCutler/dataFormat-option-7522 and squashes the following commits: b34afb4 [Bryan Cutler] [SPARK-7522] Removed angle brackets from dataFormat option --- .../org/apache/spark/examples/ml/DecisionTreeExample.scala | 2 +- .../main/scala/org/apache/spark/examples/ml/GBTExample.scala | 2 +- .../org/apache/spark/examples/ml/RandomForestExample.scala | 2 +- .../org/apache/spark/examples/mllib/DecisionTreeRunner.scala | 2 +- .../spark/examples/mllib/GradientBoostedTreesRunner.scala | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index 8340d91101ab3..54e4073941056 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -112,7 +112,7 @@ object DecisionTreeExample { .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) - opt[String]("") + opt[String]("dataFormat") .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") .action((x, c) => c.copy(dataFormat = x)) arg[String]("") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala index c5899b6683c79..33905277c7341 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala @@ -111,7 +111,7 @@ object GBTExample { .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) - opt[String]("") + opt[String]("dataFormat") .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") .action((x, c) => c.copy(dataFormat = x)) arg[String]("") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala index 7f88d2681bcaa..9f7cad68a4594 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala @@ -117,7 +117,7 @@ object RandomForestExample { .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) - opt[String]("") + opt[String]("dataFormat") .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") .action((x, c) => c.copy(dataFormat = x)) arg[String]("") diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 262fd2c9611d0..b0613632c9946 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -126,7 +126,7 @@ object DecisionTreeRunner { .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) - opt[String]("") + opt[String]("dataFormat") .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") .action((x, c) => c.copy(dataFormat = x)) arg[String]("") diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index 0763a7736305a..7416fb5a40848 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -69,7 +69,7 @@ object GradientBoostedTreesRunner { .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) - opt[String]("") + opt[String]("dataFormat") .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") .action((x, c) => c.copy(dataFormat = x)) arg[String]("") From 1b46556999ca126cb593ef052d24afcb75383223 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 11 May 2015 10:58:56 -0700 Subject: [PATCH 07/59] [SPARK-7361] [STREAMING] Throw unambiguous exception when attempting to start multiple StreamingContexts in the same JVM Currently attempt to start a streamingContext while another one is started throws a confusing exception that the action name JobScheduler is already registered. Instead its best to throw a proper exception as it is not supported. Author: Tathagata Das Closes #5907 from tdas/SPARK-7361 and squashes the following commits: fb81c4a [Tathagata Das] Fix typo a9cd5bb [Tathagata Das] Added startSite to StreamingContext 5fdfc0d [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-7361 5870e2b [Tathagata Das] Added check for multiple streaming contexts --- .../spark/streaming/StreamingContext.scala | 48 +++++++++++++++---- .../streaming/StreamingContextSuite.scala | 18 +++++++ 2 files changed, 58 insertions(+), 8 deletions(-) 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 bbdb4e8af036c..5abe1367752d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming import java.io.InputStream -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import scala.collection.Map import scala.collection.mutable.Queue @@ -28,8 +28,9 @@ import akka.actor.{Props, SupervisorStrategy} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} -import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} + import org.apache.spark._ import org.apache.spark.annotation.Experimental import org.apache.spark.input.FixedLengthBinaryInputFormat @@ -37,8 +38,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} -import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener} import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} +import org.apache.spark.util.CallSite /** * Main entry point for Spark Streaming functionality. It provides methods used to create @@ -202,6 +204,8 @@ class StreamingContext private[streaming] ( import StreamingContextState._ private[streaming] var state = Initialized + private val startSite = new AtomicReference[CallSite](null) + /** * Return the associated Spark context */ @@ -518,6 +522,7 @@ class StreamingContext private[streaming] ( * @throws SparkException if the context has already been started or stopped. */ def start(): Unit = synchronized { + import StreamingContext._ if (state == Started) { throw new SparkException("StreamingContext has already been started") } @@ -525,10 +530,15 @@ class StreamingContext private[streaming] ( throw new SparkException("StreamingContext has already been stopped") } validate() - sparkContext.setCallSite(DStream.getCreationSite()) - scheduler.start() - uiTab.foreach(_.attach()) - state = Started + startSite.set(DStream.getCreationSite()) + sparkContext.setCallSite(startSite.get) + ACTIVATION_LOCK.synchronized { + assertNoOtherContextIsActive() + scheduler.start() + uiTab.foreach(_.attach()) + state = Started + setActiveContext(this) + } } /** @@ -603,6 +613,7 @@ class StreamingContext private[streaming] ( uiTab.foreach(_.detach()) // The state should always be Stopped after calling `stop()`, even if we haven't started yet: state = Stopped + StreamingContext.setActiveContext(null) } } @@ -612,8 +623,29 @@ class StreamingContext private[streaming] ( */ object StreamingContext extends Logging { + /** + * Lock that guards access to global variables that track active StreamingContext. + */ + private val ACTIVATION_LOCK = new Object() - private[streaming] val DEFAULT_CLEANER_TTL = 3600 + private val activeContext = new AtomicReference[StreamingContext](null) + + private def assertNoOtherContextIsActive(): Unit = { + ACTIVATION_LOCK.synchronized { + if (activeContext.get() != null) { + throw new SparkException( + "Only one StreamingContext may be started in this JVM. " + + "Currently running StreamingContext was started at" + + activeContext.get.startSite.get.longForm) + } + } + } + + private def setActiveContext(ssc: StreamingContext): Unit = { + ACTIVATION_LOCK.synchronized { + activeContext.set(ssc) + } + } @deprecated("Replaced by implicit functions in the DStream companion object. This is " + "kept here only for backward compatibility.", "1.3.0") 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 a589deb1fa579..11c7fd835bfcd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -480,6 +480,24 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } + test("multiple streaming contexts") { + sc = new SparkContext(new SparkConf().setMaster(master).setAppName(appName)) + ssc = new StreamingContext(sc, Seconds(1)) + val input = addInputStream(ssc) + input.foreachRDD { rdd => rdd.count } + ssc.start() + + // Creating another streaming context should not create errors + val anotherSsc = new StreamingContext(sc, Seconds(10)) + val anotherInput = addInputStream(anotherSsc) + anotherInput.foreachRDD { rdd => rdd.count } + + val exception = intercept[SparkException] { + anotherSsc.start() + } + assert(exception.getMessage.contains("StreamingContext"), "Did not get the right exception") + } + test("DStream and generated RDD creation sites") { testPackage.test() } From 0a4844f90a712e796c9404b422cea76d21a5d2e3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 May 2015 11:35:16 -0700 Subject: [PATCH 08/59] [SPARK-7462] By default retain group by columns in aggregate Updated Java, Scala, Python, and R. Author: Reynold Xin Author: Shivaram Venkataraman Closes #5996 from rxin/groupby-retain and squashes the following commits: aac7119 [Reynold Xin] Merge branch 'groupby-retain' of github.com:rxin/spark into groupby-retain f6858f6 [Reynold Xin] Merge branch 'master' into groupby-retain 5f923c0 [Reynold Xin] Merge pull request #15 from shivaram/sparkr-groupby-retrain c1de670 [Shivaram Venkataraman] Revert workaround in SparkR to retain grouped cols Based on reverting code added in commit https://github.com/amplab-extras/spark/commit/9a6be746efc9fafad88122fa2267862ef87aa0e1 b8b87e1 [Reynold Xin] Fixed DataFrameJoinSuite. d910141 [Reynold Xin] Updated rest of the files 1e6e666 [Reynold Xin] [SPARK-7462] By default retain group by columns in aggregate --- R/pkg/R/group.R | 4 +- python/pyspark/sql/dataframe.py | 2 +- .../org/apache/spark/sql/GroupedData.scala | 15 +- .../scala/org/apache/spark/sql/SQLConf.scala | 6 + .../org/apache/spark/sql/api/r/SQLUtils.scala | 11 - .../sql/execution/stat/StatFunctions.scala | 2 +- .../spark/sql/DataFrameAggregateSuite.scala | 193 ++++++++++++++++++ .../apache/spark/sql/DataFrameJoinSuite.scala | 4 +- .../org/apache/spark/sql/DataFrameSuite.scala | 151 +------------- .../scala/org/apache/spark/sql/TestData.scala | 2 - 10 files changed, 218 insertions(+), 172 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 5a7a8a2caba13..b758481997574 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -102,9 +102,7 @@ setMethod("agg", } } jcols <- lapply(cols, function(c) { c@jc }) - # the GroupedData.agg(col, cols*) API does not contain grouping Column - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "aggWithGrouping", - x@sgd, listToSeq(jcols)) + sdf <- callJMethod(x@sgd, "agg", jcols[[1]], listToSeq(jcols[-1])) } else { stop("agg can only support Column or character") } diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index a9697999e82cb..c2fa6c87384a6 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1069,7 +1069,7 @@ def agg(self, *exprs): >>> from pyspark.sql import functions as F >>> gdf.agg(F.min(df.age)).collect() - [Row(MIN(age)=2), Row(MIN(age)=5)] + [Row(name=u'Alice', MIN(age)=2), Row(name=u'Bob', MIN(age)=5)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): 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 53ad67372e024..003a620dcc8ce 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 @@ -135,8 +135,9 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) } /** - * Compute aggregates by specifying a series of aggregate columns. Unlike other methods in this - * class, the resulting [[DataFrame]] won't automatically include the grouping columns. + * Compute aggregates by specifying a series of aggregate columns. Note that this function by + * default retains the grouping columns in its output. To not retain grouping columns, set + * `spark.sql.retainGroupColumns` to false. * * The available aggregate methods are defined in [[org.apache.spark.sql.functions]]. * @@ -158,7 +159,15 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.prettyString)() } - DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) + if (df.sqlContext.conf.dataFrameRetainGroupColumns) { + val retainedExprs = groupingExprs.map { + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.prettyString)() + } + DataFrame(df.sqlContext, Aggregate(groupingExprs, retainedExprs ++ aggExprs, df.logicalPlan)) + } else { + DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 98a75bb4ed2df..dcac97beafb04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -74,6 +74,9 @@ private[spark] object SQLConf { // See SPARK-6231. val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = "spark.sql.selfJoinAutoResolveAmbiguity" + // Whether to retain group by columns or not in GroupedData.agg. + val DATAFRAME_RETAIN_GROUP_COLUMNS = "spark.sql.retainGroupColumns" + val USE_SQL_SERIALIZER2 = "spark.sql.useSerializer2" val USE_JACKSON_STREAMING_API = "spark.sql.json.useJacksonStreamingAPI" @@ -242,6 +245,9 @@ private[sql] class SQLConf extends Serializable with CatalystConf { private[spark] def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY, "true").toBoolean + + private[spark] def dataFrameRetainGroupColumns: Boolean = + getConf(DATAFRAME_RETAIN_GROUP_COLUMNS, "true").toBoolean /** ********************** SQLConf functionality methods ************ */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index ae77f72998a22..423ecdff5804a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -72,17 +72,6 @@ private[r] object SQLUtils { sqlContext.createDataFrame(rowRDD, schema) } - // A helper to include grouping columns in Agg() - def aggWithGrouping(gd: GroupedData, exprs: Column*): DataFrame = { - val aggExprs = exprs.map { col => - col.expr match { - case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.simpleString)() - } - } - gd.toDF(aggExprs) - } - def dfToRowRDD(df: DataFrame): JavaRDD[Array[Byte]] = { df.map(r => rowToRBytes(r)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index 71b7f6c2a6756..d22f5fd2d439c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -104,7 +104,7 @@ private[sql] object StatFunctions extends Logging { /** Generate a table of frequencies for the elements of two columns. */ private[sql] def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = { val tableName = s"${col1}_$col2" - val counts = df.groupBy(col1, col2).agg(col(col1), col(col2), count("*")).take(1e6.toInt) + val counts = df.groupBy(col1, col2).agg(count("*")).take(1e6.toInt) if (counts.length == 1e6.toInt) { logWarning("The maximum limit of 1e6 pairs have been collected, which may not be all of " + "the pairs. Please try reducing the amount of distinct items in your columns.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala new file mode 100644 index 0000000000000..35a574f354741 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -0,0 +1,193 @@ +/* + * 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 + +import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.types.DecimalType + + +class DataFrameAggregateSuite extends QueryTest { + + test("groupBy") { + checkAnswer( + testData2.groupBy("a").agg(sum($"b")), + Seq(Row(1, 3), Row(2, 3), Row(3, 3)) + ) + checkAnswer( + testData2.groupBy("a").agg(sum($"b").as("totB")).agg(sum('totB)), + Row(9) + ) + checkAnswer( + testData2.groupBy("a").agg(count("*")), + Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil + ) + checkAnswer( + testData2.groupBy("a").agg(Map("*" -> "count")), + Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil + ) + checkAnswer( + testData2.groupBy("a").agg(Map("b" -> "sum")), + Row(1, 3) :: Row(2, 3) :: Row(3, 3) :: Nil + ) + + val df1 = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")) + .toDF("key", "value1", "value2", "rest") + + checkAnswer( + df1.groupBy("key").min(), + df1.groupBy("key").min("value1", "value2").collect() + ) + checkAnswer( + df1.groupBy("key").min("value2"), + Seq(Row("a", 0), Row("b", 4)) + ) + } + + test("spark.sql.retainGroupColumns config") { + checkAnswer( + testData2.groupBy("a").agg(sum($"b")), + Seq(Row(1, 3), Row(2, 3), Row(3, 3)) + ) + + TestSQLContext.conf.setConf("spark.sql.retainGroupColumns", "false") + checkAnswer( + testData2.groupBy("a").agg(sum($"b")), + Seq(Row(3), Row(3), Row(3)) + ) + TestSQLContext.conf.setConf("spark.sql.retainGroupColumns", "true") + } + + test("agg without groups") { + checkAnswer( + testData2.agg(sum('b)), + Row(9) + ) + } + + test("average") { + checkAnswer( + testData2.agg(avg('a)), + Row(2.0)) + + // Also check mean + checkAnswer( + testData2.agg(mean('a)), + Row(2.0)) + + checkAnswer( + testData2.agg(avg('a), sumDistinct('a)), // non-partial + Row(2.0, 6.0) :: Nil) + + checkAnswer( + decimalData.agg(avg('a)), + Row(new java.math.BigDecimal(2.0))) + checkAnswer( + decimalData.agg(avg('a), sumDistinct('a)), // non-partial + Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) + + checkAnswer( + decimalData.agg(avg('a cast DecimalType(10, 2))), + Row(new java.math.BigDecimal(2.0))) + // non-partial + checkAnswer( + decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), + Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) + } + + test("null average") { + checkAnswer( + testData3.agg(avg('b)), + Row(2.0)) + + checkAnswer( + testData3.agg(avg('b), countDistinct('b)), + Row(2.0, 1)) + + checkAnswer( + testData3.agg(avg('b), sumDistinct('b)), // non-partial + Row(2.0, 2.0)) + } + + test("zero average") { + val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") + checkAnswer( + emptyTableData.agg(avg('a)), + Row(null)) + + checkAnswer( + emptyTableData.agg(avg('a), sumDistinct('b)), // non-partial + Row(null, null)) + } + + test("count") { + assert(testData2.count() === testData2.map(_ => 1).count()) + + checkAnswer( + testData2.agg(count('a), sumDistinct('a)), // non-partial + Row(6, 6.0)) + } + + test("null count") { + checkAnswer( + testData3.groupBy('a).agg(count('b)), + Seq(Row(1,0), Row(2, 1)) + ) + + checkAnswer( + testData3.groupBy('a).agg(count('a + 'b)), + Seq(Row(1,0), Row(2, 1)) + ) + + checkAnswer( + testData3.agg(count('a), count('b), count(lit(1)), countDistinct('a), countDistinct('b)), + Row(2, 1, 2, 2, 1) + ) + + checkAnswer( + testData3.agg(count('b), countDistinct('b), sumDistinct('b)), // non-partial + Row(1, 1, 2) + ) + } + + test("zero count") { + val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") + assert(emptyTableData.count() === 0) + + checkAnswer( + emptyTableData.agg(count('a), sumDistinct('a)), // non-partial + Row(0, null)) + } + + test("zero sum") { + val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") + checkAnswer( + emptyTableData.agg(sum('a)), + Row(null)) + } + + test("zero sum distinct") { + val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") + checkAnswer( + emptyTableData.agg(sumDistinct('a)), + Row(null)) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index f005f55b6432e..787f3f175fea2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -77,8 +77,8 @@ class DataFrameJoinSuite extends QueryTest { df.join(df, df("key") === df("key") && df("value") === 1), Row(1, "1", 1, "1") :: Nil) - val left = df.groupBy("key").agg($"key", count("*")) - val right = df.groupBy("key").agg($"key", sum("key")) + val left = df.groupBy("key").agg(count("*")) + val right = df.groupBy("key").agg(sum("key")) checkAnswer( left.join(right, left("key") === right("key")), Row(1, 1, 1, 1) :: Row(2, 1, 2, 2) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index cf590cbd5219c..7552c1288165f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -22,7 +22,6 @@ import scala.language.postfixOps import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint, TestSQLContext} -import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -63,7 +62,7 @@ class DataFrameSuite extends QueryTest { val df = Seq((1,(1,1))).toDF() checkAnswer( - df.groupBy("_1").agg(col("_1"), sum("_2._1")).toDF("key", "total"), + df.groupBy("_1").agg(sum("_2._1")).toDF("key", "total"), Row(1, 1) :: Nil) } @@ -128,7 +127,7 @@ class DataFrameSuite extends QueryTest { df2 .select('_1 as 'letter, 'number) .groupBy('letter) - .agg('letter, countDistinct('number)), + .agg(countDistinct('number)), Row("a", 3) :: Row("b", 2) :: Row("c", 1) :: Nil ) } @@ -165,48 +164,6 @@ class DataFrameSuite extends QueryTest { testData.select('key).collect().toSeq) } - test("groupBy") { - checkAnswer( - testData2.groupBy("a").agg($"a", sum($"b")), - Seq(Row(1, 3), Row(2, 3), Row(3, 3)) - ) - checkAnswer( - testData2.groupBy("a").agg($"a", sum($"b").as("totB")).agg(sum('totB)), - Row(9) - ) - checkAnswer( - testData2.groupBy("a").agg(col("a"), count("*")), - Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil - ) - checkAnswer( - testData2.groupBy("a").agg(Map("*" -> "count")), - Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil - ) - checkAnswer( - testData2.groupBy("a").agg(Map("b" -> "sum")), - Row(1, 3) :: Row(2, 3) :: Row(3, 3) :: Nil - ) - - val df1 = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")) - .toDF("key", "value1", "value2", "rest") - - checkAnswer( - df1.groupBy("key").min(), - df1.groupBy("key").min("value1", "value2").collect() - ) - checkAnswer( - df1.groupBy("key").min("value2"), - Seq(Row("a", 0), Row("b", 4)) - ) - } - - test("agg without groups") { - checkAnswer( - testData2.agg(sum('b)), - Row(9) - ) - } - test("convert $\"attribute name\" into unresolved attribute") { checkAnswer( testData.where($"key" === lit(1)).select($"value"), @@ -303,110 +260,6 @@ class DataFrameSuite extends QueryTest { mapData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq))) } - test("average") { - checkAnswer( - testData2.agg(avg('a)), - Row(2.0)) - - // Also check mean - checkAnswer( - testData2.agg(mean('a)), - Row(2.0)) - - checkAnswer( - testData2.agg(avg('a), sumDistinct('a)), // non-partial - Row(2.0, 6.0) :: Nil) - - checkAnswer( - decimalData.agg(avg('a)), - Row(new java.math.BigDecimal(2.0))) - checkAnswer( - decimalData.agg(avg('a), sumDistinct('a)), // non-partial - Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) - - checkAnswer( - decimalData.agg(avg('a cast DecimalType(10, 2))), - Row(new java.math.BigDecimal(2.0))) - // non-partial - checkAnswer( - decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), - Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) - } - - test("null average") { - checkAnswer( - testData3.agg(avg('b)), - Row(2.0)) - - checkAnswer( - testData3.agg(avg('b), countDistinct('b)), - Row(2.0, 1)) - - checkAnswer( - testData3.agg(avg('b), sumDistinct('b)), // non-partial - Row(2.0, 2.0)) - } - - test("zero average") { - checkAnswer( - emptyTableData.agg(avg('a)), - Row(null)) - - checkAnswer( - emptyTableData.agg(avg('a), sumDistinct('b)), // non-partial - Row(null, null)) - } - - test("count") { - assert(testData2.count() === testData2.map(_ => 1).count()) - - checkAnswer( - testData2.agg(count('a), sumDistinct('a)), // non-partial - Row(6, 6.0)) - } - - test("null count") { - checkAnswer( - testData3.groupBy('a).agg('a, count('b)), - Seq(Row(1,0), Row(2, 1)) - ) - - checkAnswer( - testData3.groupBy('a).agg('a, count('a + 'b)), - Seq(Row(1,0), Row(2, 1)) - ) - - checkAnswer( - testData3.agg(count('a), count('b), count(lit(1)), countDistinct('a), countDistinct('b)), - Row(2, 1, 2, 2, 1) - ) - - checkAnswer( - testData3.agg(count('b), countDistinct('b), sumDistinct('b)), // non-partial - Row(1, 1, 2) - ) - } - - test("zero count") { - assert(emptyTableData.count() === 0) - - checkAnswer( - emptyTableData.agg(count('a), sumDistinct('a)), // non-partial - Row(0, null)) - } - - test("zero sum") { - checkAnswer( - emptyTableData.agg(sum('a)), - Row(null)) - } - - test("zero sum distinct") { - checkAnswer( - emptyTableData.agg(sumDistinct('a)), - Row(null)) - } - test("except") { checkAnswer( lowerCaseData.except(upperCaseData), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 225b51bd73d6c..446771ab2a5a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -86,8 +86,6 @@ object TestData { TestData3(2, Some(2)) :: Nil).toDF() testData3.registerTempTable("testData3") - val emptyTableData = logical.LocalRelation($"a".int, $"b".int) - case class UpperCaseData(N: Int, L: String) val upperCaseData = TestSQLContext.sparkContext.parallelize( From 82fee9d9aad2c9ba2fb4bd658579fe99218cafac Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 11 May 2015 12:09:39 -0700 Subject: [PATCH 09/59] [SPARK-6470] [YARN] Add support for YARN node labels. This is difficult to write a test for because it relies on the latest version of YARN, but I verified manually that the patch does pass along the label expression on this version and containers are successfully launched. Author: Sandy Ryza Closes #5242 from sryza/sandy-spark-6470 and squashes the following commits: 6af87b9 [Sandy Ryza] Change info to warning 6e22d99 [Sandy Ryza] [YARN] SPARK-6470. Add support for YARN node labels. --- docs/running-on-yarn.md | 9 ++++++ .../spark/deploy/yarn/YarnAllocator.scala | 31 ++++++++++++++++++- 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 4fb4a90307ec8..51c1339165024 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -220,6 +220,15 @@ Most of the configs are the same for Spark on YARN as for other deployment modes Otherwise, the client process will exit after submission. + + spark.yarn.executor.nodeLabelExpression + (none) + + A YARN node label expression that restricts the set of nodes executors will be scheduled on. + Only versions of YARN greater than or equal to 2.6 support node label expressions, so when + running against earlier versions, this property will be ignored. + + # Launching Spark on YARN 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 88d68d5556162..8a08f561a2df2 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 @@ -117,6 +117,24 @@ private[yarn] class YarnAllocator( // For testing private val launchContainers = sparkConf.getBoolean("spark.yarn.launchContainers", true) + private val labelExpression = sparkConf.getOption("spark.yarn.executor.nodeLabelExpression") + + // ContainerRequest constructor that can take a node label expression. We grab it through + // reflection because it's only available in later versions of YARN. + private val nodeLabelConstructor = labelExpression.flatMap { expr => + try { + Some(classOf[ContainerRequest].getConstructor(classOf[Resource], + classOf[Array[String]], classOf[Array[String]], classOf[Priority], classOf[Boolean], + classOf[String])) + } catch { + case e: NoSuchMethodException => { + logWarning(s"Node label expression $expr will be ignored because YARN version on" + + " classpath does not support it.") + None + } + } + } + def getNumExecutorsRunning: Int = numExecutorsRunning def getNumExecutorsFailed: Int = numExecutorsFailed @@ -211,7 +229,7 @@ private[yarn] class YarnAllocator( s"cores and ${resource.getMemory} MB memory including $memoryOverhead MB overhead") for (i <- 0 until missing) { - val request = new ContainerRequest(resource, null, null, RM_REQUEST_PRIORITY) + val request = createContainerRequest(resource) amClient.addContainerRequest(request) val nodes = request.getNodes val hostStr = if (nodes == null || nodes.isEmpty) "Any" else nodes.last @@ -230,6 +248,17 @@ private[yarn] class YarnAllocator( } } + /** + * Creates a container request, handling the reflection required to use YARN features that were + * added in recent versions. + */ + private def createContainerRequest(resource: Resource): ContainerRequest = { + nodeLabelConstructor.map { constructor => + constructor.newInstance(resource, null, null, RM_REQUEST_PRIORITY, true: java.lang.Boolean, + labelExpression.orNull) + }.getOrElse(new ContainerRequest(resource, null, null, RM_REQUEST_PRIORITY)) + } + /** * Handle containers granted by the RM by launching executors on them. * From 7ce2a33c3acffa17ae32f48ebb40b69b9b36dae4 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 11 May 2015 13:35:06 -0700 Subject: [PATCH 10/59] [SPARK-7508] JettyUtils-generated servlets to log & report all errors MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Patch for SPARK-7508 This logs warn then generates a response which include the message body and stack trace as text/plain, no-cache. The exit code is 500. In practise (in some tests in SPARK-1537 to be precise), jetty is getting in between this servlet and the web response the user sees —the body of the response is lost for any error response (500, even 404 and bad request). The standard Jetty handlers must be getting in the way. This patch doesn't address that, it ensures that 1. if the jetty handlers were put to one side the users would see the errors 2. at least the exceptions appear in the server-side logs. This is better to users saying "I saw a 500 error" and you not having anything in the logs to see what went wrong. Author: Steve Loughran Closes #6033 from steveloughran/stevel/feature/SPARK-7508-JettyUtils and squashes the following commits: 584836f [Steve Loughran] SPARK-7508 drop trailing semicolon ad6f185 [Steve Loughran] SPARK-7508: jetty handles exception reporting itself; spark just sets this up and logs exceptions before being relayed 258d9f9 [Steve Loughran] SPARK-7508 fix typo manually-edited before patch pushed 69c8263 [Steve Loughran] SPARK-7508 JettyUtils-generated servlets to log & report all errors --- core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index dfd6fdb5e9993..06e616220c706 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -78,6 +78,9 @@ private[spark] object JettyUtils extends Logging { } catch { case e: IllegalArgumentException => response.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage) + case e: Exception => + logWarning(s"GET ${request.getRequestURI} failed: $e", e) + throw e } } // SPARK-5983 ensure TRACE is not supported @@ -217,6 +220,9 @@ private[spark] object JettyUtils extends Logging { val pool = new QueuedThreadPool pool.setDaemon(true) server.setThreadPool(pool) + val errorHandler = new ErrorHandler() + errorHandler.setShowStacks(true) + server.addBean(errorHandler) server.setHandler(collection) try { server.start() From 6e9910c21ada19ae14122b5f14a6a30845b98229 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 11 May 2015 14:19:11 -0700 Subject: [PATCH 11/59] [SPARK-7515] [DOC] Update documentation for PySpark on YARN with cluster mode Now PySpark on YARN with cluster mode is supported so let's update doc. Author: Kousuke Saruta Closes #6040 from sarutak/update-doc-for-pyspark-on-yarn and squashes the following commits: ad9f88c [Kousuke Saruta] Brushed up sentences 469fd2e [Kousuke Saruta] Merge branch 'master' of https://github.com/apache/spark into update-doc-for-pyspark-on-yarn fcfdb92 [Kousuke Saruta] Updated doc for PySpark on YARN with cluster mode --- docs/submitting-applications.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 3ecbf2308cd44..e58645274e525 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -59,7 +59,7 @@ for applications that involve the REPL (e.g. Spark shell). Alternatively, if your application is submitted from a machine far from the worker machines (e.g. locally on your laptop), it is common to use `cluster` mode to minimize network latency between the drivers and the executors. Note that `cluster` mode is currently not supported for -Mesos clusters or Python applications. +Mesos clusters. Currently only YARN supports cluster mode for Python applications. For Python applications, simply pass a `.py` file in the place of `` instead of a JAR, and add Python `.zip`, `.egg` or `.py` files to the search path with `--py-files`. From 8e674331d9ce98068b44e4d483b6d35cef0648fa Mon Sep 17 00:00:00 2001 From: gchen Date: Mon, 11 May 2015 14:37:18 -0700 Subject: [PATCH 12/59] [SPARK-7516] [Minor] [DOC] Replace depreciated inferSchema() with createDataFrame() JIRA: https://issues.apache.org/jira/browse/SPARK-7516 In sql-programming-guide, deprecated python data frame api inferSchema() should be replaced by createDataFrame(): schemaPeople = sqlContext.inferSchema(people) -> schemaPeople = sqlContext.createDataFrame(people) Author: gchen Closes #6041 from gchen/python-docs and squashes the following commits: c27eb7c [gchen] replace inferSchema() with createDataFrame() --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index df4c123bdd86c..6af10432b9d03 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -470,7 +470,7 @@ parts = lines.map(lambda l: l.split(",")) people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) # Infer the schema, and register the DataFrame as a table. -schemaPeople = sqlContext.inferSchema(people) +schemaPeople = sqlContext.createDataFrame(people) schemaPeople.registerTempTable("people") # SQL can be run over DataFrames that have been registered as a table. From 25c01c54840a9ab768f8b917de7edc2bc2d61b9e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 11 May 2015 14:38:58 -0700 Subject: [PATCH 13/59] [STREAMING] [MINOR] Close files correctly when iterator is finished in streaming WAL recovery Currently there's no chance to close the file correctly after the iteration is finished, change to `CompletionIterator` to avoid resource leakage. Author: jerryshao Closes #6050 from jerryshao/close-file-correctly and squashes the following commits: 52dfaf5 [jerryshao] Close files correctly when iterator is finished --- .../apache/spark/streaming/util/FileBasedWriteAheadLog.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index 9985fedc35141..87ba4f84a9ceb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -26,7 +26,7 @@ import scala.language.postfixOps import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{CompletionIterator, ThreadUtils} import org.apache.spark.{Logging, SparkConf} /** @@ -124,7 +124,8 @@ private[streaming] class FileBasedWriteAheadLog( logFilesToRead.iterator.map { file => logDebug(s"Creating log reader with $file") - new FileBasedWriteAheadLogReader(file, hadoopConf) + val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) + CompletionIterator[ByteBuffer, Iterator[ByteBuffer]](reader, reader.close _) } flatMap { x => x } } From a8ea09683acc071cd81b244e8d2b7d9638b1aced Mon Sep 17 00:00:00 2001 From: LCY Vincent Date: Mon, 11 May 2015 14:48:10 -0700 Subject: [PATCH 14/59] Update Documentation: leftsemi instead of semijoin should sync up with here? https://github.com/apache/spark/blob/119f45d61d7b48d376cca05e1b4f0c7fcf65bfa8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala#L26 Author: LCY Vincent Closes #5944 from vincentlaucy/master and squashes the following commits: fc0e454 [LCY Vincent] Update DataFrame.scala --- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 7947042c14299..c92ca607fbd54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -433,7 +433,7 @@ class DataFrame private[sql]( * * @param right Right side of the join. * @param joinExprs Join expression. - * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. + * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `leftsemi`. * @group dfops */ def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { From 91dc3dfd77126afbb894a783b7201b87b59f73cb Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 12 May 2015 00:25:43 +0100 Subject: [PATCH 15/59] [MINOR] [DOCS] Fix the link to test building info on the wiki Author: Sean Owen Closes #6063 from srowen/FixRunningTestsLink and squashes the following commits: db62018 [Sean Owen] Fix the link to test building info on the wiki --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c3afc4db9c63c..9c09d40e2bdae 100644 --- a/README.md +++ b/README.md @@ -76,7 +76,7 @@ can be run using: ./dev/run-tests Please see the guidance on how to -[run all automated tests](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting). +[run tests for a module, or individual tests](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools). ## A Note About Hadoop Versions From 4f4dbb030c208caba18f314a1ef1751696627d26 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 May 2015 17:02:11 -0700 Subject: [PATCH 16/59] [SQL] Show better error messages for incorrect join types in DataFrames. As a follow-up to https://github.com/apache/spark/pull/5944 Author: Reynold Xin Closes #6064 from rxin/jointype-better-error and squashes the following commits: 7629bf7 [Reynold Xin] [SQL] Show better error messages for incorrect join types in DataFrames. --- .../apache/spark/sql/catalyst/plans/joinTypes.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 5dc0539caec24..77dec7ca6e2b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -24,6 +24,16 @@ object JoinType { case "leftouter" | "left" => LeftOuter case "rightouter" | "right" => RightOuter case "leftsemi" => LeftSemi + case _ => + val supported = Seq( + "inner", + "outer", "full", "fullouter", + "leftouter", "left", + "rightouter", "right", + "leftsemi") + + throw new IllegalArgumentException(s"Unsupported join type '$typ'. " + + "Supported join types include: " + supported.mkString("'", "', '", "'") + ".") } } From 57255dcd794222f4db5df1e549ebc7b896cebfdc Mon Sep 17 00:00:00 2001 From: madhukar Date: Mon, 11 May 2015 17:04:11 -0700 Subject: [PATCH 17/59] [SPARK-7084] improve saveAsTable documentation Author: madhukar Closes #5654 from phatak-dev/master and squashes the following commits: 386f407 [madhukar] #5654 updated for all the methods 2c997c5 [madhukar] Merge branch 'master' of https://github.com/apache/spark 00bc819 [madhukar] Merge branch 'master' of https://github.com/apache/spark 2a802c6 [madhukar] #5654 updated the doc according to comments 866e8df [madhukar] [SPARK-7084] improve saveAsTable documentation --- .../scala/org/apache/spark/sql/DataFrame.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index c92ca607fbd54..729cfc1da25b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1192,6 +1192,9 @@ class DataFrame private[sql]( * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * + * Also note that while this function can persist the table metadata into Hive's metastore, + * the table will NOT be accessible from Hive. * @group output */ @Experimental @@ -1208,6 +1211,9 @@ class DataFrame private[sql]( * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * + * Also note that while this function can persist the table metadata into Hive's metastore, + * the table will NOT be accessible from Hive. * @group output */ @Experimental @@ -1232,6 +1238,9 @@ class DataFrame private[sql]( * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * + * Also note that while this function can persist the table metadata into Hive's metastore, + * the table will NOT be accessible from Hive. * @group output */ @Experimental @@ -1248,6 +1257,9 @@ class DataFrame private[sql]( * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * + * Also note that while this function can persist the table metadata into Hive's metastore, + * the table will NOT be accessible from Hive. * @group output */ @Experimental @@ -1264,6 +1276,9 @@ class DataFrame private[sql]( * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * + * Also note that while this function can persist the table metadata into Hive's metastore, + * the table will NOT be accessible from Hive. * @group output */ @Experimental @@ -1285,6 +1300,9 @@ class DataFrame private[sql]( * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * + * Also note that while this function can persist the table metadata into Hive's metastore, + * the table will NOT be accessible from Hive. * @group output */ @Experimental From 3a9b6997df3fef1052d8c410f32319018c52acff Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 May 2015 18:07:12 -0700 Subject: [PATCH 18/59] [SPARK-7462][SQL] Update documentation for retaining grouping columns in DataFrames. Author: Reynold Xin Closes #6062 from rxin/agg-retain-doc and squashes the following commits: 43e511e [Reynold Xin] [SPARK-7462][SQL] Update documentation for retaining grouping columns in DataFrames. --- docs/sql-programming-guide.md | 60 ++++++++++++++++++- python/pyspark/sql/_types.py | 2 + .../org/apache/spark/sql/GroupedData.scala | 14 ++++- 3 files changed, 73 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6af10432b9d03..6b7b867ea6b5d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1594,6 +1594,64 @@ options. # Migration Guide +## Upgrading from Spark SQL 1.3 to 1.4 + +Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. + +
    +
    +{% highlight scala %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg($"department", max("age"), sum("expense")) + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")) + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    +{% highlight java %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(col("department"), max("age"), sum("expense")); + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")); + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false"); + +{% endhighlight %} +
    + +
    +{% highlight python %} + +import pyspark.sql.functions as func + +# In 1.3.x, in order for the grouping column "department" to show up, +# it must be included explicitly as part of the agg function call. +df.groupBy("department").agg("department"), func.max("age"), func.sum("expense")) + +# In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(func.max("age"), func.sum("expense")) + +# Revert to 1.3.x behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    + + ## Upgrading from Spark SQL 1.0-1.2 to 1.3 In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the @@ -1651,7 +1709,7 @@ moved into the udf object in `SQLContext`.
    -{% highlight java %} +{% highlight scala %} sqlContext.udf.register("strLen", (s: String) => s.length()) diff --git a/python/pyspark/sql/_types.py b/python/pyspark/sql/_types.py index fd98e116d2cf1..b96851a174d49 100644 --- a/python/pyspark/sql/_types.py +++ b/python/pyspark/sql/_types.py @@ -1228,12 +1228,14 @@ def __getattr__(self, item): raise AttributeError(item) def __reduce__(self): + """Returns a tuple so Python knows how to pickle Row.""" if hasattr(self, "__fields__"): return (_create_row, (self.__fields__, tuple(self))) else: return tuple.__reduce__(self) def __repr__(self): + """Printable representation of Row used in Python REPL.""" if hasattr(self, "__fields__"): return "Row(%s)" % ", ".join("%s=%r" % (k, v) for k, v in zip(self.__fields__, tuple(self))) 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 003a620dcc8ce..543320e471bf7 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 @@ -146,11 +146,21 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) * * // Scala: * import org.apache.spark.sql.functions._ - * df.groupBy("department").agg($"department", max($"age"), sum($"expense")) + * df.groupBy("department").agg(max("age"), sum("expense")) * * // Java: * import static org.apache.spark.sql.functions.*; - * df.groupBy("department").agg(col("department"), max(col("age")), sum(col("expense"))); + * df.groupBy("department").agg(max("age"), sum("expense")); + * }}} + * + * Note that before Spark 1.4, the default behavior is to NOT retain grouping columns. To change + * to that behavior, set config variable `spark.sql.retainGroupColumns` to `false`. + * {{{ + * // Scala, 1.3.x: + * df.groupBy("department").agg($"department", max("age"), sum("expense")) + * + * // Java, 1.3.x: + * df.groupBy("department").agg(col("department"), max("age"), sum("expense")); * }}} */ @scala.annotation.varargs From 87229c95c6b597f5b84e36d518b9830e3ba63424 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 May 2015 18:10:45 -0700 Subject: [PATCH 19/59] Updated DataFrame.saveAsTable Hive warning to include SPARK-7550 ticket. So users that are interested in this can track it easily. Author: Reynold Xin Closes #6067 from rxin/SPARK-7550 and squashes the following commits: ee0e34c [Reynold Xin] Updated DataFrame.saveAsTable Hive warning to include SPARK-7550 ticket. --- .../main/scala/org/apache/spark/sql/DataFrame.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 729cfc1da25b2..2472999de37f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1194,7 +1194,7 @@ class DataFrame private[sql]( * be the target of an `insertInto`. * * Also note that while this function can persist the table metadata into Hive's metastore, - * the table will NOT be accessible from Hive. + * the table will NOT be accessible from Hive, until SPARK-7550 is resolved. * @group output */ @Experimental @@ -1213,7 +1213,7 @@ class DataFrame private[sql]( * be the target of an `insertInto`. * * Also note that while this function can persist the table metadata into Hive's metastore, - * the table will NOT be accessible from Hive. + * the table will NOT be accessible from Hive, until SPARK-7550 is resolved. * @group output */ @Experimental @@ -1240,7 +1240,7 @@ class DataFrame private[sql]( * be the target of an `insertInto`. * * Also note that while this function can persist the table metadata into Hive's metastore, - * the table will NOT be accessible from Hive. + * the table will NOT be accessible from Hive, until SPARK-7550 is resolved. * @group output */ @Experimental @@ -1259,7 +1259,7 @@ class DataFrame private[sql]( * be the target of an `insertInto`. * * Also note that while this function can persist the table metadata into Hive's metastore, - * the table will NOT be accessible from Hive. + * the table will NOT be accessible from Hive, until SPARK-7550 is resolved. * @group output */ @Experimental @@ -1278,7 +1278,7 @@ class DataFrame private[sql]( * be the target of an `insertInto`. * * Also note that while this function can persist the table metadata into Hive's metastore, - * the table will NOT be accessible from Hive. + * the table will NOT be accessible from Hive, until SPARK-7550 is resolved. * @group output */ @Experimental @@ -1302,7 +1302,7 @@ class DataFrame private[sql]( * be the target of an `insertInto`. * * Also note that while this function can persist the table metadata into Hive's metastore, - * the table will NOT be accessible from Hive. + * the table will NOT be accessible from Hive, until SPARK-7550 is resolved. * @group output */ @Experimental From 35fb42a0b01d3043b7d5e27256d1b45a08583aab Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 11 May 2015 18:41:22 -0700 Subject: [PATCH 20/59] [SPARK-5893] [ML] Add bucketizer JIRA issue [here](https://issues.apache.org/jira/browse/SPARK-5893). One thing to make clear, the `buckets` parameter, which is an array of `Double`, performs as split points. Say, ```scala buckets = Array(-0.5, 0.0, 0.5) ``` splits the real number into 4 ranges, (-inf, -0.5], (-0.5, 0.0], (0.0, 0.5], (0.5, +inf), which is encoded as 0, 1, 2, 3. Author: Xusen Yin Author: Joseph K. Bradley Closes #5980 from yinxusen/SPARK-5893 and squashes the following commits: dc8c843 [Xusen Yin] Merge pull request #4 from jkbradley/yinxusen-SPARK-5893 1ca973a [Joseph K. Bradley] one more bucketizer test 34f124a [Joseph K. Bradley] Removed lowerInclusive, upperInclusive params from Bucketizer, and used splits instead. eacfcfa [Xusen Yin] change ML attribute from splits into buckets c3cc770 [Xusen Yin] add more unit test for binary search 3a16cc2 [Xusen Yin] refine comments and names ac77859 [Xusen Yin] fix style error fb30d79 [Xusen Yin] fix and test binary search 2466322 [Xusen Yin] refactor Bucketizer 11fb00a [Xusen Yin] change it into an Estimator 998bc87 [Xusen Yin] check buckets 4024cf1 [Xusen Yin] add test suite 5fe190e [Xusen Yin] add bucketizer --- .../apache/spark/ml/feature/Bucketizer.scala | 131 ++++++++++++++++ .../apache/spark/ml/util/SchemaUtils.scala | 11 ++ .../spark/ml/feature/BucketizerSuite.scala | 148 ++++++++++++++++++ 3 files changed, 290 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala new file mode 100644 index 0000000000000..7dba64bc3506f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -0,0 +1,131 @@ +/* + * 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.ml.feature + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.util.SchemaUtils +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} + +/** + * :: AlphaComponent :: + * `Bucketizer` maps a column of continuous features to a column of feature buckets. + */ +@AlphaComponent +final class Bucketizer private[ml] (override val parent: Estimator[Bucketizer]) + extends Model[Bucketizer] with HasInputCol with HasOutputCol { + + def this() = this(null) + + /** + * Parameter for mapping continuous features into buckets. With n splits, there are n+1 buckets. + * A bucket defined by splits x,y holds values in the range [x,y). Splits should be strictly + * increasing. Values at -inf, inf must be explicitly provided to cover all Double values; + * otherwise, values outside the splits specified will be treated as errors. + * @group param + */ + val splits: Param[Array[Double]] = new Param[Array[Double]](this, "splits", + "Split points for mapping continuous features into buckets. With n splits, there are n+1 " + + "buckets. A bucket defined by splits x,y holds values in the range [x,y). The splits " + + "should be strictly increasing. Values at -inf, inf must be explicitly provided to cover" + + " all Double values; otherwise, values outside the splits specified will be treated as" + + " errors.", + Bucketizer.checkSplits) + + /** @group getParam */ + def getSplits: Array[Double] = $(splits) + + /** @group setParam */ + def setSplits(value: Array[Double]): this.type = set(splits, value) + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema) + val bucketizer = udf { feature: Double => + Bucketizer.binarySearchForBuckets($(splits), feature) + } + val newCol = bucketizer(dataset($(inputCol))) + val newField = prepOutputField(dataset.schema) + dataset.withColumn($(outputCol), newCol.as($(outputCol), newField.metadata)) + } + + private def prepOutputField(schema: StructType): StructField = { + val buckets = $(splits).sliding(2).map(bucket => bucket.mkString(", ")).toArray + val attr = new NominalAttribute(name = Some($(outputCol)), isOrdinal = Some(true), + values = Some(buckets)) + attr.toStructField() + } + + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) + SchemaUtils.appendColumn(schema, prepOutputField(schema)) + } +} + +private[feature] object Bucketizer { + /** We require splits to be of length >= 3 and to be in strictly increasing order. */ + def checkSplits(splits: Array[Double]): Boolean = { + if (splits.length < 3) { + false + } else { + var i = 0 + while (i < splits.length - 1) { + if (splits(i) >= splits(i + 1)) return false + i += 1 + } + true + } + } + + /** + * Binary searching in several buckets to place each data point. + * @throws RuntimeException if a feature is < splits.head or >= splits.last + */ + def binarySearchForBuckets( + splits: Array[Double], + feature: Double): Double = { + // Check bounds. We make an exception for +inf so that it can exist in some bin. + if ((feature < splits.head) || (feature >= splits.last && feature != Double.PositiveInfinity)) { + throw new RuntimeException(s"Feature value $feature out of Bucketizer bounds" + + s" [${splits.head}, ${splits.last}). Check your features, or loosen " + + s"the lower/upper bound constraints.") + } + var left = 0 + var right = splits.length - 2 + while (left < right) { + val mid = (left + right) / 2 + val split = splits(mid + 1) + if (feature < split) { + right = mid + } else { + left = mid + 1 + } + } + left + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala index 0383bf0b382b7..11592b77eb356 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -58,4 +58,15 @@ object SchemaUtils { val outputFields = schema.fields :+ StructField(colName, dataType, nullable = false) StructType(outputFields) } + + /** + * Appends a new column to the input schema. This fails if the given output column already exists. + * @param schema input schema + * @param col New column schema + * @return new schema with the input column appended + */ + def appendColumn(schema: StructType, col: StructField): StructType = { + require(!schema.fieldNames.contains(col.name), s"Column ${col.name} already exists.") + StructType(schema.fields :+ col) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala new file mode 100644 index 0000000000000..acb46c0a35709 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala @@ -0,0 +1,148 @@ +/* + * 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.ml.feature + +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +class BucketizerSuite extends FunSuite with MLlibTestSparkContext { + + @transient private var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("Bucket continuous features, without -inf,inf") { + // Check a set of valid feature values. + val splits = Array(-0.5, 0.0, 0.5) + val validData = Array(-0.5, -0.3, 0.0, 0.2) + val expectedBuckets = Array(0.0, 0.0, 1.0, 1.0) + val dataFrame: DataFrame = + sqlContext.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") + + val bucketizer: Bucketizer = new Bucketizer() + .setInputCol("feature") + .setOutputCol("result") + .setSplits(splits) + + bucketizer.transform(dataFrame).select("result", "expected").collect().foreach { + case Row(x: Double, y: Double) => + assert(x === y, + s"The feature value is not correct after bucketing. Expected $y but found $x") + } + + // Check for exceptions when using a set of invalid feature values. + val invalidData1: Array[Double] = Array(-0.9) ++ validData + val invalidData2 = Array(0.5) ++ validData + val badDF1 = sqlContext.createDataFrame(invalidData1.zipWithIndex).toDF("feature", "idx") + intercept[RuntimeException]{ + bucketizer.transform(badDF1).collect() + println("Invalid feature value -0.9 was not caught as an invalid feature!") + } + val badDF2 = sqlContext.createDataFrame(invalidData2.zipWithIndex).toDF("feature", "idx") + intercept[RuntimeException]{ + bucketizer.transform(badDF2).collect() + println("Invalid feature value 0.5 was not caught as an invalid feature!") + } + } + + test("Bucket continuous features, with -inf,inf") { + val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) + val validData = Array(-0.9, -0.5, -0.3, 0.0, 0.2, 0.5, 0.9) + val expectedBuckets = Array(0.0, 1.0, 1.0, 2.0, 2.0, 3.0, 3.0) + val dataFrame: DataFrame = + sqlContext.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") + + val bucketizer: Bucketizer = new Bucketizer() + .setInputCol("feature") + .setOutputCol("result") + .setSplits(splits) + + bucketizer.transform(dataFrame).select("result", "expected").collect().foreach { + case Row(x: Double, y: Double) => + assert(x === y, + s"The feature value is not correct after bucketing. Expected $y but found $x") + } + } + + test("Binary search correctness on hand-picked examples") { + import BucketizerSuite.checkBinarySearch + // length 3, with -inf + checkBinarySearch(Array(Double.NegativeInfinity, 0.0, 1.0)) + // length 4 + checkBinarySearch(Array(-1.0, -0.5, 0.0, 1.0)) + // length 5 + checkBinarySearch(Array(-1.0, -0.5, 0.0, 1.0, 1.5)) + // length 3, with inf + checkBinarySearch(Array(0.0, 1.0, Double.PositiveInfinity)) + // length 3, with -inf and inf + checkBinarySearch(Array(Double.NegativeInfinity, 1.0, Double.PositiveInfinity)) + // length 4, with -inf and inf + checkBinarySearch(Array(Double.NegativeInfinity, 0.0, 1.0, Double.PositiveInfinity)) + } + + test("Binary search correctness in contrast with linear search, on random data") { + val data = Array.fill(100)(Random.nextDouble()) + val splits: Array[Double] = Double.NegativeInfinity +: + Array.fill(10)(Random.nextDouble()).sorted :+ Double.PositiveInfinity + val bsResult = Vectors.dense(data.map(x => Bucketizer.binarySearchForBuckets(splits, x))) + val lsResult = Vectors.dense(data.map(x => BucketizerSuite.linearSearchForBuckets(splits, x))) + assert(bsResult ~== lsResult absTol 1e-5) + } +} + +private object BucketizerSuite extends FunSuite { + /** Brute force search for buckets. Bucket i is defined by the range [split(i), split(i+1)). */ + def linearSearchForBuckets(splits: Array[Double], feature: Double): Double = { + require(feature >= splits.head) + var i = 0 + while (i < splits.length - 1) { + if (feature < splits(i + 1)) return i + i += 1 + } + throw new RuntimeException( + s"linearSearchForBuckets failed to find bucket for feature value $feature") + } + + /** Check all values in splits, plus values between all splits. */ + def checkBinarySearch(splits: Array[Double]): Unit = { + def testFeature(feature: Double, expectedBucket: Double): Unit = { + assert(Bucketizer.binarySearchForBuckets(splits, feature) === expectedBucket, + s"Expected feature value $feature to be in bucket $expectedBucket with splits:" + + s" ${splits.mkString(", ")}") + } + var i = 0 + while (i < splits.length - 1) { + testFeature(splits(i), i) // Split i should fall in bucket i. + testFeature((splits(i) + splits(i + 1)) / 2, i) // Value between splits i,i+1 should be in i. + i += 1 + } + if (splits.last === Double.PositiveInfinity) { + testFeature(Double.PositiveInfinity, splits.length - 2) + } + } +} From f9c7580adadce75a94bd2854cf4f743d8cbd1d23 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 11 May 2015 18:53:50 -0700 Subject: [PATCH 21/59] [SPARK-7530] [STREAMING] Added StreamingContext.getState() to expose the current state of the context Author: Tathagata Das Closes #6058 from tdas/SPARK-7530 and squashes the following commits: 80ee0e6 [Tathagata Das] STARTED --> ACTIVE 3da6547 [Tathagata Das] Added synchronized dd88444 [Tathagata Das] Added more docs e1a8505 [Tathagata Das] Fixed comment length 89f9980 [Tathagata Das] Change to Java enum and added Java test 7c57351 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-7530 dd4e702 [Tathagata Das] Addressed comments. 3d56106 [Tathagata Das] Added Mima excludes 2b86ba1 [Tathagata Das] Added scala docs. 1722433 [Tathagata Das] Fixed style 976b094 [Tathagata Das] Added license 0585130 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-7530 e0f0a05 [Tathagata Das] Added getState and exposed StreamingContextState --- project/MimaExcludes.scala | 4 + .../spark/streaming/StreamingContext.scala | 75 +++++++++++-------- .../streaming/StreamingContextState.java | 45 +++++++++++ .../api/java/JavaStreamingContext.scala | 22 ++++++ .../apache/spark/streaming/JavaAPISuite.java | 14 ++++ .../spark/streaming/JavaTestUtils.scala | 1 + .../streaming/StreamingContextSuite.scala | 19 ++++- 7 files changed, 147 insertions(+), 33 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index cfe387faec14b..ad3d8426bdedd 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -106,6 +106,10 @@ object MimaExcludes { "org.apache.spark.sql.parquet.ParquetTestData$"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.parquet.TestGroupWriteSupport") + ) ++ Seq( + // SPARK-7530 Added StreamingContext.getState() + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.streaming.StreamingContext.state_=") ) case v if v.startsWith("1.3") => 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 5abe1367752d9..2c5834defa8c8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -32,10 +32,11 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.input.FixedLengthBinaryInputFormat import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContextState._ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener} @@ -195,14 +196,7 @@ class StreamingContext private[streaming] ( assert(env.metricsSystem != null) env.metricsSystem.registerSource(streamingSource) - /** Enumeration to identify current state of the StreamingContext */ - private[streaming] object StreamingContextState extends Enumeration { - type CheckpointState = Value - val Initialized, Started, Stopped = Value - } - - import StreamingContextState._ - private[streaming] var state = Initialized + private var state: StreamingContextState = INITIALIZED private val startSite = new AtomicReference[CallSite](null) @@ -516,6 +510,21 @@ class StreamingContext private[streaming] ( ) } + /** + * :: DeveloperApi :: + * + * Return the current state of the context. The context can be in three possible states - + * - StreamingContextState.INTIALIZED - The context has been created, but not been started yet. + * Input DStreams, transformations and output operations can be created on the context. + * - StreamingContextState.ACTIVE - The context has been started, and been not stopped. + * Input DStreams, transformations and output operations cannot be created on the context. + * - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more. + */ + @DeveloperApi + def getState(): StreamingContextState = synchronized { + state + } + /** * Start the execution of the streams. * @@ -523,11 +532,13 @@ class StreamingContext private[streaming] ( */ def start(): Unit = synchronized { import StreamingContext._ - if (state == Started) { - throw new SparkException("StreamingContext has already been started") - } - if (state == Stopped) { - throw new SparkException("StreamingContext has already been stopped") + state match { + case INITIALIZED => + // good to start + case ACTIVE => + throw new SparkException("StreamingContext has already been started") + case STOPPED => + throw new SparkException("StreamingContext has already been stopped") } validate() startSite.set(DStream.getCreationSite()) @@ -536,7 +547,7 @@ class StreamingContext private[streaming] ( assertNoOtherContextIsActive() scheduler.start() uiTab.foreach(_.attach()) - state = Started + state = StreamingContextState.ACTIVE setActiveContext(this) } } @@ -598,22 +609,26 @@ class StreamingContext private[streaming] ( * received data to be completed */ def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = synchronized { - state match { - case Initialized => logWarning("StreamingContext has not been started yet") - case Stopped => logWarning("StreamingContext has already been stopped") - case Started => - scheduler.stop(stopGracefully) - logInfo("StreamingContext stopped successfully") - waiter.notifyStop() + try { + state match { + case INITIALIZED => + logWarning("StreamingContext has not been started yet") + case STOPPED => + logWarning("StreamingContext has already been stopped") + case ACTIVE => + scheduler.stop(stopGracefully) + uiTab.foreach(_.detach()) + StreamingContext.setActiveContext(null) + waiter.notifyStop() + logInfo("StreamingContext stopped successfully") + } + // 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() + } finally { + // The state should always be Stopped after calling `stop()`, even if we haven't started yet + state = STOPPED } - // Even if the streaming context has not been started, we still need to stop the SparkContext. - // 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 - StreamingContext.setActiveContext(null) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java new file mode 100644 index 0000000000000..d7b639383ee34 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java @@ -0,0 +1,45 @@ +/* + * 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.apache.spark.annotation.DeveloperApi; + +/** + * :: DeveloperApi :: + * + * Represents the state of a StreamingContext. + */ +@DeveloperApi +public enum StreamingContextState { + /** + * The context has been created, but not been started yet. + * Input DStreams, transformations and output operations can be created on the context. + */ + INITIALIZED, + + /** + * The context has been started, and been not stopped. + * Input DStreams, transformations and output operations cannot be created on the context. + */ + ACTIVE, + + /** + * The context has been stopped and cannot be used any more. + */ + STOPPED +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 572d7d8e8753d..d8fbed2c50644 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -578,6 +578,28 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { ssc.addStreamingListener(streamingListener) } + /** + * :: DeveloperApi :: + * + * Return the current state of the context. The context can be in three possible states - + *
      + *
    • + * StreamingContextState.INTIALIZED - The context has been created, but not been started yet. + * Input DStreams, transformations and output operations can be created on the context. + *
    • + *
    • + * StreamingContextState.ACTIVE - The context has been started, and been not stopped. + * Input DStreams, transformations and output operations cannot be created on the context. + *
    • + *
    • + * StreamingContextState.STOPPED - The context has been stopped and cannot be used any more. + *
    • + *
    + */ + def getState(): StreamingContextState = { + ssc.getState() + } + /** * Start the execution of the streams. */ diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index b1adf881dd0f5..2e00b980b9e44 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -70,6 +70,20 @@ public void testInitialization() { Assert.assertNotNull(ssc.sparkContext()); } + @SuppressWarnings("unchecked") + @Test + public void testContextState() { + List> inputData = Arrays.asList(Arrays.asList(1, 2, 3, 4)); + Assert.assertTrue(ssc.getState() == StreamingContextState.INITIALIZED); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaTestUtils.attachTestOutputStream(stream); + Assert.assertTrue(ssc.getState() == StreamingContextState.INITIALIZED); + ssc.start(); + Assert.assertTrue(ssc.getState() == StreamingContextState.ACTIVE); + ssc.stop(); + Assert.assertTrue(ssc.getState() == StreamingContextState.STOPPED); + } + @SuppressWarnings("unchecked") @Test public void testCount() { diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index c0ea0491c313d..bb80bff6dc2e6 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -70,6 +70,7 @@ trait JavaTestBase extends TestSuiteBase { ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = { implicit val cm: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + ssc.getState() val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput) val out = new ArrayList[JList[V]]() res.map(entry => out.append(new ArrayList[V](entry))) 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 11c7fd835bfcd..b8247db7e8950 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -109,15 +109,21 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } + test("state matching") { + import StreamingContextState._ + assert(INITIALIZED === INITIALIZED) + assert(INITIALIZED != ACTIVE) + } + test("start and stop state check") { ssc = new StreamingContext(master, appName, batchDuration) addInputStream(ssc).register() - assert(ssc.state === ssc.StreamingContextState.Initialized) + assert(ssc.getState() === StreamingContextState.INITIALIZED) ssc.start() - assert(ssc.state === ssc.StreamingContextState.Started) + assert(ssc.getState() === StreamingContextState.ACTIVE) ssc.stop() - assert(ssc.state === ssc.StreamingContextState.Stopped) + assert(ssc.getState() === StreamingContextState.STOPPED) // Make sure that the SparkContext is also stopped by default intercept[Exception] { @@ -129,9 +135,11 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc = new StreamingContext(master, appName, batchDuration) addInputStream(ssc).register() ssc.start() + assert(ssc.getState() === StreamingContextState.ACTIVE) intercept[SparkException] { ssc.start() } + assert(ssc.getState() === StreamingContextState.ACTIVE) } test("stop multiple times") { @@ -139,13 +147,16 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w addInputStream(ssc).register() ssc.start() ssc.stop() + assert(ssc.getState() === StreamingContextState.STOPPED) ssc.stop() + assert(ssc.getState() === StreamingContextState.STOPPED) } test("stop before start") { ssc = new StreamingContext(master, appName, batchDuration) addInputStream(ssc).register() ssc.stop() // stop before start should not throw exception + assert(ssc.getState() === StreamingContextState.STOPPED) } test("start after stop") { @@ -156,6 +167,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w intercept[SparkException] { ssc.start() // start after stop should throw exception } + assert(ssc.getState() === StreamingContextState.STOPPED) } test("stop only streaming context") { @@ -167,6 +179,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w addInputStream(ssc).register() ssc.start() ssc.stop(stopSparkContext = false) + assert(ssc.getState() === StreamingContextState.STOPPED) assert(sc.makeRDD(1 to 100).collect().size === 100) sc.stop() From b6bf4f76c78abfaafa99b3c3c08b498aa9644346 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 May 2015 19:15:14 -0700 Subject: [PATCH 22/59] [SPARK-7324] [SQL] DataFrame.dropDuplicates This should also close https://github.com/apache/spark/pull/5870 Author: Reynold Xin Closes #6066 from rxin/dropDups and squashes the following commits: 130692f [Reynold Xin] [SPARK-7324][SQL] DataFrame.dropDuplicates --- python/pyspark/sql/dataframe.py | 36 +++++++++++++++++- .../org/apache/spark/sql/DataFrame.scala | 38 ++++++++++++++++++- .../org/apache/spark/sql/DataFrameSuite.scala | 35 +++++++++++++++++ 3 files changed, 105 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c2fa6c87384a6..4eaa8d9c572ff 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -755,8 +755,6 @@ def groupBy(self, *cols): jdf = self._jdf.groupBy(self._jcols(*cols)) return GroupedData(jdf, self.sql_ctx) - groupby = groupBy - def agg(self, *exprs): """ Aggregate on the entire :class:`DataFrame` without groups (shorthand for ``df.groupBy.agg()``). @@ -793,6 +791,36 @@ def subtract(self, other): """ return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) + def dropDuplicates(self, subset=None): + """Return a new :class:`DataFrame` with duplicate rows removed, + optionally only considering certain columns. + + >>> from pyspark.sql import Row + >>> df = sc.parallelize([ \ + Row(name='Alice', age=5, height=80), \ + Row(name='Alice', age=5, height=80), \ + Row(name='Alice', age=10, height=80)]).toDF() + >>> df.dropDuplicates().show() + +---+------+-----+ + |age|height| name| + +---+------+-----+ + | 5| 80|Alice| + | 10| 80|Alice| + +---+------+-----+ + + >>> df.dropDuplicates(['name', 'height']).show() + +---+------+-----+ + |age|height| name| + +---+------+-----+ + | 5| 80|Alice| + +---+------+-----+ + """ + if subset is None: + jdf = self._jdf.dropDuplicates() + else: + jdf = self._jdf.dropDuplicates(self._jseq(subset)) + return DataFrame(jdf, self.sql_ctx) + def dropna(self, how='any', thresh=None, subset=None): """Returns a new :class:`DataFrame` omitting rows with null values. @@ -1012,6 +1040,10 @@ def toPandas(self): import pandas as pd return pd.DataFrame.from_records(self.collect(), columns=self.columns) + # Pandas compatibility + groupby = groupBy + drop_duplicates = dropDuplicates + # Having SchemaRDD for backward compatibility (for docs) class SchemaRDD(DataFrame): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 2472999de37f5..265a61592b943 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql import java.io.CharArrayWriter import java.sql.DriverManager - import scala.collection.JavaConversions._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -42,7 +41,7 @@ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails -import org.apache.spark.sql.json.{JacksonGenerator, JsonRDD} +import org.apache.spark.sql.json.JacksonGenerator import org.apache.spark.sql.types._ import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} import org.apache.spark.util.Utils @@ -932,6 +931,40 @@ class DataFrame private[sql]( } } + /** + * Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. + * This is an alias for `distinct`. + * @group dfops + */ + def dropDuplicates(): DataFrame = dropDuplicates(this.columns) + + /** + * (Scala-specific) Returns a new [[DataFrame]] with duplicate rows removed, considering only + * the subset of columns. + * + * @group dfops + */ + def dropDuplicates(colNames: Seq[String]): DataFrame = { + val groupCols = colNames.map(resolve) + val groupColExprIds = groupCols.map(_.exprId) + val aggCols = logicalPlan.output.map { attr => + if (groupColExprIds.contains(attr.exprId)) { + attr + } else { + Alias(First(attr), attr.name)() + } + } + Aggregate(groupCols, aggCols, logicalPlan) + } + + /** + * Returns a new [[DataFrame]] with duplicate rows removed, considering only + * the subset of columns. + * + * @group dfops + */ + def dropDuplicates(colNames: Array[String]): DataFrame = dropDuplicates(colNames.toSeq) + /** * Computes statistics for numeric columns, including count, mean, stddev, min, and max. * If no columns are given, this function computes statistics for all numerical columns. @@ -1089,6 +1122,7 @@ class DataFrame private[sql]( /** * Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. + * This is an alias for `dropDuplicates`. * @group dfops */ override def distinct: DataFrame = Distinct(logicalPlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 7552c1288165f..2ade955864b71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -457,4 +457,39 @@ class DataFrameSuite extends QueryTest { assert(complexData.filter(complexData("m")("1") === 1).count() == 1) assert(complexData.filter(complexData("s")("key") === 1).count() == 1) } + + test("SPARK-7324 dropDuplicates") { + val testData = TestSQLContext.sparkContext.parallelize( + (2, 1, 2) :: (1, 1, 1) :: + (1, 2, 1) :: (2, 1, 2) :: + (2, 2, 2) :: (2, 2, 1) :: + (2, 1, 1) :: (1, 1, 2) :: + (1, 2, 2) :: (1, 2, 1) :: Nil).toDF("key", "value1", "value2") + + checkAnswer( + testData.dropDuplicates(), + Seq(Row(2, 1, 2), Row(1, 1, 1), Row(1, 2, 1), + Row(2, 2, 2), Row(2, 1, 1), Row(2, 2, 1), + Row(1, 1, 2), Row(1, 2, 2))) + + checkAnswer( + testData.dropDuplicates(Seq("key", "value1")), + Seq(Row(2, 1, 2), Row(1, 2, 1), Row(1, 1, 1), Row(2, 2, 2))) + + checkAnswer( + testData.dropDuplicates(Seq("value1", "value2")), + Seq(Row(2, 1, 2), Row(1, 2, 1), Row(1, 1, 1), Row(2, 2, 2))) + + checkAnswer( + testData.dropDuplicates(Seq("key")), + Seq(Row(2, 1, 2), Row(1, 1, 1))) + + checkAnswer( + testData.dropDuplicates(Seq("value1")), + Seq(Row(2, 1, 2), Row(1, 2, 1))) + + checkAnswer( + testData.dropDuplicates(Seq("value2")), + Seq(Row(2, 1, 2), Row(1, 1, 1))) + } } From e35d878be3b2976333618a356b88440f5e8ba408 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 11 May 2015 19:21:16 -0700 Subject: [PATCH 23/59] [SPARK-7411] [SQL] Support SerDe for HiveQl in CTAS This is a follow up of #5876 and should be merged after #5876. Let's wait for unit testing result from Jenkins. Author: Cheng Hao Closes #5963 from chenghao-intel/useIsolatedClient and squashes the following commits: f87ace6 [Cheng Hao] remove the TODO and add `resolved condition` for HiveTable a8260e8 [Cheng Hao] Update code as feedback f4e243f [Cheng Hao] remove the serde setting for SequenceFile d166afa [Cheng Hao] style issue d25a4aa [Cheng Hao] Add SerDe support for CTAS --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 66 +++--- .../org/apache/spark/sql/hive/HiveQl.scala | 207 +++++++++++++++--- .../spark/sql/hive/client/ClientWrapper.scala | 6 + .../hive/execution/CreateTableAsSelect.scala | 12 +- .../apache/spark/sql/hive/HiveQlSuite.scala | 177 +++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 4 +- 6 files changed, 390 insertions(+), 82 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f5398605bc7eb..bbf48efb24440 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -407,64 +407,58 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { - import org.apache.hadoop.hive.ql.Context - import org.apache.hadoop.hive.ql.parse.{ASTNode, QB, SemanticAnalyzer} - def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p + case p: LogicalPlan if p.resolved => p + case p @ CreateTableAsSelect(table, child, allowExisting) => + val schema = if (table.schema.size > 0) { + table.schema + } else { + child.output.map { + attr => new HiveColumn( + attr.name, + HiveMetastoreTypes.toMetastoreType(attr.dataType), null) + } + } + + val desc = table.copy(schema = schema) - case CreateTableAsSelect(desc, child, allowExisting) => - if (hive.convertCTAS && !desc.serde.isDefined) { + if (hive.convertCTAS && table.serde.isEmpty) { // Do the conversion when spark.sql.hive.convertCTAS is true and the query // does not specify any storage format (file format and storage handler). - if (desc.specifiedDatabase.isDefined) { + if (table.specifiedDatabase.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + - "when spark.sql.hive.convertCTAS is set to true.") + "when spark.sql.hive.convertCTAS is set to true.") } val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( desc.name, - conf.defaultDataSourceName, + hive.conf.defaultDataSourceName, temporary = false, mode, options = Map.empty[String, String], child ) } else { - execution.CreateTableAsSelect( - desc.copy( - specifiedDatabase = Option(desc.specifiedDatabase.getOrElse(client.currentDatabase))), - child, - allowExisting) - } - - case p: LogicalPlan if p.resolved => p - - case p @ CreateTableAsSelect(desc, child, allowExisting) => - val (dbName, tblName) = processDatabaseAndTableName(desc.database, desc.name) - - if (hive.convertCTAS) { - if (desc.specifiedDatabase.isDefined) { - throw new AnalysisException( - "Cannot specify database name in a CTAS statement " + - "when spark.sql.hive.convertCTAS is set to true.") + val desc = if (table.serde.isEmpty) { + // add default serde + table.copy( + serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + } else { + table } - val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists - CreateTableUsingAsSelect( - tblName, - conf.defaultDataSourceName, - temporary = false, - mode, - options = Map.empty[String, String], - child - ) - } else { + val (dbName, tblName) = + processDatabaseAndTableName( + desc.specifiedDatabase.getOrElse(client.currentDatabase), desc.name) + execution.CreateTableAsSelect( - desc, + desc.copy( + specifiedDatabase = Some(dbName), + name = tblName), child, allowExisting) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 04d40bbb2bced..2cbb5ca4d2e0c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -22,14 +22,15 @@ import java.sql.Date import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.ql.{ErrorMsg, Context} import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo} import org.apache.hadoop.hive.ql.lib.Node -import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils -import org.apache.spark.sql.AnalysisException +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ @@ -62,7 +63,13 @@ case class CreateTableAsSelect( allowExisting: Boolean) extends UnaryNode with Command { override def output: Seq[Attribute] = Seq.empty[Attribute] - override lazy val resolved: Boolean = tableDesc.specifiedDatabase.isDefined && childrenResolved + override lazy val resolved: Boolean = + tableDesc.specifiedDatabase.isDefined && + tableDesc.schema.size > 0 && + tableDesc.serde.isDefined && + tableDesc.inputFormat.isDefined && + tableDesc.outputFormat.isDefined && + childrenResolved } /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ @@ -240,12 +247,23 @@ private[hive] object HiveQl { * Otherwise, there will be Null pointer exception, * when retrieving properties form HiveConf. */ - val hContext = new Context(new HiveConf()) + val hContext = new Context(hiveConf) val node = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql, hContext)) hContext.clear() node } + /** + * Returns the HiveConf + */ + private[this] def hiveConf(): HiveConf = { + val ss = SessionState.get() // SessionState is lazy initializaion, it can be null here + if (ss == null) { + new HiveConf() + } else { + ss.getConf + } + } /** Returns a LogicalPlan for a given HiveQL string. */ def parseSql(sql: String): LogicalPlan = hqlParser.parse(sql) @@ -476,8 +494,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C DropTable(tableName, ifExists.nonEmpty) // Support "ANALYZE TABLE tableNmae COMPUTE STATISTICS noscan" case Token("TOK_ANALYZE", - Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: - isNoscan) => + Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: + isNoscan) => // Reference: // https://cwiki.apache.org/confluence/display/Hive/StatsDev#StatsDev-ExistingTables if (partitionSpec.nonEmpty) { @@ -547,6 +565,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val ( Some(tableNameParts) :: _ /* likeTable */ :: + externalTable :: Some(query) :: allowExisting +: ignores) = @@ -554,6 +573,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Seq( "TOK_TABNAME", "TOK_LIKETABLE", + "EXTERNAL", "TOK_QUERY", "TOK_IFNOTEXISTS", "TOK_TABLECOMMENT", @@ -576,43 +596,153 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C children) val (db, tableName) = extractDbNameTableName(tableNameParts) - var tableDesc = - HiveTable( - specifiedDatabase = db, - name = tableName, - schema = Seq.empty, - partitionColumns = Seq.empty, - properties = Map.empty, - serdeProperties = Map.empty, - tableType = ManagedTable, - location = None, - inputFormat = None, - outputFormat = None, - serde = None) - - // TODO: Handle all the cases here... - children.foreach { - case Token("TOK_TBLRCFILE", Nil) => - import org.apache.hadoop.hive.ql.io.{RCFileInputFormat, RCFileOutputFormat} + // TODO add bucket support + var tableDesc: HiveTable = HiveTable( + specifiedDatabase = db, + name = tableName, + schema = Seq.empty[HiveColumn], + partitionColumns = Seq.empty[HiveColumn], + properties = Map[String, String](), + serdeProperties = Map[String, String](), + tableType = if (externalTable.isDefined) ExternalTable else ManagedTable, + location = None, + inputFormat = None, + outputFormat = None, + serde = None, + viewText = None) + + // default storage type abbriviation (e.g. RCFile, ORC, PARQUET etc.) + val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT) + // handle the default format for the storage type abbriviation + tableDesc = if ("SequenceFile".equalsIgnoreCase(defaultStorageType)) { + tableDesc.copy( + inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), + outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")) + } else if ("RCFile".equalsIgnoreCase(defaultStorageType)) { + tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"), + serde = Option(hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTRCFILESERDE))) + } else if ("ORC".equalsIgnoreCase(defaultStorageType)) { + tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"), + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } else if ("PARQUET".equalsIgnoreCase(defaultStorageType)) { + tableDesc.copy( + inputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + serde = + Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + } else { + tableDesc.copy( + inputFormat = + Option("org.apache.hadoop.mapred.TextInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) + } + + children.collect { + case list @ Token("TOK_TABCOLLIST", _) => + val cols = BaseSemanticAnalyzer.getColumns(list, true) + if (cols != null) { + tableDesc = tableDesc.copy( + schema = cols.map { field => + HiveColumn(field.getName, field.getType, field.getComment) + }) + } + case Token("TOK_TABLECOMMENT", child :: Nil) => + val comment = BaseSemanticAnalyzer.unescapeSQLString(child.getText) + // TODO support the sql text + tableDesc = tableDesc.copy(viewText = Option(comment)) + case Token("TOK_TABLEPARTCOLS", list @ Token("TOK_TABCOLLIST", _) :: Nil) => + val cols = BaseSemanticAnalyzer.getColumns(list(0), false) + if (cols != null) { + tableDesc = tableDesc.copy( + partitionColumns = cols.map { field => + HiveColumn(field.getName, field.getType, field.getComment) + }) + } + case Token("TOK_TABLEROWFORMAT", Token("TOK_SERDEPROPS", child :: Nil) :: Nil)=> + val serdeParams = new java.util.HashMap[String, String]() + child match { + case Token("TOK_TABLEROWFORMATFIELD", rowChild1 :: rowChild2) => + val fieldDelim = BaseSemanticAnalyzer.unescapeSQLString (rowChild1.getText()) + serdeParams.put(serdeConstants.FIELD_DELIM, fieldDelim) + serdeParams.put(serdeConstants.SERIALIZATION_FORMAT, fieldDelim) + if (rowChild2.length > 1) { + val fieldEscape = BaseSemanticAnalyzer.unescapeSQLString (rowChild2(0).getText) + serdeParams.put(serdeConstants.ESCAPE_CHAR, fieldEscape) + } + case Token("TOK_TABLEROWFORMATCOLLITEMS", rowChild :: Nil) => + val collItemDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) + serdeParams.put(serdeConstants.COLLECTION_DELIM, collItemDelim) + case Token("TOK_TABLEROWFORMATMAPKEYS", rowChild :: Nil) => + val mapKeyDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) + serdeParams.put(serdeConstants.MAPKEY_DELIM, mapKeyDelim) + case Token("TOK_TABLEROWFORMATLINES", rowChild :: Nil) => + val lineDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) + if (!(lineDelim == "\n") && !(lineDelim == "10")) { + throw new AnalysisException( + SemanticAnalyzer.generateErrorMessage( + rowChild, + ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg)) + } + serdeParams.put(serdeConstants.LINE_DELIM, lineDelim) + case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) => + val nullFormat = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) + // TODO support the nullFormat + case _ => assert(false) + } + tableDesc = tableDesc.copy( + serdeProperties = tableDesc.serdeProperties ++ serdeParams) + case Token("TOK_TABLELOCATION", child :: Nil) => + var location = BaseSemanticAnalyzer.unescapeSQLString(child.getText) + location = EximUtil.relativeToAbsolutePath(hiveConf, location) + tableDesc = tableDesc.copy(location = Option(location)) + case Token("TOK_TABLESERIALIZER", child :: Nil) => tableDesc = tableDesc.copy( - outputFormat = Option(classOf[RCFileOutputFormat].getName), - inputFormat = Option(classOf[RCFileInputFormat[_, _]].getName)) + serde = Option(BaseSemanticAnalyzer.unescapeSQLString(child.getChild(0).getText))) + if (child.getChildCount == 2) { + val serdeParams = new java.util.HashMap[String, String]() + BaseSemanticAnalyzer.readProps( + (child.getChild(1).getChild(0)).asInstanceOf[ASTNode], serdeParams) + tableDesc = tableDesc.copy(serdeProperties = tableDesc.serdeProperties ++ serdeParams) + } + case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) => + throw new SemanticException( + "Unrecognized file format in STORED AS clause:${child.getText}") + case Token("TOK_TBLRCFILE", Nil) => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) if (tableDesc.serde.isEmpty) { tableDesc = tableDesc.copy( serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) } + case Token("TOK_TBLORCFILE", Nil) => tableDesc = tableDesc.copy( inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"), - serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy( + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } case Token("TOK_TBLPARQUETFILE", Nil) => tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), - serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + inputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = + Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy( + serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + } case Token("TOK_TABLESERIALIZER", Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => @@ -627,13 +757,20 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_TABLEPROPERTIES", list :: Nil) => tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) - - case _ => + case list @ Token("TOK_TABLEFILEFORMAT", _) => + tableDesc = tableDesc.copy( + inputFormat = + Option(BaseSemanticAnalyzer.unescapeSQLString(list.getChild(0).getText)), + outputFormat = + Option(BaseSemanticAnalyzer.unescapeSQLString(list.getChild(1).getText))) + case Token("TOK_STORAGEHANDLER", _) => + throw new AnalysisException(ErrorMsg.CREATE_NON_NATIVE_AS.getMsg()) + case _ => // Unsupport features } CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting != None) - // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. + // If its not a "CTAS" like above then take it as a native command case Token("TOK_CREATETABLE", _) => NativePlaceholder // Support "TRUNCATE TABLE table_name [PARTITION partition_spec]" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index 6bca9d0179fe3..99aa0f1ded3f8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -225,6 +225,12 @@ private[hive] class ClientWrapper( table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) table.properties.foreach { case (k, v) => qlTable.setProperty(k, v) } table.serdeProperties.foreach { case (k, v) => qlTable.setSerdeParam(k, v) } + + // set owner + qlTable.setOwner(conf.getUser) + // set create time + qlTable.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) + version match { case hive.v12 => table.location.map(new URI(_)).foreach(u => qlTable.call[URI, Unit]("setDataLocation", u)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 91e6ac4032204..7d3ec12c4eb05 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -17,10 +17,8 @@ package org.apache.spark.sql.hive.execution -import org.apache.hadoop.hive.ql.plan.CreateTableDesc - import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.RunnableCommand @@ -29,13 +27,10 @@ import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, HiveMetastoreT /** * Create table and insert the query result into it. - * @param database the database name of the new relation - * @param tableName the table name of the new relation + * @param tableDesc the Table Describe, which may contains serde, storage handler etc. * @param query the query whose result will be insert into the new relation * @param allowExisting allow continue working if it's already exists, otherwise * raise exception - * @param desc the CreateTableDesc, which may contains serde, storage handler etc. - */ private[hive] case class CreateTableAsSelect( @@ -80,8 +75,7 @@ case class CreateTableAsSelect( if (allowExisting) { // table already exists, will do nothing, to keep consistent with Hive } else { - throw - new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName") + throw new AnalysisException(s"$database.$tableName already exists.") } } else { hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala new file mode 100644 index 0000000000000..941a2941649b8 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala @@ -0,0 +1,177 @@ +/* + * 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 + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.hive.client.{ManagedTable, HiveColumn, ExternalTable, HiveTable} +import org.scalatest.{BeforeAndAfterAll, FunSuite} + + +class HiveQlSuite extends FunSuite with BeforeAndAfterAll { + override def beforeAll() { + if (SessionState.get() == null) { + SessionState.start(new HiveConf()) + } + } + + private def extractTableDesc(sql: String): (HiveTable, Boolean) = { + HiveQl.createPlan(sql).collect { + case CreateTableAsSelect(desc, child, allowExisting) => (desc, allowExisting) + }.head + } + + test("Test CTAS #1") { + val s1 = + """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view + |(viewTime INT, + |userid BIGINT, + |page_url STRING, + |referrer_url STRING, + |ip STRING COMMENT 'IP Address of the User', + |country STRING COMMENT 'country of origination') + |COMMENT 'This is the staging page view table' + |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') + |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\054' STORED AS RCFILE + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src""".stripMargin + + val (desc, exists) = extractTableDesc(s1) + assert(exists == true) + assert(desc.specifiedDatabase == Some("mydb")) + assert(desc.name == "page_view") + assert(desc.tableType == ExternalTable) + assert(desc.location == Some("/user/external/page_view")) + assert(desc.schema == + HiveColumn("viewtime", "int", null) :: + HiveColumn("userid", "bigint", null) :: + HiveColumn("page_url", "string", null) :: + HiveColumn("referrer_url", "string", null) :: + HiveColumn("ip", "string", "IP Address of the User") :: + HiveColumn("country", "string", "country of origination") :: Nil) + // TODO will be SQLText + assert(desc.viewText == Option("This is the staging page view table")) + assert(desc.partitionColumns == + HiveColumn("dt", "string", "date type") :: + HiveColumn("hour", "string", "hour of the day") :: Nil) + assert(desc.serdeProperties == + Map((serdeConstants.SERIALIZATION_FORMAT, "\054"), (serdeConstants.FIELD_DELIM, "\054"))) + assert(desc.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.serde == Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) + } + + test("Test CTAS #2") { + val s2 = + """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view + |(viewTime INT, + |userid BIGINT, + |page_url STRING, + |referrer_url STRING, + |ip STRING COMMENT 'IP Address of the User', + |country STRING COMMENT 'country of origination') + |COMMENT 'This is the staging page view table' + |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') + |ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe' + | STORED AS + | INPUTFORMAT 'parquet.hive.DeprecatedParquetInputFormat' + | OUTPUTFORMAT 'parquet.hive.DeprecatedParquetOutputFormat' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src""".stripMargin + + val (desc, exists) = extractTableDesc(s2) + assert(exists == true) + assert(desc.specifiedDatabase == Some("mydb")) + assert(desc.name == "page_view") + assert(desc.tableType == ExternalTable) + assert(desc.location == Some("/user/external/page_view")) + assert(desc.schema == + HiveColumn("viewtime", "int", null) :: + HiveColumn("userid", "bigint", null) :: + HiveColumn("page_url", "string", null) :: + HiveColumn("referrer_url", "string", null) :: + HiveColumn("ip", "string", "IP Address of the User") :: + HiveColumn("country", "string", "country of origination") :: Nil) + // TODO will be SQLText + assert(desc.viewText == Option("This is the staging page view table")) + assert(desc.partitionColumns == + HiveColumn("dt", "string", "date type") :: + HiveColumn("hour", "string", "hour of the day") :: Nil) + assert(desc.serdeProperties == Map()) + assert(desc.inputFormat == Option("parquet.hive.DeprecatedParquetInputFormat")) + assert(desc.outputFormat == Option("parquet.hive.DeprecatedParquetOutputFormat")) + assert(desc.serde == Option("parquet.hive.serde.ParquetHiveSerDe")) + assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) + } + + test("Test CTAS #3") { + val s3 = """CREATE TABLE page_view AS SELECT * FROM src""" + val (desc, exists) = extractTableDesc(s3) + assert(exists == false) + assert(desc.specifiedDatabase == None) + assert(desc.name == "page_view") + assert(desc.tableType == ManagedTable) + assert(desc.location == None) + assert(desc.schema == Seq.empty[HiveColumn]) + assert(desc.viewText == None) // TODO will be SQLText + assert(desc.serdeProperties == Map()) + assert(desc.inputFormat == Option("org.apache.hadoop.mapred.TextInputFormat")) + assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) + assert(desc.serde.isEmpty) + assert(desc.properties == Map()) + } + + test("Test CTAS #4") { + val s4 = + """CREATE TABLE page_view + |STORED BY 'storage.handler.class.name' AS SELECT * FROM src""".stripMargin + intercept[AnalysisException] { + extractTableDesc(s4) + } + } + + test("Test CTAS #5") { + val s5 = """CREATE TABLE ctas2 + | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + | WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") + | STORED AS RCFile + | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") + | AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin + val (desc, exists) = extractTableDesc(s5) + assert(exists == false) + assert(desc.specifiedDatabase == None) + assert(desc.name == "ctas2") + assert(desc.tableType == ManagedTable) + assert(desc.location == None) + assert(desc.schema == Seq.empty[HiveColumn]) + assert(desc.viewText == None) // TODO will be SQLText + assert(desc.serdeProperties == Map(("serde_p1" -> "p1"), ("serde_p2" -> "p2"))) + assert(desc.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.serde == Option("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) + assert(desc.properties == Map(("tbl_p1" -> "p11"), ("tbl_p2" -> "p22"))) + } +} 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 1353802604402..0d739dead4c73 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 @@ -201,7 +201,7 @@ class SQLQuerySuite extends QueryTest { var message = intercept[AnalysisException] { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") }.getMessage - assert(message.contains("Table ctas1 already exists")) + assert(message.contains("ctas1 already exists")) checkRelation("ctas1", true) sql("DROP TABLE ctas1") @@ -314,7 +314,7 @@ class SQLQuerySuite extends QueryTest { SELECT key, value FROM src ORDER BY key, value""").collect().toSeq) - intercept[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] { + intercept[AnalysisException] { sql( """CREATE TABLE ctas4 AS | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect() From 4b5e1fe94c65f0feb77d849e7defa42b6007628d Mon Sep 17 00:00:00 2001 From: Zhongshuai Pei <799203320@qq.com> Date: Mon, 11 May 2015 19:22:44 -0700 Subject: [PATCH 24/59] [SPARK-7437] [SQL] Fold "literal in (item1, item2, ..., literal, ...)" into true or false directly SQL ``` select key from src where 3 in (4, 5); ``` Before ``` == Optimized Logical Plan == Project [key#12] Filter 3 INSET (5,4) MetastoreRelation default, src, None ``` After ``` == Optimized Logical Plan == LocalRelation [key#228], [] ``` Author: Zhongshuai Pei <799203320@qq.com> Author: DoingDone9 <799203320@qq.com> Closes #5972 from DoingDone9/InToFalse and squashes the following commits: 4c722a2 [Zhongshuai Pei] Update predicates.scala abe2bbb [Zhongshuai Pei] Update Optimizer.scala fa461a5 [Zhongshuai Pei] Update Optimizer.scala e34c28a [Zhongshuai Pei] Update predicates.scala 24739bd [Zhongshuai Pei] Update ConstantFoldingSuite.scala f4dbf50 [Zhongshuai Pei] Update ConstantFoldingSuite.scala 35ceb7a [Zhongshuai Pei] Update Optimizer.scala 36c194e [Zhongshuai Pei] Update Optimizer.scala 2e8f6ca [Zhongshuai Pei] Update Optimizer.scala 14952e2 [Zhongshuai Pei] Merge pull request #13 from apache/master f03fe7f [Zhongshuai Pei] Merge pull request #12 from apache/master f12fa50 [Zhongshuai Pei] Merge pull request #10 from apache/master f61210c [Zhongshuai Pei] Merge pull request #9 from apache/master 34b1a9a [Zhongshuai Pei] Merge pull request #8 from apache/master 802261c [DoingDone9] Merge pull request #7 from apache/master d00303b [DoingDone9] Merge pull request #6 from apache/master 98b134f [DoingDone9] Merge pull request #5 from apache/master 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 --- .../sql/catalyst/expressions/predicates.scala | 1 + .../sql/catalyst/optimizer/Optimizer.scala | 4 +-- .../optimizer/ConstantFoldingSuite.scala | 33 +++++++++++++++++++ 3 files changed, 36 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 50b0f3ee5f93f..1d72a9eb834b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -110,6 +110,7 @@ case class InSet(value: Expression, hset: Set[Any]) override def children: Seq[Expression] = value :: Nil + override def foldable: Boolean = value.foldable override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN. override def toString: String = s"$value INSET ${hset.mkString("(", ",", ")")}" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index d7b2f203a6934..1ee5fb245fbb2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -46,13 +46,13 @@ object DefaultOptimizer extends Optimizer { CombineLimits) :: Batch("ConstantFolding", FixedPoint(100), NullPropagation, + OptimizeIn, ConstantFolding, LikeSimplification, BooleanSimplification, SimplifyFilters, SimplifyCasts, - SimplifyCaseConversionExpressions, - OptimizeIn) :: + SimplifyCaseConversionExpressions) :: Batch("Decimal Optimizations", FixedPoint(100), DecimalAggregates) :: Batch("LocalRelation", FixedPoint(100), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 6b7d9a85c341b..5697c2272b8e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -35,6 +35,7 @@ class ConstantFoldingSuite extends PlanTest { Batch("AnalysisNodes", Once, EliminateSubQueries) :: Batch("ConstantFolding", Once, + OptimizeIn, ConstantFolding, BooleanSimplification) :: Nil } @@ -247,4 +248,36 @@ class ConstantFoldingSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("Constant folding test: Fold In(v, list) into true or false") { + var originalQuery = + testRelation + .select('a) + .where(In(Literal(1), Seq(Literal(1), Literal(2)))) + + var optimized = Optimize.execute(originalQuery.analyze) + + var correctAnswer = + testRelation + .select('a) + .where(Literal(true)) + .analyze + + comparePlans(optimized, correctAnswer) + + originalQuery = + testRelation + .select('a) + .where(In(Literal(1), Seq(Literal(1), 'a.attr))) + + optimized = Optimize.execute(originalQuery.analyze) + + correctAnswer = + testRelation + .select('a) + .where(Literal(true)) + .analyze + + comparePlans(optimized, correctAnswer) + } } From 028ad4bd29106747089bb9a863e9a8dee738770e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 May 2015 20:04:36 -0700 Subject: [PATCH 25/59] [SPARK-7509][SQL] DataFrame.drop in Python for dropping columns. Author: Reynold Xin Closes #6068 from rxin/drop-column and squashes the following commits: 9d7d5ec [Reynold Xin] [SPARK-7509][SQL] DataFrame.drop in Python for dropping columns. --- python/pyspark/sql/dataframe.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 4eaa8d9c572ff..72180f6d05fbc 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1014,7 +1014,7 @@ def withColumn(self, colName, col): @ignore_unicode_prefix def withColumnRenamed(self, existing, new): - """REturns a new :class:`DataFrame` by renaming an existing column. + """Returns a new :class:`DataFrame` by renaming an existing column. :param existing: string, name of the existing column to rename. :param col: string, new name of the column. @@ -1027,6 +1027,18 @@ def withColumnRenamed(self, existing, new): for c in self.columns] return self.select(*cols) + @ignore_unicode_prefix + def drop(self, colName): + """Returns a new :class:`DataFrame` that drops the specified column. + + :param colName: string, name of the column to drop. + + >>> df.drop('age').collect() + [Row(name=u'Alice'), Row(name=u'Bob')] + """ + jdf = self._jdf.drop(colName) + return DataFrame(jdf, self.sql_ctx) + def toPandas(self): """Returns the contents of this :class:`DataFrame` as Pandas ``pandas.DataFrame``. From b94a93371cf219a88edee7677d22f1eaefc1ea5b Mon Sep 17 00:00:00 2001 From: Joshi Date: Mon, 11 May 2015 21:02:34 -0700 Subject: [PATCH 26/59] [SPARK-7435] [SPARKR] Make DataFrame.show() consistent with that of Scala and pySpark Author: Joshi Author: Rekha Joshi Closes #5989 from rekhajoshm/fix/SPARK-7435 and squashes the following commits: cfc9e02 [Joshi] Spark-7435[R]: updated patch for review comments 62becc1 [Joshi] SPARK-7435: Update to DataFrame e3677c9 [Rekha Joshi] Merge pull request #1 from apache/master --- R/pkg/R/DataFrame.R | 7 ++++--- R/pkg/inst/tests/test_sparkSQL.R | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 8a9d2dd45c588..2705817531019 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -150,7 +150,7 @@ setMethod("isLocal", callJMethod(x@sdf, "isLocal") }) -#' ShowDF +#' showDF #' #' Print the first numRows rows of a DataFrame #' @@ -170,7 +170,8 @@ setMethod("isLocal", setMethod("showDF", signature(x = "DataFrame"), function(x, numRows = 20) { - callJMethod(x@sdf, "showString", numToInt(numRows)) + s <- callJMethod(x@sdf, "showString", numToInt(numRows)) + cat(s) }) #' show @@ -187,7 +188,7 @@ setMethod("showDF", #' sqlCtx <- sparkRSQL.init(sc) #' path <- "path/to/file.json" #' df <- jsonFile(sqlCtx, path) -#' show(df) +#' df #'} setMethod("show", "DataFrame", function(object) { diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index dbb535e245321..99c28830c6237 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -653,7 +653,8 @@ test_that("toJSON() returns an RDD of the correct values", { test_that("showDF()", { df <- jsonFile(sqlCtx, jsonPath) - expect_output(showDF(df), "+----+-------+\n| age| name|\n+----+-------+\n|null|Michael|\n| 30| Andy|\n| 19| Justin|\n+----+-------+\n") + s <- capture.output(showDF(df)) + expect_output(s , "+----+-------+\n| age| name|\n+----+-------+\n|null|Michael|\n| 30| Andy|\n| 19| Justin|\n+----+-------+\n") }) test_that("isLocal()", { From 16696759e9a292378cbfdf695a63d6d0cff0d79a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 May 2015 22:06:56 -0700 Subject: [PATCH 27/59] [SQL] Rename Dialect -> ParserDialect. Author: Reynold Xin Closes #6071 from rxin/parserdialect and squashes the following commits: ca2eb31 [Reynold Xin] Rename Dialect -> ParserDialect. --- .../catalyst/{Dialect.scala => ParserDialect.scala} | 2 +- .../main/scala/org/apache/spark/sql/SQLContext.scala | 12 ++++++------ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 4 ++-- .../org/apache/spark/sql/hive/HiveContext.scala | 4 ++-- .../spark/sql/hive/execution/SQLQuerySuite.scala | 6 +++--- 5 files changed, 14 insertions(+), 14 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/{Dialect.scala => ParserDialect.scala} (97%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/Dialect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ParserDialect.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/Dialect.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ParserDialect.scala index 977003493d471..05a92b06f9fd9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/Dialect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ParserDialect.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan * */ @DeveloperApi -abstract class Dialect { +abstract class ParserDialect { // this is the main function that will be implemented by sql parser. def parse(sqlText: String): LogicalPlan } 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 28fc9d04436f7..648021806f3ee 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 @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.errors.DialectException import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.Dialect +import org.apache.spark.sql.catalyst.ParserDialect import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, expressions} import org.apache.spark.sql.execution.{Filter, _} import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} @@ -49,7 +49,7 @@ import org.apache.spark.{Partition, SparkContext} /** * Currently we support the default dialect named "sql", associated with the class - * [[DefaultDialect]] + * [[DefaultParserDialect]] * * And we can also provide custom SQL Dialect, for example in Spark SQL CLI: * {{{ @@ -74,7 +74,7 @@ import org.apache.spark.{Partition, SparkContext} *-- "hiveql" (for HiveContext) * }}} */ -private[spark] class DefaultDialect extends Dialect { +private[spark] class DefaultParserDialect extends ParserDialect { @transient protected val sqlParser = new catalyst.SqlParser @@ -176,10 +176,10 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] val sqlParser = new SparkSQLParser(getSQLDialect().parse(_)) - protected[sql] def getSQLDialect(): Dialect = { + protected[sql] def getSQLDialect(): ParserDialect = { try { val clazz = Utils.classForName(dialectClassName) - clazz.newInstance().asInstanceOf[Dialect] + clazz.newInstance().asInstanceOf[ParserDialect] } catch { case NonFatal(e) => // Since we didn't find the available SQL Dialect, it will fail even for SET command: @@ -209,7 +209,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] val defaultSession = createSession() protected[sql] def dialectClassName = if (conf.dialect == "sql") { - classOf[DefaultDialect].getCanonicalName + classOf[DefaultParserDialect].getCanonicalName } else { conf.dialect } 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 b44eb223c80c5..ec0e76cde6f7c 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} import org.apache.spark.sql.types._ /** A SQL Dialect for testing purpose, and it can not be nested type */ -class MyDialect extends DefaultDialect +class MyDialect extends DefaultParserDialect class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { // Make sure the tables are loaded. @@ -94,7 +94,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { newContext.sql("SELECT 1") } // test if the dialect set back to DefaultSQLDialect - assert(newContext.getSQLDialect().getClass === classOf[DefaultDialect]) + assert(newContext.getSQLDialect().getClass === classOf[DefaultParserDialect]) } test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { 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 3bab648e31d1f..61e8c154e8c3c 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 @@ -22,7 +22,7 @@ import java.sql.Timestamp import java.util.{ArrayList => JArrayList} import org.apache.hadoop.hive.ql.parse.VariableSubstitution -import org.apache.spark.sql.catalyst.Dialect +import org.apache.spark.sql.catalyst.ParserDialect import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -54,7 +54,7 @@ import org.apache.spark.util.Utils /** * This is the HiveQL Dialect, this dialect is strongly bind with HiveContext */ -private[hive] class HiveQLDialect extends Dialect { +private[hive] class HiveQLDialect extends ParserDialect { override def parse(sqlText: String): LogicalPlan = { HiveQl.parseSql(sqlText) } 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 0d739dead4c73..a5744ccc68a47 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.errors.DialectException -import org.apache.spark.sql.DefaultDialect +import org.apache.spark.sql.DefaultParserDialect import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SQLConf} import org.apache.spark.sql.hive.MetastoreRelation import org.apache.spark.sql.hive.test.TestHive @@ -53,7 +53,7 @@ case class WindowData( area: String, product: Int) /** A SQL Dialect for testing purpose, and it can not be nested type */ -class MyDialect extends DefaultDialect +class MyDialect extends DefaultParserDialect /** * A collection of hive query tests where we generate the answers ourselves instead of depending on @@ -247,7 +247,7 @@ class SQLQuerySuite extends QueryTest { // set the dialect back to the DefaultSQLDialect sql("SET spark.sql.dialect=sql") - assert(getSQLDialect().getClass === classOf[DefaultDialect]) + assert(getSQLDialect().getClass === classOf[DefaultParserDialect]) sql("SET spark.sql.dialect=hiveql") assert(getSQLDialect().getClass === classOf[HiveQLDialect]) From 640f63b959f936ac142ce429262ffc3db4536619 Mon Sep 17 00:00:00 2001 From: vidmantas zemleris Date: Mon, 11 May 2015 22:29:24 -0700 Subject: [PATCH 28/59] [SPARK-6994][SQL] Update docs for fetching Row fields by name add docs for https://issues.apache.org/jira/browse/SPARK-6994 Author: vidmantas zemleris Closes #6030 from vidma/docs/row-with-named-fields and squashes the following commits: 241b401 [vidmantas zemleris] [SPARK-6994][SQL] Update docs for fetching Row fields by name --- docs/sql-programming-guide.md | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6b7b867ea6b5d..78b8e8ad515a0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -367,11 +367,18 @@ val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(" people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. -val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +val teenagers = sqlContext.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19") // The results of SQL queries are DataFrames and support all the normal RDD operations. -// The columns of a row in the result can be accessed by ordinal. +// The columns of a row in the result can be accessed by field index: teenagers.map(t => "Name: " + t(0)).collect().foreach(println) + +// or by field name: +teenagers.map(t => "Name: " + t.getAs[String]("name")).collect().foreach(println) + +// row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T] +teenagers.map(_.getValuesMap[Any](List("name", "age"))).collect().foreach(println) +// Map("name" -> "Justin", "age" -> 19) {% endhighlight %}
    @@ -538,7 +545,7 @@ peopleDataFrame.registerTempTable("people") val results = sqlContext.sql("SELECT name FROM people") // The results of SQL queries are DataFrames and support all the normal RDD operations. -// The columns of a row in the result can be accessed by ordinal. +// The columns of a row in the result can be accessed by field index or by field name. results.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} From 8a4edecc483490da4fce406ac3b401860834ac1e Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 12 May 2015 01:34:33 -0700 Subject: [PATCH 29/59] [SPARK-7534] [CORE] [WEBUI] Fix the Stage table when a stage is missing Just improved the Stage table when a stage is missing. Before: ![screen shot 2015-05-11 at 10 11 51 am](https://cloud.githubusercontent.com/assets/1000778/7570842/2ba37380-f7c8-11e4-9b5f-cf1a6264b2a4.png) After: ![screen shot 2015-05-11 at 10 26 09 am](https://cloud.githubusercontent.com/assets/1000778/7570848/33703152-f7c8-11e4-81a8-d53dd72d7b8d.png) Author: zsxwing Closes #6061 from zsxwing/SPARK-7534 and squashes the following commits: 09fe862 [zsxwing] Leave it blank rather than '-' 6299197 [zsxwing] Fix the Stage table when a stage is missing --- .../org/apache/spark/ui/jobs/StageTable.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index a33243d4252bf..82ba561eefb16 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -121,10 +121,23 @@ private[ui] class StageTableBase(
    {stageDesc.getOrElse("")} {killLink} {nameLink} {details}
    } + protected def missingStageRow(stageId: Int): Seq[Node] = { + {stageId} ++ + {if (isFairScheduler) {-} else Seq.empty} ++ + No data available for this stage ++ // Description + ++ // Submitted + ++ // Duration + ++ // Tasks: Succeeded/Total + ++ // Input + ++ // Output + ++ // Shuffle Read + // Shuffle Write + } + protected def stageRow(s: StageInfo): Seq[Node] = { val stageDataOption = listener.stageIdToData.get((s.stageId, s.attemptId)) if (stageDataOption.isEmpty) { - return {s.stageId}No data available for this stage + return missingStageRow(s.stageId) } val stageData = stageDataOption.get From 984787526625b4ef8a1635faf7a5ac3cb0b758b7 Mon Sep 17 00:00:00 2001 From: linweizhong Date: Tue, 12 May 2015 01:36:27 -0700 Subject: [PATCH 30/59] [MINOR] [PYSPARK] Set PYTHONPATH to python/lib/pyspark.zip rather than python/pyspark As PR #5580 we have created pyspark.zip on building and set PYTHONPATH to python/lib/pyspark.zip, so to keep consistence update this. Author: linweizhong Closes #6047 from Sephiroth-Lin/pyspark_pythonpath and squashes the following commits: 8cc3d96 [linweizhong] Set PYTHONPATH to python/lib/pyspark.zip rather than python/pyspark as PR#5580 we have create pyspark.zip on build --- .../main/scala/org/apache/spark/api/python/PythonUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index acbaba6791850..9eff0a230cbe2 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -31,7 +31,7 @@ private[spark] object PythonUtils { def sparkPythonPath: String = { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { - pythonPath += Seq(sparkHome, "python").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.2.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) From 82e890fb19d6fbaffa69856eecb4699f2f8a81eb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 12 May 2015 01:39:21 -0700 Subject: [PATCH 31/59] [SPARK-7485] [BUILD] Remove pyspark files from assembly. The sbt part of the build is hacky; it basically tricks sbt into generating the zip by using a generator, but returns an empty list for the generated files so that nothing is actually added to the assembly. Author: Marcelo Vanzin Closes #6022 from vanzin/SPARK-7485 and squashes the following commits: 22c1e04 [Marcelo Vanzin] Remove unneeded code. 4893622 [Marcelo Vanzin] [SPARK-7485] [build] Remove pyspark files from assembly. --- core/pom.xml | 47 ---------------------------------------- mllib/pom.xml | 11 ---------- project/SparkBuild.scala | 44 +++---------------------------------- sql/core/pom.xml | 8 ------- streaming/pom.xml | 8 ------- 5 files changed, 3 insertions(+), 115 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index fc42f48973fe9..262a3320db106 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -381,35 +381,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-antrun-plugin - - - generate-resources - - run - - - - - - - - - - - maven-clean-plugin - - - - ${basedir}/../python/build - - - true - - org.apache.maven.plugins maven-dependency-plugin @@ -438,24 +409,6 @@ - - - - src/main/resources - - - ../python - - pyspark/*.py - - - - ../python/build - - py4j/*.py - - - diff --git a/mllib/pom.xml b/mllib/pom.xml index a3c57ae26000b..0c07ca1a62fd3 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -141,16 +141,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - ../python - - pyspark/mllib/*.py - pyspark/mllib/stat/*.py - pyspark/ml/*.py - pyspark/ml/param/*.py - - - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 186345af0e60e..1b87e4e98bd83 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -168,7 +168,7 @@ object SparkBuild extends PomBuild { /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) - /* Package pyspark artifacts in the main assembly. */ + /* Package pyspark artifacts in a separate zip file for YARN. */ enable(PySparkAssembly.settings)(assembly) /* Enable unidoc only for the root spark project */ @@ -373,22 +373,15 @@ object PySparkAssembly { import java.util.zip.{ZipOutputStream, ZipEntry} lazy val settings = Seq( - unmanagedJars in Compile += { BuildCommons.sparkHome / "python/lib/py4j-0.8.2.1-src.zip" }, // Use a resource generator to copy all .py files from python/pyspark into a managed directory // to be included in the assembly. We can't just add "python/" to the assembly's resource dir // list since that will copy unneeded / unwanted files. resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File => val src = new File(BuildCommons.sparkHome, "python/pyspark") - val zipFile = new File(BuildCommons.sparkHome , "python/lib/pyspark.zip") zipFile.delete() zipRecursive(src, zipFile) - - val dst = new File(outDir, "pyspark") - if (!dst.isDirectory()) { - require(dst.mkdirs()) - } - copy(src, dst) + Seq[File]() } ) @@ -416,42 +409,11 @@ object PySparkAssembly { output.write(buf, 0, n) } } + output.closeEntry() in.close() } } - private def copy(src: File, dst: File): Seq[File] = { - src.listFiles().flatMap { f => - val child = new File(dst, f.getName()) - if (f.isDirectory()) { - child.mkdir() - copy(f, child) - } else if (f.getName().endsWith(".py")) { - var in: Option[FileInputStream] = None - var out: Option[FileOutputStream] = None - try { - in = Some(new FileInputStream(f)) - out = Some(new FileOutputStream(child)) - - val bytes = new Array[Byte](1024) - var read = 0 - while (read >= 0) { - read = in.get.read(bytes) - if (read > 0) { - out.get.write(bytes, 0, read) - } - } - - Some(child) - } finally { - in.foreach(_.close()) - out.foreach(_.close()) - } - } else { - None - } - } - } } object Unidoc { diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 7d274a73e079f..ffe95bb49188f 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -103,13 +103,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - ../../python - - pyspark/sql/*.py - - - diff --git a/streaming/pom.xml b/streaming/pom.xml index 5ca55a4f680bb..5ab7f4472c38b 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -105,13 +105,5 @@ - - - ../python - - pyspark/streaming/*.py - - - From f3e8e60063ccf0d713d03e671a3231560475f90d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 May 2015 01:40:55 -0700 Subject: [PATCH 32/59] [SPARK-7467] Dag visualization: treat checkpoint as an RDD operation Such that a checkpoint RDD does not go into random scopes on the UI, e.g. `take`. We've seen this in streaming. Author: Andrew Or Closes #6004 from andrewor14/dag-viz-checkpoint and squashes the following commits: 9217439 [Andrew Or] Fix checkpoints 4ae8806 [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-viz-checkpoint 19bc07b [Andrew Or] Treat checkpoint as an RDD operation --- .../main/scala/org/apache/spark/rdd/RDD.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) 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 7dad30ecbdd2f..02a94baf372d9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1523,13 +1523,15 @@ abstract class RDD[T: ClassTag]( * has completed (therefore the RDD has been materialized and potentially stored in memory). * doCheckpoint() is called recursively on the parent RDDs. */ - private[spark] def doCheckpoint() { - if (!doCheckpointCalled) { - doCheckpointCalled = true - if (checkpointData.isDefined) { - checkpointData.get.doCheckpoint() - } else { - dependencies.foreach(_.rdd.doCheckpoint()) + private[spark] def doCheckpoint(): Unit = { + RDDOperationScope.withScope(sc, "checkpoint", false, true) { + if (!doCheckpointCalled) { + doCheckpointCalled = true + if (checkpointData.isDefined) { + checkpointData.get.doCheckpoint() + } else { + dependencies.foreach(_.rdd.doCheckpoint()) + } } } } From ec6f2a9774167014566fb9608ee4394d2ce5fd6a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 12 May 2015 08:48:24 -0700 Subject: [PATCH 33/59] [SPARK-7532] [STREAMING] StreamingContext.start() made to logWarning and not throw exception Author: Tathagata Das Closes #6060 from tdas/SPARK-7532 and squashes the following commits: 6fe2e83 [Tathagata Das] Update docs 7dadfc3 [Tathagata Das] Fixed bug again 99c7678 [Tathagata Das] Added logInfo 65aec20 [Tathagata Das] Fix bug 5bf031b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-7532 1a9a818 [Tathagata Das] Fix scaladoc c584313 [Tathagata Das] StreamingContext.start() made to logWarning and not throw exception --- .../spark/streaming/StreamingContext.scala | 27 +++++++++---------- .../streaming/StreamingContextSuite.scala | 4 +-- 2 files changed, 14 insertions(+), 17 deletions(-) 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 2c5834defa8c8..8461e901208ff 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -528,28 +528,27 @@ class StreamingContext private[streaming] ( /** * Start the execution of the streams. * - * @throws SparkException if the context has already been started or stopped. + * @throws SparkException if the StreamingContext is already stopped. */ def start(): Unit = synchronized { - import StreamingContext._ state match { case INITIALIZED => - // good to start + validate() + startSite.set(DStream.getCreationSite()) + sparkContext.setCallSite(startSite.get) + StreamingContext.ACTIVATION_LOCK.synchronized { + StreamingContext.assertNoOtherContextIsActive() + scheduler.start() + uiTab.foreach(_.attach()) + state = StreamingContextState.ACTIVE + StreamingContext.setActiveContext(this) + } + logInfo("StreamingContext started") case ACTIVE => - throw new SparkException("StreamingContext has already been started") + logWarning("StreamingContext has already been started") case STOPPED => throw new SparkException("StreamingContext has already been stopped") } - validate() - startSite.set(DStream.getCreationSite()) - sparkContext.setCallSite(startSite.get) - ACTIVATION_LOCK.synchronized { - assertNoOtherContextIsActive() - scheduler.start() - uiTab.foreach(_.attach()) - state = StreamingContextState.ACTIVE - setActiveContext(this) - } } /** 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 b8247db7e8950..47299513de095 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -136,9 +136,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w addInputStream(ssc).register() ssc.start() assert(ssc.getState() === StreamingContextState.ACTIVE) - intercept[SparkException] { - ssc.start() - } + ssc.start() assert(ssc.getState() === StreamingContextState.ACTIVE) } From d86ce845840a92b4dde7975082738ed94ab8c570 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 12 May 2015 10:23:41 -0700 Subject: [PATCH 34/59] [SPARK-6876] [PySpark] [SQL] add DataFrame na.replace in pyspark Author: Daoyuan Wang Closes #6003 from adrian-wang/pynareplace and squashes the following commits: 672efba [Daoyuan Wang] remove py2.7 feature 4a148f7 [Daoyuan Wang] to_replace support dict, value support single value, and add full tests 9e232e7 [Daoyuan Wang] rename scala map af0268a [Daoyuan Wang] remove na 63ac579 [Daoyuan Wang] add na.replace in pyspark --- .../apache/spark/api/python/PythonUtils.scala | 7 ++ python/pyspark/sql/dataframe.py | 85 +++++++++++++++++++ python/pyspark/sql/tests.py | 48 +++++++++++ 3 files changed, 140 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 9eff0a230cbe2..efb6b93cfc35d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -53,4 +53,11 @@ private[spark] object PythonUtils { def toSeq[T](cols: JList[T]): Seq[T] = { cols.toList.toSeq } + + /** + * Convert java map of K, V into Map of K, V (for calling API with varargs) + */ + def toScalaMap[K, V](jm: java.util.Map[K, V]): Map[K, V] = { + jm.toMap + } } diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 72180f6d05fbc..078acfdf7e2df 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -578,6 +578,10 @@ def _jseq(self, cols, converter=None): """Return a JVM Seq of Columns from a list of Column or names""" return _to_seq(self.sql_ctx._sc, cols, converter) + def _jmap(self, jm): + """Return a JVM Scala Map from a dict""" + return _to_scala_map(self.sql_ctx._sc, jm) + def _jcols(self, *cols): """Return a JVM Seq of Columns from a list of Column or column names @@ -924,6 +928,80 @@ def fillna(self, value, subset=None): return DataFrame(self._jdf.na().fill(value, self._jseq(subset)), self.sql_ctx) + def replace(self, to_replace, value, subset=None): + """Returns a new :class:`DataFrame` replacing a value with another value. + + :param to_replace: int, long, float, string, or list. + Value to be replaced. + If the value is a dict, then `value` is ignored and `to_replace` must be a + mapping from column name (string) to replacement value. The value to be + replaced must be an int, long, float, or string. + :param value: int, long, float, string, or list. + Value to use to replace holes. + The replacement value must be an int, long, float, or string. If `value` is a + list or tuple, `value` should be of the same length with `to_replace`. + :param subset: optional list of column names to consider. + Columns specified in subset that do not have matching data type are ignored. + For example, if `value` is a string, and subset contains a non-string column, + then the non-string column is simply ignored. + >>> df4.replace(10, 20).show() + +----+------+-----+ + | age|height| name| + +----+------+-----+ + | 20| 80|Alice| + | 5| null| Bob| + |null| null| Tom| + |null| null| null| + +----+------+-----+ + + >>> df4.replace(['Alice', 'Bob'], ['A', 'B'], 'name').show() + +----+------+----+ + | age|height|name| + +----+------+----+ + | 10| 80| A| + | 5| null| B| + |null| null| Tom| + |null| null|null| + +----+------+----+ + """ + if not isinstance(to_replace, (float, int, long, basestring, list, tuple, dict)): + raise ValueError( + "to_replace should be a float, int, long, string, list, tuple, or dict") + + if not isinstance(value, (float, int, long, basestring, list, tuple)): + raise ValueError("value should be a float, int, long, string, list, or tuple") + + rep_dict = dict() + + if isinstance(to_replace, (float, int, long, basestring)): + to_replace = [to_replace] + + if isinstance(to_replace, tuple): + to_replace = list(to_replace) + + if isinstance(value, tuple): + value = list(value) + + if isinstance(to_replace, list) and isinstance(value, list): + if len(to_replace) != len(value): + raise ValueError("to_replace and value lists should be of the same length") + rep_dict = dict(zip(to_replace, value)) + elif isinstance(to_replace, list) and isinstance(value, (float, int, long, basestring)): + rep_dict = dict([(tr, value) for tr in to_replace]) + elif isinstance(to_replace, dict): + rep_dict = to_replace + + if subset is None: + return DataFrame(self._jdf.na().replace('*', rep_dict), self.sql_ctx) + elif isinstance(subset, basestring): + subset = [subset] + + if not isinstance(subset, (list, tuple)): + raise ValueError("subset should be a list or tuple of column names") + + return DataFrame( + self._jdf.na().replace(self._jseq(subset), self._jmap(rep_dict)), self.sql_ctx) + def corr(self, col1, col2, method=None): """ Calculates the correlation of two columns of a DataFrame as a double value. Currently only @@ -1226,6 +1304,13 @@ def _to_seq(sc, cols, converter=None): return sc._jvm.PythonUtils.toSeq(cols) +def _to_scala_map(sc, jm): + """ + Convert a dict into a JVM Map. + """ + return sc._jvm.PythonUtils.toScalaMap(jm) + + def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 7e63f4d6461f6..1922d03af61da 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -665,6 +665,54 @@ def test_bitwise_operations(self): result = df.select(functions.bitwiseNOT(df.b)).collect()[0].asDict() self.assertEqual(~75, result['~b']) + def test_replace(self): + schema = StructType([ + StructField("name", StringType(), True), + StructField("age", IntegerType(), True), + StructField("height", DoubleType(), True)]) + + # replace with int + row = self.sqlCtx.createDataFrame([(u'Alice', 10, 10.0)], schema).replace(10, 20).first() + self.assertEqual(row.age, 20) + self.assertEqual(row.height, 20.0) + + # replace with double + row = self.sqlCtx.createDataFrame( + [(u'Alice', 80, 80.0)], schema).replace(80.0, 82.1).first() + self.assertEqual(row.age, 82) + self.assertEqual(row.height, 82.1) + + # replace with string + row = self.sqlCtx.createDataFrame( + [(u'Alice', 10, 80.1)], schema).replace(u'Alice', u'Ann').first() + self.assertEqual(row.name, u"Ann") + self.assertEqual(row.age, 10) + + # replace with subset specified by a string of a column name w/ actual change + row = self.sqlCtx.createDataFrame( + [(u'Alice', 10, 80.1)], schema).replace(10, 20, subset='age').first() + self.assertEqual(row.age, 20) + + # replace with subset specified by a string of a column name w/o actual change + row = self.sqlCtx.createDataFrame( + [(u'Alice', 10, 80.1)], schema).replace(10, 20, subset='height').first() + self.assertEqual(row.age, 10) + + # replace with subset specified with one column replaced, another column not in subset + # stays unchanged. + row = self.sqlCtx.createDataFrame( + [(u'Alice', 10, 10.0)], schema).replace(10, 20, subset=['name', 'age']).first() + self.assertEqual(row.name, u'Alice') + self.assertEqual(row.age, 20) + self.assertEqual(row.height, 10.0) + + # replace with subset specified but no column will be replaced + row = self.sqlCtx.createDataFrame( + [(u'Alice', 10, None)], schema).replace(10, 20, subset=['name', 'height']).first() + self.assertEqual(row.name, u'Alice') + self.assertEqual(row.age, 10) + self.assertEqual(row.height, None) + class HiveContextSQLTests(ReusedPySparkTestCase): From 831504cf6bde6b1131005d5552e56a842725c84c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 12 May 2015 10:28:40 -0700 Subject: [PATCH 35/59] [DataFrame][minor] cleanup unapply methods in DataTypes Author: Wenchen Fan Closes #6079 from cloud-fan/unapply and squashes the following commits: 40da442 [Wenchen Fan] one more 7d90a05 [Wenchen Fan] cleanup unapply in DataTypes --- .../org/apache/spark/sql/types/DataType.scala | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 0992a7c311ee2..a0b261649f66f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -47,10 +47,7 @@ abstract class DataType { * ... * }}} */ - private[sql] def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType == this => true - case _ => false - } + private[sql] def unapply(e: Expression): Boolean = e.dataType == this /** * The default size of a value of this data type, used internally for size estimation. @@ -137,10 +134,7 @@ private[sql] object IntegralType { * ... * }}} */ - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType.isInstanceOf[IntegralType] => true - case _ => false - } + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[IntegralType] } @@ -157,10 +151,7 @@ private[sql] object FractionalType { * ... * }}} */ - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType.isInstanceOf[FractionalType] => true - case _ => false - } + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[FractionalType] } From 0595b6de8f1da04baceda082553c2aa1aa2cb006 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 13 May 2015 01:32:28 +0800 Subject: [PATCH 36/59] [SPARK-3928] [SPARK-5182] [SQL] Partitioning support for the data sources API This PR adds partitioning support for the external data sources API. It aims to simplify development of file system based data sources, and provide first class partitioning support for both read path and write path. Existing data sources like JSON and Parquet can be simplified with this work. ## New features provided 1. Hive compatible partition discovery This actually generalizes the partition discovery strategy used in Parquet data source in Spark 1.3.0. 1. Generalized partition pruning optimization Now partition pruning is handled during physical planning phase. Specific data sources don't need to worry about this harness anymore. (This also implies that we can remove `CatalystScan` after migrating the Parquet data source, since now we don't need to pass Catalyst expressions to data source implementations.) 1. Insertion with dynamic partitions When inserting data to a `FSBasedRelation`, data can be partitioned dynamically by specified partition columns. ## New structures provided ### Developer API 1. `FSBasedRelation` Base abstract class for file system based data sources. 1. `OutputWriter` Base abstract class for output row writers, responsible for writing a single row object. 1. `FSBasedRelationProvider` A new relation provider for `FSBasedRelation` subclasses. Note that data sources extending `FSBasedRelation` don't need to extend `RelationProvider` and `SchemaRelationProvider`. ### User API New overloaded versions of 1. `DataFrame.save()` 1. `DataFrame.saveAsTable()` 1. `SQLContext.load()` are provided to allow users to save/load DataFrames with user defined dynamic partition columns. ### Spark SQL query planning 1. `InsertIntoFSBasedRelation` Used to implement write path for `FSBasedRelation`s. 1. New rules for `FSBasedRelation` in `DataSourceStrategy` These are added to hook `FSBasedRelation` into physical query plan in read path, and perform partition pruning. ## TODO - [ ] Use scratch directories when overwriting a table with data selected from itself. Currently, this is not supported, because the table been overwritten is always deleted before writing any data to it. - [ ] When inserting with dynamic partition columns, use external sorter to group the data first. This ensures that we only need to open a single `OutputWriter` at a time. For data sources like Parquet, `OutputWriter`s can be quite memory consuming. One issue is that, this approach breaks the row distribution in the original DataFrame. However, we did't promise to preserve data distribution when writing a DataFrame. - [x] More tests. Specifically, test cases for - [x] Self-join - [x] Loading partitioned relations with a subset of partition columns stored in data files. - [x] `SQLContext.load()` with user defined dynamic partition columns. ## Parquet data source migration Parquet data source migration is covered in PR https://github.com/liancheng/spark/pull/6, which is against this PR branch and for preview only. A formal PR need to be made after this one is merged. Author: Cheng Lian Closes #5526 from liancheng/partitioning-support and squashes the following commits: 5351a1b [Cheng Lian] Fixes compilation error introduced while rebasing 1f9b1a5 [Cheng Lian] Tweaks data schema passed to FSBasedRelations 43ba50e [Cheng Lian] Avoids serializing generated projection code edf49e7 [Cheng Lian] Removed commented stale code block 348a922 [Cheng Lian] Adds projection in FSBasedRelation.buildScan(requiredColumns, inputPaths) ad4d4de [Cheng Lian] Enables HDFS style globbing 8d12e69 [Cheng Lian] Fixes compilation error c71ac6c [Cheng Lian] Addresses comments from @marmbrus 7552168 [Cheng Lian] Fixes typo in MimaExclude.scala 0349e09 [Cheng Lian] Fixes compilation error introduced while rebasing 52b0c9b [Cheng Lian] Adjusts project/MimaExclude.scala c466de6 [Cheng Lian] Addresses comments bc3f9b4 [Cheng Lian] Uses projection to separate partition columns and data columns while inserting rows 795920a [Cheng Lian] Fixes compilation error after rebasing 0b8cd70 [Cheng Lian] Adds Scala/Catalyst row conversion when writing non-partitioned tables fa543f3 [Cheng Lian] Addresses comments 5849dd0 [Cheng Lian] Fixes doc typos. Fixes partition discovery refresh. 51be443 [Cheng Lian] Replaces FSBasedRelation.outputCommitterClass with FSBasedRelation.prepareForWrite c4ed4fe [Cheng Lian] Bug fixes and a new test suite a29e663 [Cheng Lian] Bug fix: should only pass actuall data files to FSBaseRelation.buildScan 5f423d3 [Cheng Lian] Bug fixes. Lets data source to customize OutputCommitter rather than OutputFormat 54c3d7b [Cheng Lian] Enforces that FileOutputFormat must be used be0c268 [Cheng Lian] Uses TaskAttempContext rather than Configuration in OutputWriter.init 0bc6ad1 [Cheng Lian] Resorts to new Hadoop API, and now FSBasedRelation can customize output format class f320766 [Cheng Lian] Adds prepareForWrite() hook, refactored writer containers 422ff4a [Cheng Lian] Fixes style issue ce52353 [Cheng Lian] Adds new SQLContext.load() overload with user defined dynamic partition columns 8d2ff71 [Cheng Lian] Merges partition columns when reading partitioned relations ca1805b [Cheng Lian] Removes duplicated partition discovery code in new Parquet f18dec2 [Cheng Lian] More strict schema checking b746ab5 [Cheng Lian] More tests 9b487bf [Cheng Lian] Fixes compilation errors introduced while rebasing ea6c8dd [Cheng Lian] Removes remote debugging stuff 327bb1d [Cheng Lian] Implements partitioning support for data sources API 3c5073a [Cheng Lian] Fixes SaveModes used in test cases fb5a607 [Cheng Lian] Fixes compilation error 9d17607 [Cheng Lian] Adds the contract that OutputWriter should have zero-arg constructor 5de194a [Cheng Lian] Forgot Apache licence header 95d0b4d [Cheng Lian] Renames PartitionedSchemaRelationProvider to FSBasedRelationProvider 770b5ba [Cheng Lian] Adds tests for FSBasedRelation 3ba9bbf [Cheng Lian] Adds DataFrame.saveAsTable() overrides which support partitioning 1b8231f [Cheng Lian] Renames FSBasedPrunedFilteredScan to FSBasedRelation aa8ba9a [Cheng Lian] Javadoc fix 012ed2d [Cheng Lian] Adds PartitioningOptions 7dd8dd5 [Cheng Lian] Adds new interfaces and stub methods for data sources API partitioning support --- .../apache/spark/deploy/SparkHadoopUtil.scala | 52 +- project/MimaExcludes.scala | 14 +- .../org/apache/spark/sql/DataFrame.scala | 107 +++- .../scala/org/apache/spark/sql/SQLConf.scala | 6 + .../org/apache/spark/sql/SQLContext.scala | 36 +- .../spark/sql/execution/SparkStrategies.scala | 10 +- .../apache/spark/sql/parquet/newParquet.scala | 182 +----- .../sql/sources/DataSourceStrategy.scala | 191 ++++++- .../spark/sql/sources/PartitioningUtils.scala | 207 +++++++ .../apache/spark/sql/sources/commands.scala | 406 +++++++++++++- .../org/apache/spark/sql/sources/ddl.scala | 90 ++- .../apache/spark/sql/sources/interfaces.scala | 283 +++++++++- .../org/apache/spark/sql/sources/rules.scala | 8 +- .../ParquetPartitionDiscoverySuite.scala | 3 +- .../sources/CreateTableAsSelectSuite.scala | 4 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 13 +- .../spark/sql/hive/HiveStrategies.scala | 6 +- .../spark/sql/hive/execution/commands.scala | 27 +- .../spark/sql/hive/hiveWriterContainers.scala | 2 +- .../sql/sources/FSBasedRelationSuite.scala | 525 ++++++++++++++++++ .../sql/sources/SimpleTextRelation.scala | 125 +++++ 21 files changed, 2042 insertions(+), 255 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index b563034457a91..7fa75ac8c2b54 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -22,22 +22,22 @@ import java.lang.reflect.Method import java.security.PrivilegedExceptionAction import java.util.{Arrays, Comparator} +import scala.collection.JavaConversions._ +import scala.concurrent.duration._ +import scala.language.postfixOps + import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.hadoop.fs.FileSystem.Statistics +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils - -import scala.collection.JavaConversions._ -import scala.concurrent.duration._ -import scala.language.postfixOps +import org.apache.spark.{Logging, SparkConf, SparkException} /** * :: DeveloperApi :: @@ -199,13 +199,43 @@ class SparkHadoopUtil extends Logging { * that file. */ def listLeafStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = { - def recurse(path: Path): Array[FileStatus] = { - val (directories, leaves) = fs.listStatus(path).partition(_.isDir) - leaves ++ directories.flatMap(f => listLeafStatuses(fs, f.getPath)) + listLeafStatuses(fs, fs.getFileStatus(basePath)) + } + + /** + * Get [[FileStatus]] objects for all leaf children (files) under the given base path. If the + * given path points to a file, return a single-element collection containing [[FileStatus]] of + * that file. + */ + def listLeafStatuses(fs: FileSystem, baseStatus: FileStatus): Seq[FileStatus] = { + def recurse(status: FileStatus): Seq[FileStatus] = { + val (directories, leaves) = fs.listStatus(status.getPath).partition(_.isDir) + leaves ++ directories.flatMap(f => listLeafStatuses(fs, f)) } - val baseStatus = fs.getFileStatus(basePath) - if (baseStatus.isDir) recurse(basePath) else Array(baseStatus) + if (baseStatus.isDir) recurse(baseStatus) else Seq(baseStatus) + } + + def listLeafDirStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = { + listLeafDirStatuses(fs, fs.getFileStatus(basePath)) + } + + def listLeafDirStatuses(fs: FileSystem, baseStatus: FileStatus): Seq[FileStatus] = { + def recurse(status: FileStatus): Seq[FileStatus] = { + val (directories, files) = fs.listStatus(status.getPath).partition(_.isDir) + val leaves = if (directories.isEmpty) Seq(status) else Seq.empty[FileStatus] + leaves ++ directories.flatMap(dir => listLeafDirStatuses(fs, dir)) + } + + assert(baseStatus.isDir) + recurse(baseStatus) + } + + def globPath(pattern: Path): Seq[Path] = { + val fs = pattern.getFileSystem(conf) + Option(fs.globStatus(pattern)).map { statuses => + statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toSeq + }.getOrElse(Seq.empty[Path]) } /** diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ad3d8426bdedd..a47e29e2ef365 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -94,11 +94,23 @@ object MimaExcludes { // This `protected[sql]` method was removed in 1.3.1 ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.sql.SQLContext.checkAnalysis"), - // This `private[sql]` class was removed in 1.4.0: + // These `private[sql]` class were removed in 1.4.0: ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.execution.AddExchange"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.execution.AddExchange$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.PartitionSpec"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.PartitionSpec$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.Partition"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.Partition$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.ParquetRelation2$PartitionValues"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.ParquetRelation2$PartitionValues$"), // These test support classes were moved out of src/main and into src/test: ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.parquet.ParquetTestData"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 265a61592b943..f3107f7b51ad8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -27,23 +27,23 @@ import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import com.fasterxml.jackson.core.JsonFactory - import org.apache.commons.lang3.StringUtils + import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, ResolvedStar} +import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedAttribute, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} -import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, _} +import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} +import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, ScalaReflection, SqlParser} import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.json.JacksonGenerator +import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, ResolvedDataSource} import org.apache.spark.sql.types._ -import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -400,7 +400,9 @@ class DataFrame private[sql]( joined.left, joined.right, joinType = Inner, - Some(EqualTo(joined.left.resolve(usingColumn), joined.right.resolve(usingColumn)))) + Some(expressions.EqualTo( + joined.left.resolve(usingColumn), + joined.right.resolve(usingColumn)))) ) } @@ -465,8 +467,8 @@ class DataFrame private[sql]( // By the time we get here, since we have already run analysis, all attributes should've been // resolved and become AttributeReference. val cond = plan.condition.map { _.transform { - case EqualTo(a: AttributeReference, b: AttributeReference) if a.sameRef(b) => - EqualTo(plan.left.resolve(a.name), plan.right.resolve(b.name)) + case expressions.EqualTo(a: AttributeReference, b: AttributeReference) if a.sameRef(b) => + expressions.EqualTo(plan.left.resolve(a.name), plan.right.resolve(b.name)) }} plan.copy(condition = cond) } @@ -1324,6 +1326,28 @@ class DataFrame private[sql]( saveAsTable(tableName, source, mode, options.toMap) } + /** + * :: Experimental :: + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source, [[SaveMode]] specified by mode, a set of options, and a list of + * partition columns. + * + * Note that this currently only works with DataFrames that are created from a HiveContext as + * there is no notion of a persisted catalog in a standard SQL context. Instead you can write + * an RDD out to a parquet file, and then register that file as a table. This "table" can then + * be the target of an `insertInto`. + * @group output + */ + @Experimental + def saveAsTable( + tableName: String, + source: String, + mode: SaveMode, + options: java.util.Map[String, String], + partitionColumns: java.util.List[String]): Unit = { + saveAsTable(tableName, source, mode, options.toMap, partitionColumns) + } + /** * :: Experimental :: * (Scala-specific) @@ -1350,6 +1374,7 @@ class DataFrame private[sql]( tableName, source, temporary = false, + Array.empty[String], mode, options, logicalPlan) @@ -1357,6 +1382,36 @@ class DataFrame private[sql]( sqlContext.executePlan(cmd).toRdd } + /** + * :: Experimental :: + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source, [[SaveMode]] specified by mode, a set of options, and a list of + * partition columns. + * + * Note that this currently only works with DataFrames that are created from a HiveContext as + * there is no notion of a persisted catalog in a standard SQL context. Instead you can write + * an RDD out to a parquet file, and then register that file as a table. This "table" can then + * be the target of an `insertInto`. + * @group output + */ + @Experimental + def saveAsTable( + tableName: String, + source: String, + mode: SaveMode, + options: Map[String, String], + partitionColumns: Seq[String]): Unit = { + sqlContext.executePlan( + CreateTableUsingAsSelect( + tableName, + source, + temporary = false, + partitionColumns.toArray, + mode, + options, + logicalPlan)).toRdd + } + /** * :: Experimental :: * Saves the contents of this DataFrame to the given path, @@ -1417,6 +1472,21 @@ class DataFrame private[sql]( save(source, mode, options.toMap) } + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path based on the given data source, + * [[SaveMode]] specified by mode, and partition columns specified by `partitionColumns`. + * @group output + */ + @Experimental + def save( + source: String, + mode: SaveMode, + options: java.util.Map[String, String], + partitionColumns: java.util.List[String]): Unit = { + save(source, mode, options.toMap, partitionColumns) + } + /** * :: Experimental :: * (Scala-specific) @@ -1429,7 +1499,22 @@ class DataFrame private[sql]( source: String, mode: SaveMode, options: Map[String, String]): Unit = { - ResolvedDataSource(sqlContext, source, mode, options, this) + ResolvedDataSource(sqlContext, source, Array.empty[String], mode, options, this) + } + + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path based on the given data source, + * [[SaveMode]] specified by mode, and partition columns specified by `partitionColumns`. + * @group output + */ + @Experimental + def save( + source: String, + mode: SaveMode, + options: Map[String, String], + partitionColumns: Seq[String]): Unit = { + ResolvedDataSource(sqlContext, source, partitionColumns.toArray, mode, options, this) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index dcac97beafb04..f07bb196c11ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -66,6 +66,9 @@ private[spark] object SQLConf { // to its length exceeds the threshold. val SCHEMA_STRING_LENGTH_THRESHOLD = "spark.sql.sources.schemaStringLengthThreshold" + // Whether to perform partition discovery when loading external data sources. Default to true. + val PARTITION_DISCOVERY_ENABLED = "spark.sql.sources.partitionDiscovery.enabled" + // Whether to perform eager analysis when constructing a dataframe. // Set to false when debugging requires the ability to look at invalid query plans. val DATAFRAME_EAGER_ANALYSIS = "spark.sql.eagerAnalysis" @@ -235,6 +238,9 @@ private[sql] class SQLConf extends Serializable with CatalystConf { private[spark] def defaultDataSourceName: String = getConf(DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.parquet") + private[spark] def partitionDiscoveryEnabled() = + getConf(SQLConf.PARTITION_DISCOVERY_ENABLED, "true").toBoolean + // Do not use a value larger than 4000 as the default value of this property. // See the comments of SCHEMA_STRING_LENGTH_THRESHOLD above for more information. private[spark] def schemaStringLengthThreshold: Int = 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 648021806f3ee..afee09adaae20 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 @@ -762,7 +762,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def load(source: String, options: Map[String, String]): DataFrame = { - val resolved = ResolvedDataSource(this, None, source, options) + val resolved = ResolvedDataSource(this, None, Array.empty[String], source, options) DataFrame(this, LogicalRelation(resolved.relation)) } @@ -781,6 +781,37 @@ class SQLContext(@transient val sparkContext: SparkContext) load(source, schema, options.toMap) } + /** + * :: Experimental :: + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + */ + @Experimental + def load( + source: String, + schema: StructType, + partitionColumns: Array[String], + options: java.util.Map[String, String]): DataFrame = { + load(source, schema, partitionColumns, options.toMap) + } + + /** + * :: Experimental :: + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * @group genericdata + */ + @Experimental + def load( + source: String, + schema: StructType, + options: Map[String, String]): DataFrame = { + val resolved = ResolvedDataSource(this, Some(schema), Array.empty[String], source, options) + DataFrame(this, LogicalRelation(resolved.relation)) + } + /** * :: Experimental :: * (Scala-specific) Returns the dataset specified by the given data source and @@ -791,8 +822,9 @@ class SQLContext(@transient val sparkContext: SparkContext) def load( source: String, schema: StructType, + partitionColumns: Array[String], options: Map[String, String]): DataFrame = { - val resolved = ResolvedDataSource(this, Some(schema), source, options) + val resolved = ResolvedDataSource(this, Some(schema), partitionColumns, source, options) DataFrame(this, LogicalRelation(resolved.relation)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 56a4689eb58f0..af0029cb84f9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -343,9 +343,13 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case c: CreateTableUsing if c.temporary && c.allowExisting => sys.error("allowExisting should be set to false when creating a temporary table.") - case CreateTableUsingAsSelect(tableName, provider, true, mode, opts, query) => - val cmd = - CreateTempTableUsingAsSelect(tableName, provider, mode, opts, query) + case CreateTableUsingAsSelect(tableName, provider, true, partitionsCols, mode, opts, query) + if partitionsCols.nonEmpty => + sys.error("Cannot create temporary partitioned table.") + + case CreateTableUsingAsSelect(tableName, provider, true, _, mode, opts, query) => + val cmd = CreateTempTableUsingAsSelect( + tableName, provider, Array.empty[String], mode, opts, query) ExecutedCommand(cmd) :: Nil case c: CreateTableUsingAsSelect if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") 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 85e60733bc57a..ee4b1c72a2148 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 @@ -136,10 +136,6 @@ private[sql] class DefaultSource } } -private[sql] case class Partition(values: Row, path: String) - -private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) - /** * An alternative to [[ParquetRelation]] that plugs in using the data sources API. This class is * intended as a full replacement of the Parquet support in Spark SQL. The old implementation will @@ -307,7 +303,7 @@ private[sql] case class ParquetRelation2( if (partitionDirs.nonEmpty) { // Parses names and values of partition columns, and infer their data types. - ParquetRelation2.parsePartitions(partitionDirs, defaultPartitionName) + PartitioningUtils.parsePartitions(partitionDirs, defaultPartitionName) } else { // No partition directories found, makes an empty specification PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[Partition]) @@ -805,7 +801,7 @@ private[sql] object ParquetRelation2 extends Logging { val ordinalMap = metastoreSchema.zipWithIndex.map { case (field, index) => field.name.toLowerCase -> index }.toMap - val reorderedParquetSchema = mergedParquetSchema.sortBy(f => + val reorderedParquetSchema = mergedParquetSchema.sortBy(f => ordinalMap.getOrElse(f.name.toLowerCase, metastoreSchema.size + 1)) StructType(metastoreSchema.zip(reorderedParquetSchema).map { @@ -841,178 +837,4 @@ private[sql] object ParquetRelation2 extends Logging { .filter(_.nullable) StructType(parquetSchema ++ missingFields) } - - - // TODO Data source implementations shouldn't touch Catalyst types (`Literal`). - // However, we are already using Catalyst expressions for partition pruning and predicate - // push-down here... - private[parquet] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { - require(columnNames.size == literals.size) - } - - /** - * Given a group of qualified paths, tries to parse them and returns a partition specification. - * For example, given: - * {{{ - * hdfs://:/path/to/partition/a=1/b=hello/c=3.14 - * hdfs://:/path/to/partition/a=2/b=world/c=6.28 - * }}} - * it returns: - * {{{ - * PartitionSpec( - * partitionColumns = StructType( - * StructField(name = "a", dataType = IntegerType, nullable = true), - * StructField(name = "b", dataType = StringType, nullable = true), - * StructField(name = "c", dataType = DoubleType, nullable = true)), - * partitions = Seq( - * Partition( - * values = Row(1, "hello", 3.14), - * path = "hdfs://:/path/to/partition/a=1/b=hello/c=3.14"), - * Partition( - * values = Row(2, "world", 6.28), - * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28"))) - * }}} - */ - private[parquet] def parsePartitions( - paths: Seq[Path], - defaultPartitionName: String): PartitionSpec = { - val partitionValues = resolvePartitions(paths.map(parsePartition(_, defaultPartitionName))) - val fields = { - val (PartitionValues(columnNames, literals)) = partitionValues.head - columnNames.zip(literals).map { case (name, Literal(_, dataType)) => - StructField(name, dataType, nullable = true) - } - } - - val partitions = partitionValues.zip(paths).map { - case (PartitionValues(_, literals), path) => - Partition(Row(literals.map(_.value): _*), path.toString) - } - - PartitionSpec(StructType(fields), partitions) - } - - /** - * Parses a single partition, returns column names and values of each partition column. For - * example, given: - * {{{ - * path = hdfs://:/path/to/partition/a=42/b=hello/c=3.14 - * }}} - * it returns: - * {{{ - * PartitionValues( - * Seq("a", "b", "c"), - * Seq( - * Literal.create(42, IntegerType), - * Literal.create("hello", StringType), - * Literal.create(3.14, FloatType))) - * }}} - */ - private[parquet] def parsePartition( - path: Path, - defaultPartitionName: String): PartitionValues = { - val columns = ArrayBuffer.empty[(String, Literal)] - // Old Hadoop versions don't have `Path.isRoot` - var finished = path.getParent == null - var chopped = path - - while (!finished) { - val maybeColumn = parsePartitionColumn(chopped.getName, defaultPartitionName) - maybeColumn.foreach(columns += _) - chopped = chopped.getParent - finished = maybeColumn.isEmpty || chopped.getParent == null - } - - val (columnNames, values) = columns.reverse.unzip - PartitionValues(columnNames, values) - } - - private def parsePartitionColumn( - columnSpec: String, - defaultPartitionName: String): Option[(String, Literal)] = { - val equalSignIndex = columnSpec.indexOf('=') - if (equalSignIndex == -1) { - None - } else { - val columnName = columnSpec.take(equalSignIndex) - assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") - - val rawColumnValue = columnSpec.drop(equalSignIndex + 1) - assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - - val literal = inferPartitionColumnValue(rawColumnValue, defaultPartitionName) - Some(columnName -> literal) - } - } - - /** - * Resolves possible type conflicts between partitions by up-casting "lower" types. The up- - * casting order is: - * {{{ - * NullType -> - * IntegerType -> LongType -> - * FloatType -> DoubleType -> DecimalType.Unlimited -> - * StringType - * }}} - */ - private[parquet] def resolvePartitions(values: Seq[PartitionValues]): Seq[PartitionValues] = { - // Column names of all partitions must match - val distinctPartitionsColNames = values.map(_.columnNames).distinct - assert(distinctPartitionsColNames.size == 1, { - val list = distinctPartitionsColNames.mkString("\t", "\n", "") - s"Conflicting partition column names detected:\n$list" - }) - - // Resolves possible type conflicts for each column - val columnCount = values.head.columnNames.size - val resolvedValues = (0 until columnCount).map { i => - resolveTypeConflicts(values.map(_.literals(i))) - } - - // Fills resolved literals back to each partition - values.zipWithIndex.map { case (d, index) => - d.copy(literals = resolvedValues.map(_(index))) - } - } - - /** - * Converts a string to a `Literal` with automatic type inference. Currently only supports - * [[IntegerType]], [[LongType]], [[FloatType]], [[DoubleType]], [[DecimalType.Unlimited]], and - * [[StringType]]. - */ - private[parquet] def inferPartitionColumnValue( - raw: String, - defaultPartitionName: String): Literal = { - // First tries integral types - Try(Literal.create(Integer.parseInt(raw), IntegerType)) - .orElse(Try(Literal.create(JLong.parseLong(raw), LongType))) - // Then falls back to fractional types - .orElse(Try(Literal.create(JFloat.parseFloat(raw), FloatType))) - .orElse(Try(Literal.create(JDouble.parseDouble(raw), DoubleType))) - .orElse(Try(Literal.create(new JBigDecimal(raw), DecimalType.Unlimited))) - // Then falls back to string - .getOrElse { - if (raw == defaultPartitionName) Literal.create(null, NullType) - else Literal.create(raw, StringType) - } - } - - private val upCastingOrder: Seq[DataType] = - Seq(NullType, IntegerType, LongType, FloatType, DoubleType, DecimalType.Unlimited, StringType) - - /** - * Given a collection of [[Literal]]s, resolves possible type conflicts by up-casting "lower" - * types. - */ - private def resolveTypeConflicts(literals: Seq[Literal]): Seq[Literal] = { - val desiredType = { - val topType = literals.map(_.dataType).maxBy(upCastingOrder.indexOf(_)) - // Falls back to string if all values of this column are null or empty string - if (topType == NullType) StringType else topType - } - - literals.map { case l @ Literal(_, dataType) => - Literal.create(Cast(l, desiredType).eval(), desiredType) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index b3d71f687a60a..a5410cda0fe6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -17,20 +17,25 @@ package org.apache.spark.sql.sources +import org.apache.hadoop.fs.Path + +import org.apache.spark.Logging +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.types.{UTF8String, StringType} -import org.apache.spark.sql.{Row, Strategy, execution, sources} +import org.apache.spark.sql.types.{StructType, UTF8String, StringType} +import org.apache.spark.sql._ /** * A Strategy for planning scans over data sources defined using the sources API. */ -private[sql] object DataSourceStrategy extends Strategy { +private[sql] object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: CatalystScan)) => pruneFilterProjectRaw( @@ -53,6 +58,51 @@ private[sql] object DataSourceStrategy extends Strategy { filters, (a, _) => t.buildScan(a)) :: Nil + // Scanning partitioned FSBasedRelation + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) + if t.partitionSpec.partitionColumns.nonEmpty => + val selectedPartitions = prunePartitions(filters, t.partitionSpec).toArray + + logInfo { + val total = t.partitionSpec.partitions.length + val selected = selectedPartitions.length + val percentPruned = (1 - total.toDouble / selected.toDouble) * 100 + s"Selected $selected partitions out of $total, pruned $percentPruned% partitions." + } + + // Only pushes down predicates that do not reference partition columns. + val pushedFilters = { + val partitionColumnNames = t.partitionSpec.partitionColumns.map(_.name).toSet + filters.filter { f => + val referencedColumnNames = f.references.map(_.name).toSet + referencedColumnNames.intersect(partitionColumnNames).isEmpty + } + } + + buildPartitionedTableScan( + l, + projectList, + pushedFilters, + t.partitionSpec.partitionColumns, + selectedPartitions) :: Nil + + // Scanning non-partitioned FSBasedRelation + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) => + val inputPaths = t.paths.map(new Path(_)).flatMap { path => + val fs = path.getFileSystem(t.sqlContext.sparkContext.hadoopConfiguration) + val qualifiedPath = path.makeQualified(fs.getUri, fs.getWorkingDirectory) + SparkHadoopUtil.get.listLeafStatuses(fs, qualifiedPath).map(_.getPath).filterNot { path => + val name = path.getName + name.startsWith("_") || name.startsWith(".") + }.map(fs.makeQualified(_).toString) + } + + pruneFilterProject( + l, + projectList, + filters, + (a, f) => t.buildScan(a, f, inputPaths)) :: Nil + case l @ LogicalRelation(t: TableScan) => createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil @@ -60,9 +110,144 @@ private[sql] object DataSourceStrategy extends Strategy { l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil + case i @ logical.InsertIntoTable( + l @ LogicalRelation(t: FSBasedRelation), part, query, overwrite, false) if part.isEmpty => + val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append + execution.ExecutedCommand( + InsertIntoFSBasedRelation(t, query, Array.empty[String], mode)) :: Nil + case _ => Nil } + private def buildPartitionedTableScan( + logicalRelation: LogicalRelation, + projections: Seq[NamedExpression], + filters: Seq[Expression], + partitionColumns: StructType, + partitions: Array[Partition]) = { + val output = projections.map(_.toAttribute) + val relation = logicalRelation.relation.asInstanceOf[FSBasedRelation] + + // Builds RDD[Row]s for each selected partition. + val perPartitionRows = partitions.map { case Partition(partitionValues, dir) => + // Paths to all data files within this partition + val dataFilePaths = { + val dirPath = new Path(dir) + val fs = dirPath.getFileSystem(SparkHadoopUtil.get.conf) + fs.listStatus(dirPath).map(_.getPath).filterNot { path => + val name = path.getName + name.startsWith("_") || name.startsWith(".") + }.map(fs.makeQualified(_).toString) + } + + // The table scan operator (PhysicalRDD) which retrieves required columns from data files. + // Notice that the schema of data files, represented by `relation.dataSchema`, may contain + // some partition column(s). + val scan = + pruneFilterProject( + logicalRelation, + projections, + filters, + (requiredColumns, filters) => { + val partitionColNames = partitionColumns.fieldNames + + // Don't scan any partition columns to save I/O. Here we are being optimistic and + // assuming partition columns data stored in data files are always consistent with those + // partition values encoded in partition directory paths. + val nonPartitionColumns = requiredColumns.filterNot(partitionColNames.contains) + val dataRows = relation.buildScan(nonPartitionColumns, filters, dataFilePaths) + + // Merges data values with partition values. + mergeWithPartitionValues( + relation.schema, + requiredColumns, + partitionColNames, + partitionValues, + dataRows) + }) + + scan.execute() + } + + val unionedRows = perPartitionRows.reduceOption(_ ++ _).getOrElse { + relation.sqlContext.emptyResult + } + + createPhysicalRDD(logicalRelation.relation, output, unionedRows) + } + + private def mergeWithPartitionValues( + schema: StructType, + requiredColumns: Array[String], + partitionColumns: Array[String], + partitionValues: Row, + dataRows: RDD[Row]): RDD[Row] = { + val nonPartitionColumns = requiredColumns.filterNot(partitionColumns.contains) + + // If output columns contain any partition column(s), we need to merge scanned data + // columns and requested partition columns to form the final result. + if (!requiredColumns.sameElements(nonPartitionColumns)) { + val mergers = requiredColumns.zipWithIndex.map { case (name, index) => + // To see whether the `index`-th column is a partition column... + val i = partitionColumns.indexOf(name) + if (i != -1) { + // If yes, gets column value from partition values. + (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => { + mutableRow(ordinal) = partitionValues(i) + } + } else { + // Otherwise, inherits the value from scanned data. + val i = nonPartitionColumns.indexOf(name) + (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => { + mutableRow(ordinal) = dataRow(i) + } + } + } + + dataRows.mapPartitions { iterator => + val dataTypes = requiredColumns.map(schema(_).dataType) + val mutableRow = new SpecificMutableRow(dataTypes) + iterator.map { dataRow => + var i = 0 + while (i < mutableRow.length) { + mergers(i)(mutableRow, dataRow, i) + i += 1 + } + mutableRow.asInstanceOf[expressions.Row] + } + } + } else { + dataRows + } + } + + protected def prunePartitions( + predicates: Seq[Expression], + partitionSpec: PartitionSpec): Seq[Partition] = { + val PartitionSpec(partitionColumns, partitions) = partitionSpec + val partitionColumnNames = partitionColumns.map(_.name).toSet + val partitionPruningPredicates = predicates.filter { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + + if (partitionPruningPredicates.nonEmpty) { + val predicate = + partitionPruningPredicates + .reduceOption(expressions.And) + .getOrElse(Literal(true)) + + val boundPredicate = InterpretedPredicate.create(predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }) + + partitions.filter { case Partition(values, _) => boundPredicate(values) } + } else { + partitions + } + } + // Based on Public API. protected def pruneFilterProject( relation: LogicalRelation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala new file mode 100644 index 0000000000000..d30f7f65e21c0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala @@ -0,0 +1,207 @@ +/* + * 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.sources + +import java.lang.{Double => JDouble, Float => JFloat, Long => JLong} +import java.math.{BigDecimal => JBigDecimal} + +import scala.collection.mutable.ArrayBuffer +import scala.util.Try + +import com.google.common.cache.{CacheBuilder, Cache} +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} +import org.apache.spark.sql.types._ + +private[sql] case class Partition(values: Row, path: String) + +private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) + +private[sql] object PartitioningUtils { + private[sql] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { + require(columnNames.size == literals.size) + } + + /** + * Given a group of qualified paths, tries to parse them and returns a partition specification. + * For example, given: + * {{{ + * hdfs://:/path/to/partition/a=1/b=hello/c=3.14 + * hdfs://:/path/to/partition/a=2/b=world/c=6.28 + * }}} + * it returns: + * {{{ + * PartitionSpec( + * partitionColumns = StructType( + * StructField(name = "a", dataType = IntegerType, nullable = true), + * StructField(name = "b", dataType = StringType, nullable = true), + * StructField(name = "c", dataType = DoubleType, nullable = true)), + * partitions = Seq( + * Partition( + * values = Row(1, "hello", 3.14), + * path = "hdfs://:/path/to/partition/a=1/b=hello/c=3.14"), + * Partition( + * values = Row(2, "world", 6.28), + * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28"))) + * }}} + */ + private[sql] def parsePartitions( + paths: Seq[Path], + defaultPartitionName: String): PartitionSpec = { + val partitionValues = resolvePartitions(paths.map(parsePartition(_, defaultPartitionName))) + val fields = { + val (PartitionValues(columnNames, literals)) = partitionValues.head + columnNames.zip(literals).map { case (name, Literal(_, dataType)) => + StructField(name, dataType, nullable = true) + } + } + + val partitions = partitionValues.zip(paths).map { + case (PartitionValues(_, literals), path) => + Partition(Row(literals.map(_.value): _*), path.toString) + } + + PartitionSpec(StructType(fields), partitions) + } + + /** + * Parses a single partition, returns column names and values of each partition column. For + * example, given: + * {{{ + * path = hdfs://:/path/to/partition/a=42/b=hello/c=3.14 + * }}} + * it returns: + * {{{ + * PartitionValues( + * Seq("a", "b", "c"), + * Seq( + * Literal.create(42, IntegerType), + * Literal.create("hello", StringType), + * Literal.create(3.14, FloatType))) + * }}} + */ + private[sql] def parsePartition( + path: Path, + defaultPartitionName: String): PartitionValues = { + val columns = ArrayBuffer.empty[(String, Literal)] + // Old Hadoop versions don't have `Path.isRoot` + var finished = path.getParent == null + var chopped = path + + while (!finished) { + val maybeColumn = parsePartitionColumn(chopped.getName, defaultPartitionName) + maybeColumn.foreach(columns += _) + chopped = chopped.getParent + finished = maybeColumn.isEmpty || chopped.getParent == null + } + + val (columnNames, values) = columns.reverse.unzip + PartitionValues(columnNames, values) + } + + private def parsePartitionColumn( + columnSpec: String, + defaultPartitionName: String): Option[(String, Literal)] = { + val equalSignIndex = columnSpec.indexOf('=') + if (equalSignIndex == -1) { + None + } else { + val columnName = columnSpec.take(equalSignIndex) + assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") + + val rawColumnValue = columnSpec.drop(equalSignIndex + 1) + assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") + + val literal = inferPartitionColumnValue(rawColumnValue, defaultPartitionName) + Some(columnName -> literal) + } + } + + /** + * Resolves possible type conflicts between partitions by up-casting "lower" types. The up- + * casting order is: + * {{{ + * NullType -> + * IntegerType -> LongType -> + * FloatType -> DoubleType -> DecimalType.Unlimited -> + * StringType + * }}} + */ + private[sql] def resolvePartitions(values: Seq[PartitionValues]): Seq[PartitionValues] = { + // Column names of all partitions must match + val distinctPartitionsColNames = values.map(_.columnNames).distinct + assert(distinctPartitionsColNames.size == 1, { + val list = distinctPartitionsColNames.mkString("\t", "\n", "") + s"Conflicting partition column names detected:\n$list" + }) + + // Resolves possible type conflicts for each column + val columnCount = values.head.columnNames.size + val resolvedValues = (0 until columnCount).map { i => + resolveTypeConflicts(values.map(_.literals(i))) + } + + // Fills resolved literals back to each partition + values.zipWithIndex.map { case (d, index) => + d.copy(literals = resolvedValues.map(_(index))) + } + } + + /** + * Converts a string to a `Literal` with automatic type inference. Currently only supports + * [[IntegerType]], [[LongType]], [[FloatType]], [[DoubleType]], [[DecimalType.Unlimited]], and + * [[StringType]]. + */ + private[sql] def inferPartitionColumnValue( + raw: String, + defaultPartitionName: String): Literal = { + // First tries integral types + Try(Literal.create(Integer.parseInt(raw), IntegerType)) + .orElse(Try(Literal.create(JLong.parseLong(raw), LongType))) + // Then falls back to fractional types + .orElse(Try(Literal.create(JFloat.parseFloat(raw), FloatType))) + .orElse(Try(Literal.create(JDouble.parseDouble(raw), DoubleType))) + .orElse(Try(Literal.create(new JBigDecimal(raw), DecimalType.Unlimited))) + // Then falls back to string + .getOrElse { + if (raw == defaultPartitionName) Literal.create(null, NullType) + else Literal.create(raw, StringType) + } + } + + private val upCastingOrder: Seq[DataType] = + Seq(NullType, IntegerType, LongType, FloatType, DoubleType, DecimalType.Unlimited, StringType) + + /** + * Given a collection of [[Literal]]s, resolves possible type conflicts by up-casting "lower" + * types. + */ + private def resolveTypeConflicts(literals: Seq[Literal]): Seq[Literal] = { + val desiredType = { + val topType = literals.map(_.dataType).maxBy(upCastingOrder.indexOf(_)) + // Falls back to string if all values of this column are null or empty string + if (topType == NullType) StringType else topType + } + + literals.map { case l @ Literal(_, dataType) => + Literal.create(Cast(l, desiredType).eval(), desiredType) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index dbdb0d39c26a1..127133bfaf662 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -14,12 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.sources -import org.apache.spark.sql.{DataFrame, SQLContext} +import java.util.Date + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat} +import org.apache.hadoop.util.Shell +import parquet.hadoop.util.ContextUtil + +import org.apache.spark._ +import org.apache.spark.mapred.SparkHadoopMapRedUtil +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateProjection import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} private[sql] case class InsertIntoDataSource( logicalRelation: LogicalRelation, @@ -41,3 +57,391 @@ private[sql] case class InsertIntoDataSource( Seq.empty[Row] } } + +private[sql] case class InsertIntoFSBasedRelation( + @transient relation: FSBasedRelation, + @transient query: LogicalPlan, + partitionColumns: Array[String], + mode: SaveMode) + extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + require( + relation.paths.length == 1, + s"Cannot write to multiple destinations: ${relation.paths.mkString(",")}") + + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val outputPath = new Path(relation.paths.head) + val fs = outputPath.getFileSystem(hadoopConf) + val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + + val doInsertion = (mode, fs.exists(qualifiedOutputPath)) match { + case (SaveMode.ErrorIfExists, true) => + sys.error(s"path $qualifiedOutputPath already exists.") + case (SaveMode.Overwrite, true) => + fs.delete(qualifiedOutputPath, true) + true + case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => + true + case (SaveMode.Ignore, exists) => + !exists + } + + if (doInsertion) { + val job = Job.getInstance(hadoopConf) + job.setOutputKeyClass(classOf[Void]) + job.setOutputValueClass(classOf[Row]) + FileOutputFormat.setOutputPath(job, qualifiedOutputPath) + + val df = sqlContext.createDataFrame( + DataFrame(sqlContext, query).queryExecution.toRdd, + relation.schema, + needsConversion = false) + + if (partitionColumns.isEmpty) { + insert(new DefaultWriterContainer(relation, job), df) + } else { + val writerContainer = new DynamicPartitionWriterContainer( + relation, job, partitionColumns, "__HIVE_DEFAULT_PARTITION__") + insertWithDynamicPartitions(sqlContext, writerContainer, df, partitionColumns) + } + } + + Seq.empty[Row] + } + + private def insert(writerContainer: BaseWriterContainer, df: DataFrame): Unit = { + // Uses local vals for serialization + val needsConversion = relation.needConversion + val dataSchema = relation.dataSchema + + try { + writerContainer.driverSideSetup() + df.sqlContext.sparkContext.runJob(df.queryExecution.executedPlan.execute(), writeRows _) + writerContainer.commitJob() + relation.refresh() + } catch { case cause: Throwable => + writerContainer.abortJob() + throw new SparkException("Job aborted.", cause) + } + + def writeRows(taskContext: TaskContext, iterator: Iterator[Row]): Unit = { + writerContainer.executorSideSetup(taskContext) + + try { + if (needsConversion) { + val converter = CatalystTypeConverters.createToScalaConverter(dataSchema) + while (iterator.hasNext) { + val row = converter(iterator.next()).asInstanceOf[Row] + writerContainer.outputWriterForRow(row).write(row) + } + } else { + while (iterator.hasNext) { + val row = iterator.next() + writerContainer.outputWriterForRow(row).write(row) + } + } + writerContainer.commitTask() + } catch { case cause: Throwable => + writerContainer.abortTask() + throw new SparkException("Task failed while writing rows.", cause) + } + } + } + + private def insertWithDynamicPartitions( + sqlContext: SQLContext, + writerContainer: BaseWriterContainer, + df: DataFrame, + partitionColumns: Array[String]): Unit = { + // Uses a local val for serialization + val needsConversion = relation.needConversion + val dataSchema = relation.dataSchema + + require( + df.schema == relation.schema, + s"""DataFrame must have the same schema as the relation to which is inserted. + |DataFrame schema: ${df.schema} + |Relation schema: ${relation.schema} + """.stripMargin) + + val partitionColumnsInSpec = relation.partitionColumns.fieldNames + require( + partitionColumnsInSpec.sameElements(partitionColumns), + s"""Partition columns mismatch. + |Expected: ${partitionColumnsInSpec.mkString(", ")} + |Actual: ${partitionColumns.mkString(", ")} + """.stripMargin) + + val output = df.queryExecution.executedPlan.output + val (partitionOutput, dataOutput) = output.partition(a => partitionColumns.contains(a.name)) + val codegenEnabled = df.sqlContext.conf.codegenEnabled + + try { + writerContainer.driverSideSetup() + df.sqlContext.sparkContext.runJob(df.queryExecution.executedPlan.execute(), writeRows _) + writerContainer.commitJob() + relation.refresh() + } catch { case cause: Throwable => + logError("Aborting job.", cause) + writerContainer.abortJob() + throw new SparkException("Job aborted.", cause) + } + + def writeRows(taskContext: TaskContext, iterator: Iterator[Row]): Unit = { + writerContainer.executorSideSetup(taskContext) + + val partitionProj = newProjection(codegenEnabled, partitionOutput, output) + val dataProj = newProjection(codegenEnabled, dataOutput, output) + + if (needsConversion) { + val converter = CatalystTypeConverters.createToScalaConverter(dataSchema) + while (iterator.hasNext) { + val row = iterator.next() + val partitionPart = partitionProj(row) + val dataPart = dataProj(row) + val convertedDataPart = converter(dataPart).asInstanceOf[Row] + writerContainer.outputWriterForRow(partitionPart).write(convertedDataPart) + } + } else { + while (iterator.hasNext) { + val row = iterator.next() + val partitionPart = partitionProj(row) + val dataPart = dataProj(row) + writerContainer.outputWriterForRow(partitionPart).write(dataPart) + } + } + + writerContainer.commitTask() + } + } + + // This is copied from SparkPlan, probably should move this to a more general place. + private def newProjection( + codegenEnabled: Boolean, + expressions: Seq[Expression], + inputSchema: Seq[Attribute]): Projection = { + log.debug( + s"Creating Projection: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") + if (codegenEnabled) { + GenerateProjection.generate(expressions, inputSchema) + } else { + new InterpretedProjection(expressions, inputSchema) + } + } +} + +private[sql] abstract class BaseWriterContainer( + @transient val relation: FSBasedRelation, + @transient job: Job) + extends SparkHadoopMapReduceUtil + with Logging + with Serializable { + + protected val serializableConf = new SerializableWritable(ContextUtil.getConfiguration(job)) + + // This is only used on driver side. + @transient private val jobContext: JobContext = job + + // The following fields are initialized and used on both driver and executor side. + @transient protected var outputCommitter: FileOutputCommitter = _ + @transient private var jobId: JobID = _ + @transient private var taskId: TaskID = _ + @transient private var taskAttemptId: TaskAttemptID = _ + @transient protected var taskAttemptContext: TaskAttemptContext = _ + + protected val outputPath: String = { + assert( + relation.paths.length == 1, + s"Cannot write to multiple destinations: ${relation.paths.mkString(",")}") + relation.paths.head + } + + protected val dataSchema = relation.dataSchema + + protected val outputWriterClass: Class[_ <: OutputWriter] = relation.outputWriterClass + + private var outputFormatClass: Class[_ <: OutputFormat[_, _]] = _ + + def driverSideSetup(): Unit = { + setupIDs(0, 0, 0) + setupConf() + taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) + relation.prepareForWrite(job) + outputFormatClass = job.getOutputFormatClass + outputCommitter = newOutputCommitter(taskAttemptContext) + outputCommitter.setupJob(jobContext) + } + + def executorSideSetup(taskContext: TaskContext): Unit = { + setupIDs(taskContext.stageId(), taskContext.partitionId(), taskContext.attemptNumber()) + setupConf() + taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) + outputCommitter = newOutputCommitter(taskAttemptContext) + outputCommitter.setupTask(taskAttemptContext) + initWriters() + } + + private def newOutputCommitter(context: TaskAttemptContext): FileOutputCommitter = { + outputFormatClass.newInstance().getOutputCommitter(context) match { + case f: FileOutputCommitter => f + case f => sys.error( + s"FileOutputCommitter or its subclass is expected, but got a ${f.getClass.getName}.") + } + } + + private def setupIDs(jobId: Int, splitId: Int, attemptId: Int): Unit = { + this.jobId = SparkHadoopWriter.createJobID(new Date, jobId) + this.taskId = new TaskID(this.jobId, true, splitId) + this.taskAttemptId = new TaskAttemptID(taskId, attemptId) + } + + private def setupConf(): Unit = { + serializableConf.value.set("mapred.job.id", jobId.toString) + serializableConf.value.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + serializableConf.value.set("mapred.task.id", taskAttemptId.toString) + serializableConf.value.setBoolean("mapred.task.is.map", true) + serializableConf.value.setInt("mapred.task.partition", 0) + } + + // Called on executor side when writing rows + def outputWriterForRow(row: Row): OutputWriter + + protected def initWriters(): Unit + + def commitTask(): Unit = { + SparkHadoopMapRedUtil.commitTask( + outputCommitter, taskAttemptContext, jobId.getId, taskId.getId, taskAttemptId.getId) + } + + def abortTask(): Unit = { + outputCommitter.abortTask(taskAttemptContext) + logError(s"Task attempt $taskAttemptId aborted.") + } + + def commitJob(): Unit = { + outputCommitter.commitJob(jobContext) + logInfo(s"Job $jobId committed.") + } + + def abortJob(): Unit = { + outputCommitter.abortJob(jobContext, JobStatus.State.FAILED) + logError(s"Job $jobId aborted.") + } +} + +private[sql] class DefaultWriterContainer( + @transient relation: FSBasedRelation, + @transient job: Job) + extends BaseWriterContainer(relation, job) { + + @transient private var writer: OutputWriter = _ + + override protected def initWriters(): Unit = { + writer = outputWriterClass.newInstance() + writer.init(outputCommitter.getWorkPath.toString, dataSchema, taskAttemptContext) + } + + override def outputWriterForRow(row: Row): OutputWriter = writer + + override def commitTask(): Unit = { + writer.close() + super.commitTask() + } + + override def abortTask(): Unit = { + writer.close() + super.abortTask() + } +} + +private[sql] class DynamicPartitionWriterContainer( + @transient relation: FSBasedRelation, + @transient job: Job, + partitionColumns: Array[String], + defaultPartitionName: String) + extends BaseWriterContainer(relation, job) { + + // All output writers are created on executor side. + @transient protected var outputWriters: mutable.Map[String, OutputWriter] = _ + + override protected def initWriters(): Unit = { + outputWriters = mutable.Map.empty[String, OutputWriter] + } + + override def outputWriterForRow(row: Row): OutputWriter = { + val partitionPath = partitionColumns.zip(row.toSeq).map { case (col, rawValue) => + val string = if (rawValue == null) null else String.valueOf(rawValue) + val valueString = if (string == null || string.isEmpty) { + defaultPartitionName + } else { + DynamicPartitionWriterContainer.escapePathName(string) + } + s"/$col=$valueString" + }.mkString + + outputWriters.getOrElseUpdate(partitionPath, { + val path = new Path(outputCommitter.getWorkPath, partitionPath.stripPrefix(Path.SEPARATOR)) + val writer = outputWriterClass.newInstance() + writer.init(path.toString, dataSchema, taskAttemptContext) + writer + }) + } + + override def commitTask(): Unit = { + outputWriters.values.foreach(_.close()) + super.commitTask() + } + + override def abortTask(): Unit = { + outputWriters.values.foreach(_.close()) + super.abortTask() + } +} + +private[sql] object DynamicPartitionWriterContainer { + ////////////////////////////////////////////////////////////////////////////////////////////////// + // The following string escaping code is mainly copied from Hive (o.a.h.h.common.FileUtils). + ////////////////////////////////////////////////////////////////////////////////////////////////// + + val charToEscape = { + val bitSet = new java.util.BitSet(128) + + /** + * ASCII 01-1F are HTTP control characters that need to be escaped. + * \u000A and \u000D are \n and \r, respectively. + */ + val clist = Array( + '\u0001', '\u0002', '\u0003', '\u0004', '\u0005', '\u0006', '\u0007', '\u0008', '\u0009', + '\n', '\u000B', '\u000C', '\r', '\u000E', '\u000F', '\u0010', '\u0011', '\u0012', '\u0013', + '\u0014', '\u0015', '\u0016', '\u0017', '\u0018', '\u0019', '\u001A', '\u001B', '\u001C', + '\u001D', '\u001E', '\u001F', '"', '#', '%', '\'', '*', '/', ':', '=', '?', '\\', '\u007F', + '{', '[', ']', '^') + + clist.foreach(bitSet.set(_)) + + if (Shell.WINDOWS) { + Array(' ', '<', '>', '|').foreach(bitSet.set(_)) + } + + bitSet + } + + def needsEscaping(c: Char): Boolean = { + c >= 0 && c < charToEscape.size() && charToEscape.get(c) + } + + def escapePathName(path: String): String = { + val builder = new StringBuilder() + path.foreach { c => + if (DynamicPartitionWriterContainer.needsEscaping(c)) { + builder.append('%') + builder.append(f"${c.asInstanceOf[Int]}%02x") + } else { + builder.append(c) + } + } + + builder.toString() + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 06c64f2bdd59e..595c5eb40e295 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -17,18 +17,20 @@ package org.apache.spark.sql.sources -import scala.language.existentials +import scala.language.{existentials, implicitConversions} import scala.util.matching.Regex -import scala.language.implicitConversions + +import org.apache.hadoop.fs.Path import org.apache.spark.Logging -import org.apache.spark.sql.{AnalysisException, SaveMode, DataFrame, SQLContext} -import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.types._ +import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext, SaveMode} import org.apache.spark.util.Utils /** @@ -111,6 +113,7 @@ private[sql] class DDLParser( CreateTableUsingAsSelect(tableName, provider, temp.isDefined, + Array.empty[String], mode, options, queryPlan) @@ -157,7 +160,7 @@ private[sql] class DDLParser( protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")} override implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( - s"identifier matching regex ${regex}", { + s"identifier matching regex $regex", { case lexical.Identifier(str) if regex.unapplySeq(str).isDefined => str case lexical.Keyword(str) if regex.unapplySeq(str).isDefined => str } @@ -214,6 +217,7 @@ private[sql] object ResolvedDataSource { def apply( sqlContext: SQLContext, userSpecifiedSchema: Option[StructType], + partitionColumns: Array[String], provider: String, options: Map[String, String]): ResolvedDataSource = { val clazz: Class[_] = lookupDataSource(provider) @@ -222,6 +226,27 @@ private[sql] object ResolvedDataSource { case Some(schema: StructType) => clazz.newInstance() match { case dataSource: SchemaRelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema) + case dataSource: FSBasedRelationProvider => + val maybePartitionsSchema = if (partitionColumns.isEmpty) { + None + } else { + Some(partitionColumnsSchema(schema, partitionColumns)) + } + + val caseInsensitiveOptions= new CaseInsensitiveMap(options) + val paths = { + val patternPath = new Path(caseInsensitiveOptions("path")) + SparkHadoopUtil.get.globPath(patternPath).map(_.toString).toArray + } + + val dataSchema = StructType(schema.filterNot(f => partitionColumns.contains(f.name))) + + dataSource.createRelation( + sqlContext, + paths, + Some(schema), + maybePartitionsSchema, + caseInsensitiveOptions) case dataSource: org.apache.spark.sql.sources.RelationProvider => throw new AnalysisException(s"$className does not allow user-specified schemas.") case _ => @@ -231,20 +256,39 @@ private[sql] object ResolvedDataSource { case None => clazz.newInstance() match { case dataSource: RelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) + case dataSource: FSBasedRelationProvider => + val caseInsensitiveOptions = new CaseInsensitiveMap(options) + val paths = { + val patternPath = new Path(caseInsensitiveOptions("path")) + SparkHadoopUtil.get.globPath(patternPath).map(_.toString).toArray + } + dataSource.createRelation(sqlContext, paths, None, None, caseInsensitiveOptions) case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => throw new AnalysisException( s"A schema needs to be specified when using $className.") case _ => - throw new AnalysisException(s"$className is not a RelationProvider.") + throw new AnalysisException( + s"$className is neither a RelationProvider nor a FSBasedRelationProvider.") } } new ResolvedDataSource(clazz, relation) } + private def partitionColumnsSchema( + schema: StructType, + partitionColumns: Array[String]): StructType = { + StructType(partitionColumns.map { col => + schema.find(_.name == col).getOrElse { + throw new RuntimeException(s"Partition column $col not found in schema $schema") + } + }).asNullable + } + /** Create a [[ResolvedDataSource]] for saving the content of the given [[DataFrame]]. */ def apply( sqlContext: SQLContext, provider: String, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], data: DataFrame): ResolvedDataSource = { @@ -252,6 +296,31 @@ private[sql] object ResolvedDataSource { val relation = clazz.newInstance() match { case dataSource: CreatableRelationProvider => dataSource.createRelation(sqlContext, mode, options, data) + case dataSource: FSBasedRelationProvider => + // Don't glob path for the write path. The contracts here are: + // 1. Only one output path can be specified on the write path; + // 2. Output path must be a legal HDFS style file system path; + // 3. It's OK that the output path doesn't exist yet; + val caseInsensitiveOptions = new CaseInsensitiveMap(options) + val outputPath = { + val path = new Path(caseInsensitiveOptions("path")) + val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + path.makeQualified(fs.getUri, fs.getWorkingDirectory) + } + val dataSchema = StructType(data.schema.filterNot(f => partitionColumns.contains(f.name))) + val r = dataSource.createRelation( + sqlContext, + Array(outputPath.toString), + Some(dataSchema.asNullable), + Some(partitionColumnsSchema(data.schema, partitionColumns)), + caseInsensitiveOptions) + sqlContext.executePlan( + InsertIntoFSBasedRelation( + r, + data.logicalPlan, + partitionColumns.toArray, + mode)).toRdd + r case _ => sys.error(s"${clazz.getCanonicalName} does not allow create table as select.") } @@ -310,6 +379,7 @@ private[sql] case class CreateTableUsingAsSelect( tableName: String, provider: String, temporary: Boolean, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], child: LogicalPlan) extends UnaryNode { @@ -324,8 +394,9 @@ private[sql] case class CreateTempTableUsing( provider: String, options: Map[String, String]) extends RunnableCommand { - override def run(sqlContext: SQLContext): Seq[Row] = { - val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) + def run(sqlContext: SQLContext): Seq[Row] = { + val resolved = ResolvedDataSource( + sqlContext, userSpecifiedSchema, Array.empty[String], provider, options) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) Seq.empty @@ -335,13 +406,14 @@ private[sql] case class CreateTempTableUsing( private[sql] case class CreateTempTableUsingAsSelect( tableName: String, provider: String, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], query: LogicalPlan) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) - val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) + val resolved = ResolvedDataSource(sqlContext, provider, partitionColumns, mode, options, df) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index ca53dcdb92c52..5e010d21120f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,11 +17,19 @@ package org.apache.spark.sql.sources -import org.apache.spark.annotation.{Experimental, DeveloperApi} +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SaveMode, DataFrame, Row, SQLContext} -import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, _} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection +import org.apache.spark.sql.types.{StructField, StructType} /** * ::DeveloperApi:: @@ -78,6 +86,41 @@ trait SchemaRelationProvider { schema: StructType): BaseRelation } +/** + * ::DeveloperApi:: + * Implemented by objects that produce relations for a specific kind of data source + * with a given schema and partitioned columns. When Spark SQL is given a DDL operation with a + * USING clause specified (to specify the implemented [[FSBasedRelationProvider]]), a user defined + * schema, and an optional list of partition columns, this interface is used to pass in the + * parameters specified by a user. + * + * Users may specify the fully qualified class name of a given data source. When that class is + * not found Spark SQL will append the class name `DefaultSource` to the path, allowing for + * less verbose invocation. For example, 'org.apache.spark.sql.json' would resolve to the + * data source 'org.apache.spark.sql.json.DefaultSource' + * + * A new instance of this class with be instantiated each time a DDL call is made. + * + * The difference between a [[RelationProvider]] and a [[FSBasedRelationProvider]] is + * that users need to provide a schema and a (possibly empty) list of partition columns when + * using a SchemaRelationProvider. A relation provider can inherits both [[RelationProvider]], + * and [[FSBasedRelationProvider]] if it can support schema inference, user-specified + * schemas, and accessing partitioned relations. + */ +trait FSBasedRelationProvider { + /** + * Returns a new base relation with the given parameters, a user defined schema, and a list of + * partition columns. Note: the parameters' keywords are case insensitive and this insensitivity + * is enforced by the Map that is passed to the function. + */ + def createRelation( + sqlContext: SQLContext, + paths: Array[String], + schema: Option[StructType], + partitionColumns: Option[StructType], + parameters: Map[String, String]): FSBasedRelation +} + @DeveloperApi trait CreatableRelationProvider { /** @@ -207,3 +250,235 @@ trait InsertableRelation { trait CatalystScan { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } + +/** + * ::Experimental:: + * [[OutputWriter]] is used together with [[FSBasedRelation]] for persisting rows to the + * underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor. + * An [[OutputWriter]] instance is created and initialized when a new output file is opened on + * executor side. This instance is used to persist rows to this single output file. + */ +@Experimental +abstract class OutputWriter { + /** + * Initializes this [[OutputWriter]] before any rows are persisted. + * + * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that + * this may not point to the final output file. For example, `FileOutputFormat` writes to + * temporary directories and then merge written files back to the final destination. In + * this case, `path` points to a temporary output file under the temporary directory. + * @param dataSchema Schema of the rows to be written. Partition columns are not included in the + * schema if the corresponding relation is partitioned. + * @param context The Hadoop MapReduce task context. + */ + def init( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): Unit = () + + /** + * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned + * tables, dynamic partition columns are not included in rows to be written. + */ + def write(row: Row): Unit + + /** + * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before + * the task output is committed. + */ + def close(): Unit +} + +/** + * ::Experimental:: + * A [[BaseRelation]] that provides much of the common code required for formats that store their + * data to an HDFS compatible filesystem. + * + * For the read path, similar to [[PrunedFilteredScan]], it can eliminate unneeded columns and + * filter using selected predicates before producing an RDD containing all matching tuples as + * [[Row]] objects. In addition, when reading from Hive style partitioned tables stored in file + * systems, it's able to discover partitioning information from the paths of input directories, and + * perform partition pruning before start reading the data. Subclasses of [[FSBasedRelation()]] must + * override one of the three `buildScan` methods to implement the read path. + * + * For the write path, it provides the ability to write to both non-partitioned and partitioned + * tables. Directory layout of the partitioned tables is compatible with Hive. + * + * @constructor This constructor is for internal uses only. The [[PartitionSpec]] argument is for + * implementing metastore table conversion. + * @param paths Base paths of this relation. For partitioned relations, it should be the root + * directories of all partition directories. + * @param maybePartitionSpec An [[FSBasedRelation]] can be created with an optional + * [[PartitionSpec]], so that partition discovery can be skipped. + */ +@Experimental +abstract class FSBasedRelation private[sql]( + val paths: Array[String], + maybePartitionSpec: Option[PartitionSpec]) + extends BaseRelation { + + /** + * Constructs an [[FSBasedRelation]]. + * + * @param paths Base paths of this relation. For partitioned relations, it should be either root + * directories of all partition directories. + * @param partitionColumns Partition columns of this relation. + */ + def this(paths: Array[String], partitionColumns: StructType) = + this(paths, { + if (partitionColumns.isEmpty) None + else Some(PartitionSpec(partitionColumns, Array.empty[Partition])) + }) + + /** + * Constructs an [[FSBasedRelation]]. + * + * @param paths Base paths of this relation. For partitioned relations, it should be root + * directories of all partition directories. + */ + def this(paths: Array[String]) = this(paths, None) + + private val hadoopConf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) + + private val codegenEnabled = sqlContext.conf.codegenEnabled + + private var _partitionSpec: PartitionSpec = maybePartitionSpec.map { spec => + spec.copy(partitionColumns = spec.partitionColumns.asNullable) + }.getOrElse { + if (sqlContext.conf.partitionDiscoveryEnabled()) { + discoverPartitions() + } else { + PartitionSpec(StructType(Nil), Array.empty[Partition]) + } + } + + private[sql] def partitionSpec: PartitionSpec = _partitionSpec + + /** + * Partition columns. Note that they are always nullable. + */ + def partitionColumns: StructType = partitionSpec.partitionColumns + + private[sql] def refresh(): Unit = { + if (sqlContext.conf.partitionDiscoveryEnabled()) { + _partitionSpec = discoverPartitions() + } + } + + private def discoverPartitions(): PartitionSpec = { + val basePaths = paths.map(new Path(_)) + val leafDirs = basePaths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + Try(fs.getFileStatus(path.makeQualified(fs.getUri, fs.getWorkingDirectory))) + .filter(_.isDir) + .map(SparkHadoopUtil.get.listLeafDirStatuses(fs, _)) + .getOrElse(Seq.empty[FileStatus]) + }.map(_.getPath) + + if (leafDirs.nonEmpty) { + PartitioningUtils.parsePartitions(leafDirs, "__HIVE_DEFAULT_PARTITION__") + } else { + PartitionSpec(StructType(Array.empty[StructField]), Array.empty[Partition]) + } + } + + /** + * Schema of this relation. It consists of columns appearing in [[dataSchema]] and all partition + * columns not appearing in [[dataSchema]]. + */ + override lazy val schema: StructType = { + val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet + StructType(dataSchema ++ partitionSpec.partitionColumns.filterNot { column => + dataSchemaColumnNames.contains(column.name.toLowerCase) + }) + } + + /** + * Specifies schema of actual data files. For partitioned relations, if one or more partitioned + * columns are contained in the data files, they should also appear in `dataSchema`. + */ + def dataSchema: StructType + + /** + * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within + * this relation. For partitioned relations, this method is called for each selected partition, + * and builds an `RDD[Row]` containing all rows within that single partition. + * + * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the + * relation. For a partitioned relation, it contains paths of all data files in a single + * selected partition. + */ + def buildScan(inputPaths: Array[String]): RDD[Row] = { + throw new RuntimeException( + "At least one buildScan() method should be overridden to read the relation.") + } + + /** + * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within + * this relation. For partitioned relations, this method is called for each selected partition, + * and builds an `RDD[Row]` containing all rows within that single partition. + * + * @param requiredColumns Required columns. + * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the + * relation. For a partitioned relation, it contains paths of all data files in a single + * selected partition. + */ + def buildScan(requiredColumns: Array[String], inputPaths: Array[String]): RDD[Row] = { + // Yeah, to workaround serialization... + val dataSchema = this.dataSchema + val codegenEnabled = this.codegenEnabled + + val requiredOutput = requiredColumns.map { col => + val field = dataSchema(col) + BoundReference(dataSchema.fieldIndex(col), field.dataType, field.nullable) + }.toSeq + + buildScan(inputPaths).mapPartitions { rows => + val buildProjection = if (codegenEnabled) { + GenerateMutableProjection.generate(requiredOutput, dataSchema.toAttributes) + } else { + () => new InterpretedMutableProjection(requiredOutput, dataSchema.toAttributes) + } + + val mutableProjection = buildProjection() + rows.map(mutableProjection) + } + } + + /** + * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within + * this relation. For partitioned relations, this method is called for each selected partition, + * and builds an `RDD[Row]` containing all rows within that single partition. + * + * @param requiredColumns Required columns. + * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction + * of all `filters`. The pushed down filters are currently purely an optimization as they + * will all be evaluated again. This means it is safe to use them with methods that produce + * false positives such as filtering partitions based on a bloom filter. + * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the + * relation. For a partitioned relation, it contains paths of all data files in a single + * selected partition. + */ + def buildScan( + requiredColumns: Array[String], + filters: Array[Filter], + inputPaths: Array[String]): RDD[Row] = { + buildScan(requiredColumns, inputPaths) + } + + /** + * Client side preparation for data writing can be put here. For example, user defined output + * committer can be configured here. + * + * Note that the only side effect expected here is mutating `job` via its setters. Especially, + * Spark SQL caches [[BaseRelation]] instances for performance, mutating relation internal states + * may cause unexpected behaviors. + */ + def prepareForWrite(job: Job): Unit = () + + /** + * This method is responsible for producing a new [[OutputWriter]] for each newly opened output + * file on the executor side. + */ + def outputWriterClass: Class[_ <: OutputWriter] +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 6ed68d179edc9..aad1d248d0a28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -101,13 +101,13 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } } - case i @ logical.InsertIntoTable( - l: LogicalRelation, partition, query, overwrite, ifNotExists) - if !l.isInstanceOf[InsertableRelation] => + case logical.InsertIntoTable(LogicalRelation(_: InsertableRelation), _, _, _, _) => // OK + case logical.InsertIntoTable(LogicalRelation(_: FSBasedRelation), _, _, _, _) => // OK + case logical.InsertIntoTable(l: LogicalRelation, _, _, _, _) => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") - case CreateTableUsingAsSelect(tableName, _, _, SaveMode.Overwrite, _, query) => + case CreateTableUsingAsSelect(tableName, _, _, _, SaveMode.Overwrite, _, query) => // When the SaveMode is Overwrite, we need to check if the table is an input table of // the query. If so, we will throw an AnalysisException to let users know it is not allowed. if (catalog.tableExists(Seq(tableName))) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index b7561ce7298cb..bea568ed40049 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -21,7 +21,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.parquet.ParquetRelation2._ +import org.apache.spark.sql.sources.PartitioningUtils._ +import org.apache.spark.sql.sources.{Partition, PartitionSpec} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, Row, SQLContext} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 54f2f3cdec298..4e54b2eb8df7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.sources -import java.io.{IOException, File} +import java.io.{File, IOException} -import org.apache.spark.sql.AnalysisException import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.AnalysisException import org.apache.spark.util.Utils class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index bbf48efb24440..d754c8e3a8aa1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -19,25 +19,24 @@ package org.apache.spark.sql.hive import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} - import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.metastore.Warehouse +import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata._ import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.Logging -import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} -import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, Catalog, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => ParquetPartition, PartitionSpec} -import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.parquet.ParquetRelation2 +import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, LogicalRelation, Partition => ParquetPartition, PartitionSpec, ResolvedDataSource} import org.apache.spark.sql.types._ +import org.apache.spark.sql.{AnalysisException, SQLContext, SaveMode} import org.apache.spark.util.Utils /* Implicit conversions */ @@ -98,6 +97,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive ResolvedDataSource( hive, userSpecifiedSchema, + Array.empty[String], table.properties("spark.sql.sources.provider"), options) @@ -438,6 +438,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive desc.name, hive.conf.defaultDataSourceName, temporary = false, + Array.empty[String], mode, options = Map.empty[String, String], child diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index be9249a8b1f44..d46a127d47d31 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -221,14 +221,14 @@ private[hive] trait HiveStrategies { object HiveDDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case CreateTableUsing( - tableName, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => + tableName, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => ExecutedCommand( CreateMetastoreDataSource( tableName, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath)) :: Nil - case CreateTableUsingAsSelect(tableName, provider, false, mode, opts, query) => + case CreateTableUsingAsSelect(tableName, provider, false, partitionCols, mode, opts, query) => val cmd = - CreateMetastoreDataSourceAsSelect(tableName, provider, mode, opts, query) + CreateMetastoreDataSourceAsSelect(tableName, provider, partitionCols, mode, opts, query) ExecutedCommand(cmd) :: Nil case _ => Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index abab1a223a43a..8e405e080489f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -158,6 +158,7 @@ private[hive] case class CreateMetastoreDataSourceAsSelect( tableName: String, provider: String, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], query: LogicalPlan) extends RunnableCommand { @@ -189,12 +190,12 @@ case class CreateMetastoreDataSourceAsSelect( return Seq.empty[Row] case SaveMode.Append => // Check if the specified data source match the data source of the existing table. - val resolved = - ResolvedDataSource(sqlContext, Some(query.schema), provider, optionsWithPath) + val resolved = ResolvedDataSource( + sqlContext, Some(query.schema.asNullable), partitionColumns, provider, optionsWithPath) val createdRelation = LogicalRelation(resolved.relation) EliminateSubQueries(sqlContext.table(tableName).logicalPlan) match { - case l @ LogicalRelation(i: InsertableRelation) => - if (i != createdRelation.relation) { + case l @ LogicalRelation(_: InsertableRelation | _: FSBasedRelation) => + if (l.relation != createdRelation.relation) { val errorDescription = s"Cannot append to table $tableName because the resolved relation does not " + s"match the existing relation of $tableName. " + @@ -202,14 +203,13 @@ case class CreateMetastoreDataSourceAsSelect( s"table $tableName and using its data source and options." val errorMessage = s""" - |$errorDescription - |== Relations == - |${sideBySide( - s"== Expected Relation ==" :: - l.toString :: Nil, - s"== Actual Relation ==" :: - createdRelation.toString :: Nil).mkString("\n")} - """.stripMargin + |$errorDescription + |== Relations == + |${sideBySide( + s"== Expected Relation ==" :: l.toString :: Nil, + s"== Actual Relation ==" :: createdRelation.toString :: Nil + ).mkString("\n")} + """.stripMargin throw new AnalysisException(errorMessage) } existingSchema = Some(l.schema) @@ -234,7 +234,8 @@ case class CreateMetastoreDataSourceAsSelect( } // Create the relation based on the data of df. - val resolved = ResolvedDataSource(sqlContext, provider, mode, optionsWithPath, df) + val resolved = + ResolvedDataSource(sqlContext, provider, partitionColumns, mode, optionsWithPath, df) if (createMetastoreTable) { // We will use the schema of resolved.relation as the schema of the table (instead of diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 8398da268174d..cbc381cc81b59 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -204,7 +204,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( if (string == null || string.isEmpty) { defaultPartName } else { - FileUtils.escapePathName(string) + FileUtils.escapePathName(string, defaultPartName) } s"/$col=$colString" }.mkString diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala new file mode 100644 index 0000000000000..415b1cd168848 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -0,0 +1,525 @@ +/* + * 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.sources + +import org.apache.hadoop.fs.Path + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.sql._ +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.parquet.ParquetTest +import org.apache.spark.sql.types._ + +// TODO Don't extend ParquetTest +// This test suite extends ParquetTest for some convenient utility methods. These methods should be +// moved to some more general places, maybe QueryTest. +class FSBasedRelationSuite extends QueryTest with ParquetTest { + override val sqlContext: SQLContext = TestHive + + import sqlContext._ + import sqlContext.implicits._ + + val dataSchema = + StructType( + Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = false))) + + val testDF = (1 to 3).map(i => (i, s"val_$i")).toDF("a", "b") + + val partitionedTestDF1 = (for { + i <- 1 to 3 + p2 <- Seq("foo", "bar") + } yield (i, s"val_$i", 1, p2)).toDF("a", "b", "p1", "p2") + + val partitionedTestDF2 = (for { + i <- 1 to 3 + p2 <- Seq("foo", "bar") + } yield (i, s"val_$i", 2, p2)).toDF("a", "b", "p1", "p2") + + val partitionedTestDF = partitionedTestDF1.unionAll(partitionedTestDF2) + + def checkQueries(df: DataFrame): Unit = { + // Selects everything + checkAnswer( + df, + for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) + + // Simple filtering and partition pruning + checkAnswer( + df.filter('a > 1 && 'p1 === 2), + for (i <- 2 to 3; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", 2, p2)) + + // Simple projection and filtering + checkAnswer( + df.filter('a > 1).select('b, 'a + 1), + for (i <- 2 to 3; _ <- 1 to 2; _ <- Seq("foo", "bar")) yield Row(s"val_$i", i + 1)) + + // Simple projection and partition pruning + checkAnswer( + df.filter('a > 1 && 'p1 < 2).select('b, 'p1), + for (i <- 2 to 3; _ <- Seq("foo", "bar")) yield Row(s"val_$i", 1)) + + // Self-join + df.registerTempTable("t") + withTempTable("t") { + checkAnswer( + sql( + """SELECT l.a, r.b, l.p1, r.p2 + |FROM t l JOIN t r + |ON l.a = r.a AND l.p1 = r.p1 AND l.p2 = r.p2 + """.stripMargin), + for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) + } + } + + test("save()/load() - non-partitioned table - Overwrite") { + withTempPath { file => + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) + + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + testDF.collect()) + } + } + + test("save()/load() - non-partitioned table - Append") { + withTempPath { file => + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) + + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)).orderBy("a"), + testDF.unionAll(testDF).orderBy("a").collect()) + } + } + + test("save()/load() - non-partitioned table - ErrorIfExists") { + withTempDir { file => + intercept[RuntimeException] { + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists) + } + } + } + + test("save()/load() - non-partitioned table - Ignore") { + withTempDir { file => + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore) + + val path = new Path(file.getCanonicalPath) + val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + assert(fs.listStatus(path).isEmpty) + } + } + + test("save()/load() - partitioned table - simple queries") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkQueries( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json))) + } + } + + test("save()/load() - partitioned table - simple queries - partition columns in data") { + withTempDir { file => + val basePath = new Path(file.getCanonicalPath) + val fs = basePath.getFileSystem(SparkHadoopUtil.get.conf) + val qualifiedBasePath = fs.makeQualified(basePath) + + for (p1 <- 1 to 2; p2 <- Seq("foo", "bar")) { + val partitionDir = new Path(qualifiedBasePath, s"p1=$p1/p2=$p2") + sparkContext + .parallelize(for (i <- 1 to 3) yield s"$i,val_$i,$p1") + .saveAsTextFile(partitionDir.toString) + } + + val dataSchemaWithPartition = + StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) + + checkQueries( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchemaWithPartition.json))) + } + } + + test("save()/load() - partitioned table - Overwrite") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + partitionedTestDF.collect()) + } + } + + test("save()/load() - partitioned table - Append") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + partitionedTestDF.unionAll(partitionedTestDF).collect()) + } + } + + test("save()/load() - partitioned table - Append - new partition values") { + withTempPath { file => + partitionedTestDF1.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF2.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + partitionedTestDF.collect()) + } + } + + test("save()/load() - partitioned table - ErrorIfExists") { + withTempDir { file => + intercept[RuntimeException] { + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + } + } + } + + test("save()/load() - partitioned table - Ignore") { + withTempDir { file => + partitionedTestDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore) + + val path = new Path(file.getCanonicalPath) + val fs = path.getFileSystem(SparkHadoopUtil.get.conf) + assert(fs.listStatus(path).isEmpty) + } + } + + def withTable(tableName: String)(f: => Unit): Unit = { + try f finally sql(s"DROP TABLE $tableName") + } + + test("saveAsTable()/load() - non-partitioned table - Overwrite") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + Map("dataSchema" -> dataSchema.json)) + + withTable("t") { + checkAnswer(table("t"), testDF.collect()) + } + } + + test("saveAsTable()/load() - non-partitioned table - Append") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) + + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append) + + withTable("t") { + checkAnswer(table("t"), testDF.unionAll(testDF).orderBy("a").collect()) + } + } + + test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") + + withTempTable("t") { + intercept[AnalysisException] { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists) + } + } + } + + test("saveAsTable()/load() - non-partitioned table - Ignore") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") + + withTempTable("t") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore) + + assert(table("t").collect().isEmpty) + } + } + + test("saveAsTable()/load() - partitioned table - simple queries") { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + Map("dataSchema" -> dataSchema.json)) + + withTable("t") { + checkQueries(table("t")) + } + } + + test("saveAsTable()/load() - partitioned table - Overwrite") { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + withTable("t") { + checkAnswer(table("t"), partitionedTestDF.collect()) + } + } + + test("saveAsTable()/load() - partitioned table - Append") { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + withTable("t") { + checkAnswer(table("t"), partitionedTestDF.unionAll(partitionedTestDF).collect()) + } + } + + test("saveAsTable()/load() - partitioned table - Append - new partition values") { + partitionedTestDF1.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF2.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + withTable("t") { + checkAnswer(table("t"), partitionedTestDF.collect()) + } + } + + test("saveAsTable()/load() - partitioned table - Append - mismatched partition columns") { + partitionedTestDF1.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + // Using only a subset of all partition columns + intercept[Throwable] { + partitionedTestDF2.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1")) + } + + // Using different order of partition columns + intercept[Throwable] { + partitionedTestDF2.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p2", "p1")) + } + } + + test("saveAsTable()/load() - partitioned table - ErrorIfExists") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") + + withTempTable("t") { + intercept[AnalysisException] { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + } + } + } + + test("saveAsTable()/load() - partitioned table - Ignore") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") + + withTempTable("t") { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + assert(table("t").collect().isEmpty) + } + } + + test("Hadoop style globbing") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + val df = load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> s"${file.getCanonicalPath}/p1=*/p2=???", + "dataSchema" -> dataSchema.json)) + + val expectedPaths = Set( + s"${file.getCanonicalFile}/p1=1/p2=foo", + s"${file.getCanonicalFile}/p1=2/p2=foo", + s"${file.getCanonicalFile}/p1=1/p2=bar", + s"${file.getCanonicalFile}/p1=2/p2=bar" + ).map { p => + val path = new Path(p) + val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + path.makeQualified(fs.getUri, fs.getWorkingDirectory).toString + } + + println(df.queryExecution) + + val actualPaths = df.queryExecution.analyzed.collectFirst { + case LogicalRelation(relation: FSBasedRelation) => + relation.paths.toSet + }.getOrElse { + fail("Expect an FSBasedRelation, but none could be found") + } + + assert(actualPaths === expectedPaths) + checkAnswer(df, partitionedTestDF.collect()) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala new file mode 100644 index 0000000000000..8801aba2f64c3 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -0,0 +1,125 @@ +/* + * 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.sources + +import java.text.NumberFormat +import java.util.UUID + +import com.google.common.base.Objects +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{NullWritable, Text} +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} +import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} +import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.{Row, SQLContext} + +/** + * A simple example [[FSBasedRelationProvider]]. + */ +class SimpleTextSource extends FSBasedRelationProvider { + override def createRelation( + sqlContext: SQLContext, + paths: Array[String], + schema: Option[StructType], + partitionColumns: Option[StructType], + parameters: Map[String, String]): FSBasedRelation = { + val partitionsSchema = partitionColumns.getOrElse(StructType(Array.empty[StructField])) + new SimpleTextRelation(paths, schema, partitionsSchema, parameters)(sqlContext) + } +} + +class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullWritable, Text] { + val numberFormat = NumberFormat.getInstance() + + numberFormat.setMinimumIntegerDigits(5) + numberFormat.setGroupingUsed(false) + + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + val split = context.getTaskAttemptID.getTaskID.getId + val name = FileOutputFormat.getOutputName(context) + new Path(outputFile, s"$name-${numberFormat.format(split)}-${UUID.randomUUID()}") + } +} + +class SimpleTextOutputWriter extends OutputWriter { + private var recordWriter: RecordWriter[NullWritable, Text] = _ + private var taskAttemptContext: TaskAttemptContext = _ + + override def init( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): Unit = { + recordWriter = new AppendingTextOutputFormat(new Path(path)).getRecordWriter(context) + taskAttemptContext = context + } + + override def write(row: Row): Unit = { + val serialized = row.toSeq.map(_.toString).mkString(",") + recordWriter.write(null, new Text(serialized)) + } + + override def close(): Unit = recordWriter.close(taskAttemptContext) +} + +/** + * A simple example [[FSBasedRelation]], used for testing purposes. Data are stored as comma + * separated string lines. When scanning data, schema must be explicitly provided via data source + * option `"dataSchema"`. + */ +class SimpleTextRelation( + paths: Array[String], + val maybeDataSchema: Option[StructType], + partitionsSchema: StructType, + parameters: Map[String, String])( + @transient val sqlContext: SQLContext) + extends FSBasedRelation(paths, partitionsSchema) { + + import sqlContext.sparkContext + + override val dataSchema: StructType = + maybeDataSchema.getOrElse(DataType.fromJson(parameters("dataSchema")).asInstanceOf[StructType]) + + override def equals(other: Any): Boolean = other match { + case that: SimpleTextRelation => + this.paths.sameElements(that.paths) && + this.maybeDataSchema == that.maybeDataSchema && + this.dataSchema == that.dataSchema && + this.partitionColumns == that.partitionColumns + + case _ => false + } + + override def hashCode(): Int = + Objects.hashCode(paths, maybeDataSchema, dataSchema) + + override def outputWriterClass: Class[_ <: OutputWriter] = + classOf[SimpleTextOutputWriter] + + override def buildScan(inputPaths: Array[String]): RDD[Row] = { + val fields = dataSchema.map(_.dataType) + + sparkContext.textFile(inputPaths.mkString(",")).map { record => + Row(record.split(",").zip(fields).map { case (value, dataType) => + Cast(Literal(value), dataType).eval() + }: _*) + } + } +} From bfcaf8adcdc20dec203e2e9d5a72b52dd6f226a9 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 12 May 2015 10:37:57 -0700 Subject: [PATCH 37/59] [DataFrame][minor] support column in field accessor Minor improvement, now we can use `Column` as extraction expression. Author: Wenchen Fan Closes #6080 from cloud-fan/tmp and squashes the following commits: 0fdefb7 [Wenchen Fan] support column in field accessor --- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 2 +- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) 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 e6e475bb82f82..4d50821620f5e 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 @@ -78,7 +78,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { * * @group expr_ops */ - def apply(field: Any): Column = UnresolvedExtractValue(expr, Literal(field)) + def apply(extraction: Any): Column = UnresolvedExtractValue(expr, lit(extraction).expr) /** * Unary minus, i.e. negate the expression. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 2ade955864b71..d58438e5d129c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -456,6 +456,7 @@ class DataFrameSuite extends QueryTest { assert(complexData.filter(complexData("a")(0) === 2).count() == 1) assert(complexData.filter(complexData("m")("1") === 1).count() == 1) assert(complexData.filter(complexData("s")("key") === 1).count() == 1) + assert(complexData.filter(complexData("m")(complexData("s")("value")) === 1).count() == 1) } test("SPARK-7324 dropDuplicates") { From 65697bbeafe507dda066e2dc14ca5183f278dfe9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 12 May 2015 11:17:59 -0700 Subject: [PATCH 38/59] [SPARK-7500] DAG visualization: move cluster labeling to dagre-d3 This fixes the label bleeding issue described in the JIRA and pictured in the screenshots below. I also took the opportunity to move some code to the places that they belong more to. In particular: (1) Drawing cluster labels is now implemented in my branch of dagre-d3 instead of in Spark (2) All graph styling is now moved from Scala to JS Note that these changes are related because our existing mechanism of "tacking on cluster labels" afterwards isn't flexible enough for us to fix issues like this one easily. For the other half of the changes, visit http://github.com/andrewor14/dagre-d3. ------------------- **Before.** ------------------- **After.** Author: Andrew Or Closes #6076 from andrewor14/dag-viz-bleed and squashes the following commits: 5858d7a [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-viz-bleed c686dc4 [Andrew Or] Fix tooltip placement d908c36 [Andrew Or] Add link to dagre-d3 changes (minor) 4a4fb58 [Andrew Or] Fix bleeding + move all styling to JS --- .../apache/spark/ui/static/dagre-d3.min.js | 14 ++-- .../apache/spark/ui/static/spark-dag-viz.js | 79 ++++++++----------- .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../spark/ui/scope/RDDOperationGraph.scala | 40 +++------- 4 files changed, 48 insertions(+), 87 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js index 6beb7db855ca0..acf2d93b718b2 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js @@ -1,4 +1,4 @@ -!function(e){if("object"==typeof exports&&"undefined"!=typeof module)module.exports=e();else if("function"==typeof define&&define.amd)define([],e);else{var f;"undefined"!=typeof window?f=window:"undefined"!=typeof global?f=global:"undefined"!=typeof self&&(f=self),f.dagreD3=e()}}(function(){var define,module,exports;return function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);var f=new Error("Cannot find module '"+o+"'");throw f.code="MODULE_NOT_FOUND",f}var l=n[o]={exports:{}};t[o][0].call(l.exports,function(e){var n=t[o][1][e];return s(n?n:e)},l,l.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;i0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs))); +module.exports={graphlib:require("./lib/graphlib"),dagre:require("./lib/dagre"),intersect:require("./lib/intersect"),render:require("./lib/render"),util:require("./lib/util"),version:require("./lib/version")}},{"./lib/dagre":8,"./lib/graphlib":9,"./lib/intersect":10,"./lib/render":23,"./lib/util":25,"./lib/version":26}],2:[function(require,module,exports){var util=require("./util");module.exports={"default":normal,normal:normal,vee:vee,undirected:undirected};function normal(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 0 L 10 5 L 0 10 z").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}function vee(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 0 L 10 5 L 0 10 L 4 5 z").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}function undirected(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 5 L 10 5").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}},{"./util":25}],3:[function(require,module,exports){var _=require("./lodash"),addLabel=require("./label/add-label"),util=require("./util");module.exports=createClusters;function createClusters(selection,g){var clusters=g.nodes().filter(function(v){return util.isSubgraph(g,v)}),svgClusters=selection.selectAll("g.cluster").data(clusters,function(v){return v});var makeClusterIdentifier=function(v){return"cluster_"+v.replace(/^cluster/,"")};svgClusters.enter().append("g").attr("id",makeClusterIdentifier).attr("name",function(v){return g.node(v).label}).classed("cluster",true).style("opacity",0).append("rect");var sortedClusters=util.orderByRank(g,svgClusters.data());for(var i=0;i0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;imaxPadding){maxPadding=child.paddingTop}}return maxPadding}function getRank(g,v){var maxRank=0;var children=g.children(v);for(var i=0;imaxRank){maxRank=thisRank}}return maxRank}function orderByRank(g,nodes){return nodes.sort(function(x,y){return getRank(g,x)-getRank(g,y)})}function edgeToId(e){return escapeId(e.v)+":"+escapeId(e.w)+":"+escapeId(e.name)}var ID_DELIM=/:/g;function escapeId(str){return str?String(str).replace(ID_DELIM,"\\:"):""}function applyStyle(dom,styleFn){if(styleFn){dom.attr("style",styleFn)}}function applyClass(dom,classFn,otherClasses){if(classFn){dom.attr("class",classFn).attr("class",otherClasses+" "+dom.attr("class"))}}function applyTransition(selection,g){var graph=g.graph();if(_.isPlainObject(graph)){var transition=graph.transition;if(_.isFunction(transition)){return transition(selection)}}return selection}},{"./lodash":20}],26:[function(require,module,exports){module.exports="0.4.4-pre"},{}],27:[function(require,module,exports){module.exports={graphlib:require("./lib/graphlib"),layout:require("./lib/layout"),debug:require("./lib/debug"),util:{time:require("./lib/util").time,notime:require("./lib/util").notime},version:require("./lib/version")}},{"./lib/debug":32,"./lib/graphlib":33,"./lib/layout":35,"./lib/util":55,"./lib/version":56}],28:[function(require,module,exports){"use strict";var _=require("./lodash"),greedyFAS=require("./greedy-fas");module.exports={run:run,undo:undo};function run(g){var fas=g.graph().acyclicer==="greedy"?greedyFAS(g,weightFn(g)):dfsFAS(g);_.each(fas,function(e){var label=g.edge(e);g.removeEdge(e);label.forwardName=e.name;label.reversed=true;g.setEdge(e.w,e.v,label,_.uniqueId("rev"))});function weightFn(g){return function(e){return g.edge(e).weight}}}function dfsFAS(g){var fas=[],stack={},visited={};function dfs(v){if(_.has(visited,v)){return}visited[v]=true;stack[v]=true;_.each(g.outEdges(v),function(e){if(_.has(stack,e.w)){fas.push(e)}else{dfs(e.w)}});delete stack[v]}_.each(g.nodes(),dfs);return fas}function undo(g){_.each(g.edges(),function(e){var label=g.edge(e);if(label.reversed){g.removeEdge(e);var forwardName=label.forwardName;delete label.reversed;delete label.forwardName;g.setEdge(e.w,e.v,label,forwardName)}})}},{"./greedy-fas":34,"./lodash":36}],29:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports=addBorderSegments;function addBorderSegments(g){function dfs(v){var children=g.children(v),node=g.node(v);if(children.length){_.each(children,dfs)}if(_.has(node,"minRank")){node.borderLeft=[];node.borderRight=[];for(var rank=node.minRank,maxRank=node.maxRank+1;rank0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){ +parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph();graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdx0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v; +if(!g.hasEdge(v,w)){v=edge.w;w=edge.v}var vLabel=t.node(v),wLabel=t.node(w),tailLabel=vLabel,flip=false;if(vLabel.lim>wLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}; -do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[]; - -if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId); - -}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index") .replace(/"/g, "\""); var g = graphlibDot.read(escaped_dot); - g.graph().rankSep = rankSep; var renderer = new dagreD3.render(); + preprocessGraphLayout(g, forJob); renderer(container, g); } @@ -251,50 +249,37 @@ function graphContainer() { return d3.select("#dag-viz-graph"); } function metadataContainer() { return d3.select("#dag-viz-metadata"); } /* - * Helper function to create draw a label for each cluster. - * - * We need to do this manually because dagre-d3 does not support labeling clusters. - * In general, the clustering support for dagre-d3 is quite limited at this point. + * Helper function to pre-process the graph layout. + * This step is necessary for certain styles that affect the positioning + * and sizes of graph elements, e.g. padding, font style, shape. */ -function drawClusterLabels(svgContainer, forJob) { - var clusterLabelSize, stageClusterLabelSize; +function preprocessGraphLayout(g, forJob) { + var nodes = g.nodes(); + for (var i = 0; i < nodes.length; i++) { + var isCluster = g.children(nodes[i]).length > 0; + if (!isCluster) { + var node = g.node(nodes[i]); + if (forJob) { + // Do not display RDD name on job page + node.shape = "circle"; + node.labelStyle = "font-size: 0px"; + } else { + node.labelStyle = "font-size: 12px"; + } + node.padding = "5"; + } + } + // Curve the edges + var edges = g.edges(); + for (var j = 0; j < edges.length; j++) { + edges[j].lineInterpolate = "basis"; + } + // Adjust vertical separation between nodes if (forJob) { - clusterLabelSize = JobPageVizConstants.clusterLabelSize; - stageClusterLabelSize = JobPageVizConstants.stageClusterLabelSize; + g.graph().rankSep = JobPageVizConstants.rankSep; } else { - clusterLabelSize = StagePageVizConstants.clusterLabelSize; - stageClusterLabelSize = StagePageVizConstants.stageClusterLabelSize; + g.graph().rankSep = StagePageVizConstants.rankSep; } - svgContainer.selectAll("g.cluster").each(function() { - var cluster = d3.select(this); - var isStage = cluster.attr("id").indexOf(VizConstants.stageClusterPrefix) > -1; - var labelSize = isStage ? stageClusterLabelSize : clusterLabelSize; - drawClusterLabel(cluster, labelSize); - }); -} - -/* - * Helper function to draw a label for the given cluster element based on its name. - * - * In the process, we need to expand the bounding box to make room for the label. - * We need to do this because dagre-d3 did not take this into account when it first - * rendered the bounding boxes. Note that this means we need to adjust the view box - * of the SVG afterwards since we shifted a few boxes around. - */ -function drawClusterLabel(d3cluster, fontSize) { - var cluster = d3cluster; - var rect = d3cluster.select("rect"); - rect.attr("y", toFloat(rect.attr("y")) - fontSize); - rect.attr("height", toFloat(rect.attr("height")) + fontSize); - var labelX = toFloat(rect.attr("x")) + toFloat(rect.attr("width")) - fontSize / 2; - var labelY = toFloat(rect.attr("y")) + fontSize * 1.5; - var labelText = cluster.attr("name").replace(VizConstants.clusterPrefix, ""); - cluster.append("text") - .attr("x", labelX) - .attr("y", labelY) - .attr("text-anchor", "end") - .style("font-size", fontSize + "px") - .text(labelText); } /* @@ -444,7 +429,7 @@ function addTooltipsForRDDs(svgContainer) { if (tooltipText) { node.select("circle") .attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "bottom") .attr("title", tooltipText) } }); diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 6a0f5c5d16daa..441c97d6d3e2a 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -360,7 +360,7 @@ private[spark] object UIUtils extends Logging { { graphs.map { g =>
  • } - val helpButton: Seq[Node] = helpText.map { helpText => - - (?) - - }.getOrElse(Seq.empty) + val helpButton: Seq[Node] = helpText.map(tooltip(_, "bottom")).getOrElse(Seq.empty) @@ -375,6 +371,12 @@ private[spark] object UIUtils extends Logging { } + def tooltip(text: String, position: String): Seq[Node] = { + + (?) + + } + /** Return a script element that automatically expands the DAG visualization on page load. */ def expandDagVizOnLoad(forJob: Boolean): Seq[Node] = {