From 1ad04dae038673a448f529c39b17817b78d6acd0 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 6 May 2015 16:15:51 -0700 Subject: [PATCH 01/37] [SPARK-5995] [ML] Make Prediction dev API public Changes: * Update protected prediction methods, following design doc. **<--most interesting change** * Changed abstract classes for Estimator and Model to be public. Added DeveloperApi tag. (I kept the traits for Estimator/Model Params private.) * Changed ProbabilisticClassificationModel method names to use probability instead of probabilities. CC: mengxr shivaram etrain Author: Joseph K. Bradley Closes #5913 from jkbradley/public-dev-api and squashes the following commits: e9aa0ea [Joseph K. Bradley] moved findMax to DenseVector and renamed to argmax. fixed bug for vector of length 0 15b9957 [Joseph K. Bradley] renamed probabilities to probability in method names 5cda84d [Joseph K. Bradley] regenerated sharedParams 7d1877a [Joseph K. Bradley] Made spark.ml prediction abstractions public. Organized their prediction methods for efficient computation of multiple output columns. --- .../ml/{impl/estimator => }/Predictor.scala | 50 ++------ .../spark/ml/classification/Classifier.scala | 110 ++++++------------ .../DecisionTreeClassifier.scala | 5 +- .../ml/classification/GBTClassifier.scala | 5 +- .../classification/LogisticRegression.scala | 100 ++++++---------- .../ProbabilisticClassifier.scala | 100 +++++++++++----- .../RandomForestClassifier.scala | 5 +- .../ml/param/shared/SharedParamsCodeGen.scala | 6 +- .../spark/ml/param/shared/sharedParams.scala | 4 +- .../ml/regression/DecisionTreeRegressor.scala | 5 +- .../spark/ml/regression/GBTRegressor.scala | 5 +- .../ml/regression/LinearRegression.scala | 5 +- .../ml/regression/RandomForestRegressor.scala | 5 +- .../spark/ml/regression/Regressor.scala | 42 ++----- .../spark/ml/{impl => }/tree/treeParams.scala | 4 +- .../apache/spark/mllib/linalg/Vectors.scala | 22 ++++ 16 files changed, 206 insertions(+), 267 deletions(-) rename mllib/src/main/scala/org/apache/spark/ml/{impl/estimator => }/Predictor.scala (86%) rename mllib/src/main/scala/org/apache/spark/ml/{impl => }/tree/treeParams.scala (99%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala similarity index 86% rename from mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala rename to mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index e8b3628140e99..0e53877de92db 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -15,29 +15,23 @@ * limitations under the License. */ -package org.apache.spark.ml.impl.estimator +package org.apache.spark.ml -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} +import org.apache.spark.sql.{DataFrame, Row} /** - * :: DeveloperApi :: - * - * Trait for parameters for prediction (regression and classification). - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + * (private[ml]) Trait for parameters for prediction (regression and classification). */ -@DeveloperApi -private[spark] trait PredictorParams extends Params +private[ml] trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol { /** @@ -63,7 +57,7 @@ private[spark] trait PredictorParams extends Params } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Abstraction for prediction problems (regression and classification). * @@ -73,11 +67,9 @@ private[spark] trait PredictorParams extends Params * parameter to specify the concrete type. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class Predictor[ +@DeveloperApi +abstract class Predictor[ FeaturesType, Learner <: Predictor[FeaturesType, Learner, M], M <: PredictionModel[FeaturesType, M]] @@ -104,8 +96,6 @@ private[spark] abstract class Predictor[ } /** - * :: DeveloperApi :: - * * Train a model using the given dataset and parameters. * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation * and copying parameters into the model. @@ -113,12 +103,9 @@ private[spark] abstract class Predictor[ * @param dataset Training dataset * @return Fitted model */ - @DeveloperApi protected def train(dataset: DataFrame): M /** - * :: DeveloperApi :: - * * Returns the SQL DataType corresponding to the FeaturesType type parameter. * * This is used by [[validateAndTransformSchema()]]. @@ -126,7 +113,6 @@ private[spark] abstract class Predictor[ * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ - @DeveloperApi protected def featuresDataType: DataType = new VectorUDT override def transformSchema(schema: StructType): StructType = { @@ -146,7 +132,7 @@ private[spark] abstract class Predictor[ } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Abstraction for a model for prediction tasks (regression and classification). * @@ -154,11 +140,9 @@ private[spark] abstract class Predictor[ * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] +@DeveloperApi +abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { /** @group setParam */ @@ -168,8 +152,6 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel def setPredictionCol(value: String): M = set(predictionCol, value).asInstanceOf[M] /** - * :: DeveloperApi :: - * * Returns the SQL DataType corresponding to the FeaturesType type parameter. * * This is used by [[validateAndTransformSchema()]]. @@ -177,7 +159,6 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ - @DeveloperApi protected def featuresDataType: DataType = new VectorUDT override def transformSchema(schema: StructType): StructType = { @@ -192,12 +173,8 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel * @return transformed dataset with [[predictionCol]] of type [[Double]] */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - - if ($(predictionCol) != "") { + if ($(predictionCol).nonEmpty) { dataset.withColumn($(predictionCol), callUDF(predict _, DoubleType, col($(featuresCol)))) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + @@ -207,11 +184,8 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel } /** - * :: DeveloperApi :: - * * Predict label for the given features. * This internal method is used to implement [[transform()]] and output [[predictionCol]]. */ - @DeveloperApi protected def predict(features: FeaturesType): Double } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index d3361e24705c8..263d580fe2dd3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} @@ -26,15 +26,12 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} + /** - * :: DeveloperApi :: - * Params for classification. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + * (private[spark]) Params for classification. */ -@DeveloperApi -private[spark] trait ClassifierParams extends PredictorParams - with HasRawPredictionCol { +private[spark] trait ClassifierParams + extends PredictorParams with HasRawPredictionCol { override protected def validateAndTransformSchema( schema: StructType, @@ -46,23 +43,21 @@ private[spark] trait ClassifierParams extends PredictorParams } /** - * :: AlphaComponent :: + * :: DeveloperApi :: + * * Single-label binary or multiclass classification. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam E Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class Classifier[ +@DeveloperApi +abstract class Classifier[ FeaturesType, E <: Classifier[FeaturesType, E, M], M <: ClassificationModel[FeaturesType, M]] - extends Predictor[FeaturesType, E, M] - with ClassifierParams { + extends Predictor[FeaturesType, E, M] with ClassifierParams { /** @group setParam */ def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] @@ -71,17 +66,15 @@ private[spark] abstract class Classifier[ } /** - * :: AlphaComponent :: + * :: DeveloperApi :: + * * Model produced by a [[Classifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] +@DeveloperApi abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with ClassifierParams { @@ -101,13 +94,27 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * @return transformed dataset */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + var outputData = dataset + var numColsOutput = 0 + if (getRawPredictionCol != "") { + outputData = outputData.withColumn(getRawPredictionCol, + callUDF(predictRaw _, new VectorUDT, col(getFeaturesCol))) + numColsOutput += 1 + } + if (getPredictionCol != "") { + val predUDF = if (getRawPredictionCol != "") { + callUDF(raw2prediction _, DoubleType, col(getRawPredictionCol)) + } else { + callUDF(predict _, DoubleType, col(getFeaturesCol)) + } + outputData = outputData.withColumn(getPredictionCol, predUDF) + numColsOutput += 1 + } + if (numColsOutput == 0) { logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + " since no output columns were set.") @@ -116,22 +123,17 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur } /** - * :: DeveloperApi :: - * * Predict label for the given features. * This internal method is used to implement [[transform()]] and output [[predictionCol]]. * * This default implementation for classification predicts the index of the maximum value * from [[predictRaw()]]. */ - @DeveloperApi override protected def predict(features: FeaturesType): Double = { - predictRaw(features).toArray.zipWithIndex.maxBy(_._1)._2 + raw2prediction(predictRaw(features)) } /** - * :: DeveloperApi :: - * * Raw prediction for each possible label. * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives * a measure of confidence in each possible label (where larger = more confident). @@ -141,48 +143,12 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * This raw prediction may be any real number, where a larger value indicates greater * confidence for that label. */ - @DeveloperApi protected def predictRaw(features: FeaturesType): Vector -} - -private[ml] object ClassificationModel { /** - * Added prediction column(s). This is separated from [[ClassificationModel.transform()]] - * since it is used by [[org.apache.spark.ml.classification.ProbabilisticClassificationModel]]. - * @param dataset Input dataset - * @return (number of columns added, transformed dataset) + * Given a vector of raw predictions, select the predicted label. + * This may be overridden to support thresholds which favor particular labels. + * @return predicted label */ - def transformColumnsImpl[FeaturesType]( - dataset: DataFrame, - model: ClassificationModel[FeaturesType, _]): (Int, DataFrame) = { - - // Output selected columns only. - // This is a bit complicated since it tries to avoid repeated computation. - var tmpData = dataset - var numColsOutput = 0 - if (model.getRawPredictionCol != "") { - // output raw prediction - val features2raw: FeaturesType => Vector = model.predictRaw - tmpData = tmpData.withColumn(model.getRawPredictionCol, - callUDF(features2raw, new VectorUDT, col(model.getFeaturesCol))) - numColsOutput += 1 - if (model.getPredictionCol != "") { - val raw2pred: Vector => Double = (rawPred) => { - rawPred.toArray.zipWithIndex.maxBy(_._1)._2 - } - tmpData = tmpData.withColumn(model.getPredictionCol, - callUDF(raw2pred, DoubleType, col(model.getRawPredictionCol))) - numColsOutput += 1 - } - } else if (model.getPredictionCol != "") { - // output prediction - val features2pred: FeaturesType => Double = model.predict - tmpData = tmpData.withColumn(model.getPredictionCol, - callUDF(features2pred, DoubleType, col(model.getFeaturesCol))) - numColsOutput += 1 - } - (numColsOutput, tmpData) - } - + protected def raw2prediction(rawPrediction: Vector): Double = rawPrediction.toDense.argmax } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 419e5ba05d38a..dcebea1d4b015 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.tree.{TreeClassifierParams, DecisionTreeParams, DecisionTreeModel, Node} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 534ea95b1c538..ae51b05a0c42d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -21,11 +21,10 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.regression.DecisionTreeRegressionModel -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{GBTParams, TreeClassifierParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index b73be035e29b5..550369d18cfec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -21,9 +21,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT, Vectors} +import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.functions._ import org.apache.spark.storage.StorageLevel /** @@ -99,76 +98,17 @@ class LogisticRegressionModel private[ml] ( /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) + /** Margin (rawPrediction) for class label 1. For binary classification only. */ private val margin: Vector => Double = (features) => { BLAS.dot(features, weights) + intercept } + /** Score (probability) for class label 1. For binary classification only. */ private val score: Vector => Double = (features) => { val m = margin(features) 1.0 / (1.0 + math.exp(-m)) } - override def transform(dataset: DataFrame): DataFrame = { - // This is overridden (a) to be more efficient (avoiding re-computing values when creating - // multiple output columns) and (b) to handle threshold, which the abstractions do not use. - // TODO: We should abstract away the steps defined by UDFs below so that the abstractions - // can call whichever UDFs are needed to create the output columns. - - // Check schema - transformSchema(dataset.schema, logging = true) - - // Output selected columns only. - // This is a bit complicated since it tries to avoid repeated computation. - // rawPrediction (-margin, margin) - // probability (1.0-score, score) - // prediction (max margin) - var tmpData = dataset - var numColsOutput = 0 - if ($(rawPredictionCol) != "") { - val features2raw: Vector => Vector = (features) => predictRaw(features) - tmpData = tmpData.withColumn($(rawPredictionCol), - callUDF(features2raw, new VectorUDT, col($(featuresCol)))) - numColsOutput += 1 - } - if ($(probabilityCol) != "") { - if ($(rawPredictionCol) != "") { - val raw2prob = udf { (rawPreds: Vector) => - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - Vectors.dense(1.0 - prob1, prob1): Vector - } - tmpData = tmpData.withColumn($(probabilityCol), raw2prob(col($(rawPredictionCol)))) - } else { - val features2prob = udf { (features: Vector) => predictProbabilities(features) : Vector } - tmpData = tmpData.withColumn($(probabilityCol), features2prob(col($(featuresCol)))) - } - numColsOutput += 1 - } - if ($(predictionCol) != "") { - val t = $(threshold) - if ($(probabilityCol) != "") { - val predict = udf { probs: Vector => - if (probs(1) > t) 1.0 else 0.0 - } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(probabilityCol)))) - } else if ($(rawPredictionCol) != "") { - val predict = udf { rawPreds: Vector => - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - if (prob1 > t) 1.0 else 0.0 - } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(rawPredictionCol)))) - } else { - val predict = udf { features: Vector => this.predict(features) } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(featuresCol)))) - } - numColsOutput += 1 - } - if (numColsOutput == 0) { - this.logWarning(s"$uid: LogisticRegressionModel.transform() was called as NOOP" + - " since no output columns were set.") - } - tmpData - } - override val numClasses: Int = 2 /** @@ -179,17 +119,43 @@ class LogisticRegressionModel private[ml] ( if (score(features) > getThreshold) 1 else 0 } - override protected def predictProbabilities(features: Vector): Vector = { - val s = score(features) - Vectors.dense(1.0 - s, s) + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + rawPrediction match { + case dv: DenseVector => + var i = 0 + while (i < dv.size) { + dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i))) + i += 1 + } + dv + case sv: SparseVector => + throw new RuntimeException("Unexpected error in LogisticRegressionModel:" + + " raw2probabilitiesInPlace encountered SparseVector") + } } override protected def predictRaw(features: Vector): Vector = { val m = margin(features) - Vectors.dense(0.0, m) + Vectors.dense(-m, m) } override def copy(extra: ParamMap): LogisticRegressionModel = { copyValues(new LogisticRegressionModel(parent, weights, intercept), extra) } + + override protected def raw2prediction(rawPrediction: Vector): Double = { + val t = getThreshold + val rawThreshold = if (t == 0.0) { + Double.NegativeInfinity + } else if (t == 1.0) { + Double.PositiveInfinity + } else { + Math.log(t / (1.0 - t)) + } + if (rawPrediction(1) > rawThreshold) 1 else 0 + } + + override protected def probability2prediction(probability: Vector): Double = { + if (probability(1) > getThreshold) 1 else 0 + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 8519841c5c26c..330ae2938f4e0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -17,16 +17,16 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.{DoubleType, DataType, StructType} /** - * Params for probabilistic classification. + * (private[classification]) Params for probabilistic classification. */ private[classification] trait ProbabilisticClassifierParams extends ClassifierParams with HasProbabilityCol { @@ -42,17 +42,15 @@ private[classification] trait ProbabilisticClassifierParams /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Single-label binary or multiclass classifier which can output class conditional probabilities. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam E Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class ProbabilisticClassifier[ FeaturesType, E <: ProbabilisticClassifier[FeaturesType, E, M], @@ -65,17 +63,15 @@ private[spark] abstract class ProbabilisticClassifier[ /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Model produced by a [[ProbabilisticClassifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class ProbabilisticClassificationModel[ FeaturesType, M <: ProbabilisticClassificationModel[FeaturesType, M]] @@ -95,39 +91,79 @@ private[spark] abstract class ProbabilisticClassificationModel[ * @return transformed dataset */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) - // Output selected columns only. - if ($(probabilityCol) != "") { - // output probabilities - outputData.withColumn($(probabilityCol), - callUDF(predictProbabilities _, new VectorUDT, col($(featuresCol)))) - } else { - if (numColsOutput == 0) { - this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + // This is a bit complicated since it tries to avoid repeated computation. + var outputData = dataset + var numColsOutput = 0 + if ($(rawPredictionCol).nonEmpty) { + outputData = outputData.withColumn(getRawPredictionCol, + callUDF(predictRaw _, new VectorUDT, col(getFeaturesCol))) + numColsOutput += 1 + } + if ($(probabilityCol).nonEmpty) { + val probUDF = if ($(rawPredictionCol).nonEmpty) { + callUDF(raw2probability _, new VectorUDT, col($(rawPredictionCol))) + } else { + callUDF(predictProbability _, new VectorUDT, col($(featuresCol))) + } + outputData = outputData.withColumn($(probabilityCol), probUDF) + numColsOutput += 1 + } + if ($(predictionCol).nonEmpty) { + val predUDF = if ($(rawPredictionCol).nonEmpty) { + callUDF(raw2prediction _, DoubleType, col($(rawPredictionCol))) + } else if ($(probabilityCol).nonEmpty) { + callUDF(probability2prediction _, DoubleType, col($(probabilityCol))) + } else { + callUDF(predict _, DoubleType, col($(featuresCol))) } - outputData + outputData = outputData.withColumn($(predictionCol), predUDF) + numColsOutput += 1 + } + + if (numColsOutput == 0) { + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + + " since no output columns were set.") } + outputData } /** - * :: DeveloperApi :: + * Estimate the probability of each class given the raw prediction, + * doing the computation in-place. + * These predictions are also called class conditional probabilities. + * + * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. * + * @return Estimated class conditional probabilities (modified input vector) + */ + protected def raw2probabilityInPlace(rawPrediction: Vector): Vector + + /** Non-in-place version of [[raw2probabilityInPlace()]] */ + protected def raw2probability(rawPrediction: Vector): Vector = { + val probs = rawPrediction.copy + raw2probabilityInPlace(probs) + } + + /** * Predict the probability of each class given the features. * These predictions are also called class conditional probabilities. * - * WARNING: Not all models output well-calibrated probability estimates! These probabilities - * should be treated as confidences, not precise probabilities. - * * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. + * + * @return Estimated class conditional probabilities + */ + protected def predictProbability(features: FeaturesType): Vector = { + val rawPreds = predictRaw(features) + raw2probabilityInPlace(rawPreds) + } + + /** + * Given a vector of class conditional probabilities, select the predicted label. + * This may be overridden to support thresholds which favor particular labels. + * @return predicted label */ - @DeveloperApi - protected def predictProbabilities(features: FeaturesType): Vector + protected def probability2prediction(probability: Vector): Double = probability.toDense.argmax } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 17f59bb42e129..9954893f14359 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -20,10 +20,9 @@ package org.apache.spark.ml.classification import scala.collection.mutable import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{RandomForestParams, TreeClassifierParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index d379172e0bf53..0e1ff97a8bf60 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -40,8 +40,10 @@ private[shared] object SharedParamsCodeGen { ParamDesc[String]("predictionCol", "prediction column name", Some("\"prediction\"")), ParamDesc[String]("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", Some("\"rawPrediction\"")), - ParamDesc[String]("probabilityCol", - "column name for predicted class conditional probabilities", Some("\"probability\"")), + ParamDesc[String]("probabilityCol", "Column name for predicted class conditional" + + " probabilities. Note: Not all models output well-calibrated probability estimates!" + + " These probabilities should be treated as confidences, not precise probabilities.", + Some("\"probability\"")), ParamDesc[Double]("threshold", "threshold in binary classification prediction, in range [0, 1]", isValid = "ParamValidators.inRange(0, 1)"), diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index fb1874ccfc8dc..87f86807c3c91 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -128,10 +128,10 @@ private[ml] trait HasRawPredictionCol extends Params { private[ml] trait HasProbabilityCol extends Params { /** - * Param for column name for predicted class conditional probabilities. + * Param for Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.. * @group param */ - final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "column name for predicted class conditional probabilities") + final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.") setDefault(probabilityCol, "probability") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index b07c26fe79b36..f8f0b161a4812 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.tree.{TreeRegressorParams, DecisionTreeParams, DecisionTreeModel, Node} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index bc796958e4545..461905c12701a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -21,10 +21,9 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{GBTParams, TreeRegressorParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 66c475f2d9840..e63c9a3eead52 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -25,6 +25,7 @@ import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.{HasElasticNetParam, HasMaxIter, HasRegParam, HasTol} import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -39,7 +40,7 @@ import org.apache.spark.util.StatCounter /** * Params for linear regression. */ -private[regression] trait LinearRegressionParams extends RegressorParams +private[regression] trait LinearRegressionParams extends PredictorParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol /** @@ -240,7 +241,7 @@ class LinearRegressionModel private[ml] ( * + \bar{y} / \hat{y}||^2 * = 1/2n ||\sum_i w_i^\prime x_i - y / \hat{y} + offset||^2 = 1/2n diff^2 * }}} - * where w_i^\prime is the effective weights defined by w_i/\hat{x_i}, offset is + * where w_i^\prime^ is the effective weights defined by w_i/\hat{x_i}, offset is * {{{ * - \sum_i (w_i/\hat{x_i})\bar{x_i} + \bar{y} / \hat{y}. * }}}, and diff is diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 0468a1be1ba74..dbc628927433d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree.{RandomForestParams, TreeRegressorParams} +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{RandomForestParams, TreeRegressorParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index c6b3327db6ad3..c72ef29680329 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -17,62 +17,40 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} -/** - * :: DeveloperApi :: - * Params for regression. - * Currently empty, but may add functionality later. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. - */ -@DeveloperApi -private[spark] trait RegressorParams extends PredictorParams /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Single-label regression * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class Regressor[ FeaturesType, Learner <: Regressor[FeaturesType, Learner, M], M <: RegressionModel[FeaturesType, M]] - extends Predictor[FeaturesType, Learner, M] - with RegressorParams { + extends Predictor[FeaturesType, Learner, M] with PredictorParams { // TODO: defaultEvaluator (follow-up PR) } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Model produced by a [[Regressor]]. * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam M Concrete Model type. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] - extends PredictionModel[FeaturesType, M] with RegressorParams { - - /** - * :: DeveloperApi :: - * - * Predict real-valued label for the given features. - * This internal method is used to implement [[transform()]] and output [[predictionCol]]. - */ - @DeveloperApi - protected def predict(features: FeaturesType): Double +@DeveloperApi +abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] + extends PredictionModel[FeaturesType, M] with PredictorParams { + // TODO: defaultEvaluator (follow-up PR) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala similarity index 99% rename from mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 0e225627d4ee3..816fcedf2efb3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.ml.impl.tree +package org.apache.spark.ml.tree import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.ml.impl.estimator.PredictorParams +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 188d1e542b5b5..f6bcdf83cd337 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -587,6 +587,28 @@ class DenseVector(val values: Array[Double]) extends Vector { } new SparseVector(size, ii, vv) } + + /** + * Find the index of a maximal element. Returns the first maximal element in case of a tie. + * Returns -1 if vector has length 0. + */ + private[spark] def argmax: Int = { + if (size == 0) { + -1 + } else { + var maxIdx = 0 + var maxValue = values(0) + var i = 1 + while (i < size) { + if (values(i) > maxValue) { + maxIdx = i + maxValue = values(i) + } + i += 1 + } + maxIdx + } + } } object DenseVector { From fbf1f342a02af65f69e0ee770a2b983c69e7c089 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 May 2015 17:08:39 -0700 Subject: [PATCH 02/37] [HOT FIX] [SPARK-7418] Ignore flaky SparkSubmitUtilsSuite test --- .../scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 2df2597e058cd..da9578478bed9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -104,7 +104,7 @@ class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { assert(jarPath.indexOf(ivyPath) >= 0, "should use non-default ivy path") } - test("search for artifact at other repositories") { + ignore("search for artifact at other repositories") { val path = SparkSubmitUtils.resolveMavenCoordinates("com.agimatec:agimatec-validation:0.9.3", Option("https://oss.sonatype.org/content/repositories/agimatec/"), None, true) assert(path.indexOf("agimatec-validation") >= 0, "should find package. If it doesn't, check" + From 4e930420c19ae7773b138dfc7db8fc03b4660251 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 6 May 2015 17:28:11 -0700 Subject: [PATCH 03/37] [SPARK-6799] [SPARKR] Remove SparkR RDD examples, add dataframe examples This PR also makes some of the DataFrame to RDD methods private as the RDD class is private in 1.4 cc rxin pwendell Author: Shivaram Venkataraman Closes #5949 from shivaram/sparkr-examples and squashes the following commits: 6c42fdc [Shivaram Venkataraman] Remove SparkR RDD examples, add dataframe examples --- R/pkg/NAMESPACE | 4 - R/pkg/R/DataFrame.R | 2 +- examples/src/main/r/dataframe.R | 54 +++++++++++ examples/src/main/r/kmeans.R | 93 ------------------- examples/src/main/r/linear_solver_mnist.R | 107 ---------------------- examples/src/main/r/logistic_regression.R | 62 ------------- examples/src/main/r/pi.R | 46 ---------- examples/src/main/r/wordcount.R | 42 --------- 8 files changed, 55 insertions(+), 355 deletions(-) create mode 100644 examples/src/main/r/dataframe.R delete mode 100644 examples/src/main/r/kmeans.R delete mode 100644 examples/src/main/r/linear_solver_mnist.R delete mode 100644 examples/src/main/r/logistic_regression.R delete mode 100644 examples/src/main/r/pi.R delete mode 100644 examples/src/main/r/wordcount.R diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 528e6608c3c82..3fb92be0940b7 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -45,8 +45,6 @@ exportMethods("cache", "showDF", "sortDF", "take", - "toJSON", - "toRDD", "unionAll", "unpersist", "where", @@ -95,14 +93,12 @@ export("cacheTable", "createExternalTable", "dropTempTable", "jsonFile", - "jsonRDD", "loadDF", "parquetFile", "sql", "table", "tableNames", "tables", - "toDF", "uncacheTable") export("sparkRSQL.init", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 56c305d912587..47d92f141cc7d 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -272,7 +272,7 @@ setMethod("names", setMethod("registerTempTable", signature(x = "DataFrame", tableName = "character"), function(x, tableName) { - callJMethod(x@sdf, "registerTempTable", tableName) + invisible(callJMethod(x@sdf, "registerTempTable", tableName)) }) #' insertInto diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R new file mode 100644 index 0000000000000..53b817144f6ac --- /dev/null +++ b/examples/src/main/r/dataframe.R @@ -0,0 +1,54 @@ +# +# 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. +# + +library(SparkR) + +# Initialize SparkContext and SQLContext +sc <- sparkR.init(appName="SparkR-DataFrame-example") +sqlContext <- sparkRSQL.init(sc) + +# Create a simple local data.frame +localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) + +# Convert local data frame to a SparkR DataFrame +df <- createDataFrame(sqlContext, localDF) + +# Print its schema +printSchema(df) +# root +# |-- name: string (nullable = true) +# |-- age: double (nullable = true) + +# Create a DataFrame from a JSON file +path <- file.path(Sys.getenv("SPARK_HOME"), "examples/src/main/resources/people.json") +peopleDF <- jsonFile(sqlContext, path) +printSchema(peopleDF) + +# Register this DataFrame as a table. +registerTempTable(peopleDF, "people") + +# SQL statements can be run by using the sql methods provided by sqlContext +teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") + +# Call collect to get a local data.frame +teenagersLocalDF <- collect(teenagers) + +# Print the teenagers in our dataset +print(teenagersLocalDF) + +# Stop the SparkContext now +sparkR.stop() diff --git a/examples/src/main/r/kmeans.R b/examples/src/main/r/kmeans.R deleted file mode 100644 index 6e6b5cb93789c..0000000000000 --- a/examples/src/main/r/kmeans.R +++ /dev/null @@ -1,93 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -# Logistic regression in Spark. -# Note: unlike the example in Scala, a point here is represented as a vector of -# doubles. - -parseVectors <- function(lines) { - lines <- strsplit(as.character(lines) , " ", fixed = TRUE) - list(matrix(as.numeric(unlist(lines)), ncol = length(lines[[1]]))) -} - -dist.fun <- function(P, C) { - apply( - C, - 1, - function(x) { - colSums((t(P) - x)^2) - } - ) -} - -closestPoint <- function(P, C) { - max.col(-dist.fun(P, C)) -} -# Main program - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 3) { - print("Usage: kmeans ") - q("no") -} - -sc <- sparkR.init(appName = "RKMeans") -K <- as.integer(args[[2]]) -convergeDist <- as.double(args[[3]]) - -lines <- textFile(sc, args[[1]]) -points <- cache(lapplyPartition(lines, parseVectors)) -# kPoints <- take(points, K) -kPoints <- do.call(rbind, takeSample(points, FALSE, K, 16189L)) -tempDist <- 1.0 - -while (tempDist > convergeDist) { - closest <- lapplyPartition( - lapply(points, - function(p) { - cp <- closestPoint(p, kPoints); - mapply(list, unique(cp), split.data.frame(cbind(1, p), cp), SIMPLIFY=FALSE) - }), - function(x) {do.call(c, x) - }) - - pointStats <- reduceByKey(closest, - function(p1, p2) { - t(colSums(rbind(p1, p2))) - }, - 2L) - - newPoints <- do.call( - rbind, - collect(lapply(pointStats, - function(tup) { - point.sum <- tup[[2]][, -1] - point.count <- tup[[2]][, 1] - point.sum/point.count - }))) - - D <- dist.fun(kPoints, newPoints) - tempDist <- sum(D[cbind(1:3, max.col(-D))]) - kPoints <- newPoints - cat("Finished iteration (delta = ", tempDist, ")\n") -} - -cat("Final centers:\n") -writeLines(unlist(lapply(kPoints, paste, collapse = " "))) diff --git a/examples/src/main/r/linear_solver_mnist.R b/examples/src/main/r/linear_solver_mnist.R deleted file mode 100644 index c864a4232d010..0000000000000 --- a/examples/src/main/r/linear_solver_mnist.R +++ /dev/null @@ -1,107 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Instructions: https://github.com/amplab-extras/SparkR-pkg/wiki/SparkR-Example:-Digit-Recognition-on-EC2 - -library(SparkR) -library(Matrix) - -args <- commandArgs(trailing = TRUE) - -# number of random features; default to 1100 -D <- ifelse(length(args) > 0, as.integer(args[[1]]), 1100) -# number of partitions for training dataset -trainParts <- 12 -# dimension of digits -d <- 784 -# number of test examples -NTrain <- 60000 -# number of training examples -NTest <- 10000 -# scale of features -gamma <- 4e-4 - -sc <- sparkR.init(appName = "SparkR-LinearSolver") - -# You can also use HDFS path to speed things up: -# hdfs:///train-mnist-dense-with-labels.data -file <- textFile(sc, "/data/train-mnist-dense-with-labels.data", trainParts) - -W <- gamma * matrix(nrow=D, ncol=d, data=rnorm(D*d)) -b <- 2 * pi * matrix(nrow=D, ncol=1, data=runif(D)) -broadcastW <- broadcast(sc, W) -broadcastB <- broadcast(sc, b) - -includePackage(sc, Matrix) -numericLines <- lapplyPartitionsWithIndex(file, - function(split, part) { - matList <- sapply(part, function(line) { - as.numeric(strsplit(line, ",", fixed=TRUE)[[1]]) - }, simplify=FALSE) - mat <- Matrix(ncol=d+1, data=unlist(matList, F, F), - sparse=T, byrow=T) - mat - }) - -featureLabels <- cache(lapplyPartition( - numericLines, - function(part) { - label <- part[,1] - mat <- part[,-1] - ones <- rep(1, nrow(mat)) - features <- cos( - mat %*% t(value(broadcastW)) + (matrix(ncol=1, data=ones) %*% t(value(broadcastB)))) - onesMat <- Matrix(ones) - featuresPlus <- cBind(features, onesMat) - labels <- matrix(nrow=nrow(mat), ncol=10, data=-1) - for (i in 1:nrow(mat)) { - labels[i, label[i]] <- 1 - } - list(label=labels, features=featuresPlus) - })) - -FTF <- Reduce("+", collect(lapplyPartition(featureLabels, - function(part) { - t(part$features) %*% part$features - }), flatten=F)) - -FTY <- Reduce("+", collect(lapplyPartition(featureLabels, - function(part) { - t(part$features) %*% part$label - }), flatten=F)) - -# solve for the coefficient matrix -C <- solve(FTF, FTY) - -test <- Matrix(as.matrix(read.csv("/data/test-mnist-dense-with-labels.data", - header=F), sparse=T)) -testData <- test[,-1] -testLabels <- matrix(ncol=1, test[,1]) - -err <- 0 - -# contstruct the feature maps for all examples from this digit -featuresTest <- cos(testData %*% t(value(broadcastW)) + - (matrix(ncol=1, data=rep(1, NTest)) %*% t(value(broadcastB)))) -featuresTest <- cBind(featuresTest, Matrix(rep(1, NTest))) - -# extract the one vs. all assignment -results <- featuresTest %*% C -labelsGot <- apply(results, 1, which.max) -err <- sum(testLabels != labelsGot) / nrow(testLabels) - -cat("\nFinished running. The error rate is: ", err, ".\n") diff --git a/examples/src/main/r/logistic_regression.R b/examples/src/main/r/logistic_regression.R deleted file mode 100644 index 2a86aa98160d3..0000000000000 --- a/examples/src/main/r/logistic_regression.R +++ /dev/null @@ -1,62 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 3) { - print("Usage: logistic_regression ") - q("no") -} - -# Initialize Spark context -sc <- sparkR.init(appName = "LogisticRegressionR") -iterations <- as.integer(args[[2]]) -D <- as.integer(args[[3]]) - -readPartition <- function(part){ - part = strsplit(part, " ", fixed = T) - list(matrix(as.numeric(unlist(part)), ncol = length(part[[1]]))) -} - -# Read data points and convert each partition to a matrix -points <- cache(lapplyPartition(textFile(sc, args[[1]]), readPartition)) - -# Initialize w to a random value -w <- runif(n=D, min = -1, max = 1) -cat("Initial w: ", w, "\n") - -# Compute logistic regression gradient for a matrix of data points -gradient <- function(partition) { - partition = partition[[1]] - Y <- partition[, 1] # point labels (first column of input file) - X <- partition[, -1] # point coordinates - - # For each point (x, y), compute gradient function - dot <- X %*% w - logit <- 1 / (1 + exp(-Y * dot)) - grad <- t(X) %*% ((logit - 1) * Y) - list(grad) -} - -for (i in 1:iterations) { - cat("On iteration ", i, "\n") - w <- w - reduce(lapplyPartition(points, gradient), "+") -} - -cat("Final w: ", w, "\n") diff --git a/examples/src/main/r/pi.R b/examples/src/main/r/pi.R deleted file mode 100644 index aa7a833e147a0..0000000000000 --- a/examples/src/main/r/pi.R +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -sc <- sparkR.init(appName = "PiR") - -slices <- ifelse(length(args) > 1, as.integer(args[[2]]), 2) - -n <- 100000 * slices - -piFunc <- function(elem) { - rands <- runif(n = 2, min = -1, max = 1) - val <- ifelse((rands[1]^2 + rands[2]^2) < 1, 1.0, 0.0) - val -} - - -piFuncVec <- function(elems) { - message(length(elems)) - rands1 <- runif(n = length(elems), min = -1, max = 1) - rands2 <- runif(n = length(elems), min = -1, max = 1) - val <- ifelse((rands1^2 + rands2^2) < 1, 1.0, 0.0) - sum(val) -} - -rdd <- parallelize(sc, 1:n, slices) -count <- reduce(lapplyPartition(rdd, piFuncVec), sum) -cat("Pi is roughly", 4.0 * count / n, "\n") -cat("Num elements in RDD ", count(rdd), "\n") diff --git a/examples/src/main/r/wordcount.R b/examples/src/main/r/wordcount.R deleted file mode 100644 index b734cb0ecf55b..0000000000000 --- a/examples/src/main/r/wordcount.R +++ /dev/null @@ -1,42 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 1) { - print("Usage: wordcount ") - q("no") -} - -# Initialize Spark context -sc <- sparkR.init(appName = "RwordCount") -lines <- textFile(sc, args[[1]]) - -words <- flatMap(lines, - function(line) { - strsplit(line, " ")[[1]] - }) -wordCount <- lapply(words, function(word) { list(word, 1L) }) - -counts <- reduceByKey(wordCount, "+", 2L) -output <- collect(counts) - -for (wordcount in output) { - cat(wordcount[[1]], ": ", wordcount[[2]], "\n") -} From 316a5c0423ba3688cacd3acc3c5b5571e8a71d1d Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 6 May 2015 17:44:43 -0700 Subject: [PATCH 04/37] [SPARK-7396] [STREAMING] [EXAMPLE] Update KafkaWordCountProducer to use new Producer API Otherwise it will throw exception: ``` Exception in thread "main" kafka.common.FailedToSendMessageException: Failed to send messages after 3 tries. at kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90) at kafka.producer.Producer.send(Producer.scala:77) at org.apache.spark.examples.streaming.KafkaWordCountProducer$.main(KafkaWordCount.scala:96) at org.apache.spark.examples.streaming.KafkaWordCountProducer.main(KafkaWordCount.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:623) at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:169) at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:192) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:111) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) ``` Author: jerryshao Closes #5936 from jerryshao/SPARK-7396 and squashes the following commits: 270bbe2 [jerryshao] Fix Kafka Produce throw Exception issue --- .../examples/streaming/KafkaWordCount.scala | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 387c0e421334b..f407367a54f6c 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -17,9 +17,9 @@ package org.apache.spark.examples.streaming -import java.util.Properties +import java.util.HashMap -import kafka.producer._ +import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer, ProducerRecord} import org.apache.spark.streaming._ import org.apache.spark.streaming.kafka._ @@ -77,23 +77,25 @@ object KafkaWordCountProducer { val Array(brokers, topic, messagesPerSec, wordsPerMessage) = args // Zookeeper connection properties - val props = new Properties() - props.put("metadata.broker.list", brokers) - props.put("serializer.class", "kafka.serializer.StringEncoder") + val props = new HashMap[String, Object]() + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer") - val config = new ProducerConfig(props) - val producer = new Producer[String, String](config) + val producer = new KafkaProducer[String, String](props) // Send some messages while(true) { - val messages = (1 to messagesPerSec.toInt).map { messageNum => + (1 to messagesPerSec.toInt).foreach { messageNum => val str = (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString) .mkString(" ") - new KeyedMessage[String, String](topic, str) - }.toArray + val message = new ProducerRecord[String, String](topic, null, str) + producer.send(message) + } - producer.send(messages: _*) Thread.sleep(100) } } From 8fa6829f5e6d8b8f33097c3f20fcb96198d4031b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 May 2015 17:52:34 -0700 Subject: [PATCH 05/37] [SPARK-7371] [SPARK-7377] [SPARK-7408] DAG visualization addendum (#5729) This is a follow-up patch for #5729. **[SPARK-7408]** Move as much style code from JS to CSS as possible **[SPARK-7377]** Fix JS error if a job / stage contains only one RDD **[SPARK-7371]** Decrease emphasis on RDD on stage page as requested by mateiz pwendell This patch also includes general code clean up. Author: Andrew Or Closes #5954 from andrewor14/viz-emphasize-rdd and squashes the following commits: 3c0d4f0 [Andrew Or] Guard against JS error by rendering arrows only if needed f23e15b [Andrew Or] Merge branch 'master' of github.com:apache/spark into viz-emphasize-rdd 565801f [Andrew Or] Clean up code 9dab5f0 [Andrew Or] Move styling from JS to CSS + clean up code 107c0b6 [Andrew Or] Tweak background color, stroke width, font size etc. 1610c62 [Andrew Or] Implement cluster padding for stage page --- .../apache/spark/ui/static/dagre-d3.min.js | 14 +- .../apache/spark/ui/static/spark-dag-viz.css | 83 ++++ .../apache/spark/ui/static/spark-dag-viz.js | 357 ++++++++++-------- .../scala/org/apache/spark/ui/UIUtils.scala | 14 +- .../spark/ui/scope/RDDOperationGraph.scala | 19 +- 5 files changed, 310 insertions(+), 177 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css 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 6d2da25024a83..6beb7db855ca0 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 @@ -/*v0.4.3 with 1 additional commit (see http://github.com/andrewor14/dagre-d3)*/(function(f){if(typeof exports==="object"&&typeof module!=="undefined"){module.exports=f()}else if(typeof define==="function"&&define.amd){define([],f)}else{var g;if(typeof window!=="undefined"){g=window}else if(typeof global!=="undefined"){g=global}else if(typeof self!=="undefined"){g=self}else{g=this}g.dagreD3=f()}})(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)));_.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(); +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 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});svgClusters.enter().append("g").attr("id",function(v){return"cluster_"+v.replace(/^cluster/,"")}).attr("name",function(v){return g.node(v).label}).attr("class","cluster").style("opacity",0).append("rect");util.applyTransition(svgClusters.exit(),g).style("opacity",0).remove();util.applyTransition(svgClusters,g).style("opacity",1);util.applyTransition(svgClusters.selectAll("rect"),g).attr("width",function(v){var node=g.node(v);return node.width+node.paddingLeft+node.paddingRight}).attr("height",function(v){var node=g.node(v);return node.height+node.paddingTop+node.paddingBottom}).attr("x",function(v){var node=g.node(v);return node.x-node.width/2-node.paddingLeft}).attr("y",function(v){var node=g.node(v);return node.y-node.height/2-node.paddingTop})}},{"./util":25}],4:[function(require,module,exports){"use strict";var _=require("./lodash"),addLabel=require("./label/add-label"),util=require("./util"),d3=require("./d3");module.exports=createEdgeLabels;function createEdgeLabels(selection,g){var svgEdgeLabels=selection.selectAll("g.edgeLabel").data(g.edges(),function(e){return util.edgeToId(e)}).classed("update",true);svgEdgeLabels.selectAll("*").remove();svgEdgeLabels.enter().append("g").classed("edgeLabel",true).style("opacity",0);svgEdgeLabels.each(function(e){var edge=g.edge(e),label=addLabel(d3.select(this),g.edge(e),0,0).classed("label",true),bbox=label.node().getBBox();if(edge.labelId){label.attr("id",edge.labelId)}if(!_.has(edge,"width")){edge.width=bbox.width}if(!_.has(edge,"height")){edge.height=bbox.height}});util.applyTransition(svgEdgeLabels.exit(),g).style("opacity",0).remove();return svgEdgeLabels}},{"./d3":7,"./label/add-label":18,"./lodash":20,"./util":25}],5:[function(require,module,exports){"use strict";var _=require("./lodash"),intersectNode=require("./intersect/intersect-node"),util=require("./util"),d3=require("./d3");module.exports=createEdgePaths;function createEdgePaths(selection,g,arrows){var svgPaths=selection.selectAll("g.edgePath").data(g.edges(),function(e){return util.edgeToId(e)}).classed("update",true);enter(svgPaths,g);exit(svgPaths,g);util.applyTransition(svgPaths,g).style("opacity",1);svgPaths.each(function(e){var domEdge=d3.select(this);var edge=g.edge(e);edge.elem=this;if(edge.id){domEdge.attr("id",edge.id)}util.applyClass(domEdge,edge["class"],(domEdge.classed("update")?"update ":"")+"edgePath")});svgPaths.selectAll("path.path").each(function(e){var edge=g.edge(e);edge.arrowheadId=_.uniqueId("arrowhead");var domEdge=d3.select(this).attr("marker-end",function(){return"url(#"+edge.arrowheadId+")"}).style("fill","none");util.applyTransition(domEdge,g).attr("d",function(e){return calcPoints(g,e)});util.applyStyle(domEdge,edge.style)});svgPaths.selectAll("defs *").remove();svgPaths.selectAll("defs").each(function(e){var edge=g.edge(e),arrowhead=arrows[edge.arrowhead];arrowhead(d3.select(this),edge.arrowheadId,edge,"arrowhead")});return svgPaths}function calcPoints(g,e){var edge=g.edge(e),tail=g.node(e.v),head=g.node(e.w),points=edge.points.slice(1,edge.points.length-1);points.unshift(intersectNode(tail,points[0]));points.push(intersectNode(head,points[points.length-1]));return createLine(edge,points)}function createLine(edge,points){var line=d3.svg.line().x(function(d){return d.x}).y(function(d){return d.y});if(_.has(edge,"lineInterpolate")){line.interpolate(edge.lineInterpolate)}if(_.has(edge,"lineTension")){line.tension(Number(edge.lineTension))}return line(points)}function getCoords(elem){var bbox=elem.getBBox(),matrix=elem.getTransformToElement(elem.ownerSVGElement).translate(bbox.width/2,bbox.height/2);return{x:matrix.e,y:matrix.f}}function enter(svgPaths,g){var svgPathsEnter=svgPaths.enter().append("g").attr("class","edgePath").style("opacity",0);svgPathsEnter.append("path").attr("class","path").attr("d",function(e){var edge=g.edge(e),sourceElem=g.node(e.v).elem,points=_.range(edge.points.length).map(function(){return getCoords(sourceElem)});return createLine(edge,points)});svgPathsEnter.append("defs")}function exit(svgPaths,g){var svgPathExit=svgPaths.exit();util.applyTransition(svgPathExit,g).style("opacity",0).remove();util.applyTransition(svgPathExit.select("path.path"),g).attr("d",function(e){var source=g.node(e.v);if(source){var points=_.range(this.pathSegList.length).map(function(){return source});return createLine({},points)}else{return d3.select(this).attr("d")}})}},{"./d3":7,"./intersect/intersect-node":14,"./lodash":20,"./util":25}],6:[function(require,module,exports){"use strict";var _=require("./lodash"),addLabel=require("./label/add-label"),util=require("./util"),d3=require("./d3");module.exports=createNodes;function createNodes(selection,g,shapes){var simpleNodes=g.nodes().filter(function(v){return!util.isSubgraph(g,v)});var svgNodes=selection.selectAll("g.node").data(simpleNodes,function(v){return v}).classed("update",true);svgNodes.selectAll("*").remove();svgNodes.enter().append("g").attr("id",function(v){return"node_"+v}).attr("name",function(v){return g.node(v).label}).attr("class","node").style("opacity",0);svgNodes.each(function(v){var node=g.node(v),thisGroup=d3.select(this),labelGroup=thisGroup.append("g").attr("class","label"),labelDom=addLabel(labelGroup,node),shape=shapes[node.shape],bbox=_.pick(labelDom.node().getBBox(),"width","height");node.elem=this;if(node.id){thisGroup.attr("id",node.id)}if(node.labelId){labelGroup.attr("id",node.labelId)}util.applyClass(thisGroup,node["class"],(thisGroup.classed("update")?"update ":"")+"node");if(_.has(node,"width")){bbox.width=node.width}if(_.has(node,"height")){bbox.height=node.height}bbox.width+=node.paddingLeft+node.paddingRight;bbox.height+=node.paddingTop+node.paddingBottom;labelGroup.attr("transform","translate("+(node.paddingLeft-node.paddingRight)/2+","+(node.paddingTop-node.paddingBottom)/2+")");var shapeSvg=shape(d3.select(this),bbox,node);util.applyStyle(shapeSvg,node.style);var shapeBBox=shapeSvg.node().getBBox();node.width=shapeBBox.width;node.height=shapeBBox.height});util.applyTransition(svgNodes.exit(),g).style("opacity",0).remove();return svgNodes}},{"./d3":7,"./label/add-label":18,"./lodash":20,"./util":25}],7:[function(require,module,exports){module.exports=window.d3},{}],8:[function(require,module,exports){var dagre;if(require){try{dagre=require("dagre")}catch(e){}}if(!dagre){dagre=window.dagre}module.exports=dagre},{dagre:27}],9:[function(require,module,exports){var graphlib;if(require){try{graphlib=require("graphlib")}catch(e){}}if(!graphlib){graphlib=window.graphlib}module.exports=graphlib},{graphlib:57}],10:[function(require,module,exports){module.exports={node:require("./intersect-node"),circle:require("./intersect-circle"),ellipse:require("./intersect-ellipse"),polygon:require("./intersect-polygon"),rect:require("./intersect-rect")}},{"./intersect-circle":11,"./intersect-ellipse":12,"./intersect-node":14,"./intersect-polygon":15,"./intersect-rect":16}],11:[function(require,module,exports){var intersectEllipse=require("./intersect-ellipse");module.exports=intersectCircle;function intersectCircle(node,rx,point){return intersectEllipse(node,rx,rx,point)}},{"./intersect-ellipse":12}],12:[function(require,module,exports){module.exports=intersectEllipse;function intersectEllipse(node,rx,ry,point){var cx=node.x;var cy=node.y;var px=cx-point.x;var py=cy-point.y;var det=Math.sqrt(rx*rx*py*py+ry*ry*px*px);var dx=Math.abs(rx*ry*px/det);if(point.x0}},{}],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))); -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){arguments[4][20][0].apply(exports,arguments)},{dup: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&&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;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){arguments[4][20][0].apply(exports,arguments)},{dup: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; +_.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&&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; -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(++indexMath.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 0) { - // Take into account the position and width of the last stage's container - var existingStages = stageClusters(); + var existingStages = svgContainer + .selectAll("g.cluster") + .filter("[id*=\"" + VizConstants.stageClusterPrefix + "\"]"); if (!existingStages.empty()) { - var lastStage = existingStages[0].pop(); - var lastStageId = d3.select(lastStage).attr("id"); - var lastStageWidth = toFloat(d3.select("#" + lastStageId + " rect").attr("width")); - var lastStagePosition = getAbsolutePosition(lastStageId); + var lastStage = d3.select(existingStages[0].pop()); + var lastStageId = lastStage.attr("id"); + var lastStageWidth = toFloat(svgContainer + .select("#" + lastStageId) + .select("rect") + .attr("width")); + var lastStagePosition = getAbsolutePosition(lastStage); var offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep; container.attr("transform", "translate(" + offset + ", 0)"); } } + + // Actually render the stage renderDot(dot, container); + // If there are any incoming edges into this graph, keep track of them to render // them separately later. Note that we cannot draw them now because we need to // put these edges in a separate container that is on top of all stage graphs. @@ -221,15 +218,7 @@ function renderDagVizForJob(svgContainer) { }); }); - // Draw edges that cross stages - if (crossStageEdges.length > 0) { - var container = svgContainer.append("g").attr("id", "cross-stage-edges"); - for (var i = 0; i < crossStageEdges.length; i++) { - var fromRDDId = crossStageEdges[i][0]; - var toRDDId = crossStageEdges[i][1]; - connectRDDs(fromRDDId, toRDDId, container); - } - } + drawCrossStageEdges(crossStageEdges, svgContainer); } /* Render the dot file as an SVG in the given container. */ @@ -243,99 +232,156 @@ function renderDot(dot, container) { renderer(container, g); } -/* Style the visualization we just rendered. */ -function styleDagViz(forJob) { - graphContainer().selectAll("svg g.cluster rect") - .style("fill", "white") - .style("stroke", VizConstants.rddOperationColor) - .style("stroke-width", "4px") - .style("stroke-opacity", "0.5"); - graphContainer().selectAll("svg g.cluster text") - .attr("fill", VizConstants.clusterLabelColor) - .attr("font-size", "11px"); - graphContainer().selectAll("svg path") - .style("stroke", VizConstants.edgeColor) - .style("stroke-width", VizConstants.edgeWidth); - stageClusters() - .select("rect") - .style("stroke", VizConstants.stageColor) - .style("strokeWidth", "6px"); - - // Put an arrow at the end of every edge - // We need to do this because we manually render some edges ourselves - // For these edges, we borrow the arrow marker generated by dagre-d3 - var dagreD3Marker = graphContainer().select("svg g.edgePaths marker").node(); - graphContainer().select("svg") - .append(function() { return dagreD3Marker.cloneNode(true); }) - .attr("id", "marker-arrow") - .select("path") - .attr("fill", VizConstants.edgeColor) - .attr("strokeWidth", "0px"); - graphContainer().selectAll("svg g > path").attr("marker-end", "url(#marker-arrow)"); - graphContainer().selectAll("svg g.edgePaths def").remove(); // We no longer need these - - // Apply any job or stage specific styles +/* -------------------- * + * | Helper functions | * + * -------------------- */ + +// Helper d3 accessors +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. + */ +function drawClusterLabels(svgContainer, forJob) { if (forJob) { - styleDagVizForJob(); + var clusterLabelSize = JobPageVizConstants.clusterLabelSize; + var stageClusterLabelSize = JobPageVizConstants.stageClusterLabelSize; } else { - styleDagVizForStage(); + var clusterLabelSize = StagePageVizConstants.clusterLabelSize; + var stageClusterLabelSize = StagePageVizConstants.stageClusterLabelSize; } + 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); + }); } -/* Apply job-page-specific style to the visualization. */ -function styleDagVizForJob() { - graphContainer().selectAll("svg g.node circle") - .style("fill", VizConstants.rddColor); - // TODO: add a legend to explain what a highlighted dot means - graphContainer().selectAll("svg g.cached circle") - .style("fill", VizConstants.rddCachedColor); - graphContainer().selectAll("svg g#cross-stage-edges path") - .style("fill", "none"); +/* + * 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) + .text(labelText); } -/* Apply stage-page-specific style to the visualization. */ -function styleDagVizForStage() { - graphContainer().selectAll("svg g.node rect") - .style("fill", "none") - .style("stroke", VizConstants.rddColor) - .style("stroke-width", "2px") - .attr("rx", "5") // round corners - .attr("ry", "5"); - // TODO: add a legend to explain what a highlighted RDD means - graphContainer().selectAll("svg g.cached rect") - .style("stroke", VizConstants.rddCachedColor); - graphContainer().selectAll("svg g.node g.label text tspan") - .style("fill", VizConstants.rddColor); +/* + * Helper function to size the SVG appropriately such that all elements are displyed. + * This assumes that all outermost elements are clusters (rectangles). + */ +function resizeSvg(svg) { + var allClusters = svg.selectAll("g.cluster rect")[0]; + var startX = -VizConstants.svgMarginX + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).x; + })); + var startY = -VizConstants.svgMarginY + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).y; + })); + var endX = VizConstants.svgMarginX + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e) + return getAbsolutePosition(t).x + toFloat(t.attr("width")); + })); + var endY = VizConstants.svgMarginY + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e) + return getAbsolutePosition(t).y + toFloat(t.attr("height")); + })); + var width = endX - startX; + var height = endY - startY; + svg.attr("viewBox", startX + " " + startY + " " + width + " " + height) + .attr("width", width) + .attr("height", height); } /* - * (Job page only) Helper method to compute the absolute - * position of the group element identified by the given ID. + * (Job page only) Helper function to draw edges that cross stage boundaries. + * We need to do this manually because we render each stage separately in dagre-d3. */ -function getAbsolutePosition(groupId) { - var obj = d3.select("#" + groupId).filter("g"); - var _x = 0, _y = 0; +function drawCrossStageEdges(edges, svgContainer) { + if (edges.length == 0) { + return; + } + // Draw the paths first + var edgesContainer = svgContainer.append("g").attr("id", "cross-stage-edges"); + for (var i = 0; i < edges.length; i++) { + var fromRDDId = edges[i][0]; + var toRDDId = edges[i][1]; + connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer); + } + // Now draw the arrows by borrowing the arrow marker generated by dagre-d3 + var dagreD3Marker = svgContainer.select("g.edgePaths marker").node(); + if (!dagreD3Marker.empty()) { + svgContainer + .append(function() { return dagreD3Marker.cloneNode(true); }) + .attr("id", "marker-arrow") + svgContainer.selectAll("g > path").attr("marker-end", "url(#marker-arrow)"); + svgContainer.selectAll("g.edgePaths def").remove(); // We no longer need these + } +} + +/* + * (Job page only) Helper function to compute the absolute + * position of the specified element in our graph. + */ +function getAbsolutePosition(d3selection) { + if (d3selection.empty()) { + throw "Attempted to get absolute position of an empty selection."; + } + var obj = d3selection; + var _x = toFloat(obj.attr("x")) || 0; + var _y = toFloat(obj.attr("y")) || 0; while (!obj.empty()) { var transformText = obj.attr("transform"); - var translate = d3.transform(transformText).translate - _x += translate[0]; - _y += translate[1]; - obj = d3.select(obj.node().parentNode).filter("g") + if (transformText) { + var translate = d3.transform(transformText).translate; + _x += toFloat(translate[0]); + _y += toFloat(translate[1]); + } + // Climb upwards to find how our parents are translated + obj = d3.select(obj.node().parentNode); + // Stop when we've reached the graph container itself + if (obj.node() == graphContainer().node()) { + break; + } } return { x: _x, y: _y }; } -/* (Job page only) Connect two RDD nodes with a curved edge. */ -function connectRDDs(fromRDDId, toRDDId, container) { +/* (Job page only) Helper function to connect two RDDs with a curved edge. */ +function connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer) { var fromNodeId = VizConstants.nodePrefix + fromRDDId; - var toNodeId = VizConstants.nodePrefix + toRDDId - var fromPos = getAbsolutePosition(fromNodeId); - var toPos = getAbsolutePosition(toNodeId); + var toNodeId = VizConstants.nodePrefix + toRDDId; + var fromPos = getAbsolutePosition(svgContainer.select("#" + fromNodeId)); + var toPos = getAbsolutePosition(svgContainer.select("#" + toNodeId)); // On the job page, RDDs are rendered as dots (circles). When rendering the path, // we need to account for the radii of these circles. Otherwise the arrow heads // will bleed into the circle itself. - var delta = toFloat(graphContainer() + var delta = toFloat(svgContainer .select("g.node#" + toNodeId) .select("circle") .attr("r")); @@ -375,18 +421,15 @@ function connectRDDs(fromRDDId, toRDDId, container) { } var line = d3.svg.line().interpolate("basis"); - container.append("path").datum(points).attr("d", line); + edgesContainer.append("path").datum(points).attr("d", line); } -/* Helper d3 accessor to clusters that represent stages. */ -function stageClusters() { - return graphContainer().selectAll("g.cluster").filter(function() { - return d3.select(this).attr("id").indexOf(VizConstants.stageClusterPrefix) > -1; - }); -} - -/* Helper method to convert attributes to numeric values. */ +/* Helper function to convert attributes to numeric values. */ function toFloat(f) { - return parseFloat(f.replace(/px$/, "")); + if (f) { + return parseFloat(f.toString().replace(/px$/, "")); + } else { + return f; + } } 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 2f3fb181e4026..e2d03f8342315 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -156,13 +156,10 @@ private[spark] object UIUtils extends Logging { def commonHeaderNodes: Seq[Node] = { - - - - + + + + @@ -174,6 +171,7 @@ private[spark] object UIUtils extends Logging { } def vizHeaderNodes: Seq[Node] = { + @@ -358,7 +356,7 @@ private[spark] object UIUtils extends Logging {
{ graphs.map { g => - +## ElementwiseProduct + +ElementwiseProduct multiplies each input vector by a provided "weight" vector, using element-wise multiplication. In other words, it scales each column of the dataset by a scalar multiplier. This represents the [Hadamard product](https://en.wikipedia.org/wiki/Hadamard_product_%28matrices%29) between the input vector, `v` and transforming vector, `w`, to yield a result vector. + +`\[ \begin{pmatrix} +v_1 \\ +\vdots \\ +v_N +\end{pmatrix} \circ \begin{pmatrix} + w_1 \\ + \vdots \\ + w_N + \end{pmatrix} += \begin{pmatrix} + v_1 w_1 \\ + \vdots \\ + v_N w_N + \end{pmatrix} +\]` + +[`ElementwiseProduct`](api/scala/index.html#org.apache.spark.mllib.feature.ElementwiseProduct) has the following parameter in the constructor: + +* `w`: the transforming vector. + +`ElementwiseProduct` implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) which can apply the weighting on a `Vector` to produce a transformed `Vector` or on an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +### Example + +This example below demonstrates how to load a simple vectors file, extract a set of vectors, then transform those vectors using a transforming vector value. + + +
+
+{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.ElementwiseProduct +import org.apache.spark.mllib.linalg.Vectors + +// Load and parse the data: +val data = sc.textFile("data/mllib/kmeans_data.txt") +val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))) + +val transformingVector = Vectors.dense(0.0, 1.0, 2.0) +val transformer = new ElementwiseProduct(transformingVector) + +// Batch transform and per-row transform give the same results: +val transformedData = transformer.transform(parsedData) +val transformedData2 = parsedData.map(x => transformer.transform(x)) + +{% endhighlight %} +
+
+ + diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala new file mode 100644 index 0000000000000..f8b56293e3ccc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -0,0 +1,55 @@ +/* + * 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.UnaryTransformer +import org.apache.spark.ml.param.Param +import org.apache.spark.mllib.feature +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql.types.DataType + +/** + * :: AlphaComponent :: + * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a + * provided "weight" vector. In other words, it scales each column of the dataset by a scalar + * multiplier. + */ +@AlphaComponent +class ElementwiseProduct extends UnaryTransformer[Vector, Vector, ElementwiseProduct] { + + /** + * the vector to multiply with input vectors + * @group param + */ + val scalingVec: Param[Vector] = new Param(this, "scalingVector", "vector for hadamard product") + + /** @group setParam */ + def setScalingVec(value: Vector): this.type = set(scalingVec, value) + + /** @group getParam */ + def getScalingVec: Vector = getOrDefault(scalingVec) + + override protected def createTransformFunc: Vector => Vector = { + require(params.contains(scalingVec), s"transformation requires a weight vector") + val elemScaler = new feature.ElementwiseProduct($(scalingVec)) + elemScaler.transform + } + + override protected def outputDataType: DataType = new VectorUDT() +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala new file mode 100644 index 0000000000000..b0985baf9b278 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala @@ -0,0 +1,64 @@ +/* + * 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.mllib.feature + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg._ + +/** + * :: Experimental :: + * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a + * provided "weight" vector. In other words, it scales each column of the dataset by a scalar + * multiplier. + * @param scalingVector The values used to scale the reference vector's individual components. + */ +@Experimental +class ElementwiseProduct(val scalingVector: Vector) extends VectorTransformer { + + /** + * Does the hadamard product transformation. + * + * @param vector vector to be transformed. + * @return transformed vector. + */ + override def transform(vector: Vector): Vector = { + require(vector.size == scalingVector.size, + s"vector sizes do not match: Expected ${scalingVector.size} but found ${vector.size}") + vector match { + case dv: DenseVector => + val values: Array[Double] = dv.values.clone() + val dim = scalingVector.size + var i = 0 + while (i < dim) { + values(i) *= scalingVector(i) + i += 1 + } + Vectors.dense(values) + case SparseVector(size, indices, vs) => + val values = vs.clone() + val dim = values.length + var i = 0 + while (i < dim) { + values(i) *= scalingVector(indices(i)) + i += 1 + } + Vectors.sparse(size, indices, values) + case v => throw new IllegalArgumentException("Does not support vector type " + v.getClass) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala new file mode 100644 index 0000000000000..f3a482abda873 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala @@ -0,0 +1,61 @@ +/* + * 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.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class ElementwiseProductSuite extends FunSuite with MLlibTestSparkContext { + + test("elementwise (hadamard) product should properly apply vector to dense data set") { + val denseData = Array( + Vectors.dense(1.0, 4.0, 1.9, -9.0) + ) + val scalingVec = Vectors.dense(2.0, 0.5, 0.0, 0.25) + val transformer = new ElementwiseProduct(scalingVec) + val transformedData = transformer.transform(sc.makeRDD(denseData)) + val transformedVecs = transformedData.collect() + val transformedVec = transformedVecs(0) + val expectedVec = Vectors.dense(2.0, 2.0, 0.0, -2.25) + assert(transformedVec ~== expectedVec absTol 1E-5, + s"Expected transformed vector $expectedVec but found $transformedVec") + } + + test("elementwise (hadamard) product should properly apply vector to sparse data set") { + val sparseData = Array( + Vectors.sparse(3, Seq((1, -1.0), (2, -3.0))) + ) + val dataRDD = sc.parallelize(sparseData, 3) + val scalingVec = Vectors.dense(1.0, 0.0, 0.5) + val transformer = new ElementwiseProduct(scalingVec) + val data2 = sparseData.map(transformer.transform) + val data2RDD = transformer.transform(dataRDD) + + assert((sparseData, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after hadamard product") + + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert(data2(0) ~== Vectors.sparse(3, Seq((1, 0.0), (2, -1.5))) absTol 1E-5) + } +} From e43803b8f477b2c8d28836ac163cb54328d13f1a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 7 May 2015 15:45:37 -0700 Subject: [PATCH 30/37] [SPARK-6948] [MLLIB] compress vectors in VectorAssembler The compression is based on storage. brkyvz Author: Xiangrui Meng Closes #5985 from mengxr/SPARK-6948 and squashes the following commits: df56a00 [Xiangrui Meng] update python tests 6d90d45 [Xiangrui Meng] compress vectors in VectorAssembler --- .../org/apache/spark/ml/feature/VectorAssembler.scala | 2 +- .../apache/spark/ml/feature/VectorAssemblerSuite.scala | 10 +++++++++- python/pyspark/ml/feature.py | 6 +++--- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index b5a69cee6daf3..796758a70ef18 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -102,6 +102,6 @@ object VectorAssembler { case o => throw new SparkException(s"$o of type ${o.getClass.getName} is not supported.") } - Vectors.sparse(cur, indices.result(), values.result()) + Vectors.sparse(cur, indices.result(), values.result()).compressed } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index 57d0278e03639..0db27607bc274 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.scalatest.FunSuite import org.apache.spark.SparkException -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{Row, SQLContext} @@ -48,6 +48,14 @@ class VectorAssemblerSuite extends FunSuite with MLlibTestSparkContext { } } + test("assemble should compress vectors") { + import org.apache.spark.ml.feature.VectorAssembler.assemble + val v1 = assemble(0.0, 0.0, 0.0, Vectors.dense(4.0)) + assert(v1.isInstanceOf[SparseVector]) + val v2 = assemble(1.0, 2.0, 3.0, Vectors.sparse(1, Array(0), Array(4.0))) + assert(v2.isInstanceOf[DenseVector]) + } + test("VectorAssembler") { val df = sqlContext.createDataFrame(Seq( (0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 8a0fdddd2d9b5..705a368192c24 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -121,12 +121,12 @@ class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol): >>> df = sc.parallelize([Row(a=1, b=0, c=3)]).toDF() >>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features") >>> vecAssembler.transform(df).head().features - SparseVector(3, {0: 1.0, 2: 3.0}) + DenseVector([1.0, 0.0, 3.0]) >>> vecAssembler.setParams(outputCol="freqs").transform(df).head().freqs - SparseVector(3, {0: 1.0, 2: 3.0}) + DenseVector([1.0, 0.0, 3.0]) >>> params = {vecAssembler.inputCols: ["b", "a"], vecAssembler.outputCol: "vector"} >>> vecAssembler.transform(df, params).head().vector - SparseVector(2, {1: 1.0}) + DenseVector([0.0, 1.0]) """ _java_class = "org.apache.spark.ml.feature.VectorAssembler" From 97d1182af63d55abab44521171652c81c56c6af6 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 7 May 2015 16:21:24 -0700 Subject: [PATCH 31/37] [SQL] [MINOR] make star and multialias extend NamedExpression `Star` and `MultiAlias` just used in `analyzer` and them will be substituted after analyze, So just like `Alias` they do not need extend `Attribute` Author: scwf Closes #5928 from scwf/attribute and squashes the following commits: 73a0560 [scwf] star and multialias do not need extend attribute --- .../sql/catalyst/analysis/unresolved.scala | 20 +++++-------------- .../expressions/namedExpressions.scala | 2 +- 2 files changed, 6 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 3f567e3e8b2a6..eb736ac3290e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -95,7 +95,7 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E * Represents all of the input attributes to a given relational operator, for example in * "SELECT * FROM ...". A [[Star]] gets automatically expanded during analysis. */ -trait Star extends Attribute with trees.LeafNode[Expression] { +trait Star extends NamedExpression with trees.LeafNode[Expression] { self: Product => override def name: String = throw new UnresolvedException(this, "name") @@ -103,13 +103,9 @@ trait Star extends Attribute with trees.LeafNode[Expression] { override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") override lazy val resolved = false - override def newInstance(): Star = this - override def withNullability(newNullability: Boolean): Star = this - override def withQualifiers(newQualifiers: Seq[String]): Star = this - override def withName(newName: String): Star = this - // Star gets expanded at runtime so we never evaluate a Star. override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") @@ -154,7 +150,7 @@ case class UnresolvedStar(table: Option[String]) extends Star { * @param names the names to be associated with each output of computing [[child]]. */ case class MultiAlias(child: Expression, names: Seq[String]) - extends Attribute with trees.UnaryNode[Expression] { + extends NamedExpression with trees.UnaryNode[Expression] { override def name: String = throw new UnresolvedException(this, "name") @@ -166,15 +162,9 @@ case class MultiAlias(child: Expression, names: Seq[String]) override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") - override lazy val resolved = false - - override def newInstance(): MultiAlias = this + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") - override def withNullability(newNullability: Boolean): MultiAlias = this - - override def withQualifiers(newQualifiers: Seq[String]): MultiAlias = this - - override def withName(newName: String): MultiAlias = this + override lazy val resolved = false override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 57ace2a14f77c..a9170589f8c6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -86,7 +86,7 @@ abstract class Attribute extends NamedExpression { def withQualifiers(newQualifiers: Seq[String]): Attribute def withName(newName: String): Attribute - def toAttribute: Attribute = this + override def toAttribute: Attribute = this def newInstance(): Attribute } From ea3077f19c18b5556a3632b36771aeb153746ff5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 May 2015 16:22:45 -0700 Subject: [PATCH 32/37] [SPARK-7277] [SQL] Throw exception if the property mapred.reduce.tasks is set to -1 JIRA: https://issues.apache.org/jira/browse/SPARK-7277 As automatically determining the number of reducers is not supported (`mapred.reduce.tasks` is set to `-1`), we should throw exception to users. Author: Liang-Chi Hsieh Closes #5811 from viirya/no_neg_reduce_tasks and squashes the following commits: e518f96 [Liang-Chi Hsieh] Consider other wrong setting values. fd9c817 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_neg_reduce_tasks 4ede705 [Liang-Chi Hsieh] Throw exception instead of warning message. 68a1c70 [Liang-Chi Hsieh] Show warning message if mapred.reduce.tasks is set to -1. --- .../org/apache/spark/sql/execution/commands.scala | 10 ++++++++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 98df5bef34efa..65687db4e6235 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -84,8 +84,14 @@ case class SetCommand( logWarning( s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS, value) - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) + if (value.toInt < 1) { + val msg = s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " + + "determining the number of reducers is not supported." + throw new IllegalArgumentException(msg) + } else { + sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS, value) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) + } // Configures a single property. case Some((key, Some(value))) => 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 208cec6a32d4d..77be3b8b206c0 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 @@ -871,6 +871,16 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { conf.clear() } + test("SET commands with illegal or inappropriate argument") { + conf.clear() + // Set negative mapred.reduce.tasks for automatically determing + // the number of reducers is not supported + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-2")) + conf.clear() + } + test("apply schema") { val schema1 = StructType( StructField("f1", IntegerType, false) :: From 937ba798c56770ec54276b9259e47ae65ee93967 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Thu, 7 May 2015 16:24:11 -0700 Subject: [PATCH 33/37] [SPARK-5281] [SQL] Registering table on RDD is giving MissingRequirementError Go through the context classloader when reflecting on user types in ScalaReflection. Replaced calls to `typeOf` with `typeTag[T].in(mirror)`. The convenience method assumes all types can be found in the classloader that loaded scala-reflect (the primordial classloader). This assumption is not valid in all contexts (sbt console, Eclipse launchers). Fixed SPARK-5281 Author: Iulian Dragos Closes #5981 from dragos/issue/mirrors-missing-requirement-error and squashes the following commits: d103e70 [Iulian Dragos] Go through the context classloader when reflecting on user types in ScalaReflection --- .../spark/sql/catalyst/ScalaReflection.scala | 57 ++++++++++++------- 1 file changed, 37 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index c52965507c715..6998cc8d9666d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.types._ */ object ScalaReflection extends ScalaReflection { val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + val mirror: universe.Mirror = universe.runtimeMirror(Thread.currentThread().getContextClassLoader) } /** @@ -36,6 +37,9 @@ trait ScalaReflection { /** The universe we work in (runtime or macro) */ val universe: scala.reflect.api.Universe + /** The mirror used to access types in the universe */ + val mirror: universe.Mirror + import universe._ // The Predef.Map is scala.collection.immutable.Map. @@ -52,7 +56,19 @@ trait ScalaReflection { /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor[T: TypeTag]: Schema = - ScalaReflectionLock.synchronized { schemaFor(typeOf[T]) } + ScalaReflectionLock.synchronized { schemaFor(localTypeOf[T]) } + + /** + * Return the Scala Type for `T` in the current classloader mirror. + * + * Use this method instead of the convenience method `universe.typeOf`, which + * assumes that all types can be found in the classloader that loaded scala-reflect classes. + * That's not necessarily the case when running using Eclipse launchers or even + * Sbt console or test (without `fork := true`). + * + * @see SPARK-5281 + */ + private def localTypeOf[T: TypeTag]: `Type` = typeTag[T].in(mirror).tpe /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized { @@ -67,25 +83,25 @@ trait ScalaReflection { val udt = Utils.classForName(className) .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() Schema(udt, nullable = true) - case t if t <:< typeOf[Option[_]] => + case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t Schema(schemaFor(optType).dataType, nullable = true) // Need to decide if we actually need a special type here. - case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) - case t if t <:< typeOf[Array[_]] => + case t if t <:< localTypeOf[Array[Byte]] => Schema(BinaryType, nullable = true) + case t if t <:< localTypeOf[Array[_]] => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Seq[_]] => + case t if t <:< localTypeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Map[_, _]] => + case t if t <:< localTypeOf[Map[_, _]] => val TypeRef(_, _, Seq(keyType, valueType)) = t val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if t <:< typeOf[Product] => + case t if t <:< localTypeOf[Product] => val formalTypeArgs = t.typeSymbol.asClass.typeParams val TypeRef(_, _, actualTypeArgs) = t val constructorSymbol = t.member(nme.CONSTRUCTOR) @@ -107,19 +123,20 @@ trait ScalaReflection { schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) StructField(p.name.toString, dataType, nullable) }), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) - case t if t <:< typeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) - case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) - case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) - case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) - case t if t <:< typeOf[java.lang.Double] => Schema(DoubleType, nullable = true) - case t if t <:< typeOf[java.lang.Float] => Schema(FloatType, nullable = true) - case t if t <:< typeOf[java.lang.Short] => Schema(ShortType, nullable = true) - case t if t <:< typeOf[java.lang.Byte] => Schema(ByteType, nullable = true) - case t if t <:< typeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) + case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) + case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) + case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[java.math.BigDecimal] => + Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) + case t if t <:< localTypeOf[java.lang.Long] => Schema(LongType, nullable = true) + case t if t <:< localTypeOf[java.lang.Double] => Schema(DoubleType, nullable = true) + case t if t <:< localTypeOf[java.lang.Float] => Schema(FloatType, nullable = true) + case t if t <:< localTypeOf[java.lang.Short] => Schema(ShortType, nullable = true) + case t if t <:< localTypeOf[java.lang.Byte] => Schema(ByteType, nullable = true) + case t if t <:< localTypeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) case t if t <:< definitions.IntTpe => Schema(IntegerType, nullable = false) case t if t <:< definitions.LongTpe => Schema(LongType, nullable = false) case t if t <:< definitions.DoubleTpe => Schema(DoubleType, nullable = false) From 35f0173b8f67e2e506fc4575be6430cfb66e2238 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 7 May 2015 16:26:49 -0700 Subject: [PATCH 34/37] [SPARK-2155] [SQL] [WHEN D THEN E] [ELSE F] add CaseKeyWhen for "CASE a WHEN b THEN c * END" Avoid translating to CaseWhen and evaluate the key expression many times. Author: Wenchen Fan Closes #5979 from cloud-fan/condition and squashes the following commits: 3ce54e1 [Wenchen Fan] add CaseKeyWhen --- .../apache/spark/sql/catalyst/SqlParser.scala | 10 +- .../catalyst/analysis/HiveTypeCoercion.scala | 43 +++--- .../sql/catalyst/expressions/Expression.scala | 2 +- .../sql/catalyst/expressions/predicates.scala | 135 ++++++++++++------ .../ExpressionEvaluationSuite.scala | 26 ++++ .../spark/sql/DataFrameNaFunctions.scala | 9 +- .../org/apache/spark/sql/hive/HiveQl.scala | 12 +- .../sql/hive/execution/SQLQuerySuite.scala | 7 + 8 files changed, 159 insertions(+), 85 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 1d3a2dc0d9bb0..b06bfb2ce8c8e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -296,13 +296,13 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | LOWER ~ "(" ~> expression <~ ")" ^^ { case exp => Lower(exp) } | IF ~ "(" ~> expression ~ ("," ~> expression) ~ ("," ~> expression) <~ ")" ^^ { case c ~ t ~ f => If(c, t, f) } - | CASE ~> expression.? ~ (WHEN ~> expression ~ (THEN ~> expression)).* ~ + | CASE ~> expression.? ~ rep1(WHEN ~> expression ~ (THEN ~> expression)) ~ (ELSE ~> expression).? <~ END ^^ { case casePart ~ altPart ~ elsePart => - val altExprs = altPart.flatMap { case whenExpr ~ thenExpr => - Seq(casePart.fold(whenExpr)(EqualTo(_, whenExpr)), thenExpr) - } - CaseWhen(altExprs ++ elsePart.toList) + val branches = altPart.flatMap { case whenExpr ~ thenExpr => + Seq(whenExpr, thenExpr) + } ++ elsePart + casePart.map(CaseKeyWhen(_, branches)).getOrElse(CaseWhen(branches)) } | (SUBSTR | SUBSTRING) ~ "(" ~> expression ~ ("," ~> expression) <~ ")" ^^ { case s ~ p => Substring(s, p, Literal(Integer.MAX_VALUE)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 873c75c525c3b..168a4e30eab86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -631,31 +631,24 @@ trait HiveTypeCoercion { import HiveTypeCoercion._ def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => - val valueTypes = branches.sliding(2, 2).map { - case Seq(_, value) => value.dataType - case Seq(elseVal) => elseVal.dataType - }.toSeq - - logDebug(s"Input values for null casting ${valueTypes.mkString(",")}") - - if (valueTypes.distinct.size > 1) { - val commonType = valueTypes.reduce { (v1, v2) => - findTightestCommonType(v1, v2) - .getOrElse(sys.error( - s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) - } - val transformedBranches = branches.sliding(2, 2).map { - case Seq(cond, value) if value.dataType != commonType => - Seq(cond, Cast(value, commonType)) - case Seq(elseVal) if elseVal.dataType != commonType => - Seq(Cast(elseVal, commonType)) - case s => s - }.reduce(_ ++ _) - CaseWhen(transformedBranches) - } else { - // Types match up. Hopefully some other rule fixes whatever is wrong with resolution. - cw + case cw: CaseWhenLike if !cw.resolved && cw.childrenResolved && !cw.valueTypesEqual => + logDebug(s"Input values for null casting ${cw.valueTypes.mkString(",")}") + val commonType = cw.valueTypes.reduce { (v1, v2) => + findTightestCommonType(v1, v2).getOrElse(sys.error( + s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) + } + val transformedBranches = cw.branches.sliding(2, 2).map { + case Seq(when, value) if value.dataType != commonType => + Seq(when, Cast(value, commonType)) + case Seq(elseVal) if elseVal.dataType != commonType => + Seq(Cast(elseVal, commonType)) + case s => s + }.reduce(_ ++ _) + cw match { + case _: CaseWhen => + CaseWhen(transformedBranches) + case CaseKeyWhen(key, _) => + CaseKeyWhen(key, transformedBranches) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 4fd1bc4dd642d..0837a3179d897 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -64,7 +64,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns true if all the children of this expression have been resolved to a specific schema * and false if any still contains any unresolved placeholders. */ - def childrenResolved: Boolean = !children.exists(!_.resolved) + def childrenResolved: Boolean = children.forall(_.resolved) /** * Returns a string representation of this expression that does not have developer centric 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 26c38c56c04f5..50b0f3ee5f93f 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 @@ -353,79 +353,134 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def toString: String = s"if ($predicate) $trueValue else $falseValue" } +trait CaseWhenLike extends Expression { + self: Product => + + type EvaluatedType = Any + + // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last + // element is the value for the default catch-all case (if provided). + // Hence, `branches` consists of at least two elements, and can have an odd or even length. + def branches: Seq[Expression] + + @transient lazy val whenList = + branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq + @transient lazy val thenList = + branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq + val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) + + // both then and else val should be considered. + def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType) + def valueTypesEqual: Boolean = valueTypes.distinct.size <= 1 + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") + } + valueTypes.head + } + + override def nullable: Boolean = { + // If no value is nullable and no elseValue is provided, the whole statement defaults to null. + thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) + } +} + // scalastyle:off /** * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". * Refer to this link for the corresponding semantics: * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - * - * The other form of case statements "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END" gets - * translated to this form at parsing time. Namely, such a statement gets translated to - * "CASE WHEN a=b THEN c [WHEN a=d THEN e]* [ELSE f] END". - * - * Note that `branches` are considered in consecutive pairs (cond, val), and the optional last - * element is the value for the default catch-all case (if provided). Hence, `branches` consists of - * at least two elements, and can have an odd or even length. */ // scalastyle:on -case class CaseWhen(branches: Seq[Expression]) extends Expression { - type EvaluatedType = Any +case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. + @transient private[this] lazy val branchesArr = branches.toArray override def children: Seq[Expression] = branches - override def dataType: DataType = { - if (!resolved) { - throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") + override lazy val resolved: Boolean = + childrenResolved && + whenList.forall(_.dataType == BooleanType) && + valueTypesEqual + + /** Written in imperative fashion for performance considerations. */ + override def eval(input: Row): Any = { + val len = branchesArr.length + var i = 0 + // If all branches fail and an elseVal is not provided, the whole statement + // defaults to null, according to Hive's semantics. + while (i < len - 1) { + if (branchesArr(i).eval(input) == true) { + return branchesArr(i + 1).eval(input) + } + i += 2 + } + var res: Any = null + if (i == len - 1) { + res = branchesArr(i).eval(input) } - branches(1).dataType + return res } + override def toString: String = { + "CASE" + branches.sliding(2, 2).map { + case Seq(cond, value) => s" WHEN $cond THEN $value" + case Seq(elseValue) => s" ELSE $elseValue" + }.mkString + } +} + +// scalastyle:off +/** + * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". + * Refer to this link for the corresponding semantics: + * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions + */ +// scalastyle:on +case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. @transient private[this] lazy val branchesArr = branches.toArray - @transient private[this] lazy val predicates = - branches.sliding(2, 2).collect { case Seq(cond, _) => cond }.toSeq - @transient private[this] lazy val values = - branches.sliding(2, 2).collect { case Seq(_, value) => value }.toSeq - @transient private[this] lazy val elseValue = - if (branches.length % 2 == 0) None else Option(branches.last) - override def nullable: Boolean = { - // If no value is nullable and no elseValue is provided, the whole statement defaults to null. - values.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) - } + override def children: Seq[Expression] = key +: branches - override lazy val resolved: Boolean = { - if (!childrenResolved) { - false - } else { - val allCondBooleans = predicates.forall(_.dataType == BooleanType) - // both then and else val should be considered. - val dataTypesEqual = (values ++ elseValue).map(_.dataType).distinct.size <= 1 - allCondBooleans && dataTypesEqual - } - } + override lazy val resolved: Boolean = + childrenResolved && valueTypesEqual /** Written in imperative fashion for performance considerations. */ override def eval(input: Row): Any = { + val evaluatedKey = key.eval(input) val len = branchesArr.length var i = 0 // If all branches fail and an elseVal is not provided, the whole statement // defaults to null, according to Hive's semantics. - var res: Any = null while (i < len - 1) { - if (branchesArr(i).eval(input) == true) { - res = branchesArr(i + 1).eval(input) - return res + if (equalNullSafe(evaluatedKey, branchesArr(i).eval(input))) { + return branchesArr(i + 1).eval(input) } i += 2 } + var res: Any = null if (i == len - 1) { res = branchesArr(i).eval(input) } - res + return res + } + + private def equalNullSafe(l: Any, r: Any) = { + if (l == null && r == null) { + true + } else if (l == null || r == null) { + false + } else { + l == r + } } override def toString: String = { - "CASE" + branches.sliding(2, 2).map { + s"CASE $key" + branches.sliding(2, 2).map { case Seq(cond, value) => s" WHEN $cond THEN $value" case Seq(elseValue) => s" ELSE $elseValue" }.mkString diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index faaa55aa5e6f1..88d36d153ceea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -850,6 +850,32 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) } + test("case key when") { + val row = create_row(null, 1, 2, "a", "b", "c") + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.string.at(3) + val c5 = 'a.string.at(4) + val c6 = 'a.string.at(5) + + val literalNull = Literal.create(null, BooleanType) + val literalInt = Literal(1) + val literalString = Literal("a") + + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, literalNull, c5, c6)), "b", row) + checkEvaluation(CaseKeyWhen(c2, Seq(literalInt, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(c2, Seq(c1, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c4, Seq(literalString, c2, c3)), 1, row) + checkEvaluation(CaseKeyWhen(c4, Seq(c1, c3, c5, c2, Literal(3))), 3, row) + + checkEvaluation(CaseKeyWhen(literalInt, Seq(c2, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(literalString, Seq(c5, c2, c4, c3)), 2, row) + checkEvaluation(CaseKeyWhen(literalInt, Seq(c5, c2, c4, c3)), null, row) + checkEvaluation(CaseKeyWhen(literalNull, Seq(c5, c2, c1, c3)), 2, row) + } + test("complex type") { val row = create_row( "^Ba*n", // 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 481ed4924857e..4a54120ba86f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -357,11 +357,12 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * TODO: This can be optimized to use broadcast join when replacementMap is large. */ private def replaceCol(col: StructField, replacementMap: Map[_, _]): Column = { - val branches: Seq[Expression] = replacementMap.flatMap { case (source, target) => - df.col(col.name).equalTo(lit(source).cast(col.dataType)).expr :: - lit(target).cast(col.dataType).expr :: Nil + val keyExpr = df.col(col.name).expr + def buildExpr(v: Any) = Cast(Literal(v), keyExpr.dataType) + val branches = replacementMap.flatMap { case (source, target) => + Seq(buildExpr(source), buildExpr(target)) }.toSeq - new Column(CaseWhen(branches ++ Seq(df.col(col.name).expr))).as(col.name) + new Column(CaseKeyWhen(keyExpr, branches :+ keyExpr)).as(col.name) } private def convertToDouble(v: Any): Double = v match { 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 4e51473979a2a..6176aee25e7a4 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 @@ -1246,16 +1246,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => CaseWhen(branches.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => - val transformed = branches.drop(1).sliding(2, 2).map { - case Seq(condVal, value) => - // FIXME (SPARK-2155): the key will get evaluated for multiple times in CaseWhen's eval(). - // Hence effectful / non-deterministic key expressions are *not* supported at the moment. - // We should consider adding new Expressions to get around this. - Seq(EqualTo(nodeToExpr(branches(0)), nodeToExpr(condVal)), - nodeToExpr(value)) - case Seq(elseVal) => Seq(nodeToExpr(elseVal)) - }.toSeq.reduce(_ ++ _) - CaseWhen(transformed) + val keyExpr = nodeToExpr(branches.head) + CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) /* Complex datatype manipulation */ case Token("[", child :: ordinal :: Nil) => 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 616352d223c5b..c605f1017547a 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 @@ -751,4 +751,11 @@ class SQLQuerySuite extends QueryTest { (6, "c", 0, 6) ).map(i => Row(i._1, i._2, i._3, i._4))) } + + test("test case key when") { + (1 to 5).map(i => (i, i.toString)).toDF("k", "v").registerTempTable("t") + checkAnswer( + sql("SELECT CASE k WHEN 2 THEN 22 WHEN 4 THEN 44 ELSE 0 END, v FROM t"), + Row(0, "1") :: Row(22, "2") :: Row(0, "3") :: Row(44, "4") :: Row(0, "5") :: Nil) + } } From 88063c62689135da56ee6678b2e826b88c382732 Mon Sep 17 00:00:00 2001 From: tedyu Date: Thu, 7 May 2015 16:53:59 -0700 Subject: [PATCH 35/37] [SPARK-7450] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() Author: tedyu Closes #5897 from tedyu/master and squashes the following commits: 473bf9d [tedyu] Address Josh's review comments 1719c5b [tedyu] Correct upper bound in for loop b51dcaf [tedyu] Add unit test in BitSetSuite for BitSet#anySet() 83f9f87 [tedyu] Merge branch 'master' of github.com:apache/spark 817e3f9 [tedyu] Replace constant 8 with SIZE_OF_LONG 75a467b [tedyu] Correct offset for UNSAFE.getLong() 855374b [tedyu] Remove second loop since bitSetWidthInBytes is WORD aligned 093b7a4 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() 63ee050 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() 4ca0ef6 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() 3e9b6919 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() --- .../main/java/org/apache/spark/unsafe/bitset/BitSet.java | 8 ++++++++ .../org/apache/spark/unsafe/bitset/BitSetMethods.java | 7 ++++--- .../java/org/apache/spark/unsafe/bitset/BitSetSuite.java | 7 +++++++ 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java index f72e07fce92fd..28e23da108ebe 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java @@ -102,4 +102,12 @@ public boolean isSet(int index) { public int nextSetBit(int fromIndex) { return BitSetMethods.nextSetBit(baseObject, baseOffset, fromIndex, numWords); } + + /** + * Returns {@code true} if any bit is set. + */ + public boolean anySet() { + return BitSetMethods.anySet(baseObject, baseOffset, numWords); + } + } diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index f30626d8f4317..0987191c1c636 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -70,9 +70,10 @@ public static boolean isSet(Object baseObject, long baseOffset, int index) { /** * Returns {@code true} if any bit is set. */ - public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInBytes) { - for (int i = 0; i <= bitSetWidthInBytes; i++) { - if (PlatformDependent.UNSAFE.getByte(baseObject, baseOffset + i) != 0) { + public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInWords) { + long addr = baseOffset; + for (int i = 0; i < bitSetWidthInWords; i++, addr += WORD_SIZE) { + if (PlatformDependent.UNSAFE.getLong(baseObject, addr) != 0) { return true; } } diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java index e3a824e29b768..18393db9f382f 100644 --- a/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java +++ b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java @@ -39,6 +39,8 @@ public void basicOps() { for (int i = 0; i < bs.capacity(); i++) { Assert.assertFalse(bs.isSet(i)); } + // another form of asserting that the bit set is empty + Assert.assertFalse(bs.anySet()); // Set every bit and check it. for (int i = 0; i < bs.capacity(); i++) { @@ -52,6 +54,11 @@ public void basicOps() { bs.unset(i); Assert.assertFalse(bs.isSet(i)); } + + // Make sure anySet() can detect any set bit + bs = createBitSet(256); + bs.set(64); + Assert.assertTrue(bs.anySet()); } @Test From 22ab70e06ede65ca865073fe36c859042a920aa3 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 7 May 2015 17:34:44 -0700 Subject: [PATCH 36/37] [SPARK-7305] [STREAMING] [WEBUI] Make BatchPage show friendly information when jobs are dropped by SparkListener If jobs are dropped by SparkListener, at least we can show the job ids in BatchPage. Screenshot: ![b1](https://cloud.githubusercontent.com/assets/1000778/7434968/f19aa784-eff3-11e4-8f86-36a073873574.png) Author: zsxwing Closes #5840 from zsxwing/SPARK-7305 and squashes the following commits: aca0ba6 [zsxwing] Fix the code style 718765e [zsxwing] Make generateNormalJobRow private 8073b03 [zsxwing] Merge branch 'master' into SPARK-7305 83dec11 [zsxwing] Make BatchPage show friendly information when jobs are dropped by SparkListener --- .../apache/spark/streaming/ui/BatchPage.scala | 136 ++++++++++++++---- 1 file changed, 106 insertions(+), 30 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 3f1cab69068dc..831f60e870f74 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.ui import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} +import scala.xml.{NodeSeq, Node, Text} import org.apache.commons.lang3.StringEscapeUtils @@ -28,6 +28,7 @@ import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} import org.apache.spark.streaming.ui.StreamingJobProgressListener.{SparkJobId, OutputOpId} import org.apache.spark.ui.jobs.UIData.JobUIData +private case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private val streamingListener = parent.listener @@ -44,25 +45,33 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { Error } + private def generateJobRow( + outputOpId: OutputOpId, + outputOpDescription: Seq[Node], + formattedOutputOpDuration: String, + numSparkJobRowsInOutputOp: Int, + isFirstRow: Boolean, + sparkJob: SparkJobIdWithUIData): Seq[Node] = { + if (sparkJob.jobUIData.isDefined) { + generateNormalJobRow(outputOpId, outputOpDescription, formattedOutputOpDuration, + numSparkJobRowsInOutputOp, isFirstRow, sparkJob.jobUIData.get) + } else { + generateDroppedJobRow(outputOpId, outputOpDescription, formattedOutputOpDuration, + numSparkJobRowsInOutputOp, isFirstRow, sparkJob.sparkJobId) + } + } + /** * Generate a row for a Spark Job. Because duplicated output op infos needs to be collapsed into * one cell, we use "rowspan" for the first row of a output op. */ - def generateJobRow( + private def generateNormalJobRow( outputOpId: OutputOpId, + outputOpDescription: Seq[Node], formattedOutputOpDuration: String, numSparkJobRowsInOutputOp: Int, isFirstRow: Boolean, sparkJob: JobUIData): Seq[Node] = { - val lastStageInfo = Option(sparkJob.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => sparkListener.stageIdToInfo.get(ids.max) } - val lastStageData = lastStageInfo.flatMap { s => - sparkListener.stageIdToData.get((s.stageId, s.attemptId)) - } - - val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") val duration: Option[Long] = { sparkJob.submissionTime.map { start => val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) @@ -83,9 +92,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { if (isFirstRow) { {outputOpId.toString} - - {lastStageDescription} - {lastStageName} + {outputOpDescription} {formattedOutputOpDuration} } else { @@ -122,27 +129,96 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } - private def generateOutputOpIdRow( - outputOpId: OutputOpId, sparkJobs: Seq[JobUIData]): Seq[Node] = { - val sparkjobDurations = sparkJobs.map(sparkJob => { - sparkJob.submissionTime.map { start => - val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) - end - start + /** + * If a job is dropped by sparkListener due to exceeding the limitation, we only show the job id + * with "-" cells. + */ + private def generateDroppedJobRow( + outputOpId: OutputOpId, + outputOpDescription: Seq[Node], + formattedOutputOpDuration: String, + numSparkJobRowsInOutputOp: Int, + isFirstRow: Boolean, + jobId: Int): Seq[Node] = { + // In the first row, output op id and its information needs to be shown. In other rows, these + // cells will be taken up due to "rowspan". + // scalastyle:off + val prefixCells = + if (isFirstRow) { + {outputOpId.toString} + {outputOpDescription} + {formattedOutputOpDuration} + } else { + Nil } - }) + // scalastyle:on + + + {prefixCells} + + {jobId.toString} + + + - + + - + + - + + - + + } + + private def generateOutputOpIdRow( + outputOpId: OutputOpId, sparkJobs: Seq[SparkJobIdWithUIData]): Seq[Node] = { + // We don't count the durations of dropped jobs + val sparkJobDurations = sparkJobs.filter(_.jobUIData.nonEmpty).map(_.jobUIData.get). + map(sparkJob => { + sparkJob.submissionTime.map { start => + val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) + end - start + } + }) val formattedOutputOpDuration = - if (sparkjobDurations.exists(_ == None)) { - // If any job does not finish, set "formattedOutputOpDuration" to "-" + if (sparkJobDurations.isEmpty || sparkJobDurations.exists(_ == None)) { + // If no job or any job does not finish, set "formattedOutputOpDuration" to "-" "-" } else { - SparkUIUtils.formatDuration(sparkjobDurations.flatMap(x => x).sum) + SparkUIUtils.formatDuration(sparkJobDurations.flatMap(x => x).sum) } - generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, true, sparkJobs.head) ++ + + val description = generateOutputOpDescription(sparkJobs) + + generateJobRow( + outputOpId, description, formattedOutputOpDuration, sparkJobs.size, true, sparkJobs.head) ++ sparkJobs.tail.map { sparkJob => - generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, false, sparkJob) + generateJobRow( + outputOpId, description, formattedOutputOpDuration, sparkJobs.size, false, sparkJob) }.flatMap(x => x) } + private def generateOutputOpDescription(sparkJobs: Seq[SparkJobIdWithUIData]): Seq[Node] = { + val lastStageInfo = + sparkJobs.flatMap(_.jobUIData).headOption. // Get the first JobUIData + flatMap { sparkJob => // For the first job, get the latest Stage info + if (sparkJob.stageIds.isEmpty) { + None + } else { + sparkListener.stageIdToInfo.get(sparkJob.stageIds.max) + } + } + val lastStageData = lastStageInfo.flatMap { s => + sparkListener.stageIdToData.get((s.stageId, s.attemptId)) + } + + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + + + {lastStageDescription} + ++ Text(lastStageName) + } + private def failureReasonCell(failureReason: String): Seq[Node] = { val isMultiline = failureReason.indexOf('\n') >= 0 // Display the first line by default @@ -187,10 +263,10 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { (outputOpId, outputOpIdAndSparkJobIds.map(_.sparkJobId).sorted) } sparkListener.synchronized { - val outputOpIdWithJobs: Seq[(OutputOpId, Seq[JobUIData])] = + val outputOpIdWithJobs: Seq[(OutputOpId, Seq[SparkJobIdWithUIData])] = outputOpIdToSparkJobIds.map { case (outputOpId, sparkJobIds) => - // Filter out spark Job ids that don't exist in sparkListener - (outputOpId, sparkJobIds.flatMap(getJobData)) + (outputOpId, + sparkJobIds.map(sparkJobId => SparkJobIdWithUIData(sparkJobId, getJobData(sparkJobId)))) } @@ -200,7 +276,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { { outputOpIdWithJobs.map { - case (outputOpId, jobs) => generateOutputOpIdRow(outputOpId, jobs) + case (outputOpId, sparkJobIds) => generateOutputOpIdRow(outputOpId, sparkJobIds) } } From cd1d4110cfffb413ab585cf1cc8f1264243cb393 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 May 2015 19:36:24 -0700 Subject: [PATCH 37/37] [SPARK-6908] [SQL] Use isolated Hive client This PR switches Spark SQL's Hive support to use the isolated hive client interface introduced by #5851, instead of directly interacting with the client. By using this isolated client we can now allow users to dynamically configure the version of Hive that they are connecting to by setting `spark.sql.hive.metastore.version` without the need recompile. This also greatly reduces the surface area for our interaction with the hive libraries, hopefully making it easier to support other versions in the future. Jars for the desired hive version can be configured using `spark.sql.hive.metastore.jars`, which accepts the following options: - a colon-separated list of jar files or directories for hive and hadoop. - `builtin` - attempt to discover the jars that were used to load Spark SQL and use those. This option is only valid when using the execution version of Hive. - `maven` - download the correct version of hive on demand from maven. By default, `builtin` is used for Hive 13. This PR also removes the test step for building against Hive 12, as this will no longer be required to talk to Hive 12 metastores. However, the full removal of the Shim is deferred until a later PR. Remaining TODOs: - Remove the Hive Shims and inline code for Hive 13. - Several HiveCompatibility tests are not yet passing. - `nullformatCTAS` - As detailed below, we now are handling CTAS parsing ourselves instead of hacking into the Hive semantic analyzer. However, we currently only handle the common cases and not things like CTAS where the null format is specified. - `combine1` now leaks state about compression somehow, breaking all subsequent tests. As such we currently add it to the blacklist - `part_inherit_tbl_props` and `part_inherit_tbl_props_with_star` do not work anymore. We are correctly propagating the information - "load_dyn_part14.*" - These tests pass when run on their own, but fail when run with all other tests. It seems our `RESET` mechanism may not be as robust as it used to be? Other required changes: - `CreateTableAsSelect` no longer carries parts of the HiveQL AST with it through the query execution pipeline. Instead, we parse CTAS during the HiveQL conversion and construct a `HiveTable`. The full parsing here is not yet complete as detailed above in the remaining TODOs. Since the operator is Hive specific, it is moved to the hive package. - `Command` is simplified to be a trait that simply acts as a marker for a LogicalPlan that should be eagerly evaluated. Author: Michael Armbrust Closes #5876 from marmbrus/useIsolatedClient and squashes the following commits: 258d000 [Michael Armbrust] really really correct path handling e56fd4a [Michael Armbrust] getAbsolutePath 5a259f5 [Michael Armbrust] fix typos 81bb366 [Michael Armbrust] comments from vanzin 5f3945e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient 4b5cd41 [Michael Armbrust] yin's comments f5de7de [Michael Armbrust] cleanup 11e9c72 [Michael Armbrust] better coverage in versions suite 7e8f010 [Michael Armbrust] better error messages and jar handling e7b3941 [Michael Armbrust] more permisive checking for function registration da91ba7 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient 5fe5894 [Michael Armbrust] fix serialization suite 81711c4 [Michael Armbrust] Initial support for running without maven 1d8ae44 [Michael Armbrust] fix final tests? 1c50813 [Michael Armbrust] more comments a3bee70 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient a6f5df1 [Michael Armbrust] style ab07f7e [Michael Armbrust] WIP 4d8bf02 [Michael Armbrust] Remove hive 12 compilation 8843a25 [Michael Armbrust] [SPARK-6908] [SQL] Use isolated Hive client --- dev/run-tests | 23 - project/MimaExcludes.scala | 2 + project/SparkBuild.scala | 9 +- .../plans/logical/basicOperators.scala | 16 +- .../sql/catalyst/plans/logical/commands.scala | 8 +- .../spark/sql/catalyst/SqlParserSuite.scala | 6 +- .../org/apache/spark/sql/DataFrame.scala | 1 - .../org/apache/spark/sql/SQLContext.scala | 11 +- .../apache/spark/sql/execution/commands.scala | 4 +- .../org/apache/spark/sql/sources/ddl.scala | 16 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 26 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 9 +- .../execution/HiveCompatibilitySuite.scala | 12 +- .../apache/spark/sql/hive/HiveContext.scala | 283 ++++++------ .../spark/sql/hive/HiveMetastoreCatalog.scala | 415 ++++++------------ .../org/apache/spark/sql/hive/HiveQl.scala | 126 +++++- .../apache/spark/sql/hive/TableReader.scala | 11 +- .../sql/hive/client/ClientInterface.scala | 41 +- .../spark/sql/hive/client/ClientWrapper.scala | 99 +++-- .../hive/client/IsolatedClientLoader.scala | 23 +- .../sql/hive/client/ReflectionMagic.scala | 8 + .../hive/execution/CreateTableAsSelect.scala | 33 +- .../hive/execution/InsertIntoHiveTable.scala | 33 +- .../spark/sql/hive/execution/commands.scala | 13 + .../apache/spark/sql/hive/test/TestHive.scala | 72 ++- sql/hive/src/test/resources/log4j.properties | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 22 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 26 +- .../spark/sql/hive/SerializationSuite.scala | 6 +- .../spark/sql/hive/client/VersionsSuite.scala | 78 +++- .../hive/execution/HiveComparisonTest.scala | 2 + .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../sql/hive/execution/PruningSuite.scala | 15 +- 33 files changed, 782 insertions(+), 671 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 05c63bce4d40d..ef587a1a5988c 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -142,29 +142,6 @@ CURRENT_BLOCK=$BLOCK_BUILD { HIVE_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" - HIVE_12_BUILD_ARGS="$HIVE_BUILD_ARGS -Phive-0.12.0" - - # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build - echo "[info] Compile with Hive 0.12.0" - [ -d "lib_managed" ] && rm -rf lib_managed - echo "[info] Building Spark with these arguments: $HIVE_12_BUILD_ARGS" - - if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then - build/mvn $HIVE_12_BUILD_ARGS clean package -DskipTests - else - # NOTE: echo "q" is needed because sbt on encountering a build file with failure - # (either resolution or compilation) prompts the user for input either q, r, etc - # to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - # single argument! - # QUESTION: Why doesn't 'yes "q"' work? - # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - echo -e "q\n" \ - | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - fi - - # Then build with default Hive version (0.13.1) because tests are based on this version echo "[info] Compile with Hive 0.13.1" [ -d "lib_managed" ] && rm -rf lib_managed echo "[info] Building Spark with these arguments: $HIVE_BUILD_ARGS" diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bf343d4b7e40b..cfe387faec14b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -89,6 +89,8 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Vector.numActives") ) ++ Seq( + // Execution should never be included as its always internal. + MimaBuild.excludeSparkPackage("sql.execution"), // This `protected[sql]` method was removed in 1.3.1 ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.sql.SQLContext.checkAnalysis"), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b4431c7ee05b6..026855f8f6a5a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -193,6 +193,7 @@ object SparkBuild extends PomBuild { * Usage: `build/sbt sparkShell` */ val sparkShell = taskKey[Unit]("start a spark-shell.") + val sparkSql = taskKey[Unit]("starts the spark sql CLI.") enable(Seq( connectInput in run := true, @@ -203,6 +204,12 @@ object SparkBuild extends PomBuild { sparkShell := { (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value + }, + + javaOptions in Compile += "-Dspark.master=local", + + sparkSql := { + (runMain in Compile).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value } ))(assembly) @@ -497,7 +504,7 @@ object TestSettings { // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes // launched by the tests have access to the correct test-time classpath. envVars in Test ++= Map( - "SPARK_DIST_CLASSPATH" -> + "SPARK_DIST_CLASSPATH" -> (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"), "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += "-Dspark.test.home=" + sparkHome, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index ba0abb2df596c..0f349f9d11415 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -149,16 +149,6 @@ case class InsertIntoTable( } } -case class CreateTableAsSelect[T]( - databaseName: Option[String], - tableName: String, - child: LogicalPlan, - allowExisting: Boolean, - desc: Option[T] = None) extends UnaryNode { - override def output: Seq[Attribute] = Seq.empty[Attribute] - override lazy val resolved: Boolean = databaseName != None && childrenResolved -} - /** * A container for holding named common table expressions (CTEs) and a query plan. * This operator will be removed during analysis and the relations will be substituted into child. @@ -184,10 +174,10 @@ case class WriteToFile( } /** - * @param order The ordering expressions - * @param global True means global sorting apply for entire data set, + * @param order The ordering expressions + * @param global True means global sorting apply for entire data set, * False means sorting only apply within the partition. - * @param child Child logical plan + * @param child Child logical plan */ case class Sort( order: Seq[SortOrder], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 45905f8ef98c5..246f4d7e34d3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -21,9 +21,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute /** * A logical node that represents a non-query command to be executed by the system. For example, - * commands can be used by parsers to represent DDL operations. + * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are + * eagerly executed. */ -abstract class Command extends LeafNode { - self: Product => - def output: Seq[Attribute] = Seq.empty -} +trait Command diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala index a652c70560990..890ea2a84b82e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.Command import org.scalatest.FunSuite -private[sql] case class TestCommand(cmd: String) extends Command +private[sql] case class TestCommand(cmd: String) extends LogicalPlan with Command { + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty +} private[sql] class SuperLongKeywordTestParser extends AbstractSparkSQLParser { protected val EXECUTE = Keyword("THISISASUPERLONGKEYWORDTEST") 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 79fbf50300d4d..7947042c14299 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 @@ -143,7 +143,6 @@ class DataFrame private[sql]( // happen right away to let these side effects take place eagerly. case _: Command | _: InsertIntoTable | - _: CreateTableAsSelect[_] | _: CreateTableUsingAsSelect | _: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) 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 0563430a6fdc7..0ac0936f0f592 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 @@ -70,7 +70,7 @@ import org.apache.spark.{Partition, SparkContext} * spark-sql> SELECT * FROM src LIMIT 1; * *-- Exception will be thrown and switch to dialect - *-- "sql" (for SQLContext) or + *-- "sql" (for SQLContext) or *-- "hiveql" (for HiveContext) * }}} */ @@ -107,7 +107,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * @return Spark SQL configuration */ - protected[sql] def conf = tlSession.get().conf + protected[sql] def conf = currentSession().conf /** * Set Spark SQL configuration properties. @@ -1197,13 +1197,17 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(executedPlan)} """.stripMargin.trim - override def toString: String = + override def toString: String = { + def output = + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}").mkString(", ") + // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)}) // however, the `toRdd` will cause the real execution, which is not what we want. // We need to think about how to avoid the side effect. s"""== Parsed Logical Plan == |${stringOrError(logical)} |== Analyzed Logical Plan == + |${stringOrError(output)} |${stringOrError(analyzed)} |== Optimized Logical Plan == |${stringOrError(optimizedPlan)} @@ -1212,6 +1216,7 @@ class SQLContext(@transient val sparkContext: SparkContext) |Code Generation: ${stringOrError(executedPlan.codegenEnabled)} |== RDD == """.stripMargin.trim + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 65687db4e6235..388a8184e4cfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -32,9 +32,11 @@ import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -trait RunnableCommand extends logical.Command { +private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { self: Product => + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty def run(sqlContext: SQLContext): Seq[Row] } 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 1abf3aa51cb25..06c64f2bdd59e 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 @@ -269,8 +269,10 @@ private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRel */ private[sql] case class DescribeCommand( table: LogicalPlan, - isExtended: Boolean) extends Command { - override val output = Seq( + isExtended: Boolean) extends LogicalPlan with Command { + + override def children: Seq[LogicalPlan] = Seq.empty + override val output: Seq[Attribute] = Seq( // Column names are based on Hive. AttributeReference("col_name", StringType, nullable = false, new MetadataBuilder().putString("comment", "name of the column").build())(), @@ -292,7 +294,11 @@ private[sql] case class CreateTableUsing( temporary: Boolean, options: Map[String, String], allowExisting: Boolean, - managedIfNoPath: Boolean) extends Command + managedIfNoPath: Boolean) extends LogicalPlan with Command { + + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty +} /** * A node used to support CTAS statements and saveAsTable for the data source API. @@ -318,7 +324,7 @@ private[sql] case class CreateTempTableUsing( provider: String, options: Map[String, String]) extends RunnableCommand { - def run(sqlContext: SQLContext): Seq[Row] = { + override def run(sqlContext: SQLContext): Seq[Row] = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) @@ -333,7 +339,7 @@ private[sql] case class CreateTempTableUsingAsSelect( options: Map[String, String], query: LogicalPlan) extends RunnableCommand { - def run(sqlContext: SQLContext): Seq[Row] = { + override def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) sqlContext.registerDataFrameAsTable( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index b7b6925aa87f7..deb1008c468bf 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket import org.apache.spark.Logging -import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.hive.{HiveContext, HiveShim} import org.apache.spark.util.Utils private[hive] object SparkSQLCLIDriver { @@ -74,7 +74,12 @@ private[hive] object SparkSQLCLIDriver { System.exit(1) } - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) + val cliConf = new HiveConf(classOf[SessionState]) + // Override the location of the metastore since this is only used for local execution. + HiveContext.newTemporaryConfiguration().foreach { + case (key, value) => cliConf.set(key, value) + } + val sessionState = new CliSessionState(cliConf) sessionState.in = System.in try { @@ -91,10 +96,14 @@ private[hive] object SparkSQLCLIDriver { // Set all properties specified via command line. val conf: HiveConf = sessionState.getConf - sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => - conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - sessionState.getOverriddenConfigurations.put( - item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + sessionState.cmdProperties.entrySet().foreach { item => + val key = item.getKey.asInstanceOf[String] + val value = item.getValue.asInstanceOf[String] + // We do not propagate metastore options to the execution copy of hive. + if (key != "javax.jdo.option.ConnectionURL") { + conf.set(key, value) + sessionState.getOverriddenConfigurations.put(key, value) + } } SessionState.start(sessionState) @@ -138,8 +147,9 @@ private[hive] object SparkSQLCLIDriver { case e: UnsupportedEncodingException => System.exit(3) } - // use the specified database if specified - cli.processSelectDatabase(sessionState); + if (sessionState.database != null) { + SparkSQLEnv.hiveContext.runSqlHive(s"USE ${sessionState.database}") + } // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 97b46a01ba5b4..7c0c505e2d61e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.thriftserver +import java.io.PrintStream + import scala.collection.JavaConversions._ import org.apache.spark.scheduler.StatsReportListener @@ -39,7 +41,6 @@ private[hive] object SparkSQLEnv extends Logging { sparkConf .setAppName(s"SparkSQL::${Utils.localHostName()}") - .set("spark.sql.hive.version", HiveShim.version) .set( "spark.serializer", maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer")) @@ -51,6 +52,12 @@ private[hive] object SparkSQLEnv extends Logging { sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) + hiveContext.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) + hiveContext.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) + hiveContext.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) + + hiveContext.setConf("spark.sql.hive.version", HiveShim.version) + if (log.isDebugEnabled) { hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) => logDebug(s"HiveConf var: $k=$v") diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 5e411c2fdba9d..b6245a57074c8 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -240,7 +240,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // It has a bug and it has been fixed by // https://issues.apache.org/jira/browse/HIVE-7673 (in Hive 0.14 and trunk). - "input46" + "input46", + + // These tests were broken by the hive client isolation PR. + "part_inherit_tbl_props", + "part_inherit_tbl_props_with_star", + + "nullformatCTAS", // SPARK-7411: need to finish CTAS parser + + // The isolated classloader seemed to make some of our test reset mechanisms less robust. + "combine1", // This test changes compression settings in a way that breaks all subsequent tests. + "load_dyn_part14.*" // These work alone but fail when run with other tests... ) ++ HiveShim.compatibilityBlackList /** 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 f25723e53f07c..538c6c7f0a200 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 @@ -17,8 +17,9 @@ package org.apache.spark.sql.hive -import java.io.{BufferedReader, InputStreamReader, PrintStream} +import java.io.{BufferedReader, File, InputStreamReader, PrintStream} import java.sql.Timestamp +import java.util.{ArrayList => JArrayList} import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.spark.sql.catalyst.Dialect @@ -35,15 +36,19 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubQueries, OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, QueryExecutionException, SetCommand} +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} import org.apache.spark.sql.sources.{DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + /** * This is the HiveQL Dialect, this dialect is strongly bind with HiveContext @@ -61,6 +66,8 @@ private[hive] class HiveQLDialect extends Dialect { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => + import HiveContext._ + /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive @@ -93,9 +100,118 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] def convertCTAS: Boolean = getConf("spark.sql.hive.convertCTAS", "false").toBoolean + /** + * The version of the hive client that will be used to communicate with the metastore. Note that + * this does not necessarily need to be the same version of Hive that is used internally by + * Spark SQL for execution. + */ + protected[hive] def hiveMetastoreVersion: String = + getConf(HIVE_METASTORE_VERSION, hiveExecutionVersion) + + /** + * The location of the jars that should be used to instantiate the HiveMetastoreClient. This + * property can be one of three options: + * - a classpath in the standard format for both hive and hadoop. + * - builtin - attempt to discover the jars that were used to load Spark SQL and use those. This + * option is only valid when using the execution version of Hive. + * - maven - download the correct version of hive on demand from maven. + */ + protected[hive] def hiveMetastoreJars: String = + getConf(HIVE_METASTORE_JARS, "builtin") + @transient protected[sql] lazy val substitutor = new VariableSubstitution() + /** + * The copy of the hive client that is used for execution. Currently this must always be + * Hive 13 as this is the version of Hive that is packaged with Spark SQL. This copy of the + * client is used for execution related tasks like registering temporary functions or ensuring + * that the ThreadLocal SessionState is correctly populated. This copy of Hive is *not* used + * for storing peristent metadata, and only point to a dummy metastore in a temporary directory. + */ + @transient + protected[hive] lazy val executionHive: ClientWrapper = { + logInfo(s"Initilizing execution hive, version $hiveExecutionVersion") + new ClientWrapper( + version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), + config = newTemporaryConfiguration()) + } + SessionState.setCurrentSessionState(executionHive.state) + + /** + * The copy of the Hive client that is used to retrieve metadata from the Hive MetaStore. + * The version of the Hive client that is used here must match the metastore that is configured + * in the hive-site.xml file. + */ + @transient + protected[hive] lazy val metadataHive: ClientInterface = { + val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) + + // We instantiate a HiveConf here to read in the hive-site.xml file and then pass the options + // into the isolated client loader + val metadataConf = new HiveConf() + // `configure` goes second to override other settings. + val allConfig = metadataConf.iterator.map(e => e.getKey -> e.getValue).toMap ++ configure + + val isolatedLoader = if (hiveMetastoreJars == "builtin") { + if (hiveExecutionVersion != hiveMetastoreVersion) { + throw new IllegalArgumentException( + "Builtin jars can only be used when hive execution version == hive metastore version. " + + s"Execution: ${hiveExecutionVersion} != Metastore: ${hiveMetastoreVersion}. " + + "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + + s"or change $HIVE_METASTORE_VERSION to $hiveExecutionVersion.") + } + val jars = getClass.getClassLoader match { + case urlClassLoader: java.net.URLClassLoader => urlClassLoader.getURLs + case other => + throw new IllegalArgumentException( + "Unable to locate hive jars to connect to metastore " + + s"using classloader ${other.getClass.getName}. " + + "Please set spark.sql.hive.metastore.jars") + } + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") + new IsolatedClientLoader( + version = metaVersion, + execJars = jars.toSeq, + config = allConfig, + isolationOn = true) + } else if (hiveMetastoreJars == "maven") { + // TODO: Support for loading the jars from an already downloaded location. + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using maven.") + IsolatedClientLoader.forVersion(hiveMetastoreVersion, allConfig ) + } else { + // Convert to files and expand any directories. + val jars = + hiveMetastoreJars + .split(File.pathSeparator) + .flatMap { + case path if new File(path).getName() == "*" => + val files = new File(path).getParentFile().listFiles() + if (files == null) { + logWarning(s"Hive jar path '$path' does not exist.") + Nil + } else { + files.filter(_.getName().toLowerCase().endsWith(".jar")) + } + case path => + new File(path) :: Nil + } + .map(_.toURI.toURL) + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using $jars") + new IsolatedClientLoader( + version = metaVersion, + execJars = jars.toSeq, + config = allConfig, + isolationOn = true) + } + isolatedLoader.client + } + protected[sql] override def parseSql(sql: String): LogicalPlan = { super.parseSql(substitutor.substitute(hiveconf, sql)) } @@ -178,15 +294,10 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - tableParameters.put(HiveShim.getStatsSetupConstTotalSize, newTotalSize.toString) - val hiveTTable = relation.hiveQlTable.getTTable - hiveTTable.setParameters(tableParameters) - val tableFullName = - relation.hiveQlTable.getDbName + "." + relation.hiveQlTable.getTableName - - catalog.synchronized { - catalog.client.alterTable(tableFullName, new Table(hiveTTable)) - } + catalog.client.alterTable( + relation.table.copy( + properties = relation.table.properties + + (HiveShim.getStatsSetupConstTotalSize -> newTotalSize.toString))) } case otherRelation => throw new UnsupportedOperationException( @@ -194,47 +305,19 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. - @transient - protected lazy val outputBuffer = new java.io.OutputStream { - var pos: Int = 0 - var buffer = new Array[Int](10240) - def write(i: Int): Unit = { - buffer(pos) = i - pos = (pos + 1) % buffer.size - } - - override def toString: String = { - val (end, start) = buffer.splitAt(pos) - val input = new java.io.InputStream { - val iterator = (start ++ end).iterator - - def read(): Int = if (iterator.hasNext) iterator.next() else -1 - } - val reader = new BufferedReader(new InputStreamReader(input)) - val stringBuilder = new StringBuilder - var line = reader.readLine() - while(line != null) { - stringBuilder.append(line) - stringBuilder.append("\n") - line = reader.readLine() - } - stringBuilder.toString() - } - } - - protected[hive] def sessionState = tlSession.get().asInstanceOf[this.SQLSession].sessionState - protected[hive] def hiveconf = tlSession.get().asInstanceOf[this.SQLSession].hiveconf override def setConf(key: String, value: String): Unit = { super.setConf(key, value) - runSqlHive(s"SET $key=$value") + hiveconf.set(key, value) + executionHive.runSqlHive(s"SET $key=$value") + metadataHive.runSqlHive(s"SET $key=$value") } /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + override protected[sql] lazy val catalog = + new HiveMetastoreCatalog(metadataHive, this) with OverrideCatalog // Note that HiveUDFs will be overridden by functions registered in this context. @transient @@ -261,16 +344,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { new this.SQLSession() } + /** Overridden by child classes that need to set configuration before the client init. */ + protected def configure(): Map[String, String] = Map.empty + protected[hive] class SQLSession extends super.SQLSession { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") } - protected[hive] lazy val hiveconf: HiveConf = { - setConf(sessionState.getConf.getAllProperties) - sessionState.getConf - } - /** * SQLConf and HiveConf contracts: * @@ -285,78 +366,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { state = new SessionState(new HiveConf(classOf[SessionState])) SessionState.start(state) } - if (state.out == null) { - state.out = new PrintStream(outputBuffer, true, "UTF-8") - } - if (state.err == null) { - state.err = new PrintStream(outputBuffer, true, "UTF-8") - } state } - } - - /** - * Runs the specified SQL query using Hive. - */ - protected[sql] def runSqlHive(sql: String): Seq[String] = { - val maxResults = 100000 - val results = runHive(sql, maxResults) - // It is very confusing when you only get back some of the results... - if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") - results - } - - /** - * Execute the command using Hive and return the results as a sequence. Each element - * in the sequence is one row. - */ - protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = synchronized { - try { - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hiveconf) - - // Makes sure the session represented by the `sessionState` field is activated. This implies - // Spark SQL Hive support uses a single `SessionState` for all Hive operations and breaks - // session isolation under multi-user scenarios (i.e. HiveThriftServer2). - // TODO Fix session isolation - if (SessionState.get() != sessionState) { - SessionState.start(sessionState) - } - proc match { - case driver: Driver => - val results = HiveShim.createDriverResultsArray - val response: CommandProcessorResponse = driver.run(cmd) - // Throw an exception if there is an error in query processing. - if (response.getResponseCode != 0) { - driver.close() - throw new QueryExecutionException(response.getErrorMessage) - } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.close() - HiveShim.processResults(results) - case _ => - if (sessionState.out != null) { - sessionState.out.println(tokens(0) + " " + cmd_1) - } - Seq(proc.run(cmd_1).getResponseCode.toString) - } - } catch { - case e: Exception => - logError( - s""" - |====================== - |HIVE FAILURE OUTPUT - |====================== - |${outputBuffer.toString} - |====================== - |END HIVE FAILURE OUTPUT - |====================== - """.stripMargin) - throw e + protected[hive] lazy val hiveconf: HiveConf = { + setConf(sessionState.getConf.getAllProperties) + sessionState.getConf } } @@ -391,17 +406,23 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { ) } + protected[hive] def runSqlHive(sql: String): Seq[String] = { + if (sql.toLowerCase.contains("create temporary function")) { + executionHive.runSqlHive(sql) + } else if (sql.trim.toLowerCase.startsWith("set")) { + metadataHive.runSqlHive(sql) + executionHive.runSqlHive(sql) + } else { + metadataHive.runSqlHive(sql) + } + } + @transient override protected[sql] val planner = hivePlanner /** Extends QueryExecution with hive specific features. */ protected[sql] class QueryExecution(logicalPlan: LogicalPlan) extends super.QueryExecution(logicalPlan) { - // Like what we do in runHive, makes sure the session represented by the - // `sessionState` field is activated. - if (SessionState.get() != sessionState) { - SessionState.start(sessionState) - } /** * Returns the result as a hive compatible sequence of strings. For native commands, the @@ -439,7 +460,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } -private object HiveContext { +private[hive] object HiveContext { + /** The version of hive used internally by Spark SQL. */ + val hiveExecutionVersion: String = "0.13.1" + + val HIVE_METASTORE_VERSION: String = "spark.sql.hive.metastore.version" + val HIVE_METASTORE_JARS: String = "spark.sql.hive.metastore.jars" + + /** Constructs a configuration for hive, where the metastore is located in a temp directory. */ + def newTemporaryConfiguration(): Map[String, String] = { + val tempDir = Utils.createTempDir() + val localMetastore = new File(tempDir, "metastore").getAbsolutePath + Map( + "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$localMetastore;create=true") + } + protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) 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 4d222cf88e5e8..8fcdf3d0ab119 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 @@ -22,6 +22,8 @@ import java.util.{List => JList} 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, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.metastore.{TableType, Warehouse} import org.apache.hadoop.hive.ql.metadata._ @@ -32,6 +34,7 @@ import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.sql.hive.client.IsolatedClientLoader import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NoSuchTableException, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ @@ -39,6 +42,7 @@ 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, DDLParser, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ @@ -47,11 +51,10 @@ import org.apache.spark.util.Utils /* Implicit conversions */ import scala.collection.JavaConversions._ -private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { - import org.apache.spark.sql.hive.HiveMetastoreTypes._ +private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: HiveContext) + extends Catalog with Logging { - /** Connection to hive metastore. Usages should lock on `this`. */ - protected[hive] val client = Hive.get(hive.hiveconf) + import org.apache.spark.sql.hive.HiveMetastoreTypes._ /** Usages should lock on `this`. */ protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) @@ -67,14 +70,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { override def load(in: QualifiedTableName): LogicalPlan = { logDebug(s"Creating new cached data source for $in") - val table = HiveMetastoreCatalog.this.synchronized { - client.getTable(in.database, in.name) - } + val table = client.getTable(in.database, in.name) def schemaStringFromParts: Option[String] = { - Option(table.getProperty("spark.sql.sources.schema.numParts")).map { numParts => + table.properties.get("spark.sql.sources.schema.numParts").map { numParts => val parts = (0 until numParts.toInt).map { index => - val part = table.getProperty(s"spark.sql.sources.schema.part.${index}") + val part = table.properties.get(s"spark.sql.sources.schema.part.${index}").orNull if (part == null) { throw new AnalysisException( s"Could not read schema from the metastore because it is corrupted " + @@ -92,20 +93,20 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // After SPARK-6024, we removed this flag. // Although we are not using spark.sql.sources.schema any more, we need to still support. val schemaString = - Option(table.getProperty("spark.sql.sources.schema")).orElse(schemaStringFromParts) + table.properties.get("spark.sql.sources.schema").orElse(schemaStringFromParts) val userSpecifiedSchema = schemaString.map(s => DataType.fromJson(s).asInstanceOf[StructType]) // It does not appear that the ql client for the metastore has a way to enumerate all the // SerDe properties directly... - val options = table.getTTable.getSd.getSerdeInfo.getParameters.toMap + val options = table.serdeProperties val resolvedRelation = ResolvedDataSource( hive, userSpecifiedSchema, - table.getProperty("spark.sql.sources.provider"), + table.properties("spark.sql.sources.provider"), options) LogicalRelation(resolvedRelation.relation) @@ -144,49 +145,53 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with options: Map[String, String], isExternal: Boolean): Unit = { val (dbName, tblName) = processDatabaseAndTableName("default", tableName) - val tbl = new Table(dbName, tblName) - - tbl.setProperty("spark.sql.sources.provider", provider) + val tableProperties = new scala.collection.mutable.HashMap[String, String] + tableProperties.put("spark.sql.sources.provider", provider) if (userSpecifiedSchema.isDefined) { val threshold = hive.conf.schemaStringLengthThreshold val schemaJsonString = userSpecifiedSchema.get.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq - tbl.setProperty("spark.sql.sources.schema.numParts", parts.size.toString) + tableProperties.put("spark.sql.sources.schema.numParts", parts.size.toString) parts.zipWithIndex.foreach { case (part, index) => - tbl.setProperty(s"spark.sql.sources.schema.part.${index}", part) + tableProperties.put(s"spark.sql.sources.schema.part.${index}", part) } } - options.foreach { case (key, value) => tbl.setSerdeParam(key, value) } - if (isExternal) { - tbl.setProperty("EXTERNAL", "TRUE") - tbl.setTableType(TableType.EXTERNAL_TABLE) + val tableType = if (isExternal) { + tableProperties.put("EXTERNAL", "TRUE") + ExternalTable } else { - tbl.setProperty("EXTERNAL", "FALSE") - tbl.setTableType(TableType.MANAGED_TABLE) - } - - // create the table - synchronized { - client.createTable(tbl, false) - } + tableProperties.put("EXTERNAL", "FALSE") + ManagedTable + } + + client.createTable( + HiveTable( + specifiedDatabase = Option(dbName), + name = tblName, + schema = Seq.empty, + partitionColumns = Seq.empty, + tableType = tableType, + properties = tableProperties.toMap, + serdeProperties = options)) } - def hiveDefaultTableFilePath(tableName: String): String = synchronized { - val currentDatabase = client.getDatabase(hive.sessionState.getCurrentDatabase) - - hiveWarehouse.getTablePath(currentDatabase, tableName).toString + def hiveDefaultTableFilePath(tableName: String): String = { + // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) + new Path( + new Path(client.getDatabase(client.currentDatabase).location), + tableName.toLowerCase).toString } - def tableExists(tableIdentifier: Seq[String]): Boolean = synchronized { + def tableExists(tableIdentifier: Seq[String]): Boolean = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent .lift(tableIdent.size - 2) - .getOrElse(hive.sessionState.getCurrentDatabase) + .getOrElse(client.currentDatabase) val tblName = tableIdent.last - client.getTable(databaseName, tblName, false) != null + client.getTableOption(databaseName, tblName).isDefined } def lookupRelation( @@ -194,18 +199,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with alias: Option[String]): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( - hive.sessionState.getCurrentDatabase) + client.currentDatabase) val tblName = tableIdent.last - val table = try { - synchronized { - client.getTable(databaseName, tblName) - } - } catch { - case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => - throw new NoSuchTableException - } + val table = client.getTable(databaseName, tblName) - if (table.getProperty("spark.sql.sources.provider") != null) { + if (table.properties.get("spark.sql.sources.provider").isDefined) { val dataSourceTable = cachedDataSourceTables(QualifiedTableName(databaseName, tblName).toLowerCase) // Then, if alias is specified, wrap the table with a Subquery using the alias. @@ -215,22 +213,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Subquery(tableIdent.last, dataSourceTable)) withAlias - } else if (table.isView) { - // if the unresolved relation is from hive view - // parse the text into logic node. - HiveQl.createPlanForView(table, alias) + } else if (table.tableType == VirtualView) { + val viewText = table.viewText.getOrElse(sys.error("Invalid view without text.")) + alias match { + // because hive use things like `_c0` to build the expanded text + // currently we cannot support view from "create view v1(c1) as ..." + case None => Subquery(table.name, HiveQl.createPlan(viewText)) + case Some(aliasText) => Subquery(aliasText, HiveQl.createPlan(viewText)) + } } else { - val partitions: Seq[Partition] = - if (table.isPartitioned) { - synchronized { - HiveShim.getAllPartitionsOf(client, table).toSeq - } - } else { - Nil - } - - MetastoreRelation(databaseName, tblName, alias)( - table.getTTable, partitions.map(part => part.getTPartition))(hive) + MetastoreRelation(databaseName, tblName, alias)(table)(hive) } } @@ -318,178 +310,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with result.newInstance() } - override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = synchronized { - val dbName = if (!caseSensitive) { - if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None - } else { - databaseName - } - val db = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - - client.getAllTables(db).map(tableName => (tableName, false)) - } - - /** - * Create table with specified database, table name, table description and schema - * @param databaseName Database Name - * @param tableName Table Name - * @param schema Schema of the new table, if not specified, will use the schema - * specified in crtTbl - * @param allowExisting if true, ignore AlreadyExistsException - * @param desc CreateTableDesc object which contains the SerDe info. Currently - * we support most of the features except the bucket. - */ - def createTable( - databaseName: String, - tableName: String, - schema: Seq[Attribute], - allowExisting: Boolean = false, - desc: Option[CreateTableDesc] = None) { - val hconf = hive.hiveconf - - val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) - val tbl = new Table(dbName, tblName) - - val crtTbl: CreateTableDesc = desc.getOrElse(null) - - // We should respect the passed in schema, unless it's not set - val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) { - crtTbl.getCols - } else { - schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), null)) - } - tbl.setFields(hiveSchema) - - // Most of code are similar with the DDLTask.createTable() of Hive, - if (crtTbl != null && crtTbl.getTblProps() != null) { - tbl.getTTable().getParameters().putAll(crtTbl.getTblProps()) - } - - if (crtTbl != null && crtTbl.getPartCols() != null) { - tbl.setPartCols(crtTbl.getPartCols()) - } - - if (crtTbl != null && crtTbl.getStorageHandler() != null) { - tbl.setProperty( - org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, - crtTbl.getStorageHandler()) - } - - /* - * We use LazySimpleSerDe by default. - * - * If the user didn't specify a SerDe, and any of the columns are not simple - * types, we will have to use DynamicSerDe instead. - */ - if (crtTbl == null || crtTbl.getSerName() == null) { - val storageHandler = tbl.getStorageHandler() - if (storageHandler == null) { - logInfo(s"Default to LazySimpleSerDe for table $dbName.$tblName") - tbl.setSerializationLib(classOf[LazySimpleSerDe].getName()) - - import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - import org.apache.hadoop.io.Text - import org.apache.hadoop.mapred.TextInputFormat - - tbl.setInputFormatClass(classOf[TextInputFormat]) - tbl.setOutputFormatClass(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]]) - tbl.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") - } else { - val serDeClassName = storageHandler.getSerDeClass().getName() - logInfo(s"Use StorageHandler-supplied $serDeClassName for table $dbName.$tblName") - tbl.setSerializationLib(serDeClassName) - } - } else { - // let's validate that the serde exists - val serdeName = crtTbl.getSerName() - try { - val d = ReflectionUtils.newInstance(hconf.getClassByName(serdeName), hconf) - if (d != null) { - logDebug("Found class for $serdeName") - } - } catch { - case e: SerDeException => throw new HiveException("Cannot validate serde: " + serdeName, e) - } - tbl.setSerializationLib(serdeName) - } - - if (crtTbl != null && crtTbl.getFieldDelim() != null) { - tbl.setSerdeParam(serdeConstants.FIELD_DELIM, crtTbl.getFieldDelim()) - tbl.setSerdeParam(serdeConstants.SERIALIZATION_FORMAT, crtTbl.getFieldDelim()) - } - if (crtTbl != null && crtTbl.getFieldEscape() != null) { - tbl.setSerdeParam(serdeConstants.ESCAPE_CHAR, crtTbl.getFieldEscape()) - } - - if (crtTbl != null && crtTbl.getCollItemDelim() != null) { - tbl.setSerdeParam(serdeConstants.COLLECTION_DELIM, crtTbl.getCollItemDelim()) - } - if (crtTbl != null && crtTbl.getMapKeyDelim() != null) { - tbl.setSerdeParam(serdeConstants.MAPKEY_DELIM, crtTbl.getMapKeyDelim()) - } - if (crtTbl != null && crtTbl.getLineDelim() != null) { - tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim()) - } - HiveShim.setTblNullFormat(crtTbl, tbl) - - if (crtTbl != null && crtTbl.getSerdeProps() != null) { - val iter = crtTbl.getSerdeProps().entrySet().iterator() - while (iter.hasNext()) { - val m = iter.next() - tbl.setSerdeParam(m.getKey(), m.getValue()) - } - } - - if (crtTbl != null && crtTbl.getComment() != null) { - tbl.setProperty("comment", crtTbl.getComment()) - } - - if (crtTbl != null && crtTbl.getLocation() != null) { - HiveShim.setLocation(tbl, crtTbl) - } - - if (crtTbl != null && crtTbl.getSkewedColNames() != null) { - tbl.setSkewedColNames(crtTbl.getSkewedColNames()) - } - if (crtTbl != null && crtTbl.getSkewedColValues() != null) { - tbl.setSkewedColValues(crtTbl.getSkewedColValues()) - } - - if (crtTbl != null) { - tbl.setStoredAsSubDirectories(crtTbl.isStoredAsSubDirectories()) - tbl.setInputFormatClass(crtTbl.getInputFormat()) - tbl.setOutputFormatClass(crtTbl.getOutputFormat()) - } - - tbl.getTTable().getSd().setInputFormat(tbl.getInputFormatClass().getName()) - tbl.getTTable().getSd().setOutputFormat(tbl.getOutputFormatClass().getName()) - - if (crtTbl != null && crtTbl.isExternal()) { - tbl.setProperty("EXTERNAL", "TRUE") - tbl.setTableType(TableType.EXTERNAL_TABLE) - } - - // set owner - try { - tbl.setOwner(hive.hiveconf.getUser) - } catch { - case e: IOException => throw new HiveException("Unable to get current user", e) - } - - // set create time - tbl.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - - // TODO add bucket support - // TODO set more info if Hive upgrade + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + val db = databaseName.getOrElse(client.currentDatabase) - // create the table - synchronized { - try client.createTable(tbl, allowExisting) catch { - case e: org.apache.hadoop.hive.metastore.api.AlreadyExistsException - if allowExisting => // Do nothing - case e: Throwable => throw e - } - } + client.listTables(db).map(tableName => (tableName, false)) } protected def processDatabaseAndTableName( @@ -598,42 +422,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - // TODO extra is in type of ASTNode which means the logical plan is not resolved - // Need to think about how to implement the CreateTableAsSelect.resolved - case CreateTableAsSelect(db, tableName, child, allowExisting, Some(extra: ASTNode)) => - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - - // Get the CreateTableDesc from Hive SemanticAnalyzer - val desc: Option[CreateTableDesc] = if (tableExists(Seq(databaseName, tblName))) { - None - } else { - val sa = new SemanticAnalyzer(hive.hiveconf) { - override def analyzeInternal(ast: ASTNode) { - // A hack to intercept the SemanticAnalyzer.analyzeInternal, - // to ignore the SELECT clause of the CTAS - val method = classOf[SemanticAnalyzer].getDeclaredMethod( - "analyzeCreateTable", classOf[ASTNode], classOf[QB]) - method.setAccessible(true) - method.invoke(this, ast, this.getQB) - } - } - - sa.analyze(extra, new Context(hive.hiveconf)) - Some(sa.getQB().getTableDesc) - } - - // Check if the query specifies file format or storage handler. - val hasStorageSpec = desc match { - case Some(crtTbl) => - crtTbl != null && (crtTbl.getSerName != null || crtTbl.getStorageHandler != null) - case None => false - } - - if (hive.convertCTAS && !hasStorageSpec) { + case CreateTableAsSelect(desc, child, allowExisting) => + if (hive.convertCTAS && !desc.serde.isDefined) { // 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 (dbName.isDefined) { + if (desc.specifiedDatabase.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + "when spark.sql.hive.convertCTAS is set to true.") @@ -641,7 +434,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( - tblName, + desc.name, hive.conf.defaultDataSourceName, temporary = false, mode, @@ -650,19 +443,19 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with ) } else { execution.CreateTableAsSelect( - databaseName, - tableName, + desc.copy( + specifiedDatabase = Option(desc.specifiedDatabase.getOrElse(client.currentDatabase))), child, - allowExisting, - desc) + allowExisting) } case p: LogicalPlan if p.resolved => p - case p @ CreateTableAsSelect(db, tableName, child, allowExisting, None) => - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + case p @ CreateTableAsSelect(desc, child, allowExisting) => + val (dbName, tblName) = processDatabaseAndTableName(desc.database, desc.name) + if (hive.convertCTAS) { - if (dbName.isDefined) { + if (desc.specifiedDatabase.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + "when spark.sql.hive.convertCTAS is set to true.") @@ -678,13 +471,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with child ) } else { - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) execution.CreateTableAsSelect( - databaseName, - tableName, + desc, child, - allowExisting, - None) + allowExisting) } } } @@ -767,7 +557,7 @@ private[hive] case class InsertIntoHiveTable( private[hive] case class MetastoreRelation (databaseName: String, tableName: String, alias: Option[String]) - (val table: TTable, val partitions: Seq[TPartition]) + (val table: HiveTable) (@transient sqlContext: SQLContext) extends LeafNode with MultiInstanceRelation { @@ -786,16 +576,63 @@ private[hive] case class MetastoreRelation Objects.hashCode(databaseName, tableName, alias, output) } - // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and - // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. - // Right now, using org.apache.hadoop.hive.ql.metadata.Table and - // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException - // which indicates the SerDe we used is not Serializable. + @transient val hiveQlTable: Table = { + // We start by constructing an API table as Hive performs several important transformations + // internally when converting an API table to a QL table. + val tTable = new org.apache.hadoop.hive.metastore.api.Table() + tTable.setTableName(table.name) + tTable.setDbName(table.database) + + val tableParameters = new java.util.HashMap[String, String]() + tTable.setParameters(tableParameters) + table.properties.foreach { case (k, v) => tableParameters.put(k, v) } + + tTable.setTableType(table.tableType.name) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tTable.setSd(sd) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + tTable.setPartitionKeys( + table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + + table.location.foreach(sd.setLocation) + table.inputFormat.foreach(sd.setInputFormat) + table.outputFormat.foreach(sd.setOutputFormat) + + val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo + sd.setSerdeInfo(serdeInfo) + table.serde.foreach(serdeInfo.setSerializationLib) + val serdeParameters = new java.util.HashMap[String, String]() + serdeInfo.setParameters(serdeParameters) + table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + + new Table(tTable) + } + + @transient val hiveQlPartitions: Seq[Partition] = table.getAllPartitions.map { p => + val tPartition = new org.apache.hadoop.hive.metastore.api.Partition + tPartition.setDbName(databaseName) + tPartition.setTableName(tableName) + tPartition.setValues(p.values) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tPartition.setSd(sd) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + + sd.setLocation(p.storage.location) + sd.setInputFormat(p.storage.inputFormat) + sd.setOutputFormat(p.storage.outputFormat) + + val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo + sd.setSerdeInfo(serdeInfo) + serdeInfo.setSerializationLib(p.storage.serde) - @transient val hiveQlTable: Table = new Table(table) + val serdeParameters = new java.util.HashMap[String, String]() + serdeInfo.setParameters(serdeParameters) + table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + p.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } - @transient val hiveQlPartitions: Seq[Partition] = partitions.map { p => - new Partition(hiveQlTable, p) + new Partition(hiveQlTable, tPartition) } @transient override lazy val statistics: Statistics = Statistics( @@ -865,7 +702,7 @@ private[hive] case class MetastoreRelation val columnOrdinals = AttributeMap(attributes.zipWithIndex) override def newInstance(): MetastoreRelation = { - MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) + MetastoreRelation(databaseName, tableName, alias)(table)(sqlContext) } } 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 6176aee25e7a4..f30b196734c40 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 @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.sources.DescribeCommand +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler @@ -50,7 +51,19 @@ import scala.collection.JavaConversions._ * back for Hive to execute natively. Will be replaced with a native command that contains the * cmd string. */ -private[hive] case object NativePlaceholder extends Command +private[hive] case object NativePlaceholder extends LogicalPlan { + override def children: Seq[LogicalPlan] = Seq.empty + override def output: Seq[Attribute] = Seq.empty +} + +case class CreateTableAsSelect( + tableDesc: HiveTable, + child: LogicalPlan, + allowExisting: Boolean) extends UnaryNode with Command { + + override def output: Seq[Attribute] = Seq.empty[Attribute] + override lazy val resolved: Boolean = tableDesc.specifiedDatabase.isDefined && childrenResolved +} /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { @@ -78,16 +91,16 @@ private[hive] object HiveQl { "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", - + "TOK_CREATEDATABASE", "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_CREATEROLE", "TOK_CREATEVIEW", - + "TOK_DESCDATABASE", "TOK_DESCFUNCTION", - + "TOK_DROPDATABASE", "TOK_DROPFUNCTION", "TOK_DROPINDEX", @@ -95,22 +108,22 @@ private[hive] object HiveQl { "TOK_DROPTABLE_PROPERTIES", "TOK_DROPVIEW", "TOK_DROPVIEW_PROPERTIES", - + "TOK_EXPORT", - + "TOK_GRANT", "TOK_GRANT_ROLE", - + "TOK_IMPORT", - + "TOK_LOAD", - + "TOK_LOCKTABLE", - + "TOK_MSCK", - + "TOK_REVOKE", - + "TOK_SHOW_COMPACTIONS", "TOK_SHOW_CREATETABLE", "TOK_SHOW_GRANT", @@ -127,9 +140,9 @@ private[hive] object HiveQl { "TOK_SHOWINDEXES", "TOK_SHOWLOCKS", "TOK_SHOWPARTITIONS", - + "TOK_SWITCHDATABASE", - + "TOK_UNLOCKTABLE" ) @@ -259,6 +272,7 @@ private[hive] object HiveQl { case otherMessage => throw new AnalysisException(otherMessage) } + case e: MatchError => throw e case e: Exception => throw new AnalysisException(e.getMessage) case e: NotImplementedError => @@ -272,14 +286,6 @@ private[hive] object HiveQl { } } - /** Creates LogicalPlan for a given VIEW */ - def createPlanForView(view: Table, alias: Option[String]): Subquery = alias match { - // because hive use things like `_c0` to build the expanded text - // currently we cannot support view from "create view v1(c1) as ..." - case None => Subquery(view.getTableName, createPlan(view.getViewExpandedText)) - case Some(aliasText) => Subquery(aliasText, createPlan(view.getViewExpandedText)) - } - def parseDdl(ddl: String): Seq[Attribute] = { val tree = try { @@ -453,6 +459,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C (keys, bitmasks) } + protected def getProperties(node: Node): Seq[(String, String)] = node match { + case Token("TOK_TABLEPROPLIST", list) => + list.map { + case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) => + (unquoteString(key) -> unquoteString(value)) + } + } + protected def nodeToPlan(node: Node): LogicalPlan = node match { // Special drop table that also uncaches. case Token("TOK_DROPTABLE", @@ -562,7 +576,62 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C children) val (db, tableName) = extractDbNameTableName(tableNameParts) - CreateTableAsSelect(db, tableName, nodeToPlan(query), allowExisting != None, Some(node)) + 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} + tableDesc = tableDesc.copy( + outputFormat = Option(classOf[RCFileOutputFormat].getName), + inputFormat = Option(classOf[RCFileInputFormat[_, _]].getName)) + + 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")) + + 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")) + + case Token("TOK_TABLESERIALIZER", + Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => + tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName))) + + otherProps match { + case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => + tableDesc = tableDesc.copy( + serdeProperties = tableDesc.serdeProperties ++ getProperties(list)) + case Nil => + } + + case Token("TOK_TABLEPROPERTIES", list :: Nil) => + tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) + + case _ => + } + + 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. case Token("TOK_CREATETABLE", _) => NativePlaceholder @@ -759,7 +828,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_CUBE_GROUPBY", children) => Cube(children.map(nodeToExpr), withLateralView, selectExpressions) case _ => sys.error("Expect WITH CUBE") - }), + }), Some(Project(selectExpressions, withLateralView))).flatten.head } @@ -1077,6 +1146,15 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } protected val escapedIdentifier = "`([^`]+)`".r + protected val doubleQuotedString = "\"([^\"]+)\"".r + protected val singleQuotedString = "'([^']+)'".r + + protected def unquoteString(str: String) = str match { + case singleQuotedString(s) => s + case doubleQuotedString(s) => s + case other => other + } + /** Strips backticks from ident if present */ protected def cleanIdentifier(ident: String): String = ident match { case escapedIdentifier(i) => i diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index e556c74ffb015..b69312f0f8717 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast +import org.apache.spark.Logging import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DateUtils @@ -57,7 +58,7 @@ class HadoopTableReader( @transient relation: MetastoreRelation, @transient sc: HiveContext, @transient hiveExtraConf: HiveConf) - extends TableReader { + extends TableReader with Logging { // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local". // https://hadoop.apache.org/docs/r1.0.4/mapred-default.html @@ -78,7 +79,7 @@ class HadoopTableReader( makeRDDForTable( hiveTable, Class.forName( - relation.tableDesc.getSerdeClassName, true, sc.sessionState.getConf.getClassLoader) + relation.tableDesc.getSerdeClassName, true, Utils.getSparkClassLoader) .asInstanceOf[Class[Deserializer]], filterOpt = None) @@ -145,7 +146,7 @@ class HadoopTableReader( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[Row] = { - + // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists def verifyPartitionPath( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): @@ -288,7 +289,7 @@ class HadoopTableReader( } } -private[hive] object HadoopTableReader extends HiveInspectors { +private[hive] object HadoopTableReader extends HiveInspectors with Logging { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to * instantiate a HadoopRDD. @@ -329,6 +330,8 @@ private[hive] object HadoopTableReader extends HiveInspectors { tableDeser.getObjectInspector).asInstanceOf[StructObjectInspector] } + logDebug(soi.toString) + val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) => soi.getStructFieldRef(attr.name) -> ordinal }.unzip diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala index a863aa77cb7e0..0a1d761a52f88 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala @@ -17,30 +17,35 @@ package org.apache.spark.sql.hive.client +import java.io.PrintStream +import java.util.{Map => JMap} + import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} -case class HiveDatabase( +private[hive] case class HiveDatabase( name: String, location: String) -abstract class TableType { val name: String } -case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } -case object IndexTable extends TableType { override val name = "INDEX_TABLE" } -case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } -case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } +private[hive] abstract class TableType { val name: String } +private[hive] case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } +private[hive] case object IndexTable extends TableType { override val name = "INDEX_TABLE" } +private[hive] case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } +private[hive] case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } -case class HiveStorageDescriptor( +// TODO: Use this for Tables and Partitions +private[hive] case class HiveStorageDescriptor( location: String, inputFormat: String, outputFormat: String, - serde: String) + serde: String, + serdeProperties: Map[String, String]) -case class HivePartition( +private[hive] case class HivePartition( values: Seq[String], storage: HiveStorageDescriptor) -case class HiveColumn(name: String, hiveType: String, comment: String) -case class HiveTable( +private[hive] case class HiveColumn(name: String, hiveType: String, comment: String) +private[hive] case class HiveTable( specifiedDatabase: Option[String], name: String, schema: Seq[HiveColumn], @@ -51,7 +56,8 @@ case class HiveTable( location: Option[String] = None, inputFormat: Option[String] = None, outputFormat: Option[String] = None, - serde: Option[String] = None) { + serde: Option[String] = None, + viewText: Option[String] = None) { @transient private[client] var client: ClientInterface = _ @@ -76,13 +82,17 @@ case class HiveTable( * internal and external classloaders for a given version of Hive and thus must expose only * shared classes. */ -trait ClientInterface { +private[hive] trait ClientInterface { /** * Runs a HiveQL command using Hive, returning the results as a list of strings. Each row will * result in one string. */ def runSqlHive(sql: String): Seq[String] + def setOut(stream: PrintStream): Unit + def setInfo(stream: PrintStream): Unit + def setError(stream: PrintStream): Unit + /** Returns the names of all tables in the given database. */ def listTables(dbName: String): Seq[String] @@ -114,6 +124,11 @@ trait ClientInterface { /** Creates a new database with the given name. */ def createDatabase(database: HiveDatabase): Unit + /** Returns the specified paritition or None if it does not exist. */ + def getPartitionOption( + hTable: HiveTable, + partitionSpec: JMap[String, String]): Option[HivePartition] + /** Returns all partitions for the given table. */ def getAllPartitions(hTable: HiveTable): Seq[HivePartition] 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 ea52fea037f1f..6bca9d0179fe3 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.client import java.io.{BufferedReader, InputStreamReader, File, PrintStream} import java.net.URI -import java.util.{ArrayList => JArrayList} +import java.util.{ArrayList => JArrayList, Map => JMap, List => JList, Set => JSet} import scala.collection.JavaConversions._ import scala.language.reflectiveCalls @@ -27,6 +27,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.Database import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.TableType import org.apache.hadoop.hive.metastore.api import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata @@ -54,19 +55,13 @@ import org.apache.spark.sql.execution.QueryExecutionException * @param config a collection of configuration options that will be added to the hive conf before * opening the hive client. */ -class ClientWrapper( +private[hive] class ClientWrapper( version: HiveVersion, config: Map[String, String]) extends ClientInterface with Logging with ReflectionMagic { - private val conf = new HiveConf(classOf[SessionState]) - config.foreach { case (k, v) => - logDebug(s"Hive Config: $k=$v") - conf.set(k, v) - } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. private val outputBuffer = new java.io.OutputStream { var pos: Int = 0 @@ -99,17 +94,31 @@ class ClientWrapper( val original = Thread.currentThread().getContextClassLoader Thread.currentThread().setContextClassLoader(getClass.getClassLoader) val ret = try { - val newState = new SessionState(conf) - SessionState.start(newState) - newState.out = new PrintStream(outputBuffer, true, "UTF-8") - newState.err = new PrintStream(outputBuffer, true, "UTF-8") - newState + val oldState = SessionState.get() + if (oldState == null) { + val initialConf = new HiveConf(classOf[SessionState]) + config.foreach { case (k, v) => + logDebug(s"Hive Config: $k=$v") + initialConf.set(k, v) + } + val newState = new SessionState(initialConf) + SessionState.start(newState) + newState.out = new PrintStream(outputBuffer, true, "UTF-8") + newState.err = new PrintStream(outputBuffer, true, "UTF-8") + newState + } else { + oldState + } } finally { Thread.currentThread().setContextClassLoader(original) } ret } + /** Returns the configuration for the current session. */ + def conf: HiveConf = SessionState.get().getConf + + // TODO: should be a def?s private val client = Hive.get(conf) /** @@ -133,6 +142,18 @@ class ClientWrapper( ret } + def setOut(stream: PrintStream): Unit = withHiveState { + state.out = stream + } + + def setInfo(stream: PrintStream): Unit = withHiveState { + state.info = stream + } + + def setError(stream: PrintStream): Unit = withHiveState { + state.err = stream + } + override def currentDatabase: String = withHiveState { state.getCurrentDatabase } @@ -171,14 +192,20 @@ class ClientWrapper( partitionColumns = h.getPartCols.map(f => HiveColumn(f.getName, f.getType, f.getComment)), properties = h.getParameters.toMap, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, - tableType = ManagedTable, // TODO + tableType = h.getTableType match { + case TableType.MANAGED_TABLE => ManagedTable + case TableType.EXTERNAL_TABLE => ExternalTable + case TableType.VIRTUAL_VIEW => VirtualView + case TableType.INDEX_TABLE => IndexTable + }, location = version match { case hive.v12 => Option(h.call[URI]("getDataLocation")).map(_.toString) case hive.v13 => Option(h.call[Path]("getDataLocation")).map(_.toString) }, inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), - serde = Option(h.getSerializationLib)).withClient(this) + serde = Option(h.getSerializationLib), + viewText = Option(h.getViewExpandedText)).withClient(this) } converted } @@ -223,27 +250,40 @@ class ClientWrapper( client.alterTable(table.qualifiedName, qlTable) } + private def toHivePartition(partition: metadata.Partition): HivePartition = { + val apiPartition = partition.getTPartition + HivePartition( + values = Option(apiPartition.getValues).map(_.toSeq).getOrElse(Seq.empty), + storage = HiveStorageDescriptor( + location = apiPartition.getSd.getLocation, + inputFormat = apiPartition.getSd.getInputFormat, + outputFormat = apiPartition.getSd.getOutputFormat, + serde = apiPartition.getSd.getSerdeInfo.getSerializationLib, + serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.toMap)) + } + + override def getPartitionOption( + table: HiveTable, + partitionSpec: JMap[String, String]): Option[HivePartition] = withHiveState { + + val qlTable = toQlTable(table) + val qlPartition = client.getPartition(qlTable, partitionSpec, false) + Option(qlPartition).map(toHivePartition) + } + override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState { val qlTable = toQlTable(hTable) val qlPartitions = version match { case hive.v12 => - client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsForPruner", qlTable) + client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsForPruner", qlTable) case hive.v13 => - client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsOf", qlTable) + client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsOf", qlTable) } - qlPartitions.map(_.getTPartition).map { p => - HivePartition( - values = Option(p.getValues).map(_.toSeq).getOrElse(Seq.empty), - storage = HiveStorageDescriptor( - location = p.getSd.getLocation, - inputFormat = p.getSd.getInputFormat, - outputFormat = p.getSd.getOutputFormat, - serde = p.getSd.getSerdeInfo.getSerializationLib)) - }.toSeq + qlPartitions.toSeq.map(toHivePartition) } override def listTables(dbName: String): Seq[String] = withHiveState { - client.getAllTables + client.getAllTables(dbName) } /** @@ -267,11 +307,12 @@ class ClientWrapper( try { val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") + // The remainder of the command. val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc: CommandProcessor = version match { case hive.v12 => classOf[CommandProcessorFactory] - .callStatic[String, HiveConf, CommandProcessor]("get", cmd_1, conf) + .callStatic[String, HiveConf, CommandProcessor]("get", tokens(0), conf) case hive.v13 => classOf[CommandProcessorFactory] .callStatic[Array[String], HiveConf, CommandProcessor]("get", Array(tokens(0)), conf) @@ -294,7 +335,7 @@ class ClientWrapper( res.toSeq case hive.v13 => val res = new JArrayList[Object] - driver.call[JArrayList[Object], Boolean]("getResults", res) + driver.call[JList[Object], Boolean]("getResults", res) res.map { r => r match { case s: String => s diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 710dbca6e3c66..7f94c93ba49c1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.client import java.io.File -import java.net.URLClassLoader +import java.net.{URL, URLClassLoader} import java.util import scala.language.reflectiveCalls @@ -30,9 +30,10 @@ import org.apache.spark.Logging import org.apache.spark.deploy.SparkSubmitUtils import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.hive.HiveContext /** Factory for `IsolatedClientLoader` with specific versions of hive. */ -object IsolatedClientLoader { +private[hive] object IsolatedClientLoader { /** * Creates isolated Hive client loaders by downloading the requested version from maven. */ @@ -49,7 +50,7 @@ object IsolatedClientLoader { case "13" | "0.13" | "0.13.0" | "0.13.1" => hive.v13 } - private def downloadVersion(version: HiveVersion): Seq[File] = { + private def downloadVersion(version: HiveVersion): Seq[URL] = { val hiveArtifacts = (Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") ++ (if (version.hasBuiltinsJar) "hive-builtins" :: Nil else Nil)) @@ -72,10 +73,10 @@ object IsolatedClientLoader { tempDir.mkdir() allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) - tempDir.listFiles() + tempDir.listFiles().map(_.toURL) } - private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[File]] + private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[URL]] } /** @@ -99,9 +100,9 @@ object IsolatedClientLoader { * @param baseClassLoader The spark classloader that is used to load shared classes. * */ -class IsolatedClientLoader( +private[hive] class IsolatedClientLoader( val version: HiveVersion, - val execJars: Seq[File] = Seq.empty, + val execJars: Seq[URL] = Seq.empty, val config: Map[String, String] = Map.empty, val isolationOn: Boolean = true, val rootClassLoader: ClassLoader = ClassLoader.getSystemClassLoader.getParent.getParent, @@ -112,7 +113,7 @@ class IsolatedClientLoader( assert(Try(baseClassLoader.loadClass("org.apache.hive.HiveConf")).isFailure) /** All jars used by the hive specific classloader. */ - protected def allJars = execJars.map(_.toURI.toURL).toArray + protected def allJars = execJars.toArray protected def isSharedClass(name: String): Boolean = name.contains("slf4j") || @@ -166,6 +167,12 @@ class IsolatedClientLoader( .getConstructors.head .newInstance(version, config) .asInstanceOf[ClientInterface] + } catch { + case ReflectionException(cnf: NoClassDefFoundError) => + throw new ClassNotFoundException( + s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" + + "Please make sure that jars for your version of hive and hadoop are included in the " + + s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.") } finally { Thread.currentThread.setContextClassLoader(baseClassLoader) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala index 90d03049356b5..c600b158c5460 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala @@ -19,6 +19,14 @@ package org.apache.spark.sql.hive.client import scala.reflect._ +/** Unwraps reflection exceptions. */ +private[client] object ReflectionException { + def unapply(a: Throwable): Option[Throwable] = a match { + case ite: java.lang.reflect.InvocationTargetException => Option(ite.getCause) + case _ => None + } +} + /** * Provides implicit functions on any object for calling methods reflectively. */ 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 76a1965f3cb25..91e6ac4032204 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 @@ -24,8 +24,8 @@ import org.apache.spark.sql.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 -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.MetastoreRelation +import org.apache.spark.sql.hive.client.{HiveTable, HiveColumn} +import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, HiveMetastoreTypes} /** * Create table and insert the query result into it. @@ -39,17 +39,34 @@ import org.apache.spark.sql.hive.MetastoreRelation */ private[hive] case class CreateTableAsSelect( - database: String, - tableName: String, + tableDesc: HiveTable, query: LogicalPlan, - allowExisting: Boolean, - desc: Option[CreateTableDesc]) extends RunnableCommand { + allowExisting: Boolean) + extends RunnableCommand { + + def database: String = tableDesc.database + def tableName: String = tableDesc.name override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] lazy val metastoreRelation: MetastoreRelation = { - // Create Hive Table - hiveContext.catalog.createTable(database, tableName, query.output, allowExisting, desc) + import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + import org.apache.hadoop.io.Text + import org.apache.hadoop.mapred.TextInputFormat + + val withSchema = + tableDesc.copy( + schema = + query.output.map(c => + HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), null)), + inputFormat = + tableDesc.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), + outputFormat = + tableDesc.outputFormat + .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), + serde = tableDesc.serde.orElse(Some(classOf[LazySimpleSerDe].getName()))) + hiveContext.catalog.client.createTable(withSchema) // Get the Metastore Relation hiveContext.catalog.lookupRelation(Seq(database, tableName), None) match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 89995a91b1a92..de8954d5dec99 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -200,9 +200,7 @@ case class InsertIntoHiveTable( orderedPartitionSpec.put(entry.getName,partitionSpec.get(entry.getName).getOrElse("")) } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) - catalog.synchronized { - catalog.client.validatePartitionNameCharacters(partVals) - } + // inheritTableSpecs is set to true. It should be set to false for a IMPORT query // which is currently considered as a Hive native command. val inheritTableSpecs = true @@ -211,7 +209,7 @@ case class InsertIntoHiveTable( if (numDynamicPartitions > 0) { catalog.synchronized { catalog.client.loadDynamicPartitions( - outputPath, + outputPath.toString, qualifiedTableName, orderedPartitionSpec, overwrite, @@ -224,31 +222,28 @@ case class InsertIntoHiveTable( // ifNotExists is only valid with static partition, refer to // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries // scalastyle:on - val oldPart = catalog.synchronized { - catalog.client.getPartition( - catalog.client.getTable(qualifiedTableName), partitionSpec, false) - } - if (oldPart == null || !ifNotExists) { - catalog.synchronized { + val oldPart = + catalog.client.getPartitionOption( + catalog.client.getTable(table.databaseName, table.tableName), + partitionSpec) + + if (oldPart.isEmpty || !ifNotExists) { catalog.client.loadPartition( - outputPath, + outputPath.toString, qualifiedTableName, orderedPartitionSpec, overwrite, holdDDLTime, inheritTableSpecs, isSkewedStoreAsSubdir) - } } } } else { - catalog.synchronized { - catalog.client.loadTable( - outputPath, - qualifiedTableName, - overwrite, - holdDDLTime) - } + catalog.client.loadTable( + outputPath.toString, // TODO: URI + qualifiedTableName, + overwrite, + holdDDLTime) } // Invalidate the cache. 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 a40a1e53117cd..abab1a223a43a 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * Analyzes the given table in the current database to generate statistics, which will be @@ -84,8 +85,20 @@ case class AddJar(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] + val currentClassLoader = Utils.getContextOrSparkClassLoader + + // Add jar to current context + val jarURL = new java.io.File(path).toURL + val newClassLoader = new java.net.URLClassLoader(Array(jarURL), currentClassLoader) + Thread.currentThread.setContextClassLoader(newClassLoader) + org.apache.hadoop.hive.ql.metadata.Hive.get().getConf().setClassLoader(newClassLoader) + + // Add jar to isolated hive classloader hiveContext.runSqlHive(s"ADD JAR $path") + + // Add jar to executors hiveContext.sparkContext.addJar(path) + Seq(Row(0)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index ca84b43a998b8..1f40a5340c2ce 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table @@ -62,6 +63,8 @@ object TestHive class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { self => + import HiveContext._ + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") @@ -70,24 +73,16 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveconf.set("hive.plan.serialization.format", "javaXML") lazy val warehousePath = Utils.createTempDir() - lazy val metastorePath = Utils.createTempDir() /** Sets up the system initially or after a RESET command */ - protected def configure(): Unit = { - warehousePath.delete() - metastorePath.delete() - setConf("javax.jdo.option.ConnectionURL", - s"jdbc:derby:;databaseName=$metastorePath;create=true") - setConf("hive.metastore.warehouse.dir", warehousePath.toString) - } + protected override def configure(): Map[String, String] = + newTemporaryConfiguration() ++ Map("hive.metastore.warehouse.dir" -> warehousePath.toString) val testTempDir = Utils.createTempDir() // For some hive test case which contain ${system:test.tmp.dir} System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) - configure() // Must be called before initializing the catalog below. - /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") /** The location of the hive source code. */ @@ -195,6 +190,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { * A list of test tables and the DDL required to initialize them. A test table is loaded on * demand when a query are run against it. */ + @transient lazy val testTables = new mutable.HashMap[String, TestTable]() def registerTestTable(testTable: TestTable): Unit = { @@ -204,6 +200,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // The test tables that are defined in the Hive QTestUtil. // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient val hiveQTestUtilTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, @@ -236,16 +233,18 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} import org.apache.thrift.protocol.TBinaryProtocol - val srcThrift = new Table("default", "src_thrift") - srcThrift.setFields(Nil) - srcThrift.setInputFormatClass(classOf[SequenceFileInputFormat[_,_]].getName) - // In Hive, SequenceFileOutputFormat will be substituted by HiveSequenceFileOutputFormat. - srcThrift.setOutputFormatClass(classOf[SequenceFileOutputFormat[_,_]].getName) - srcThrift.setSerializationLib(classOf[ThriftDeserializer].getName) - srcThrift.setSerdeParam("serialization.class", classOf[Complex].getName) - srcThrift.setSerdeParam("serialization.format", classOf[TBinaryProtocol].getName) - catalog.client.createTable(srcThrift) - + runSqlHive( + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='${classOf[Complex].getName}', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_,_]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_,_]].getName}' + """.stripMargin) runSqlHive( s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") @@ -367,7 +366,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. loadedTables += name - logInfo(s"Loading test table $name") + logDebug(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -384,9 +383,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { */ protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames - // Database default may not exist in 0.13.1, create it if not exist - HiveShim.createDefaultDBIfNeeded(this) - /** * Resets the test instance by deleting any tables that have been created. * TODO: also clear out UDFs, views, etc. @@ -401,24 +397,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { cacheManager.clearCache() loadedTables.clear() catalog.cachedDataSourceTables.invalidateAll() - catalog.client.getAllTables("default").foreach { t => - logDebug(s"Deleting table $t") - val table = catalog.client.getTable("default", t) - - catalog.client.getIndexes("default", t, 255).foreach { index => - catalog.client.dropIndex("default", t, index.getIndexName, true) - } - - if (!table.isIndexTable) { - catalog.client.dropTable("default", t) - } - } - - catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - logDebug(s"Dropping Database: $db") - catalog.client.dropDatabase(db, true, false, true) - } - + catalog.client.reset() catalog.unregisterAllTables() FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName => @@ -429,7 +408,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveconf.set("fs.default.name", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. - runSqlHive("RESET") + executionHive.runSqlHive("RESET") + metadataHive.runSqlHive("RESET") // For some reason, RESET does not reset the following variables... // https://issues.apache.org/jira/browse/HIVE-9004 runSqlHive("set hive.table.parameters.default=") @@ -437,7 +417,11 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { runSqlHive("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") - configure() + + configure().foreach { + case (k, v) => + metadataHive.runSqlHive(s"SET $k=$v") + } runSqlHive("USE default") diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index 5bc08062d30eb..92eaf1f2795ba 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -33,7 +33,7 @@ log4j.appender.FA.layout=org.apache.log4j.PatternLayout log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n # Set the logger level of File Appender to WARN -log4j.appender.FA.Threshold = INFO +log4j.appender.FA.Threshold = DEBUG # Some packages are noisy for no good reason. log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index d960a30e00738..30f5313d2b812 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.hive -import java.io.{OutputStream, PrintStream} - import scala.util.Try import org.scalatest.BeforeAndAfter +import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.{AnalysisException, QueryTest} @@ -109,25 +108,6 @@ class ErrorPositionSuite extends QueryTest with BeforeAndAfter { "SELECT 1 + array(1)", "1 + array") } - /** Hive can be very noisy, messing up the output of our tests. */ - private def quietly[A](f: => A): A = { - val origErr = System.err - val origOut = System.out - try { - System.setErr(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) - System.setOut(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) - - f - } finally { - System.setErr(origErr) - System.setOut(origOut) - } - } - /** * Creates a test that checks to see if the error thrown when analyzing a given query includes * the location of the given token in the query string. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 0538aa203c5a0..47c60f651d14c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.InvalidInputException import org.apache.spark.sql._ import org.apache.spark.util.Utils import org.apache.spark.sql.types._ +import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.parquet.ParquetRelation2 @@ -686,16 +687,21 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { test("SPARK-6655 still support a schema stored in spark.sql.sources.schema") { val tableName = "spark6655" val schema = StructType(StructField("int", IntegerType, true) :: Nil) - // Manually create the metadata in metastore. - val tbl = new Table("default", tableName) - tbl.setProperty("spark.sql.sources.provider", "json") - tbl.setProperty("spark.sql.sources.schema", schema.json) - tbl.setProperty("EXTERNAL", "FALSE") - tbl.setTableType(TableType.MANAGED_TABLE) - tbl.setSerdeParam("path", catalog.hiveDefaultTableFilePath(tableName)) - catalog.synchronized { - catalog.client.createTable(tbl) - } + + val hiveTable = HiveTable( + specifiedDatabase = Some("default"), + name = tableName, + schema = Seq.empty, + partitionColumns = Seq.empty, + properties = Map( + "spark.sql.sources.provider" -> "json", + "spark.sql.sources.schema" -> schema.json, + "EXTERNAL" -> "FALSE"), + tableType = ManagedTable, + serdeProperties = Map( + "path" -> catalog.hiveDefaultTableFilePath(tableName))) + + catalog.client.createTable(hiveTable) invalidateTable(tableName) val actualSchema = table(tableName).schema diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala index d6ddd539d159d..8afe5459d4f1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala @@ -26,8 +26,10 @@ import org.apache.spark.sql.hive.test.TestHive class SerializationSuite extends FunSuite { test("[SPARK-5840] HiveContext should be serializable") { - val hiveContext = new HiveContext(TestHive.sparkContext) + val hiveContext = TestHive hiveContext.hiveconf - new JavaSerializer(new SparkConf()).newInstance().serialize(hiveContext) + val serializer = new JavaSerializer(new SparkConf()).newInstance() + val bytes = serializer.serialize(hiveContext) + val deSer = serializer.deserialize[AnyRef](bytes) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 81e77ba257bf1..321dc8d7322b8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -22,9 +22,13 @@ import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.util.Utils import org.scalatest.FunSuite +/** + * A simple set of tests that call the methods of a hive ClientInterface, loading different version + * of hive from maven central. These tests are simple in that they are mostly just testing to make + * sure that reflective calls are not throwing NoSuchMethod error, but the actually functionallity + * is not fully tested. + */ class VersionsSuite extends FunSuite with Logging { - val testType = "derby" - private def buildConf() = { lazy val warehousePath = Utils.createTempDir() lazy val metastorePath = Utils.createTempDir() @@ -50,6 +54,14 @@ class VersionsSuite extends FunSuite with Logging { causes } + private val emptyDir = Utils.createTempDir().getCanonicalPath + + private def partSpec = { + val hashMap = new java.util.LinkedHashMap[String, String] + hashMap.put("key", "1") + hashMap + } + // Its actually pretty easy to mess things up and have all of your tests "pass" by accidentally // connecting to an auto-populated, in-process metastore. Let's make sure we are getting the // versions right by forcing a known compatibility failure. @@ -66,10 +78,9 @@ class VersionsSuite extends FunSuite with Logging { private var client: ClientInterface = null versions.foreach { version => - test(s"$version: listTables") { + test(s"$version: create client") { client = null client = IsolatedClientLoader.forVersion(version, buildConf()).client - client.listTables("default") } test(s"$version: createDatabase") { @@ -101,5 +112,64 @@ class VersionsSuite extends FunSuite with Logging { test(s"$version: getTable") { client.getTable("default", "src") } + + test(s"$version: listTables") { + assert(client.listTables("default") === Seq("src")) + } + + test(s"$version: currentDatabase") { + assert(client.currentDatabase === "default") + } + + test(s"$version: getDatabase") { + client.getDatabase("default") + } + + test(s"$version: alterTable") { + client.alterTable(client.getTable("default", "src")) + } + + test(s"$version: set command") { + client.runSqlHive("SET spark.sql.test.key=1") + } + + test(s"$version: create partitioned table DDL") { + client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key INT)") + client.runSqlHive("ALTER TABLE src_part ADD PARTITION (key = '1')") + } + + test(s"$version: getPartitions") { + client.getAllPartitions(client.getTable("default", "src_part")) + } + + test(s"$version: loadPartition") { + client.loadPartition( + emptyDir, + "default.src_part", + partSpec, + false, + false, + false, + false) + } + + test(s"$version: loadTable") { + client.loadTable( + emptyDir, + "src", + false, + false) + } + + test(s"$version: loadDynamicPartitions") { + client.loadDynamicPartitions( + emptyDir, + "default.src_part", + partSpec, + false, + 1, + false, + false) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a3eacbd4e3981..9c056e493bfde 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -300,6 +300,8 @@ abstract class HiveComparisonTest val hiveQueries = queryList.map(new TestHive.QueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. + // Note this must only look at the logical plan as we might not be able to analyze if + // other DDL has not been executed yet. hiveQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { case ((queryString, i), hiveQuery, cachedAnswerFile)=> diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index ac10b173307d8..7d728fe87bda7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -900,7 +900,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |DROP TABLE IF EXISTS dynamic_part_table; """.stripMargin) - test("Dynamic partition folder layout") { + ignore("Dynamic partition folder layout") { sql("DROP TABLE IF EXISTS dynamic_part_table") sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") sql("SET hive.exec.dynamic.partition.mode=nonstrict") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 45f10e2fe64aa..de6a41ce5bfcb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -150,20 +150,21 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => val columnNames = columns.map(_.name) - val partValues = p.prunePartitions(relation.hiveQlPartitions).map(_.getValues) + val partValues = if (relation.table.isPartitioned) { + p.prunePartitions(relation.hiveQlPartitions).map(_.getValues) + } else { + Seq.empty + } (columnNames, partValues) }.head assert(actualOutputColumns === expectedOutputColumns, "Output columns mismatch") assert(actualScannedColumns === expectedScannedColumns, "Scanned columns mismatch") - assert( - actualPartValues.length === expectedPartValues.length, - "Partition value count mismatches") + val actualPartitions = actualPartValues.map(_.toSeq.mkString(",")).sorted + val expectedPartitions = expectedPartValues.map(_.mkString(",")).sorted - for ((actual, expected) <- actualPartValues.zip(expectedPartValues)) { - assert(actual sameElements expected, "Partition values mismatch") - } + assert(actualPartitions === expectedPartitions, "Partitions selected do not match") } // Creates a query test to compare query results generated by Hive and Catalyst.