From 7051722023b98f1720142c7b3b41948d275ea455 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 1 May 2016 19:05:20 -0700 Subject: [PATCH 001/313] [SPARK-13425][SQL] Documentation for CSV datasource options ## What changes were proposed in this pull request? This PR adds the explanation and documentation for CSV options for reading and writing. ## How was this patch tested? Style tests with `./dev/run_tests` for documentation style. Author: hyukjinkwon Author: Hyukjin Kwon Closes #12817 from HyukjinKwon/SPARK-13425. (cherry picked from commit a832cef11233c6357c7ba7ede387b432e6b0ed71) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 52 +++++++++++++++++++ .../apache/spark/sql/DataFrameReader.scala | 47 +++++++++++++++-- .../apache/spark/sql/DataFrameWriter.scala | 8 +++ 3 files changed, 103 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index ed9e716ab78e..cc5e93dcadf4 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -282,6 +282,45 @@ def csv(self, paths): :param paths: string, or list of strings, for input path(s). + You can set the following CSV-specific options to deal with CSV files: + * ``sep`` (default ``,``): sets the single character as a separator \ + for each field and value. + * ``charset`` (default ``UTF-8``): decodes the CSV files by the given \ + encoding type. + * ``quote`` (default ``"``): sets the single character used for escaping \ + quoted values where the separator can be part of the value. + * ``escape`` (default ``\``): sets the single character used for escaping quotes \ + inside an already quoted value. + * ``comment`` (default empty string): sets the single character used for skipping \ + lines beginning with this character. By default, it is disabled. + * ``header`` (default ``false``): uses the first line as names of columns. + * ``ignoreLeadingWhiteSpace`` (default ``false``): defines whether or not leading \ + whitespaces from values being read should be skipped. + * ``ignoreTrailingWhiteSpace`` (default ``false``): defines whether or not trailing \ + whitespaces from values being read should be skipped. + * ``nullValue`` (default empty string): sets the string representation of a null value. + * ``nanValue`` (default ``NaN``): sets the string representation of a non-number \ + value. + * ``positiveInf`` (default ``Inf``): sets the string representation of a positive \ + infinity value. + * ``negativeInf`` (default ``-Inf``): sets the string representation of a negative \ + infinity value. + * ``dateFormat`` (default ``None``): sets the string that indicates a date format. \ + Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This \ + applies to both date type and timestamp type. By default, it is None which means \ + trying to parse times and date by ``java.sql.Timestamp.valueOf()`` and \ + ``java.sql.Date.valueOf()``. + * ``maxColumns`` (default ``20480``): defines a hard limit of how many columns \ + a record can have. + * ``maxCharsPerColumn`` (default ``1000000``): defines the maximum number of \ + characters allowed for any given value being read. + * ``mode`` (default ``PERMISSIVE``): allows a mode for dealing with corrupt records \ + during parsing. + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. \ + When a schema is set by user, it sets ``null`` for extra fields. + * ``DROPMALFORMED`` : ignores the whole corrupted records. + * ``FAILFAST`` : throws an exception when it meets corrupted records. + >>> df = sqlContext.read.csv('python/test_support/sql/ages.csv') >>> df.dtypes [('C0', 'string'), ('C1', 'string')] @@ -663,6 +702,19 @@ def csv(self, path, mode=None, compression=None): known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). + You can set the following CSV-specific options to deal with CSV files: + * ``sep`` (default ``,``): sets the single character as a separator \ + for each field and value. + * ``quote`` (default ``"``): sets the single character used for escaping \ + quoted values where the separator can be part of the value. + * ``escape`` (default ``\``): sets the single character used for escaping quotes \ + inside an already quoted value. + * ``header`` (default ``false``): writes the names of columns as the first line. + * ``nullValue`` (default empty string): sets the string representation of a null value. + * ``compression``: compression codec to use when saving to file. This can be one of \ + the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and \ + deflate). + >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 3d43f2022f66..2d4a68f3c3a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -290,7 +290,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers * (e.g. 00012)
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
  • + * during parsing.
  • *
      *
    • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the * malformed string into a new field configured by `columnNameOfCorruptRecord`. When @@ -300,7 +300,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    *
  • `columnNameOfCorruptRecord` (default `_corrupt_record`): allows renaming the new field * having malformed string created by `PERMISSIVE` mode. This overrides - * `spark.sql.columnNameOfCorruptRecord`.
  • + * `spark.sql.columnNameOfCorruptRecord`.
  • * * @since 1.4.0 */ @@ -326,7 +326,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
  • + * during parsing.
  • *
      *
    • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the * malformed string into a new field configured by `columnNameOfCorruptRecord`. When @@ -336,7 +336,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    *
  • `columnNameOfCorruptRecord` (default `_corrupt_record`): allows renaming the new field * having malformed string created by `PERMISSIVE` mode. This overrides - * `spark.sql.columnNameOfCorruptRecord`.
  • + * `spark.sql.columnNameOfCorruptRecord`.
  • * * @since 1.6.0 */ @@ -393,6 +393,45 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * This function goes through the input once to determine the input schema. To avoid going * through the entire data once, specify the schema explicitly using [[schema]]. * + * You can set the following CSV-specific options to deal with CSV files: + *
  • `sep` (default `,`): sets the single character as a separator for each + * field and value.
  • + *
  • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding + * type.
  • + *
  • `quote` (default `"`): sets the single character used for escaping quoted values where + * the separator can be part of the value.
  • + *
  • `escape` (default `\`): sets the single character used for escaping quotes inside + * an already quoted value.
  • + *
  • `comment` (default empty string): sets the single character used for skipping lines + * beginning with this character. By default, it is disabled.
  • + *
  • `header` (default `false`): uses the first line as names of columns.
  • + *
  • `ignoreLeadingWhiteSpace` (default `false`): defines whether or not leading whitespaces + * from values being read should be skipped.
  • + *
  • `ignoreTrailingWhiteSpace` (default `fDataFraalse`): defines whether or not trailing + * whitespaces from values being read should be skipped.
  • + *
  • `nullValue` (default empty string): sets the string representation of a null value.
  • + *
  • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
  • + *
  • `positiveInf` (default `Inf`): sets the string representation of a positive infinity + * value.
  • + *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity + * value.
  • + *
  • `dateFormat` (default `null`): sets the string that indicates a date format. Custom date + * formats follow the formats at `java.text.SimpleDateFormat`. This applies to both date type + * and timestamp type. By default, it is `null` which means trying to parse times and date by + * `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()`.
  • + *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns + * a record can have.
  • + *
  • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed + * for any given value being read.
  • + *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records + * during parsing.
  • + *
      + *
    • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + * a schema is set by user, it sets `null` for extra fields.
    • + *
    • `DROPMALFORMED` : ignores the whole corrupted records.
    • + *
    • `FAILFAST` : throws an exception when it meets corrupted records.
    • + *
    + * * @since 2.0.0 */ @scala.annotation.varargs diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 28f5ccd26bc5..a57d47d28ceb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -606,6 +606,14 @@ final class DataFrameWriter private[sql](df: DataFrame) { * }}} * * You can set the following CSV-specific option(s) for writing CSV files: + *
  • `sep` (default `,`): sets the single character as a separator for each + * field and value.
  • + *
  • `quote` (default `"`): sets the single character used for escaping quoted values where + * the separator can be part of the value.
  • + *
  • `escape` (default `\`): sets the single character used for escaping quotes inside + * an already quoted value.
  • + *
  • `header` (default `false`): writes the names of columns as the first line.
  • + *
  • `nullValue` (default empty string): sets the string representation of a null value.
  • *
  • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
  • From 7d63c36e1efe8baec96cdc16a997249728e204fd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 1 May 2016 20:21:02 -0700 Subject: [PATCH 002/313] [SPARK-15049] Rename NewAccumulator to AccumulatorV2 ## What changes were proposed in this pull request? NewAccumulator isn't the best name if we ever come up with v3 of the API. ## How was this patch tested? Updated tests to reflect the change. Author: Reynold Xin Closes #12827 from rxin/SPARK-15049. (cherry picked from commit 44da8d8eabeccc12bfed0d43b37d54e0da845c66) Signed-off-by: Reynold Xin --- ...wAccumulator.scala => AccumulatorV2.scala} | 69 ++++++++++--------- .../org/apache/spark/ContextCleaner.scala | 2 +- .../org/apache/spark/HeartbeatReceiver.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../org/apache/spark/TaskContextImpl.scala | 2 +- .../org/apache/spark/TaskEndReason.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 4 +- .../apache/spark/executor/TaskMetrics.scala | 18 ++--- .../apache/spark/scheduler/DAGScheduler.scala | 10 +-- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../apache/spark/scheduler/TaskResult.scala | 8 +-- .../spark/scheduler/TaskScheduler.scala | 4 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 2 +- .../spark/InternalAccumulatorSuite.scala | 2 +- .../spark/executor/TaskMetricsSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 6 +- .../ExternalClusterManagerSuite.scala | 4 +- .../spark/scheduler/TaskSetManagerSuite.scala | 6 +- .../sql/execution/metric/SQLMetrics.scala | 6 +- 23 files changed, 85 insertions(+), 84 deletions(-) rename core/src/main/scala/org/apache/spark/{NewAccumulator.scala => AccumulatorV2.scala} (82%) diff --git a/core/src/main/scala/org/apache/spark/NewAccumulator.scala b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala similarity index 82% rename from core/src/main/scala/org/apache/spark/NewAccumulator.scala rename to core/src/main/scala/org/apache/spark/AccumulatorV2.scala index 1571e15b76ac..c65108a55eba 100644 --- a/core/src/main/scala/org/apache/spark/NewAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala @@ -21,9 +21,6 @@ import java.{lang => jl} import java.io.ObjectInputStream import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong -import javax.annotation.concurrent.GuardedBy - -import scala.collection.JavaConverters._ import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.util.Utils @@ -39,7 +36,7 @@ private[spark] case class AccumulatorMetadata( * The base class for accumulators, that can accumulate inputs of type `IN`, and produce output of * type `OUT`. */ -abstract class NewAccumulator[IN, OUT] extends Serializable { +abstract class AccumulatorV2[IN, OUT] extends Serializable { private[spark] var metadata: AccumulatorMetadata = _ private[this] var atDriverSide = true @@ -95,7 +92,7 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { } /** - * Creates an [[AccumulableInfo]] representation of this [[NewAccumulator]] with the provided + * Creates an [[AccumulableInfo]] representation of this [[AccumulatorV2]] with the provided * values. */ private[spark] def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { @@ -106,16 +103,16 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { final private[spark] def isAtDriverSide: Boolean = atDriverSide /** - * Tells if this accumulator is zero value or not. e.g. for a counter accumulator, 0 is zero + * Returns if this accumulator is zero value or not. e.g. for a counter accumulator, 0 is zero * value; for a list accumulator, Nil is zero value. */ - def isZero(): Boolean + def isZero: Boolean /** * Creates a new copy of this accumulator, which is zero value. i.e. call `isZero` on the copy * must return true. */ - def copyAndReset(): NewAccumulator[IN, OUT] + def copyAndReset(): AccumulatorV2[IN, OUT] /** * Takes the inputs and accumulates. e.g. it can be a simple `+=` for counter accumulator. @@ -126,7 +123,7 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { * Merges another same-type accumulator into this one and update its state, i.e. this should be * merge-in-place. */ - def merge(other: NewAccumulator[IN, OUT]): Unit + def merge(other: AccumulatorV2[IN, OUT]): Unit /** * Access this accumulator's current value; only allowed on driver. @@ -155,7 +152,7 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { "Accumulator must be registered before send to executor") } val copy = copyAndReset() - assert(copy.isZero(), "copyAndReset must return a zero value copy") + assert(copy.isZero, "copyAndReset must return a zero value copy") copy.metadata = metadata copy } else { @@ -191,6 +188,9 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { } +/** + * An internal class used to track accumulators by Spark itself. + */ private[spark] object AccumulatorContext { /** @@ -199,20 +199,21 @@ private[spark] object AccumulatorContext { * once the RDDs and user-code that reference them are cleaned up. * TODO: Don't use a global map; these should be tied to a SparkContext (SPARK-13051). */ - private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[NewAccumulator[_, _]]] + private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[AccumulatorV2[_, _]]] private[this] val nextId = new AtomicLong(0L) /** - * Return a globally unique ID for a new [[Accumulator]]. + * Returns a globally unique ID for a new [[Accumulator]]. * Note: Once you copy the [[Accumulator]] the ID is no longer unique. */ def newId(): Long = nextId.getAndIncrement + /** Returns the number of accumulators registered. Used in testing. */ def numAccums: Int = originals.size /** - * Register an [[Accumulator]] created on the driver such that it can be used on the executors. + * Registers an [[Accumulator]] created on the driver such that it can be used on the executors. * * All accumulators registered here can later be used as a container for accumulating partial * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. @@ -222,21 +223,21 @@ private[spark] object AccumulatorContext { * If an [[Accumulator]] with the same ID was already registered, this does nothing instead * of overwriting it. We will never register same accumulator twice, this is just a sanity check. */ - def register(a: NewAccumulator[_, _]): Unit = { - originals.putIfAbsent(a.id, new jl.ref.WeakReference[NewAccumulator[_, _]](a)) + def register(a: AccumulatorV2[_, _]): Unit = { + originals.putIfAbsent(a.id, new jl.ref.WeakReference[AccumulatorV2[_, _]](a)) } /** - * Unregister the [[Accumulator]] with the given ID, if any. + * Unregisters the [[Accumulator]] with the given ID, if any. */ def remove(id: Long): Unit = { originals.remove(id) } /** - * Return the [[Accumulator]] registered with the given ID, if any. + * Returns the [[Accumulator]] registered with the given ID, if any. */ - def get(id: Long): Option[NewAccumulator[_, _]] = { + def get(id: Long): Option[AccumulatorV2[_, _]] = { Option(originals.get(id)).map { ref => // Since we are storing weak references, we must check whether the underlying data is valid. val acc = ref.get @@ -248,7 +249,7 @@ private[spark] object AccumulatorContext { } /** - * Clear all registered [[Accumulator]]s. For testing only. + * Clears all registered [[Accumulator]]s. For testing only. */ def clear(): Unit = { originals.clear() @@ -256,10 +257,10 @@ private[spark] object AccumulatorContext { } -class LongAccumulator extends NewAccumulator[jl.Long, jl.Long] { +class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { private[this] var _sum = 0L - override def isZero(): Boolean = _sum == 0 + override def isZero: Boolean = _sum == 0 override def copyAndReset(): LongAccumulator = new LongAccumulator @@ -269,7 +270,7 @@ class LongAccumulator extends NewAccumulator[jl.Long, jl.Long] { def sum: Long = _sum - override def merge(other: NewAccumulator[jl.Long, jl.Long]): Unit = other match { + override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match { case o: LongAccumulator => _sum += o.sum case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") @@ -281,10 +282,10 @@ class LongAccumulator extends NewAccumulator[jl.Long, jl.Long] { } -class DoubleAccumulator extends NewAccumulator[jl.Double, jl.Double] { +class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[this] var _sum = 0.0 - override def isZero(): Boolean = _sum == 0.0 + override def isZero: Boolean = _sum == 0.0 override def copyAndReset(): DoubleAccumulator = new DoubleAccumulator @@ -294,7 +295,7 @@ class DoubleAccumulator extends NewAccumulator[jl.Double, jl.Double] { def sum: Double = _sum - override def merge(other: NewAccumulator[jl.Double, jl.Double]): Unit = other match { + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { case o: DoubleAccumulator => _sum += o.sum case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") @@ -306,11 +307,11 @@ class DoubleAccumulator extends NewAccumulator[jl.Double, jl.Double] { } -class AverageAccumulator extends NewAccumulator[jl.Double, jl.Double] { +class AverageAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[this] var _sum = 0.0 private[this] var _count = 0L - override def isZero(): Boolean = _sum == 0.0 && _count == 0 + override def isZero: Boolean = _sum == 0.0 && _count == 0 override def copyAndReset(): AverageAccumulator = new AverageAccumulator @@ -324,7 +325,7 @@ class AverageAccumulator extends NewAccumulator[jl.Double, jl.Double] { _count += 1 } - override def merge(other: NewAccumulator[jl.Double, jl.Double]): Unit = other match { + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { case o: AverageAccumulator => _sum += o.sum _count += o.count @@ -344,16 +345,16 @@ class AverageAccumulator extends NewAccumulator[jl.Double, jl.Double] { } -class ListAccumulator[T] extends NewAccumulator[T, java.util.List[T]] { +class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { private[this] val _list: java.util.List[T] = new java.util.ArrayList[T] - override def isZero(): Boolean = _list.isEmpty + override def isZero: Boolean = _list.isEmpty override def copyAndReset(): ListAccumulator[T] = new ListAccumulator override def add(v: T): Unit = _list.add(v) - override def merge(other: NewAccumulator[T, java.util.List[T]]): Unit = other match { + override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match { case o: ListAccumulator[T] => _list.addAll(o.localValue) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") @@ -370,10 +371,10 @@ class ListAccumulator[T] extends NewAccumulator[T, java.util.List[T]] { class LegacyAccumulatorWrapper[R, T]( initialValue: R, - param: org.apache.spark.AccumulableParam[R, T]) extends NewAccumulator[T, R] { + param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] { private[spark] var _value = initialValue // Current value on driver - override def isZero(): Boolean = _value == param.zero(initialValue) + override def isZero: Boolean = _value == param.zero(initialValue) override def copyAndReset(): LegacyAccumulatorWrapper[R, T] = { val acc = new LegacyAccumulatorWrapper(initialValue, param) @@ -383,7 +384,7 @@ class LegacyAccumulatorWrapper[R, T]( override def add(v: T): Unit = _value = param.addAccumulator(_value, v) - override def merge(other: NewAccumulator[T, R]): Unit = other match { + override def merge(other: AccumulatorV2[T, R]): Unit = other match { case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.localValue) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 63a00a84af3c..a51338c01707 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -144,7 +144,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { registerForCleanup(rdd, CleanRDD(rdd.id)) } - def registerAccumulatorForCleanup(a: NewAccumulator[_, _]): Unit = { + def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { registerForCleanup(a, CleanAccum(a.id)) } diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 9eac05fdf9f3..29018c75b922 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} */ private[spark] case class Heartbeat( executorId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], // taskId -> accumulator updates + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates blockManagerId: BlockManagerId) /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2cb3ed0296a4..d0f88d4e4d10 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1282,7 +1282,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register the given accumulator. Note that accumulators must be registered before use, or it * will throw exception. */ - def register(acc: NewAccumulator[_, _]): Unit = { + def register(acc: AccumulatorV2[_, _]): Unit = { acc.register(this) } @@ -1290,7 +1290,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register the given accumulator with given name. Note that accumulators must be registered * before use, or it will throw exception. */ - def register(acc: NewAccumulator[_, _], name: String): Unit = { + def register(acc: AccumulatorV2[_, _], name: String): Unit = { acc.register(this, name = Some(name)) } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 9e5325746286..1a8f8cf11c1b 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -188,6 +188,6 @@ abstract class TaskContext extends Serializable { * Register an accumulator that belongs to this task. Accumulators must call this method when * deserializing in executors. */ - private[spark] def registerAccumulator(a: NewAccumulator[_, _]): Unit + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index bc3807f5db18..c904e083911c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -122,7 +122,7 @@ private[spark] class TaskContextImpl( override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) - private[spark] override def registerAccumulator(a: NewAccumulator[_, _]): Unit = { + private[spark] override def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { taskMetrics.registerAccumulator(a) } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 82ba2d0c274b..ef333e397f64 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -118,7 +118,7 @@ case class ExceptionFailure( fullStackTrace: String, private val exceptionWrapper: Option[ThrowableSerializationWrapper], accumUpdates: Seq[AccumulableInfo] = Seq.empty, - private[spark] var accums: Seq[NewAccumulator[_, _]] = Nil) + private[spark] var accums: Seq[AccumulatorV2[_, _]] = Nil) extends TaskFailedReason { /** @@ -138,7 +138,7 @@ case class ExceptionFailure( this(e, accumUpdates, preserveCause = true) } - private[spark] def withAccums(accums: Seq[NewAccumulator[_, _]]): ExceptionFailure = { + private[spark] def withAccums(accums: Seq[AccumulatorV2[_, _]]): ExceptionFailure = { this.accums = accums this } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4d61f7e23248..4f74dc92d7df 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -353,7 +353,7 @@ private[spark] class Executor( logError(s"Exception in $taskName (TID $taskId)", t) // Collect latest accumulator values to report back to the driver - val accums: Seq[NewAccumulator[_, _]] = + val accums: Seq[AccumulatorV2[_, _]] = if (task != null) { task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart) task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) @@ -478,7 +478,7 @@ private[spark] class Executor( /** Reports heartbeat and metrics for active tasks to the driver. */ private def reportHeartBeat(): Unit = { // list of (task id, accumUpdates) to send back to the driver - val accumUpdates = new ArrayBuffer[(Long, Seq[NewAccumulator[_, _]])]() + val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() for (taskRunner <- runningTasks.values().asScala) { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 0b64917219a7..56d034fd033e 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -201,7 +201,7 @@ class TaskMetrics private[spark] () extends Serializable { output.RECORDS_WRITTEN -> outputMetrics._recordsWritten ) ++ testAccum.map(TEST_ACCUM -> _) - @transient private[spark] lazy val internalAccums: Seq[NewAccumulator[_, _]] = + @transient private[spark] lazy val internalAccums: Seq[AccumulatorV2[_, _]] = nameToAccums.values.toIndexedSeq /* ========================== * @@ -217,13 +217,13 @@ class TaskMetrics private[spark] () extends Serializable { /** * External accumulators registered with this task. */ - @transient private lazy val externalAccums = new ArrayBuffer[NewAccumulator[_, _]] + @transient private lazy val externalAccums = new ArrayBuffer[AccumulatorV2[_, _]] - private[spark] def registerAccumulator(a: NewAccumulator[_, _]): Unit = { + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { externalAccums += a } - private[spark] def accumulators(): Seq[NewAccumulator[_, _]] = internalAccums ++ externalAccums + private[spark] def accumulators(): Seq[AccumulatorV2[_, _]] = internalAccums ++ externalAccums } @@ -271,15 +271,15 @@ private[spark] object TaskMetrics extends Logging { /** * Construct a [[TaskMetrics]] object from a list of accumulator updates, called on driver only. */ - def fromAccumulators(accums: Seq[NewAccumulator[_, _]]): TaskMetrics = { + def fromAccumulators(accums: Seq[AccumulatorV2[_, _]]): TaskMetrics = { val tm = new TaskMetrics val (internalAccums, externalAccums) = accums.partition(a => a.name.isDefined && tm.nameToAccums.contains(a.name.get)) internalAccums.foreach { acc => - val tmAcc = tm.nameToAccums(acc.name.get).asInstanceOf[NewAccumulator[Any, Any]] + val tmAcc = tm.nameToAccums(acc.name.get).asInstanceOf[AccumulatorV2[Any, Any]] tmAcc.metadata = acc.metadata - tmAcc.merge(acc.asInstanceOf[NewAccumulator[Any, Any]]) + tmAcc.merge(acc.asInstanceOf[AccumulatorV2[Any, Any]]) } tm.externalAccums ++= externalAccums @@ -289,7 +289,7 @@ private[spark] object TaskMetrics extends Logging { private[spark] class BlockStatusesAccumulator - extends NewAccumulator[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { + extends AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { private[this] var _seq = ArrayBuffer.empty[(BlockId, BlockStatus)] override def isZero(): Boolean = _seq.isEmpty @@ -298,7 +298,7 @@ private[spark] class BlockStatusesAccumulator override def add(v: (BlockId, BlockStatus)): Unit = _seq += v - override def merge(other: NewAccumulator[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) + override def merge(other: AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) : Unit = other match { case o: BlockStatusesAccumulator => _seq ++= o.localValue case _ => throw new UnsupportedOperationException( diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a96d5f6fbf08..4dfd532e9362 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -209,7 +209,7 @@ class DAGScheduler( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo): Unit = { eventProcessLoop.post( CompletionEvent(task, reason, result, accumUpdates, taskInfo)) @@ -1091,14 +1091,14 @@ class DAGScheduler( event.accumUpdates.foreach { updates => val id = updates.id // Find the corresponding accumulator on the driver and update it - val acc: NewAccumulator[Any, Any] = AccumulatorContext.get(id) match { - case Some(accum) => accum.asInstanceOf[NewAccumulator[Any, Any]] + val acc: AccumulatorV2[Any, Any] = AccumulatorContext.get(id) match { + case Some(accum) => accum.asInstanceOf[AccumulatorV2[Any, Any]] case None => throw new SparkException(s"attempted to access non-existent accumulator $id") } - acc.merge(updates.asInstanceOf[NewAccumulator[Any, Any]]) + acc.merge(updates.asInstanceOf[AccumulatorV2[Any, Any]]) // To avoid UI cruft, ignore cases where value wasn't updated - if (acc.name.isDefined && !updates.isZero()) { + if (acc.name.isDefined && !updates.isZero) { stage.latestInfo.accumulables(id) = acc.toInfo(None, Some(acc.value)) event.taskInfo.accumulables += acc.toInfo(Some(updates.value), Some(acc.value)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index e57a2246d872..0a2c2dc03937 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -71,7 +71,7 @@ private[scheduler] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index e7ca6efd84ae..362f8e51ce05 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -153,7 +153,7 @@ private[spark] abstract class Task[T]( * Collect the latest values of accumulators used in this task. If the task failed, * filter out the accumulators whose values should not be included on failures. */ - def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[NewAccumulator[_, _]] = { + def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[AccumulatorV2[_, _]] = { if (context != null) { context.taskMetrics.accumulators().filter { a => !taskFailed || a.countFailedValues } } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index b472c5511b73..69ce00f30d9a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{NewAccumulator, SparkEnv} +import org.apache.spark.{AccumulatorV2, SparkEnv} import org.apache.spark.storage.BlockId import org.apache.spark.util.Utils @@ -36,7 +36,7 @@ private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] class DirectTaskResult[T]( var valueBytes: ByteBuffer, - var accumUpdates: Seq[NewAccumulator[_, _]]) + var accumUpdates: Seq[AccumulatorV2[_, _]]) extends TaskResult[T] with Externalizable { private var valueObjectDeserialized = false @@ -61,9 +61,9 @@ private[spark] class DirectTaskResult[T]( if (numUpdates == 0) { accumUpdates = null } else { - val _accumUpdates = new ArrayBuffer[NewAccumulator[_, _]] + val _accumUpdates = new ArrayBuffer[AccumulatorV2[_, _]] for (i <- 0 until numUpdates) { - _accumUpdates += in.readObject.asInstanceOf[NewAccumulator[_, _]] + _accumUpdates += in.readObject.asInstanceOf[AccumulatorV2[_, _]] } accumUpdates = _accumUpdates } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 75a0c5631197..9881a1018cf5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.NewAccumulator +import org.apache.spark.AccumulatorV2 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId @@ -67,7 +67,7 @@ private[spark] trait TaskScheduler { */ def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 8fa4aa121c12..666b6365585c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -384,7 +384,7 @@ private[spark] class TaskSchedulerImpl( */ override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b79f643a7481..b724050f5bb8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -647,7 +647,7 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - var accumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty + var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + reason.asInstanceOf[TaskFailedReason].toErrorString val failureException: Option[Throwable] = reason match { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 9c90049715dd..09eb9c1dbdc6 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -273,7 +273,7 @@ private[spark] object AccumulatorSuite { * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the * info as an accumulator update. */ - def makeInfo(a: NewAccumulator[_, _]): AccumulableInfo = a.toInfo(Some(a.localValue), None) + def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.localValue), None) /** * Run one or more Spark jobs and verify that in at least one job the peak execution memory diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 688eb6bde904..25977a466024 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -213,7 +213,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { private class SaveAccumContextCleaner(sc: SparkContext) extends ContextCleaner(sc) { private val accumsRegistered = new ArrayBuffer[Long] - override def registerAccumulatorForCleanup(a: NewAccumulator[_, _]): Unit = { + override def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { accumsRegistered += a.id super.registerAccumulatorForCleanup(a) } diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 94f6e1a3a77c..27a1e7bb35ba 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -203,7 +203,7 @@ class TaskMetricsSuite extends SparkFunSuite { acc1.add(1) acc2.add(2) val newUpdates = tm.accumulators() - .map(a => (a.id, a.asInstanceOf[NewAccumulator[Any, Any]])).toMap + .map(a => (a.id, a.asInstanceOf[AccumulatorV2[Any, Any]])).toMap assert(newUpdates.contains(acc1.id)) assert(newUpdates.contains(acc2.id)) assert(newUpdates.contains(acc3.id)) @@ -230,8 +230,8 @@ private[spark] object TaskMetricsSuite extends Assertions { * Note: this does NOT check accumulator ID equality. */ def assertUpdatesEquals( - updates1: Seq[NewAccumulator[_, _]], - updates2: Seq[NewAccumulator[_, _]]): Unit = { + updates1: Seq[AccumulatorV2[_, _]], + updates2: Seq[AccumulatorV2[_, _]]): Unit = { assert(updates1.size === updates2.size) updates1.zip(updates2).foreach { case (acc1, acc2) => // do not assert ID equals here diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9912d1f3bc5a..5a5c3a0cd193 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -112,7 +112,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def stop() = {} override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager @@ -483,7 +483,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def defaultParallelism(): Int = 2 override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def applicationAttemptId(): Option[String] = None @@ -2012,7 +2012,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou task: Task[_], reason: TaskEndReason, result: Any, - extraAccumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty, + extraAccumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty, taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { val accumUpdates = reason match { case Success => task.metrics.accumulators() diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 16027d944fdf..72ac848f12b4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.{LocalSparkContext, NewAccumulator, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{AccumulatorV2, LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId @@ -67,6 +67,6 @@ private class DummyTaskScheduler extends TaskScheduler { override def applicationAttemptId(): Option[String] = None def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 339fc4254d53..122a3ecb4954 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -37,7 +37,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } @@ -184,7 +184,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val accumUpdatesByTask: Array[Seq[NewAccumulator[_, _]]] = taskSet.tasks.map { task => + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } @@ -791,7 +791,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg private def createTaskResult( id: Int, - accumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty): DirectTaskResult[Int] = { + accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 7bf922527261..40c00ee1881f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution.metric import java.text.NumberFormat -import org.apache.spark.{NewAccumulator, SparkContext} +import org.apache.spark.{AccumulatorV2, SparkContext} import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.util.Utils -class SQLMetric(val metricType: String, initValue: Long = 0L) extends NewAccumulator[Long, Long] { +class SQLMetric(val metricType: String, initValue: Long = 0L) extends AccumulatorV2[Long, Long] { // This is a workaround for SPARK-11013. // We may use -1 as initial value of the accumulator, if the accumulator is valid, we will // update it at the end of task and the value will be at least 0. Then we can filter out the -1 @@ -33,7 +33,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends NewAccumul override def copyAndReset(): SQLMetric = new SQLMetric(metricType, initValue) - override def merge(other: NewAccumulator[Long, Long]): Unit = other match { + override def merge(other: AccumulatorV2[Long, Long]): Unit = other match { case o: SQLMetric => _value += o.localValue case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") From ccb53a20e4c3bff02a17542cad13a1fe36d7a7ea Mon Sep 17 00:00:00 2001 From: Ben McCann Date: Sun, 1 May 2016 22:43:28 -0700 Subject: [PATCH 003/313] Fix reference to external metrics documentation Author: Ben McCann Closes #12833 from benmccann/patch-1. (cherry picked from commit 214d1be4fd4a34399b6a2adb2618784de459a48d) Signed-off-by: Reynold Xin --- docs/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index 9912cde743a4..88002ebdc39c 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -341,7 +341,7 @@ keep the paths consistent in both modes. # Metrics Spark has a configurable metrics system based on the -[Coda Hale Metrics Library](http://metrics.codahale.com/). +[Dropwizard Metrics Library](http://metrics.dropwizard.io/). This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV files. The metrics system is configured via a configuration file that Spark expects to be present at `$SPARK_HOME/conf/metrics.properties`. A custom file location can be specified via the From 1145ea01b994faf458ab00301b8c4ac757d0dd0b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 2 May 2016 10:21:14 -0700 Subject: [PATCH 004/313] [SPARK-14637][SQL] object expressions cleanup ## What changes were proposed in this pull request? Simplify and clean up some object expressions: 1. simplify the logic to handle `propagateNull` 2. add `propagateNull` parameter to `Invoke` 3. simplify the unbox logic in `Invoke` 4. other minor cleanup TODO: simplify `MapObjects` ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #12399 from cloud-fan/object. (cherry picked from commit 0513c3ac93e0a25d6eedbafe6c0561e71c92880a) Signed-off-by: Michael Armbrust --- .../sql/catalyst/expressions/objects.scala | 218 ++++++++---------- 1 file changed, 100 insertions(+), 118 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala index 1e418540a262..523eed825f26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala @@ -64,33 +64,29 @@ case class StaticInvoke( val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") - if (propagateNull) { - val objNullCheck = if (ctx.defaultValue(dataType) == "null") { - s"${ev.isNull} = ${ev.value} == null;" - } else { - "" - } - - val argsNonNull = s"!(${argGen.map(_.isNull).mkString(" || ")})" - ev.copy(code = s""" - ${argGen.map(_.code).mkString("\n")} - - boolean ${ev.isNull} = !$argsNonNull; - $javaType ${ev.value} = ${ctx.defaultValue(dataType)}; + val callFunc = s"$objectName.$functionName($argString)" - if ($argsNonNull) { - ${ev.value} = $objectName.$functionName($argString); - $objNullCheck - } - """) + val setIsNull = if (propagateNull && arguments.nonEmpty) { + s"boolean ${ev.isNull} = ${argGen.map(_.isNull).mkString(" || ")};" } else { - ev.copy(code = s""" - ${argGen.map(_.code).mkString("\n")} + s"boolean ${ev.isNull} = false;" + } - $javaType ${ev.value} = $objectName.$functionName($argString); - final boolean ${ev.isNull} = ${ev.value} == null; - """) + // If the function can return null, we do an extra check to make sure our null bit is still set + // correctly. + val postNullCheck = if (ctx.defaultValue(dataType) == "null") { + s"${ev.isNull} = ${ev.value} == null;" + } else { + "" } + + val code = s""" + ${argGen.map(_.code).mkString("\n")} + $setIsNull + final $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : $callFunc; + $postNullCheck + """ + ev.copy(code = code) } } @@ -111,7 +107,8 @@ case class Invoke( targetObject: Expression, functionName: String, dataType: DataType, - arguments: Seq[Expression] = Nil) extends Expression with NonSQLExpression { + arguments: Seq[Expression] = Nil, + propagateNull: Boolean = true) extends Expression with NonSQLExpression { override def nullable: Boolean = true override def children: Seq[Expression] = targetObject +: arguments @@ -130,60 +127,53 @@ case class Invoke( case _ => None } - lazy val unboxer = (dataType, method.map(_.getReturnType.getName).getOrElse("")) match { - case (IntegerType, "java.lang.Object") => (s: String) => - s"((java.lang.Integer)$s).intValue()" - case (LongType, "java.lang.Object") => (s: String) => - s"((java.lang.Long)$s).longValue()" - case (FloatType, "java.lang.Object") => (s: String) => - s"((java.lang.Float)$s).floatValue()" - case (ShortType, "java.lang.Object") => (s: String) => - s"((java.lang.Short)$s).shortValue()" - case (ByteType, "java.lang.Object") => (s: String) => - s"((java.lang.Byte)$s).byteValue()" - case (DoubleType, "java.lang.Object") => (s: String) => - s"((java.lang.Double)$s).doubleValue()" - case (BooleanType, "java.lang.Object") => (s: String) => - s"((java.lang.Boolean)$s).booleanValue()" - case _ => identity[String] _ - } - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) val obj = targetObject.genCode(ctx) val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") - // If the function can return null, we do an extra check to make sure our null bit is still set - // correctly. - val objNullCheck = if (ctx.defaultValue(dataType) == "null") { - s"boolean ${ev.isNull} = ${ev.value} == null;" + val callFunc = if (method.isDefined && method.get.getReturnType.isPrimitive) { + s"${obj.value}.$functionName($argString)" } else { - ev.isNull = obj.isNull - "" + s"(${ctx.boxedType(javaType)}) ${obj.value}.$functionName($argString)" } - val value = unboxer(s"${obj.value}.$functionName($argString)") + val setIsNull = if (propagateNull && arguments.nonEmpty) { + s"boolean ${ev.isNull} = ${obj.isNull} || ${argGen.map(_.isNull).mkString(" || ")};" + } else { + s"boolean ${ev.isNull} = ${obj.isNull};" + } val evaluate = if (method.forall(_.getExceptionTypes.isEmpty)) { - s"$javaType ${ev.value} = ${obj.isNull} ? ${ctx.defaultValue(dataType)} : ($javaType) $value;" + s"final $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : $callFunc;" } else { s""" $javaType ${ev.value} = ${ctx.defaultValue(javaType)}; try { - ${ev.value} = ${obj.isNull} ? ${ctx.defaultValue(javaType)} : ($javaType) $value; + ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(javaType)} : $callFunc; } catch (Exception e) { org.apache.spark.unsafe.Platform.throwException(e); } """ } - ev.copy(code = s""" + // If the function can return null, we do an extra check to make sure our null bit is still set + // correctly. + val postNullCheck = if (ctx.defaultValue(dataType) == "null") { + s"${ev.isNull} = ${ev.value} == null;" + } else { + "" + } + + val code = s""" ${obj.code} ${argGen.map(_.code).mkString("\n")} + $setIsNull $evaluate - $objNullCheck - """) + $postNullCheck + """ + ev.copy(code = code) } override def toString: String = s"$targetObject.$functionName" @@ -246,11 +236,13 @@ case class NewInstance( val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) - val setup = - s""" - ${argGen.map(_.code).mkString("\n")} - ${outer.map(_.code).getOrElse("")} - """.stripMargin + var isNull = ev.isNull + val setIsNull = if (propagateNull && arguments.nonEmpty) { + s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" + } else { + isNull = "false" + "" + } val constructorCall = outer.map { gen => s"""${gen.value}.new ${cls.getSimpleName}($argString)""" @@ -258,27 +250,13 @@ case class NewInstance( s"new $className($argString)" } - if (propagateNull && argGen.nonEmpty) { - val argsNonNull = s"!(${argGen.map(_.isNull).mkString(" || ")})" - - ev.copy(code = s""" - $setup - - boolean ${ev.isNull} = true; - $javaType ${ev.value} = ${ctx.defaultValue(dataType)}; - if ($argsNonNull) { - ${ev.value} = $constructorCall; - ${ev.isNull} = false; - } - """) - } else { - ev.copy(code = s""" - $setup - - final $javaType ${ev.value} = $constructorCall; - final boolean ${ev.isNull} = false; - """) - } + val code = s""" + ${argGen.map(_.code).mkString("\n")} + ${outer.map(_.code).getOrElse("")} + $setIsNull + final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; + """ + ev.copy(code = code, isNull = isNull) } override def toString: String = s"newInstance($cls)" @@ -306,13 +284,14 @@ case class UnwrapOption( val javaType = ctx.javaType(dataType) val inputObject = child.genCode(ctx) - ev.copy(code = s""" + val code = s""" ${inputObject.code} - boolean ${ev.isNull} = ${inputObject.value} == null || ${inputObject.value}.isEmpty(); + final boolean ${ev.isNull} = ${inputObject.isNull} || ${inputObject.value}.isEmpty(); $javaType ${ev.value} = - ${ev.isNull} ? ${ctx.defaultValue(dataType)} : ($javaType)${inputObject.value}.get(); - """) + ${ev.isNull} ? ${ctx.defaultValue(javaType)} : ($javaType) ${inputObject.value}.get(); + """ + ev.copy(code = code) } } @@ -338,14 +317,14 @@ case class WrapOption(child: Expression, optType: DataType) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val inputObject = child.genCode(ctx) - ev.copy(code = s""" + val code = s""" ${inputObject.code} - boolean ${ev.isNull} = false; scala.Option ${ev.value} = ${inputObject.isNull} ? scala.Option$$.MODULE$$.apply(null) : new scala.Some(${inputObject.value}); - """) + """ + ev.copy(code = code, isNull = "false") } } @@ -474,7 +453,7 @@ case class MapObjects private( s"${loopVar.isNull} = ${genInputData.isNull} || ${loopVar.value} == null;" } - ev.copy(code = s""" + val code = s""" ${genInputData.code} boolean ${ev.isNull} = ${genInputData.value} == null; @@ -504,7 +483,8 @@ case class MapObjects private( ${ev.isNull} = false; ${ev.value} = new ${classOf[GenericArrayData].getName}($convertedArray); } - """) + """ + ev.copy(code = code) } } @@ -539,14 +519,16 @@ case class CreateExternalRow(children: Seq[Expression], schema: StructType) } """ } + val childrenCode = ctx.splitExpressions(ctx.INPUT_ROW, childrenCodes) val schemaField = ctx.addReferenceObj("schema", schema) - ev.copy(code = s""" - boolean ${ev.isNull} = false; + + val code = s""" $values = new Object[${children.size}]; $childrenCode final ${classOf[Row].getName} ${ev.value} = new $rowClass($values, this.$schemaField); - """) + """ + ev.copy(code = code, isNull = "false") } } @@ -579,14 +561,14 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) // Code to serialize. val input = child.genCode(ctx) - ev.copy(code = s""" + val javaType = ctx.javaType(dataType) + val serialize = s"$serializer.serialize(${input.value}, null).array()" + + val code = s""" ${input.code} - final boolean ${ev.isNull} = ${input.isNull}; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - ${ev.value} = $serializer.serialize(${input.value}, null).array(); - } - """) + final $javaType ${ev.value} = ${input.isNull} ? ${ctx.defaultValue(javaType)} : $serialize; + """ + ev.copy(code = code, isNull = input.isNull) } override def dataType: DataType = BinaryType @@ -617,17 +599,17 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B serializer, s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") - // Code to serialize. + // Code to deserialize. val input = child.genCode(ctx) - ev.copy(code = s""" + val javaType = ctx.javaType(dataType) + val deserialize = + s"($javaType) $serializer.deserialize(java.nio.ByteBuffer.wrap(${input.value}), null)" + + val code = s""" ${input.code} - final boolean ${ev.isNull} = ${input.isNull}; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - ${ev.value} = (${ctx.javaType(dataType)}) - $serializer.deserialize(java.nio.ByteBuffer.wrap(${input.value}), null); - } - """) + final $javaType ${ev.value} = ${input.isNull} ? ${ctx.defaultValue(javaType)} : $deserialize; + """ + ev.copy(code = code, isNull = input.isNull) } override def dataType: DataType = ObjectType(tag.runtimeClass) @@ -658,15 +640,13 @@ case class InitializeJavaBean(beanInstance: Expression, setters: Map[String, Exp """ } - ev.isNull = instanceGen.isNull - ev.value = instanceGen.value - - ev.copy(code = s""" + val code = s""" ${instanceGen.code} if (!${instanceGen.isNull}) { ${initialize.mkString("\n")} } - """) + """ + ev.copy(code = code, isNull = instanceGen.isNull, value = instanceGen.value) } } @@ -696,13 +676,15 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) "If the schema is inferred from a Scala tuple/case class, or a Java bean, " + "please try to use scala.Option[_] or other nullable types " + "(e.g. java.lang.Integer instead of int/scala.Int)." - val idx = ctx.references.length - ctx.references += errMsg - ExprCode(code = s""" + val errMsgField = ctx.addReferenceObj("errMsg", errMsg) + + val code = s""" ${childGen.code} if (${childGen.isNull}) { - throw new RuntimeException((String) references[$idx]); - }""", isNull = "false", value = childGen.value) + throw new RuntimeException(this.$errMsgField); + } + """ + ev.copy(code = code, isNull = "false", value = childGen.value) } } From eb7336a754574879fc28c3c10cb68651329a346d Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Mon, 2 May 2016 11:03:37 -0700 Subject: [PATCH 005/313] =?UTF-8?q?[SPARK-14845][SPARK=5FSUBMIT][YARN]=20s?= =?UTF-8?q?park.files=20in=20properties=20file=20is=20n=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? initialize SparkSubmitArgument#files first from spark-submit arguments then from properties file, so that sys property spark.yarn.dist.files will be set correctly. ``` OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.files"), ``` ## How was this patch tested? manul test. file defined in properties file is also distributed to driver in yarn-cluster mode. Author: Jeff Zhang Closes #12656 from zjffdu/SPARK-14845. (cherry picked from commit 0a3026990bd0cbad53f0001da793349201104958) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 78da1b70c54a..206c130c7637 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -173,6 +173,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orNull name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull + files = Option(files).orElse(sparkProperties.get("spark.files")).orNull ivyRepoPath = sparkProperties.get("spark.jars.ivy").orNull packages = Option(packages).orElse(sparkProperties.get("spark.jars.packages")).orNull packagesExclusions = Option(packagesExclusions) From 08ae32e6104e998b3c9a4822e563e63aeae55578 Mon Sep 17 00:00:00 2001 From: Andrew Ray Date: Mon, 2 May 2016 11:12:55 -0700 Subject: [PATCH 006/313] [SPARK-13749][SQL] Faster pivot implementation for many distinct values with two phase aggregation ## What changes were proposed in this pull request? The existing implementation of pivot translates into a single aggregation with one aggregate per distinct pivot value. When the number of distinct pivot values is large (say 1000+) this can get extremely slow since each input value gets evaluated on every aggregate even though it only affects the value of one of them. I'm proposing an alternate strategy for when there are 10+ (somewhat arbitrary threshold) distinct pivot values. We do two phases of aggregation. In the first we group by the grouping columns plus the pivot column and perform the specified aggregations (one or sometimes more). In the second aggregation we group by the grouping columns and use the new (non public) PivotFirst aggregate that rearranges the outputs of the first aggregation into an array indexed by the pivot value. Finally we do a project to extract the array entries into the appropriate output column. ## How was this patch tested? Additional unit tests in DataFramePivotSuite and manual larger scale testing. Author: Andrew Ray Closes #11583 from aray/fast-pivot. (cherry picked from commit 99274418684ebae5b98d15b4686b95c1ac029e94) Signed-off-by: Yin Huai --- .../sql/catalyst/analysis/Analyzer.scala | 85 ++++++---- .../expressions/aggregate/PivotFirst.scala | 152 ++++++++++++++++++ .../spark/sql/DataFramePivotSuite.scala | 92 ++++++++++- 3 files changed, 296 insertions(+), 33 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e98036a970d4..2f8ab3f43586 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -363,43 +363,68 @@ class Analyzer( object ResolvePivot extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p: Pivot if !p.childrenResolved | !p.aggregates.forall(_.resolved) => p + case p: Pivot if !p.childrenResolved | !p.aggregates.forall(_.resolved) + | !p.groupByExprs.forall(_.resolved) | !p.pivotColumn.resolved => p case Pivot(groupByExprs, pivotColumn, pivotValues, aggregates, child) => val singleAgg = aggregates.size == 1 - val pivotAggregates: Seq[NamedExpression] = pivotValues.flatMap { value => - def ifExpr(expr: Expression) = { - If(EqualTo(pivotColumn, value), expr, Literal(null)) + def outputName(value: Literal, aggregate: Expression): String = { + if (singleAgg) value.toString else value + "_" + aggregate.sql + } + if (aggregates.forall(a => PivotFirst.supportsDataType(a.dataType))) { + // Since evaluating |pivotValues| if statements for each input row can get slow this is an + // alternate plan that instead uses two steps of aggregation. + val namedAggExps: Seq[NamedExpression] = aggregates.map(a => Alias(a, a.sql)()) + val namedPivotCol = pivotColumn match { + case n: NamedExpression => n + case _ => Alias(pivotColumn, "__pivot_col")() + } + val bigGroup = groupByExprs :+ namedPivotCol + val firstAgg = Aggregate(bigGroup, bigGroup ++ namedAggExps, child) + val castPivotValues = pivotValues.map(Cast(_, pivotColumn.dataType).eval(EmptyRow)) + val pivotAggs = namedAggExps.map { a => + Alias(PivotFirst(namedPivotCol.toAttribute, a.toAttribute, castPivotValues) + .toAggregateExpression() + , "__pivot_" + a.sql)() + } + val secondAgg = Aggregate(groupByExprs, groupByExprs ++ pivotAggs, firstAgg) + val pivotAggAttribute = pivotAggs.map(_.toAttribute) + val pivotOutputs = pivotValues.zipWithIndex.flatMap { case (value, i) => + aggregates.zip(pivotAggAttribute).map { case (aggregate, pivotAtt) => + Alias(ExtractValue(pivotAtt, Literal(i), resolver), outputName(value, aggregate))() + } } - aggregates.map { aggregate => - val filteredAggregate = aggregate.transformDown { - // Assumption is the aggregate function ignores nulls. This is true for all current - // AggregateFunction's with the exception of First and Last in their default mode - // (which we handle) and possibly some Hive UDAF's. - case First(expr, _) => - First(ifExpr(expr), Literal(true)) - case Last(expr, _) => - Last(ifExpr(expr), Literal(true)) - case a: AggregateFunction => - a.withNewChildren(a.children.map(ifExpr)) - }.transform { - // We are duplicating aggregates that are now computing a different value for each - // pivot value. - // TODO: Don't construct the physical container until after analysis. - case ae: AggregateExpression => ae.copy(resultId = NamedExpression.newExprId) + Project(groupByExprs ++ pivotOutputs, secondAgg) + } else { + val pivotAggregates: Seq[NamedExpression] = pivotValues.flatMap { value => + def ifExpr(expr: Expression) = { + If(EqualTo(pivotColumn, value), expr, Literal(null)) } - if (filteredAggregate.fastEquals(aggregate)) { - throw new AnalysisException( - s"Aggregate expression required for pivot, found '$aggregate'") + aggregates.map { aggregate => + val filteredAggregate = aggregate.transformDown { + // Assumption is the aggregate function ignores nulls. This is true for all current + // AggregateFunction's with the exception of First and Last in their default mode + // (which we handle) and possibly some Hive UDAF's. + case First(expr, _) => + First(ifExpr(expr), Literal(true)) + case Last(expr, _) => + Last(ifExpr(expr), Literal(true)) + case a: AggregateFunction => + a.withNewChildren(a.children.map(ifExpr)) + }.transform { + // We are duplicating aggregates that are now computing a different value for each + // pivot value. + // TODO: Don't construct the physical container until after analysis. + case ae: AggregateExpression => ae.copy(resultId = NamedExpression.newExprId) + } + if (filteredAggregate.fastEquals(aggregate)) { + throw new AnalysisException( + s"Aggregate expression required for pivot, found '$aggregate'") + } + Alias(filteredAggregate, outputName(value, aggregate))() } - val name = if (singleAgg) value.toString else value + "_" + aggregate.sql - Alias(filteredAggregate, name)() } + Aggregate(groupByExprs, groupByExprs ++ pivotAggregates, child) } - val newGroupByExprs = groupByExprs.map { - case UnresolvedAlias(e, _) => e - case e => e - } - Aggregate(newGroupByExprs, groupByExprs ++ pivotAggregates, child) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala new file mode 100644 index 000000000000..9154e96e34e9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import scala.collection.immutable.HashMap + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.types._ + +object PivotFirst { + + def supportsDataType(dataType: DataType): Boolean = updateFunction.isDefinedAt(dataType) + + // Currently UnsafeRow does not support the generic update method (throws + // UnsupportedOperationException), so we need to explicitly support each DataType. + private val updateFunction: PartialFunction[DataType, (MutableRow, Int, Any) => Unit] = { + case DoubleType => + (row, offset, value) => row.setDouble(offset, value.asInstanceOf[Double]) + case IntegerType => + (row, offset, value) => row.setInt(offset, value.asInstanceOf[Int]) + case LongType => + (row, offset, value) => row.setLong(offset, value.asInstanceOf[Long]) + case FloatType => + (row, offset, value) => row.setFloat(offset, value.asInstanceOf[Float]) + case BooleanType => + (row, offset, value) => row.setBoolean(offset, value.asInstanceOf[Boolean]) + case ShortType => + (row, offset, value) => row.setShort(offset, value.asInstanceOf[Short]) + case ByteType => + (row, offset, value) => row.setByte(offset, value.asInstanceOf[Byte]) + case d: DecimalType => + (row, offset, value) => row.setDecimal(offset, value.asInstanceOf[Decimal], d.precision) + } +} + +/** + * PivotFirst is a aggregate function used in the second phase of a two phase pivot to do the + * required rearrangement of values into pivoted form. + * + * For example on an input of + * A | B + * --+-- + * x | 1 + * y | 2 + * z | 3 + * + * with pivotColumn=A, valueColumn=B, and pivotColumnValues=[z,y] the output is [3,2]. + * + * @param pivotColumn column that determines which output position to put valueColumn in. + * @param valueColumn the column that is being rearranged. + * @param pivotColumnValues the list of pivotColumn values in the order of desired output. Values + * not listed here will be ignored. + */ +case class PivotFirst( + pivotColumn: Expression, + valueColumn: Expression, + pivotColumnValues: Seq[Any], + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends ImperativeAggregate { + + override val children: Seq[Expression] = pivotColumn :: valueColumn :: Nil + + override lazy val inputTypes: Seq[AbstractDataType] = children.map(_.dataType) + + override val nullable: Boolean = false + + val valueDataType = valueColumn.dataType + + override val dataType: DataType = ArrayType(valueDataType) + + val pivotIndex = HashMap(pivotColumnValues.zipWithIndex: _*) + + val indexSize = pivotIndex.size + + private val updateRow: (MutableRow, Int, Any) => Unit = PivotFirst.updateFunction(valueDataType) + + override def update(mutableAggBuffer: MutableRow, inputRow: InternalRow): Unit = { + val pivotColValue = pivotColumn.eval(inputRow) + if (pivotColValue != null) { + // We ignore rows whose pivot column value is not in the list of pivot column values. + val index = pivotIndex.getOrElse(pivotColValue, -1) + if (index >= 0) { + val value = valueColumn.eval(inputRow) + if (value != null) { + updateRow(mutableAggBuffer, mutableAggBufferOffset + index, value) + } + } + } + } + + override def merge(mutableAggBuffer: MutableRow, inputAggBuffer: InternalRow): Unit = { + for (i <- 0 until indexSize) { + if (!inputAggBuffer.isNullAt(inputAggBufferOffset + i)) { + val value = inputAggBuffer.get(inputAggBufferOffset + i, valueDataType) + updateRow(mutableAggBuffer, mutableAggBufferOffset + i, value) + } + } + } + + override def initialize(mutableAggBuffer: MutableRow): Unit = valueDataType match { + case d: DecimalType => + // Per doc of setDecimal we need to do this instead of setNullAt for DecimalType. + for (i <- 0 until indexSize) { + mutableAggBuffer.setDecimal(mutableAggBufferOffset + i, null, d.precision) + } + case _ => + for (i <- 0 until indexSize) { + mutableAggBuffer.setNullAt(mutableAggBufferOffset + i) + } + } + + override def eval(input: InternalRow): Any = { + val result = new Array[Any](indexSize) + for (i <- 0 until indexSize) { + result(i) = input.get(mutableAggBufferOffset + i, valueDataType) + } + new GenericArrayData(result) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + + override lazy val aggBufferAttributes: Seq[AttributeReference] = + pivotIndex.toList.sortBy(_._2).map(kv => AttributeReference(kv._1.toString, valueDataType)()) + + override lazy val aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + + override lazy val inputAggBufferAttributes: Seq[AttributeReference] = + aggBufferAttributes.map(_.newInstance()) +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index 368aa5cd141f..b17284aa94d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.expressions.aggregate.PivotFirst import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ import testImplicits._ - test("pivot courses with literals") { + test("pivot courses") { checkAnswer( courseSales.groupBy("year").pivot("course", Seq("dotNET", "Java")) .agg(sum($"earnings")), @@ -32,14 +34,14 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ ) } - test("pivot year with literals") { + test("pivot year") { checkAnswer( courseSales.groupBy("course").pivot("year", Seq(2012, 2013)).agg(sum($"earnings")), Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil ) } - test("pivot courses with literals and multiple aggregations") { + test("pivot courses with multiple aggregations") { checkAnswer( courseSales.groupBy($"year") .pivot("course", Seq("dotNET", "Java")) @@ -94,4 +96,88 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ Row(2012, 15000.0, 20000.0) :: Row(2013, 48000.0, 30000.0) :: Nil ) } + + // Tests for optimized pivot (with PivotFirst) below + + test("optimized pivot planned") { + val df = courseSales.groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + val queryExecution = sqlContext.executePlan(df.queryExecution.logical) + assert(queryExecution.simpleString.contains("pivotfirst")) + } + + + test("optimized pivot courses with literals") { + checkAnswer( + courseSales.groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + .select("year", "dotNET", "Java"), + Row(2012, 15000.0, 20000.0) :: Row(2013, 48000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot year with literals") { + checkAnswer( + courseSales.groupBy($"course") + // pivot with extra columns to trigger optimization + .pivot("year", Seq(2012, 2013) ++ (1 to 10)) + .agg(sum($"earnings")) + .select("course", "2012", "2013"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot year with string values (cast)") { + checkAnswer( + courseSales.groupBy("course") + // pivot with extra columns to trigger optimization + .pivot("year", Seq("2012", "2013") ++ (1 to 10).map(_.toString)) + .sum("earnings") + .select("course", "2012", "2013"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot DecimalType") { + val df = courseSales.select($"course", $"year", $"earnings".cast(DecimalType(10, 2))) + .groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + .select("year", "dotNET", "Java") + + assertResult(IntegerType)(df.schema("year").dataType) + assertResult(DecimalType(20, 2))(df.schema("Java").dataType) + assertResult(DecimalType(20, 2))(df.schema("dotNET").dataType) + + checkAnswer(df, Row(2012, BigDecimal(1500000, 2), BigDecimal(2000000, 2)) :: + Row(2013, BigDecimal(4800000, 2), BigDecimal(3000000, 2)) :: Nil) + } + + test("PivotFirst supported datatypes") { + val supportedDataTypes: Seq[DataType] = DoubleType :: IntegerType :: LongType :: FloatType :: + BooleanType :: ShortType :: ByteType :: Nil + for (datatype <- supportedDataTypes) { + assertResult(true)(PivotFirst.supportsDataType(datatype)) + } + assertResult(true)(PivotFirst.supportsDataType(DecimalType(10, 1))) + assertResult(false)(PivotFirst.supportsDataType(null)) + assertResult(false)(PivotFirst.supportsDataType(ArrayType(IntegerType))) + } + + test("optimized pivot with multiple aggregations") { + checkAnswer( + courseSales.groupBy($"year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings"), avg($"earnings")), + Row(Seq(2012, 15000.0, 7500.0, 20000.0, 20000.0) ++ Seq.fill(20)(null): _*) :: + Row(Seq(2013, 48000.0, 48000.0, 30000.0, 30000.0) ++ Seq.fill(20)(null): _*) :: Nil + ) + } + } From 1c2082b643dc01fdeb2405c97dcf5a9551cc782d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 2 May 2016 11:28:21 -0700 Subject: [PATCH 007/313] [SPARK-14579][SQL] Fix the race condition in StreamExecution.processAllAvailable again ## What changes were proposed in this pull request? #12339 didn't fix the race condition. MemorySinkSuite is still flaky: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.2/814/testReport/junit/org.apache.spark.sql.streaming/MemorySinkSuite/registering_as_a_table/ Here is an execution order to reproduce it. | Time |Thread 1 | MicroBatchThread | |:-------------:|:-------------:|:-----:| | 1 | | `MemorySink.getOffset` | | 2 | | availableOffsets ++= newData (availableOffsets is not changed here) | | 3 | addData(newData) | | | 4 | Set `noNewData` to `false` in processAllAvailable | | | 5 | | `dataAvailable` returns `false` | | 6 | | noNewData = true | | 7 | `noNewData` is true so just return | | | 8 | assert results and fail | | | 9 | | `dataAvailable` returns true so process the new batch | This PR expands the scope of `awaitBatchLock.synchronized` to eliminate the above race. ## How was this patch tested? test("stress test"). It always failed before this patch. And it will pass after applying this patch. Ignore this test in the PR as it takes several minutes to finish. Author: Shixiong Zhu Closes #12582 from zsxwing/SPARK-14579-2. (cherry picked from commit a35a67a83dbb450d26ce0d142ab106e952670842) Signed-off-by: Michael Armbrust --- .../sql/execution/streaming/StreamExecution.scala | 10 +++++----- .../apache/spark/sql/streaming/MemorySinkSuite.scala | 9 +++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index fc18e5f065a0..ce68c0968fb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -242,12 +242,12 @@ class StreamExecution( // method. See SPARK-14131. // // Check to see what new data is available. - val newData = microBatchThread.runUninterruptibly { - uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - } - availableOffsets ++= newData - val hasNewData = awaitBatchLock.synchronized { + val newData = microBatchThread.runUninterruptibly { + uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) + } + availableOffsets ++= newData + if (dataAvailable) { true } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala index 1f2834054519..74ca3977d663 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala @@ -26,6 +26,15 @@ class MemorySinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ test("registering as a table") { + testRegisterAsTable() + } + + ignore("stress test") { + // Ignore the stress test as it takes several minutes to run + (0 until 1000).foreach(_ => testRegisterAsTable()) + } + + private def testRegisterAsTable(): Unit = { val input = MemoryStream[Int] val query = input.toDF().write .format("memory") From 972fd22e3933e58e637781a1da0b6a18afaced17 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 2 May 2016 12:40:21 -0700 Subject: [PATCH 008/313] [SPARK-14830][SQL] Add RemoveRepetitionFromGroupExpressions optimizer. ## What changes were proposed in this pull request? This PR aims to optimize GroupExpressions by removing repeating expressions. `RemoveRepetitionFromGroupExpressions` is added. **Before** ```scala scala> sql("select a+1 from values 1,2 T(a) group by a+1, 1+a, A+1, 1+A").explain() == Physical Plan == WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1)#6,(1 + a#0)#7,(A#0 + 1)#8,(1 + A#0)#9], functions=[], output=[(a + 1)#5]) : +- INPUT +- Exchange hashpartitioning((a#0 + 1)#6, (1 + a#0)#7, (A#0 + 1)#8, (1 + A#0)#9, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1) AS (a#0 + 1)#6,(1 + a#0) AS (1 + a#0)#7,(A#0 + 1) AS (A#0 + 1)#8,(1 + A#0) AS (1 + A#0)#9], functions=[], output=[(a#0 + 1)#6,(1 + a#0)#7,(A#0 + 1)#8,(1 + A#0)#9]) : +- INPUT +- LocalTableScan [a#0], [[1],[2]] ``` **After** ```scala scala> sql("select a+1 from values 1,2 T(a) group by a+1, 1+a, A+1, 1+A").explain() == Physical Plan == WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1)#6], functions=[], output=[(a + 1)#5]) : +- INPUT +- Exchange hashpartitioning((a#0 + 1)#6, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1) AS (a#0 + 1)#6], functions=[], output=[(a#0 + 1)#6]) : +- INPUT +- LocalTableScan [a#0], [[1],[2]] ``` ## How was this patch tested? Pass the Jenkins tests (with a new testcase) Author: Dongjoon Hyun Closes #12590 from dongjoon-hyun/SPARK-14830. (cherry picked from commit 6e6320122ea84247c67e2d0fb0e6af54e2c5bb31) Signed-off-by: Michael Armbrust --- .../sql/catalyst/optimizer/Optimizer.scala | 15 +++++++++++++- .../optimizer/AggregateOptimizeSuite.scala | 20 ++++++++++++++++++- 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 0b70edec8e37..a147fff27413 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -68,7 +68,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) ReplaceExceptWithAntiJoin, ReplaceDistinctWithAggregate) :: Batch("Aggregate", fixedPoint, - RemoveLiteralFromGroupExpressions) :: + RemoveLiteralFromGroupExpressions, + RemoveRepetitionFromGroupExpressions) :: Batch("Operator Optimizations", fixedPoint, // Operator push down SetOperationPushDown, @@ -1439,6 +1440,18 @@ object RemoveLiteralFromGroupExpressions extends Rule[LogicalPlan] { } } +/** + * Removes repetition from group expressions in [[Aggregate]], as they have no effect to the result + * but only makes the grouping key bigger. + */ +object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a @ Aggregate(grouping, _, _) => + val newGrouping = ExpressionSet(grouping).toSeq + a.copy(groupingExpressions = newGrouping) + } +} + /** * Computes the current date and time to make sure we return the same result in a single query. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala index e458eb8a1d36..c94dcb33546f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.sql.catalyst.SimpleCatalystConf +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.Literal @@ -25,10 +28,14 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor class AggregateOptimizeSuite extends PlanTest { + val conf = new SimpleCatalystConf(caseSensitiveAnalysis = false) + val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) + val analyzer = new Analyzer(catalog, conf) object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Aggregate", FixedPoint(100), - RemoveLiteralFromGroupExpressions) :: Nil + RemoveLiteralFromGroupExpressions, + RemoveRepetitionFromGroupExpressions) :: Nil } test("remove literals in grouping expression") { @@ -42,4 +49,15 @@ class AggregateOptimizeSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("remove repetition in grouping expression") { + val input = LocalRelation('a.int, 'b.int, 'c.int) + + val query = input.groupBy('a + 1, 'b + 2, Literal(1) + 'A, Literal(2) + 'B)(sum('c)) + val optimized = Optimize.execute(analyzer.execute(query)) + + val correctAnswer = analyzer.execute(input.groupBy('a + 1, 'b + 2)(sum('c))) + + comparePlans(optimized, correctAnswer) + } } From 4a7e75a203b0a8ecabcb241208aaee5201f6c6e6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 2 May 2016 12:58:59 -0700 Subject: [PATCH 009/313] [SPARK-14781] [SQL] support nested predicate subquery ## What changes were proposed in this pull request? In order to support nested predicate subquery, this PR introduce an internal join type ExistenceJoin, which will emit all the rows from left, plus an additional column, which presents there are any rows matched from right or not (it's not null-aware right now). This additional column could be used to replace the subquery in Filter. In theory, all the predicate subquery could use this join type, but it's slower than LeftSemi and LeftAnti, so it's only used for nested subquery (subquery inside OR). For example, the following SQL: ```sql SELECT a FROM t WHERE EXISTS (select 0) OR EXISTS (select 1) ``` This PR also fix a bug in predicate subquery push down through join (they should not). Nested null-aware subquery is still not supported. For example, `a > 3 OR b NOT IN (select bb from t)` After this, we could run TPCDS query Q10, Q35, Q45 ## How was this patch tested? Added unit tests. Author: Davies Liu Closes #12820 from davies/or_exists. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 5 +- .../sql/catalyst/expressions/subquery.scala | 15 ++- .../sql/catalyst/optimizer/Optimizer.scala | 41 ++++++-- .../spark/sql/catalyst/plans/joinTypes.scala | 10 ++ .../plans/logical/basicLogicalOperators.scala | 4 + .../analysis/AnalysisErrorSuite.scala | 11 ++- .../spark/sql/execution/SparkStrategies.scala | 1 + .../joins/BroadcastHashJoinExec.scala | 66 ++++++++++++- .../joins/BroadcastNestedLoopJoinExec.scala | 94 ++++++++++++++----- .../spark/sql/execution/joins/HashJoin.scala | 31 +++++- .../joins/ShuffledHashJoinExec.scala | 13 +-- .../execution/joins/SortMergeJoinExec.scala | 40 ++++++++ .../org/apache/spark/sql/SubquerySuite.scala | 25 +++++ .../execution/joins/ExistenceJoinSuite.scala | 50 +++++++++- 14 files changed, 345 insertions(+), 61 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 61a7d9ea24f4..6e3a14dfb920 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -115,8 +115,9 @@ trait CheckAnalysis extends PredicateHelper { case f @ Filter(condition, child) => splitConjunctivePredicates(condition).foreach { case _: PredicateSubquery | Not(_: PredicateSubquery) => - case e if PredicateSubquery.hasPredicateSubquery(e) => - failAnalysis(s"Predicate sub-queries cannot be used in nested conditions: $e") + case e if PredicateSubquery.hasNullAwarePredicateWithinNot(e) => + failAnalysis(s"Null-aware predicate sub-queries cannot be used in nested" + + s" conditions: $e") case e => } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index cd6d3a00b7cf..eed062f8bc18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -92,7 +92,7 @@ case class PredicateSubquery( extends SubqueryExpression with Predicate with Unevaluable { override lazy val resolved = childrenResolved && query.resolved override lazy val references: AttributeSet = super.references -- query.outputSet - override def nullable: Boolean = false + override def nullable: Boolean = nullAware override def plan: LogicalPlan = SubqueryAlias(toString, query) override def withNewPlan(plan: LogicalPlan): PredicateSubquery = copy(query = plan) override def toString: String = s"predicate-subquery#${exprId.id} $conditionString" @@ -105,6 +105,19 @@ object PredicateSubquery { case _ => false }.isDefined } + + /** + * Returns whether there are any null-aware predicate subqueries inside Not. If not, we could + * turn the null-aware predicate into not-null-aware predicate. + */ + def hasNullAwarePredicateWithinNot(e: Expression): Boolean = { + e.find{ x => + x.isInstanceOf[Not] && e.find { + case p: PredicateSubquery => p.nullAware + case _ => false + }.isDefined + }.isDefined + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index a147fff27413..e1c969f50f2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -100,8 +100,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) EliminateSorts, SimplifyCasts, SimplifyCaseConversionExpressions, - EliminateSerialization, - RewritePredicateSubquery) :: + EliminateSerialization) :: Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: Batch("Typed Filter Optimization", fixedPoint, @@ -109,7 +108,10 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("LocalRelation", fixedPoint, ConvertToLocalRelation) :: Batch("OptimizeCodegen", Once, - OptimizeCodegen(conf)) :: Nil + OptimizeCodegen(conf)) :: + Batch("RewriteSubquery", Once, + RewritePredicateSubquery, + CollapseProject) :: Nil } /** @@ -1078,7 +1080,14 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { def createOrderedJoin(input: Seq[LogicalPlan], conditions: Seq[Expression]): LogicalPlan = { assert(input.size >= 2) if (input.size == 2) { - Join(input(0), input(1), Inner, conditions.reduceLeftOption(And)) + val (joinConditions, others) = conditions.partition( + e => !PredicateSubquery.hasPredicateSubquery(e)) + val join = Join(input(0), input(1), Inner, joinConditions.reduceLeftOption(And)) + if (others.nonEmpty) { + Filter(others.reduceLeft(And), join) + } else { + join + } } else { val left :: rest = input.toList // find out the first join that have at least one join condition @@ -1091,7 +1100,8 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val right = conditionalJoin.getOrElse(rest.head) val joinedRefs = left.outputSet ++ right.outputSet - val (joinConditions, others) = conditions.partition(_.references.subsetOf(joinedRefs)) + val (joinConditions, others) = conditions.partition( + e => e.references.subsetOf(joinedRefs) && !PredicateSubquery.hasPredicateSubquery(e)) val joined = Join(left, right, Inner, joinConditions.reduceLeftOption(And)) // should not have reference to same logical plan @@ -1201,9 +1211,16 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) val newRight = rightFilterConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) - val newJoinCond = (commonFilterCondition ++ joinCondition).reduceLeftOption(And) + val (newJoinConditions, others) = + commonFilterCondition.partition(e => !PredicateSubquery.hasPredicateSubquery(e)) + val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And) - Join(newLeft, newRight, Inner, newJoinCond) + val join = Join(newLeft, newRight, Inner, newJoinCond) + if (others.nonEmpty) { + Filter(others.reduceLeft(And), join) + } else { + join + } case RightOuter => // push down the right side only `where` condition val newLeft = left @@ -1543,6 +1560,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Note that will almost certainly be planned as a Broadcast Nested Loop join. Use EXISTS // if performance matters to you. Join(p, sub, LeftAnti, Option(Or(anyNull, condition))) + case (p, predicate) => + var joined = p + val replaced = predicate transformUp { + case PredicateSubquery(sub, conditions, nullAware, _) => + // TODO: support null-aware join + val exists = AttributeReference("exists", BooleanType, false)() + joined = Join(joined, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) + exists + } + Project(p.output, Filter(replaced, joined)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 13f57c54a562..80674d9b4bc9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Attribute object JoinType { def apply(typ: String): JoinType = typ.toLowerCase.replace("_", "") match { @@ -69,6 +70,14 @@ case object LeftAnti extends JoinType { override def sql: String = "LEFT ANTI" } +case class ExistenceJoin(exists: Attribute) extends JoinType { + override def sql: String = { + // This join type is only used in the end of optimizer and physical plans, we will not + // generate SQL for this join type + throw new UnsupportedOperationException + } +} + case class NaturalJoin(tpe: JoinType) extends JoinType { require(Seq(Inner, LeftOuter, RightOuter, FullOuter).contains(tpe), "Unsupported natural join type " + tpe) @@ -84,6 +93,7 @@ case class UsingJoin(tpe: JoinType, usingColumns: Seq[UnresolvedAttribute]) exte object LeftExistence { def unapply(joinType: JoinType): Option[JoinType] = joinType match { case LeftSemi | LeftAnti => Some(joinType) + case j: ExistenceJoin => Some(joinType) case _ => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b2297bbcaa9c..830a7ac77dd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -273,6 +273,8 @@ case class Join( override def output: Seq[Attribute] = { joinType match { + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case LeftOuter => @@ -295,6 +297,8 @@ case class Join( case LeftSemi if condition.isDefined => left.constraints .union(splitConjunctivePredicates(condition.get).toSet) + case j: ExistenceJoin => + left.constraints case Inner => left.constraints.union(right.constraints) case LeftExistence(_) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 1b08913ddd0e..10bff3d6d82e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -459,11 +459,14 @@ class AnalysisErrorSuite extends AnalysisTest { val a = AttributeReference("a", IntegerType)() val b = AttributeReference("b", IntegerType)() val c = AttributeReference("c", BooleanType)() - val plan1 = Filter(Cast(In(a, Seq(ListQuery(LocalRelation(b)))), BooleanType), LocalRelation(a)) - assertAnalysisError(plan1, "Predicate sub-queries cannot be used in nested conditions" :: Nil) + val plan1 = Filter(Cast(Not(In(a, Seq(ListQuery(LocalRelation(b))))), BooleanType), + LocalRelation(a)) + assertAnalysisError(plan1, + "Null-aware predicate sub-queries cannot be used in nested conditions" :: Nil) - val plan2 = Filter(Or(In(a, Seq(ListQuery(LocalRelation(b)))), c), LocalRelation(a, c)) - assertAnalysisError(plan2, "Predicate sub-queries cannot be used in nested conditions" :: Nil) + val plan2 = Filter(Or(Not(In(a, Seq(ListQuery(LocalRelation(b))))), c), LocalRelation(a, c)) + assertAnalysisError(plan2, + "Null-aware predicate sub-queries cannot be used in nested conditions" :: Nil) } test("PredicateSubQuery correlated predicate is nested in an illegal plan") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 238334e26b45..9747e58f4371 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -118,6 +118,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { private def canBuildRight(joinType: JoinType): Boolean = joinType match { case Inner | LeftOuter | LeftSemi | LeftAnti => true + case j: ExistenceJoin => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 587c603192cc..7c194ab72643 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -48,8 +48,6 @@ case class BroadcastHashJoinExec( override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) buildSide match { @@ -85,6 +83,7 @@ case class BroadcastHashJoinExec( case LeftOuter | RightOuter => codegenOuter(ctx, input) case LeftSemi => codegenSemi(ctx, input) case LeftAnti => codegenAnti(ctx, input) + case j: ExistenceJoin => codegenExistence(ctx, input) case x => throw new IllegalArgumentException( s"BroadcastHashJoin should not take $x as the JoinType") @@ -407,4 +406,67 @@ case class BroadcastHashJoinExec( """.stripMargin } } + + /** + * Generates the code for existence join. + */ + private def codegenExistence(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + val existsVar = ctx.freshName("exists") + + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + // filter the output via condition + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |$eval + |${ev.code} + |$existsVar = !${ev.isNull} && ${ev.value}; + """.stripMargin + } else { + s"$existsVar = true;" + } + + val resultVar = input ++ Seq(ExprCode("", "false", existsVar)) + if (broadcastRelation.value.keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |boolean $existsVar = false; + |if ($matched != null) { + | $checkCondition + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $existsVar = false; + |if ($matches != null) { + | while (!$existsVar && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition + | } + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index a659bf26e32d..2a250ecce6de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -50,19 +50,16 @@ case class BroadcastNestedLoopJoinExec( UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil } - private[this] def genResultProjection: InternalRow => InternalRow = { - if (joinType == LeftSemi) { + private[this] def genResultProjection: InternalRow => InternalRow = joinType match { + case LeftExistence(j) => UnsafeProjection.create(output, output) - } else { + case other => // Always put the stream side on left to simplify implementation // both of left and right side could be null UnsafeProjection.create( output, (streamed.output ++ broadcast.output).map(_.withNullability(true))) - } } - override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def output: Seq[Attribute] = { joinType match { case Inner => @@ -73,6 +70,8 @@ case class BroadcastNestedLoopJoinExec( left.output.map(_.withNullability(true)) ++ right.output case FullOuter => left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case x => @@ -197,6 +196,28 @@ case class BroadcastNestedLoopJoinExec( } } + private def existenceJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = { + assert(buildSide == BuildRight) + streamed.execute().mapPartitionsInternal { streamedIter => + val buildRows = relation.value + val joinedRow = new JoinedRow + + if (condition.isDefined) { + val resultRow = new GenericMutableRow(Array[Any](null)) + streamedIter.map { row => + val result = buildRows.exists(r => boundCondition(joinedRow(row, r))) + resultRow.setBoolean(0, result) + joinedRow(row, resultRow) + } + } else { + val resultRow = new GenericMutableRow(Array[Any](buildRows.nonEmpty)) + streamedIter.map { row => + joinedRow(row, resultRow) + } + } + } + } + /** * The implementation for these joins: * @@ -204,7 +225,8 @@ case class BroadcastNestedLoopJoinExec( * RightOuter with BuildRight * FullOuter * LeftSemi with BuildLeft - * Anti with BuildLeft + * LeftAnti with BuildLeft + * ExistenceJoin with BuildLeft */ private def defaultJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = { /** All rows that either match both-way, or rows from streamed joined with nulls. */ @@ -231,27 +253,50 @@ case class BroadcastNestedLoopJoinExec( new BitSet(relation.value.length) )(_ | _) - if (joinType == LeftSemi) { - assert(buildSide == BuildLeft) - val buf: CompactBuffer[InternalRow] = new CompactBuffer() - var i = 0 - val rel = relation.value - while (i < rel.length) { - if (matchedBroadcastRows.get(i)) { - buf += rel(i).copy() + joinType match { + case LeftSemi => + assert(buildSide == BuildLeft) + val buf: CompactBuffer[InternalRow] = new CompactBuffer() + var i = 0 + val rel = relation.value + while (i < rel.length) { + if (matchedBroadcastRows.get(i)) { + buf += rel(i).copy() + } + i += 1 } - i += 1 - } - return sparkContext.makeRDD(buf) + return sparkContext.makeRDD(buf) + case j: ExistenceJoin => + val buf: CompactBuffer[InternalRow] = new CompactBuffer() + var i = 0 + val rel = relation.value + while (i < rel.length) { + val result = new GenericInternalRow(Array[Any](matchedBroadcastRows.get(i))) + buf += new JoinedRow(rel(i).copy(), result) + i += 1 + } + return sparkContext.makeRDD(buf) + case LeftAnti => + val notMatched: CompactBuffer[InternalRow] = new CompactBuffer() + var i = 0 + val rel = relation.value + while (i < rel.length) { + if (!matchedBroadcastRows.get(i)) { + notMatched += rel(i).copy() + } + i += 1 + } + return sparkContext.makeRDD(notMatched) + case o => } val notMatchedBroadcastRows: Seq[InternalRow] = { val nulls = new GenericMutableRow(streamed.output.size) val buf: CompactBuffer[InternalRow] = new CompactBuffer() - var i = 0 - val buildRows = relation.value val joinedRow = new JoinedRow joinedRow.withLeft(nulls) + var i = 0 + val buildRows = relation.value while (i < buildRows.length) { if (!matchedBroadcastRows.get(i)) { buf += joinedRow.withRight(buildRows(i)).copy() @@ -261,10 +306,6 @@ case class BroadcastNestedLoopJoinExec( buf } - if (joinType == LeftAnti) { - return sparkContext.makeRDD(notMatchedBroadcastRows) - } - val matchedStreamRows = streamRdd.mapPartitionsInternal { streamedIter => val buildRows = relation.value val joinedRow = new JoinedRow @@ -308,13 +349,16 @@ case class BroadcastNestedLoopJoinExec( leftExistenceJoin(broadcastedRelation, exists = true) case (LeftAnti, BuildRight) => leftExistenceJoin(broadcastedRelation, exists = false) + case (j: ExistenceJoin, BuildRight) => + existenceJoin(broadcastedRelation) case _ => /** * LeftOuter with BuildLeft * RightOuter with BuildRight * FullOuter * LeftSemi with BuildLeft - * Anti with BuildLeft + * LeftAnti with BuildLeft + * ExistenceJoin with BuildLeft */ defaultJoin(broadcastedRelation) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 9c173d7bf101..d46a80423fa3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{RowIterator, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.{IntegralType, LongType} @@ -43,6 +44,8 @@ trait HashJoin { left.output ++ right.output.map(_.withNullability(true)) case RightOuter => left.output.map(_.withNullability(true)) ++ right.output + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case x => @@ -50,6 +53,8 @@ trait HashJoin { } } + override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning + protected lazy val (buildPlan, streamedPlan) = buildSide match { case BuildLeft => (left, right) case BuildRight => (right, left) @@ -110,15 +115,14 @@ trait HashJoin { (r: InternalRow) => true } - protected def createResultProjection(): (InternalRow) => InternalRow = { - if (joinType == LeftSemi) { + protected def createResultProjection(): (InternalRow) => InternalRow = joinType match { + case LeftExistence(_) => UnsafeProjection.create(output, output) - } else { + case _ => // Always put the stream side on left to simplify implementation // both of left and right side could be null UnsafeProjection.create( output, (streamedPlan.output ++ buildPlan.output).map(_.withNullability(true))) - } } private def innerJoin( @@ -184,6 +188,23 @@ trait HashJoin { } } + private def existenceJoin( + streamIter: Iterator[InternalRow], + hashedRelation: HashedRelation): Iterator[InternalRow] = { + val joinKeys = streamSideKeyGenerator() + val result = new GenericMutableRow(Array[Any](null)) + val joinedRow = new JoinedRow + streamIter.map { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + val exists = !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + result.setBoolean(0, exists) + joinedRow(current, result) + } + } + private def antiJoin( streamIter: Iterator[InternalRow], hashedRelation: HashedRelation): Iterator[InternalRow] = { @@ -212,6 +233,8 @@ trait HashJoin { semiJoin(streamedIter, hashed) case LeftAnti => antiJoin(streamedIter, hashed) + case j: ExistenceJoin => + existenceJoin(streamedIter, hashed) case x => throw new IllegalArgumentException( s"BroadcastHashJoin should not take $x as the JoinType") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 3ef2fec35220..0036f9aadc5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} @@ -44,17 +44,6 @@ case class ShuffledHashJoinExec( "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) - override def outputPartitioning: Partitioning = joinType match { - case Inner => PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning)) - case LeftAnti => left.outputPartitioning - case LeftSemi => left.outputPartitioning - case LeftOuter => left.outputPartitioning - case RightOuter => right.outputPartitioning - case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) - case x => - throw new IllegalArgumentException(s"ShuffledHashJoin should not take $x as the JoinType") - } - override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 775f8ac50818..f0efa52c3d86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -53,6 +53,8 @@ case class SortMergeJoinExec( left.output.map(_.withNullability(true)) ++ right.output case FullOuter => (left.output ++ right.output).map(_.withNullability(true)) + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case x => @@ -269,6 +271,44 @@ case class SortMergeJoinExec( override def getRow: InternalRow = currentLeftRow }.toScala + case j: ExistenceJoin => + new RowIterator { + private[this] var currentLeftRow: InternalRow = _ + private[this] val result: MutableRow = new GenericMutableRow(Array[Any](null)) + private[this] val smjScanner = new SortMergeJoinScanner( + createLeftKeyGenerator(), + createRightKeyGenerator(), + keyOrdering, + RowIterator.fromScala(leftIter), + RowIterator.fromScala(rightIter) + ) + private[this] val joinRow = new JoinedRow + + override def advanceNext(): Boolean = { + while (smjScanner.findNextOuterJoinRows()) { + currentLeftRow = smjScanner.getStreamedRow + val currentRightMatches = smjScanner.getBufferedMatches + var found = false + if (currentRightMatches != null) { + var i = 0 + while (!found && i < currentRightMatches.length) { + joinRow(currentLeftRow, currentRightMatches(i)) + if (boundCondition(joinRow)) { + found = true + } + i += 1 + } + } + result.setBoolean(0, found) + numOutputRows += 1 + return true + } + false + } + + override def getRow: InternalRow = resultProj(joinRow(currentLeftRow, result)) + }.toScala + case x => throw new IllegalArgumentException( s"SortMergeJoin should not take $x as the JoinType") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 0bf4c6f96063..ff3f9bb33f9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -152,6 +152,19 @@ class SubquerySuite extends QueryTest with SharedSQLContext { Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) } + test("EXISTS predicate subquery within OR") { + checkAnswer( + sql("select * from l where exists (select * from r where l.a = r.c)" + + " or exists (select * from r where l.a = r.c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + checkAnswer( + sql("select * from l where not exists (select * from r where l.a = r.c and l.b < r.d)" + + " or not exists (select * from r where l.a = r.c)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + test("IN predicate subquery") { checkAnswer( sql("select * from l where l.a in (select c from r)"), @@ -187,6 +200,18 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } + test("IN predicate subquery within OR") { + checkAnswer( + sql("select * from l where l.a in (select c from r)" + + " or l.a in (select c from r where l.b < r.d)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + intercept[AnalysisException] { + sql("select * from l where a not in (select c from r)" + + " or a not in (select c from r where c is not null)") + } + } + test("complex IN predicate subquery") { checkAnswer( sql("select * from l where (a, b) not in (select c, d from r)"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index b32b6444b6d9..8093054b6dd1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -18,15 +18,15 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys -import org.apache.spark.sql.catalyst.plans.{Inner, JoinType, LeftAnti, LeftSemi} +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.Join -import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} +import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StructType} class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { @@ -89,6 +89,18 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { ExtractEquiJoinKeys.unapply(join) } + val existsAttr = AttributeReference("exists", BooleanType, false)() + val leftSemiPlus = ExistenceJoin(existsAttr) + def createLeftSemiPlusJoin(join: SparkPlan): SparkPlan = { + val output = join.output.dropRight(1) + val condition = if (joinType == LeftSemi) { + existsAttr + } else { + Not(existsAttr) + } + ProjectExec(output, FilterExec(condition, join)) + } + test(s"$testName using ShuffledHashJoin") { extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { @@ -98,6 +110,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(ShuffledHashJoinExec( + leftKeys, rightKeys, leftSemiPlus, BuildRight, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) } } } @@ -111,6 +129,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastHashJoinExec( + leftKeys, rightKeys, leftSemiPlus, BuildRight, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) } } } @@ -123,6 +147,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { SortMergeJoinExec(leftKeys, rightKeys, joinType, boundCondition, left, right)), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(SortMergeJoinExec( + leftKeys, rightKeys, leftSemiPlus, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) } } } @@ -134,6 +164,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { BroadcastNestedLoopJoinExec(left, right, BuildLeft, joinType, Some(condition))), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastNestedLoopJoinExec( + left, right, BuildLeft, leftSemiPlus, Some(condition)))), + expectedAnswer, + sortAnswers = true) } } @@ -144,6 +180,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { BroadcastNestedLoopJoinExec(left, right, BuildRight, joinType, Some(condition))), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastNestedLoopJoinExec( + left, right, BuildRight, leftSemiPlus, Some(condition)))), + expectedAnswer, + sortAnswers = true) } } } From 56dbf165c0206a59701f61649ec654b9a0b15a3f Mon Sep 17 00:00:00 2001 From: Pete Robbins Date: Mon, 2 May 2016 13:16:46 -0700 Subject: [PATCH 010/313] [SPARK-13745] [SQL] Support columnar in memory representation on Big Endian platforms ## What changes were proposed in this pull request? parquet datasource and ColumnarBatch tests fail on big-endian platforms This patch adds support for the little-endian byte arrays being correctly interpreted on a big-endian platform ## How was this patch tested? Spark test builds ran on big endian z/Linux and regression build on little endian amd64 Author: Pete Robbins Closes #12397 from robbinspg/master. (cherry picked from commit 8a1ce4899fb9f751dedaaa34ea654dfbc8330852) Signed-off-by: Davies Liu --- .../parquet/VectorizedPlainValuesReader.java | 28 +++++++++- .../vectorized/OffHeapColumnVector.java | 54 +++++++++++++++---- .../vectorized/OnHeapColumnVector.java | 41 ++++++++++---- .../vectorized/ColumnarBatchSuite.scala | 9 ++++ 4 files changed, 110 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java index 2672e0453b39..9475c853a03f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.parquet; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import org.apache.spark.sql.execution.vectorized.ColumnVector; import org.apache.spark.unsafe.Platform; @@ -31,6 +33,9 @@ public class VectorizedPlainValuesReader extends ValuesReader implements Vectori private byte[] buffer; private int offset; private int bitOffset; // Only used for booleans. + private ByteBuffer byteBuffer; // used to wrap the byte array buffer + + private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); public VectorizedPlainValuesReader() { } @@ -39,6 +44,9 @@ public VectorizedPlainValuesReader() { public void initFromPage(int valueCount, byte[] bytes, int offset) throws IOException { this.buffer = bytes; this.offset = offset + Platform.BYTE_ARRAY_OFFSET; + if (bigEndianPlatform) { + byteBuffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + } } @Override @@ -103,6 +111,9 @@ public final boolean readBoolean() { @Override public final int readInteger() { int v = Platform.getInt(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Integer.reverseBytes(v); + } offset += 4; return v; } @@ -110,6 +121,9 @@ public final int readInteger() { @Override public final long readLong() { long v = Platform.getLong(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Long.reverseBytes(v); + } offset += 8; return v; } @@ -121,14 +135,24 @@ public final byte readByte() { @Override public final float readFloat() { - float v = Platform.getFloat(buffer, offset); + float v; + if (!bigEndianPlatform) { + v = Platform.getFloat(buffer, offset); + } else { + v = byteBuffer.getFloat(offset - Platform.BYTE_ARRAY_OFFSET); + } offset += 4; return v; } @Override public final double readDouble() { - double v = Platform.getDouble(buffer, offset); + double v; + if (!bigEndianPlatform) { + v = Platform.getDouble(buffer, offset); + } else { + v = byteBuffer.getDouble(offset - Platform.BYTE_ARRAY_OFFSET); + } offset += 8; return v; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index b1901411351a..b8dd16227ec1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.nio.ByteBuffer; import java.nio.ByteOrder; import org.apache.commons.lang.NotImplementedException; @@ -28,6 +29,9 @@ * Column data backed using offheap memory. */ public final class OffHeapColumnVector extends ColumnVector { + + private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these two allocations need to maintain binary compatible. We can // directly pass this buffer to external components. private long nulls; @@ -39,9 +43,7 @@ public final class OffHeapColumnVector extends ColumnVector { protected OffHeapColumnVector(int capacity, DataType type) { super(capacity, type, MemoryMode.OFF_HEAP); - if (!ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { - throw new NotImplementedException("Only little endian is supported."); - } + nulls = 0; data = 0; lengthData = 0; @@ -221,8 +223,16 @@ public void putInts(int rowId, int count, int[] src, int srcIndex) { @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - null, data + 4 * rowId, count * 4); + if (!bigEndianPlatform) { + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 4 * rowId, count * 4); + } else { + int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; + long offset = data + 4 * rowId; + for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { + Platform.putInt(null, offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); + } + } } @Override @@ -259,8 +269,16 @@ public void putLongs(int rowId, int count, long[] src, int srcIndex) { @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - null, data + 8 * rowId, count * 8); + if (!bigEndianPlatform) { + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 8 * rowId, count * 8); + } else { + int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; + long offset = data + 8 * rowId; + for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { + Platform.putLong(null, offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); + } + } } @Override @@ -297,8 +315,16 @@ public void putFloats(int rowId, int count, float[] src, int srcIndex) { @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data + rowId * 4, count * 4); + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 4, count * 4); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + long offset = data + 4 * rowId; + for (int i = 0; i < count; ++i, offset += 4) { + Platform.putFloat(null, offset, bb.getFloat(srcIndex + (4 * i))); + } + } } @Override @@ -336,8 +362,16 @@ public void putDoubles(int rowId, int count, double[] src, int srcIndex) { @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId * 8, count * 8); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + long offset = data + 8 * rowId; + for (int i = 0; i < count; ++i, offset += 8) { + Platform.putDouble(null, offset, bb.getDouble(srcIndex + (8 * i))); + } + } } @Override diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index e97276800daa..b1ffe4c21049 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.Arrays; import org.apache.spark.memory.MemoryMode; @@ -27,6 +29,9 @@ * and a java array for the values. */ public final class OnHeapColumnVector extends ColumnVector { + + private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these arrays need to maintain binary compatible. We can // directly pass this buffer to external components. @@ -211,10 +216,11 @@ public void putInts(int rowId, int count, int[] src, int srcIndex) { @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - for (int i = 0; i < count; ++i) { + for (int i = 0; i < count; ++i, srcOffset += 4) { intData[i + rowId] = Platform.getInt(src, srcOffset); - srcIndex += 4; - srcOffset += 4; + if (bigEndianPlatform) { + intData[i + rowId] = java.lang.Integer.reverseBytes(intData[i + rowId]); + } } } @@ -251,10 +257,11 @@ public void putLongs(int rowId, int count, long[] src, int srcIndex) { @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - for (int i = 0; i < count; ++i) { + for (int i = 0; i < count; ++i, srcOffset += 8) { longData[i + rowId] = Platform.getLong(src, srcOffset); - srcIndex += 8; - srcOffset += 8; + if (bigEndianPlatform) { + longData[i + rowId] = java.lang.Long.reverseBytes(longData[i + rowId]); + } } } @@ -286,8 +293,15 @@ public void putFloats(int rowId, int count, float[] src, int srcIndex) { @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - floatData, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, floatData, + Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + for (int i = 0; i < count; ++i) { + floatData[i + rowId] = bb.getFloat(srcIndex + (4 * i)); + } + } } @Override @@ -320,8 +334,15 @@ public void putDoubles(int rowId, int count, double[] src, int srcIndex) { @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, - Platform.DOUBLE_ARRAY_OFFSET + rowId * 8, count * 8); + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, + Platform.DOUBLE_ARRAY_OFFSET + rowId * 8, count * 8); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + for (int i = 0; i < count; ++i) { + doubleData[i + rowId] = bb.getDouble(srcIndex + (8 * i)); + } + } } @Override diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index a63007fc3bf2..7e576a865799 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.execution.vectorized import java.nio.charset.StandardCharsets +import java.nio.ByteBuffer +import java.nio.ByteOrder import scala.collection.JavaConverters._ import scala.collection.mutable @@ -280,6 +282,13 @@ class ColumnarBatchSuite extends SparkFunSuite { Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, 2.234) Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, 1.123) + if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { + // Ensure array contains Liitle Endian doubles + var bb = ByteBuffer.wrap(buffer).order(ByteOrder.LITTLE_ENDIAN) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, bb.getDouble(0)) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, bb.getDouble(8)) + } + column.putDoubles(idx, 1, buffer, 8) column.putDoubles(idx + 1, 1, buffer, 0) reference += 1.123 From 740f96f6362a49fd95a6d56d93b966094166bbf2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 May 2016 14:57:00 -0700 Subject: [PATCH 011/313] [SPARK-15054] Deprecate old accumulator API ## What changes were proposed in this pull request? This patch deprecates the old accumulator API. ## How was this patch tested? N/A Author: Reynold Xin Closes #12832 from rxin/SPARK-15054. (cherry picked from commit d5c79f564f3557037c5526e2ee5f963dd100fb34) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/Accumulable.scala | 6 ++++-- .../scala/org/apache/spark/Accumulator.scala | 19 +++++++++++-------- .../scala/org/apache/spark/SparkContext.scala | 5 +++++ 3 files changed, 20 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index c76720c4bb8b..799c7e4fd500 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -17,14 +17,13 @@ package org.apache.spark -import java.io.{ObjectInputStream, Serializable} +import java.io.Serializable import scala.collection.generic.Growable import scala.reflect.ClassTag import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.util.Utils /** @@ -49,6 +48,7 @@ import org.apache.spark.util.Utils * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ +@deprecated("use AccumulatorV2", "2.0.0") class Accumulable[R, T] private ( val id: Long, // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile @@ -162,6 +162,7 @@ class Accumulable[R, T] private ( * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ +@deprecated("use AccumulatorV2", "2.0.0") trait AccumulableParam[R, T] extends Serializable { /** * Add additional data to the accumulator value. Is allowed to modify and return `r` @@ -191,6 +192,7 @@ trait AccumulableParam[R, T] extends Serializable { } +@deprecated("use AccumulatorV2", "2.0.0") private[spark] class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] extends AccumulableParam[R, T] { diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index 9b007b977608..e52d36b7b564 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -17,13 +17,6 @@ package org.apache.spark -import java.util.concurrent.atomic.AtomicLong -import javax.annotation.concurrent.GuardedBy - -import scala.collection.mutable -import scala.ref.WeakReference - -import org.apache.spark.internal.Logging import org.apache.spark.storage.{BlockId, BlockStatus} @@ -58,7 +51,8 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * @param name human-readable name associated with this accumulator * @param countFailedValues whether to accumulate values from failed tasks * @tparam T result type - */ +*/ +@deprecated("use AccumulatorV2", "2.0.0") class Accumulator[T] private[spark] ( // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile @transient private val initialValue: T, @@ -75,6 +69,7 @@ class Accumulator[T] private[spark] ( * * @tparam T type of value to accumulate */ +@deprecated("use AccumulatorV2", "2.0.0") trait AccumulatorParam[T] extends AccumulableParam[T, T] { def addAccumulator(t1: T, t2: T): T = { addInPlace(t1, t2) @@ -82,6 +77,7 @@ trait AccumulatorParam[T] extends AccumulableParam[T, T] { } +@deprecated("use AccumulatorV2", "2.0.0") object AccumulatorParam { // The following implicit objects were in SparkContext before 1.2 and users had to @@ -89,21 +85,25 @@ object AccumulatorParam { // them automatically. However, as there are duplicate codes in SparkContext for backward // compatibility, please update them accordingly if you modify the following implicit objects. + @deprecated("use AccumulatorV2", "2.0.0") implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double): Double = 0.0 } + @deprecated("use AccumulatorV2", "2.0.0") implicit object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int): Int = 0 } + @deprecated("use AccumulatorV2", "2.0.0") implicit object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long): Long = t1 + t2 def zero(initialValue: Long): Long = 0L } + @deprecated("use AccumulatorV2", "2.0.0") implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float): Float = t1 + t2 def zero(initialValue: Float): Float = 0f @@ -112,6 +112,7 @@ object AccumulatorParam { // Note: when merging values, this param just adopts the newer value. This is used only // internally for things that shouldn't really be accumulated across tasks, like input // read method, which should be the same across all tasks in the same stage. + @deprecated("use AccumulatorV2", "2.0.0") private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { def addInPlace(t1: String, t2: String): String = t2 def zero(initialValue: String): String = "" @@ -119,12 +120,14 @@ object AccumulatorParam { // Note: this is expensive as it makes a copy of the list every time the caller adds an item. // A better way to use this is to first accumulate the values yourself then them all at once. + @deprecated("use AccumulatorV2", "2.0.0") private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] } // For the internal metric that records what blocks are updated in a particular task + @deprecated("use AccumulatorV2", "2.0.0") private[spark] object UpdatedBlockStatusesAccumulatorParam extends ListAccumulatorParam[(BlockId, BlockStatus)] diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d0f88d4e4d10..302dec25c66b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1219,6 +1219,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `+=` method. Only the driver can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] = { val acc = new Accumulator(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) @@ -1230,6 +1231,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the * driver can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) : Accumulator[T] = { val acc = new Accumulator(initialValue, param, Some(name)) @@ -1243,6 +1245,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param) @@ -1257,6 +1260,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param, Some(name)) @@ -1270,6 +1274,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by * standard mutable collections. So you can use this with mutable Map, Set, etc. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R, T] From 990611cd879d443298e61f672fee41432aab36ef Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 May 2016 15:27:16 -0700 Subject: [PATCH 012/313] [SPARK-15052][SQL] Use builder pattern to create SparkSession ## What changes were proposed in this pull request? This patch creates a builder pattern for creating SparkSession. The new code is unused and mostly deadcode. I'm putting it up here for feedback. There are a few TODOs that can be done as follow-up pull requests: - [ ] Update tests to use this - [ ] Update examples to use this - [ ] Clean up SQLContext code w.r.t. this one (i.e. SparkSession shouldn't call into SQLContext.getOrCreate; it should be the other way around) - [ ] Remove SparkSession.withHiveSupport - [ ] Disable the old constructor (by making it private) so the only way to start a SparkSession is through this builder pattern ## How was this patch tested? Part of the future pull request is to clean this up and switch existing tests to use this. Author: Reynold Xin Closes #12830 from rxin/sparksession-builder. (cherry picked from commit ca1b2198581b8de1651a88fc97540570a2347dc9) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/SparkSession.scala | 131 +++++++++++++++++- 1 file changed, 127 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 4c2a7b8ae906..3836ce2daa56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -18,9 +18,7 @@ package org.apache.spark.sql import java.beans.Introspector -import java.util.Properties -import scala.collection.immutable import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag @@ -30,7 +28,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{CATALOG_IMPLEMENTATION, ConfigEntry} +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst._ @@ -49,7 +47,16 @@ import org.apache.spark.util.Utils /** - * The entry point to Spark execution. + * The entry point to programming Spark with the Dataset and DataFrame API. + * + * To create a SparkSession, use the following builder pattern: + * + * {{{ + * SparkSession.builder() + * .master("local") + * .config("spark.some.config.option", "some-value"). + * .getOrCreate() + * }}} */ class SparkSession private( @transient val sparkContext: SparkContext, @@ -635,6 +642,122 @@ class SparkSession private( object SparkSession { + /** + * Builder for [[SparkSession]]. + */ + class Builder { + + private[this] val options = new scala.collection.mutable.HashMap[String, String] + + /** + * Sets a name for the application, which will be shown in the Spark web UI. + * + * @since 2.0.0 + */ + def appName(name: String): Builder = config("spark.app.name", name) + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: String): Builder = synchronized { + options += key -> value + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: Long): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: Double): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: Boolean): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a list of config options based on the given [[SparkConf]]. + * + * @since 2.0.0 + */ + def config(conf: SparkConf): Builder = synchronized { + conf.getAll.foreach { case (k, v) => options += k -> v } + this + } + + /** + * Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. + * + * @since 2.0.0 + */ + def master(master: String): Builder = config("spark.master", master) + + /** + * Enables Hive support, including connectivity to a persistent Hive metastore, support for + * Hive serdes, and Hive user-defined functions. + * + * @since 2.0.0 + */ + def enableHiveSupport(): Builder = synchronized { + if (hiveClassesArePresent) { + config(CATALOG_IMPLEMENTATION.key, "hive") + } else { + throw new IllegalArgumentException( + "Unable to instantiate SparkSession with Hive support because " + + "Hive classes are not found.") + } + } + + /** + * Gets an existing [[SparkSession]] or, if there is no existing one, creates a new one + * based on the options set in this builder. + * + * @since 2.0.0 + */ + def getOrCreate(): SparkSession = synchronized { + // Step 1. Create a SparkConf + // Step 2. Get a SparkContext + // Step 3. Get a SparkSession + val sparkConf = new SparkConf() + options.foreach { case (k, v) => sparkConf.set(k, v) } + val sparkContext = SparkContext.getOrCreate(sparkConf) + + SQLContext.getOrCreate(sparkContext).sparkSession + } + } + + /** + * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. + * @since 2.0.0 + */ + def builder: Builder = new Builder + private val HIVE_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" private val HIVE_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" From 05bb5b6f64d8b4114e3434bc467385d8cba86fd0 Mon Sep 17 00:00:00 2001 From: poolis Date: Mon, 2 May 2016 16:15:07 -0700 Subject: [PATCH 013/313] [SPARK-12928][SQL] Oracle FLOAT datatype is not properly handled when reading via JDBC The contribution is my original work and that I license the work to the project under the project's open source license. Author: poolis Author: Greg Michalopoulos Closes #10899 from poolis/spark-12928. (cherry picked from commit 917d05f43bddc1728735979fe7e62fe631b35e6f) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/jdbc/OracleDialect.scala | 6 ++++++ .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala index 46b3877a7cab..b795e8b42df0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -38,6 +38,12 @@ private case object OracleDialect extends JdbcDialect { // This is sub-optimal as we have to pick a precision/scale in advance whereas the data // in Oracle is allowed to have different precision/scale for each value. Option(DecimalType(DecimalType.MAX_PRECISION, 10)) + } else if (sqlType == Types.NUMERIC && md.build().getLong("scale") == -127) { + // Handle FLOAT fields in a special way because JDBC ResultSetMetaData converts + // this to NUMERIC with -127 scale + // Not sure if there is a more robust way to identify the field as a float (or other + // numeric types that do not specify a scale. + Option(DecimalType(DecimalType.MAX_PRECISION, 10)) } else { None } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 783511b78109..47a1017caa53 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -600,6 +600,15 @@ class JDBCSuite extends SparkFunSuite assert(derbyDialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "BOOLEAN") } + test("OracleDialect jdbc type mapping") { + val oracleDialect = JdbcDialects.get("jdbc:oracle") + val metadata = new MetadataBuilder().putString("name", "test_column").putLong("scale", -127) + assert(oracleDialect.getCatalystType(java.sql.Types.NUMERIC, "float", 1, metadata) == + Some(DecimalType(DecimalType.MAX_PRECISION, 10))) + assert(oracleDialect.getCatalystType(java.sql.Types.NUMERIC, "numeric", 0, null) == + Some(DecimalType(DecimalType.MAX_PRECISION, 10))) + } + test("table exists query by jdbc dialect") { val MySQL = JdbcDialects.get("jdbc:mysql://127.0.0.1/db") val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") From fbc73f73186873cfd60581e58aff4a8d919e39b4 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 2 May 2016 16:32:31 -0700 Subject: [PATCH 014/313] [SPARK-14785] [SQL] Support correlated scalar subqueries ## What changes were proposed in this pull request? In this PR we add support for correlated scalar subqueries. An example of such a query is: ```SQL select * from tbl1 a where a.value > (select max(value) from tbl2 b where b.key = a.key) ``` The implementation adds the `RewriteCorrelatedScalarSubquery` rule to the Optimizer. This rule plans these subqueries using `LEFT OUTER` joins. It currently supports rewrites for `Project`, `Aggregate` & `Filter` logical plans. I could not find a well defined semantics for the use of scalar subqueries in an `Aggregate`. The current implementation currently evaluates the scalar subquery *before* aggregation. This means that you either have to make scalar subquery part of the grouping expression, or that you have to aggregate it further on. I am open to suggestions on this. The implementation currently forces the uniqueness of a scalar subquery by enforcing that it is aggregated and that the resulting column is wrapped in an `AggregateExpression`. ## How was this patch tested? Added tests to `SubquerySuite`. Author: Herman van Hovell Closes #12822 from hvanhovell/SPARK-14785. --- .../sql/catalyst/analysis/Analyzer.scala | 11 ++- .../sql/catalyst/analysis/CheckAnalysis.scala | 42 +++++++++- .../sql/catalyst/expressions/subquery.scala | 39 +++++---- .../sql/catalyst/optimizer/Optimizer.scala | 82 +++++++++++++++++-- .../plans/logical/basicLogicalOperators.scala | 2 +- .../analysis/AnalysisErrorSuite.scala | 11 +-- .../org/apache/spark/sql/SubquerySuite.scala | 47 +++++++++++ 7 files changed, 195 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2f8ab3f43586..59af5b7095a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1081,10 +1081,10 @@ class Analyzer( // Step 2: Pull out the predicates if the plan is resolved. if (current.resolved) { // Make sure the resolved query has the required number of output columns. This is only - // needed for IN expressions. + // needed for Scalar and IN subqueries. if (requiredColumns > 0 && requiredColumns != current.output.size) { - failAnalysis(s"The number of fields in the value ($requiredColumns) does not " + - s"match with the number of columns in the subquery (${current.output.size})") + failAnalysis(s"The number of columns in the subquery (${current.output.size}) " + + s"does not match the required number of columns ($requiredColumns)") } // Pullout predicates and construct a new plan. f.tupled(rewriteSubQuery(current, plans)) @@ -1099,8 +1099,11 @@ class Analyzer( */ private def resolveSubQueries(plan: LogicalPlan, plans: Seq[LogicalPlan]): LogicalPlan = { plan transformExpressions { + case s @ ScalarSubquery(sub, conditions, exprId) + if sub.resolved && conditions.isEmpty && sub.output.size != 1 => + failAnalysis(s"Scalar subquery must return only one column, but got ${sub.output.size}") case s @ ScalarSubquery(sub, _, exprId) if !sub.resolved => - resolveSubQuery(s, plans)(ScalarSubquery(_, _, exprId)) + resolveSubQuery(s, plans, 1)(ScalarSubquery(_, _, exprId)) case e @ Exists(sub, exprId) => resolveSubQuery(e, plans)(PredicateSubquery(_, _, nullAware = false, exprId)) case In(e, Seq(l @ ListQuery(_, exprId))) if e.resolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 6e3a14dfb920..800bf01abd67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.plans.{Inner, RightOuter, UsingJoin} +import org.apache.spark.sql.catalyst.plans.UsingJoin import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ @@ -60,9 +60,6 @@ trait CheckAnalysis extends PredicateHelper { val from = operator.inputSet.map(_.name).mkString(", ") a.failAnalysis(s"cannot resolve '${a.sql}' given input columns: [$from]") - case ScalarSubquery(_, conditions, _) if conditions.nonEmpty => - failAnalysis("Correlated scalar subqueries are not supported.") - case e: Expression if e.checkInputDataTypes().isFailure => e.checkInputDataTypes() match { case TypeCheckResult.TypeCheckFailure(message) => @@ -104,6 +101,36 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis(s"Window specification $s is not valid because $m") case None => w } + + case s @ ScalarSubquery(query, conditions, _) if conditions.nonEmpty => + // Make sure we are using equi-joins. + conditions.foreach { + case _: EqualTo | _: EqualNullSafe => // ok + case e => failAnalysis( + s"The correlated scalar subquery can only contain equality predicates: $e") + } + + // Make sure correlated scalar subqueries contain one row for every outer row by + // enforcing that they are aggregates which contain exactly one aggregate expressions. + // The analyzer has already checked that subquery contained only one output column, and + // added all the grouping expressions to the aggregate. + def checkAggregate(a: Aggregate): Unit = { + val aggregates = a.expressions.flatMap(_.collect { + case a: AggregateExpression => a + }) + if (aggregates.isEmpty) { + failAnalysis("The output of a correlated scalar subquery must be aggregated") + } + } + + query match { + case a: Aggregate => checkAggregate(a) + case Filter(_, a: Aggregate) => checkAggregate(a) + case Project(_, a: Aggregate) => checkAggregate(a) + case Project(_, Filter(_, a: Aggregate)) => checkAggregate(a) + case fail => failAnalysis(s"Correlated scalar subqueries must be Aggregated: $fail") + } + s } operator match { @@ -220,6 +247,13 @@ trait CheckAnalysis extends PredicateHelper { | but one table has '${firstError.output.length}' columns and another table has | '${s.children.head.output.length}' columns""".stripMargin) + case p if p.expressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) => + p match { + case _: Filter | _: Aggregate | _: Project => // Ok + case other => failAnalysis( + s"Correlated scalar sub-queries can only be used in a Filter/Aggregate/Project: $p") + } + case p if p.expressions.exists(PredicateSubquery.hasPredicateSubquery) => failAnalysis(s"Predicate sub-queries can only be used in a Filter: $p") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index eed062f8bc18..5001f9a41e07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -44,6 +44,15 @@ abstract class SubqueryExpression extends Expression { protected def conditionString: String = children.mkString("[", " && ", "]") } +object SubqueryExpression { + def hasCorrelatedSubquery(e: Expression): Boolean = { + e.find { + case e: SubqueryExpression if e.children.nonEmpty => true + case _ => false + }.isDefined + } +} + /** * A subquery that will return only one row and one column. This will be converted into a physical * scalar subquery during planning. @@ -55,28 +64,26 @@ case class ScalarSubquery( children: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId) extends SubqueryExpression with Unevaluable { - - override def plan: LogicalPlan = SubqueryAlias(toString, query) - override lazy val resolved: Boolean = childrenResolved && query.resolved - - override def dataType: DataType = query.schema.fields.head.dataType - - override def checkInputDataTypes(): TypeCheckResult = { - if (query.schema.length != 1) { - TypeCheckResult.TypeCheckFailure("Scalar subquery must return only one column, but got " + - query.schema.length.toString) - } else { - TypeCheckResult.TypeCheckSuccess - } + override lazy val references: AttributeSet = { + if (query.resolved) super.references -- query.outputSet + else super.references } - + override def dataType: DataType = query.schema.fields.head.dataType override def foldable: Boolean = false override def nullable: Boolean = true - + override def plan: LogicalPlan = SubqueryAlias(toString, query) override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(query = plan) + override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" +} - override def toString: String = s"subquery#${exprId.id} $conditionString" +object ScalarSubquery { + def hasCorrelatedScalarSubquery(e: Expression): Boolean = { + e.find { + case e: ScalarSubquery if e.children.nonEmpty => true + case _ => false + }.isDefined + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e1c969f50f2b..a3ab89dc7114 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec import scala.collection.immutable.HashSet +import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, DistinctAggregationRewriter, EliminateSubqueryAliases, EmptyFunctionRegistry} @@ -100,6 +101,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) EliminateSorts, SimplifyCasts, SimplifyCaseConversionExpressions, + RewriteCorrelatedScalarSubquery, EliminateSerialization) :: Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: @@ -1081,7 +1083,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { assert(input.size >= 2) if (input.size == 2) { val (joinConditions, others) = conditions.partition( - e => !PredicateSubquery.hasPredicateSubquery(e)) + e => !SubqueryExpression.hasCorrelatedSubquery(e)) val join = Join(input(0), input(1), Inner, joinConditions.reduceLeftOption(And)) if (others.nonEmpty) { Filter(others.reduceLeft(And), join) @@ -1101,7 +1103,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val joinedRefs = left.outputSet ++ right.outputSet val (joinConditions, others) = conditions.partition( - e => e.references.subsetOf(joinedRefs) && !PredicateSubquery.hasPredicateSubquery(e)) + e => e.references.subsetOf(joinedRefs) && !SubqueryExpression.hasCorrelatedSubquery(e)) val joined = Join(left, right, Inner, joinConditions.reduceLeftOption(And)) // should not have reference to same logical plan @@ -1134,7 +1136,7 @@ object OuterJoinElimination extends Rule[LogicalPlan] with PredicateHelper { * Returns whether the expression returns null or false when all inputs are nulls. */ private def canFilterOutNull(e: Expression): Boolean = { - if (!e.deterministic || PredicateSubquery.hasPredicateSubquery(e)) return false + if (!e.deterministic || SubqueryExpression.hasCorrelatedSubquery(e)) return false val attributes = e.references.toSeq val emptyRow = new GenericInternalRow(attributes.length) val v = BindReferences.bindReference(e, attributes).eval(emptyRow) @@ -1203,7 +1205,6 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition)) => val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = split(splitConjunctivePredicates(filterCondition), left, right) - joinType match { case Inner => // push down the single side `where` condition into respective sides @@ -1212,7 +1213,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newRight = rightFilterConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val (newJoinConditions, others) = - commonFilterCondition.partition(e => !PredicateSubquery.hasPredicateSubquery(e)) + commonFilterCondition.partition(e => !SubqueryExpression.hasCorrelatedSubquery(e)) val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And) val join = Join(newLeft, newRight, Inner, newJoinCond) @@ -1573,3 +1574,74 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { } } } + +/** + * This rule rewrites correlated [[ScalarSubquery]] expressions into LEFT OUTER joins. + */ +object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { + /** + * Extract all correlated scalar subqueries from an expression. The subqueries are collected using + * the given collector. The expression is rewritten and returned. + */ + private def extractCorrelatedScalarSubqueries[E <: Expression]( + expression: E, + subqueries: ArrayBuffer[ScalarSubquery]): E = { + val newExpression = expression transform { + case s: ScalarSubquery if s.children.nonEmpty => + subqueries += s + s.query.output.head + } + newExpression.asInstanceOf[E] + } + + /** + * Construct a new child plan by left joining the given subqueries to a base plan. + */ + private def constructLeftJoins( + child: LogicalPlan, + subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = { + subqueries.foldLeft(child) { + case (currentChild, ScalarSubquery(query, conditions, _)) => + Project( + currentChild.output :+ query.output.head, + Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + } + } + + /** + * Rewrite [[Filter]], [[Project]] and [[Aggregate]] plans containing correlated scalar + * subqueries. + */ + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a @ Aggregate(grouping, expressions, child) => + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) + if (subqueries.nonEmpty) { + // We currently only allow correlated subqueries in an aggregate if they are part of the + // grouping expressions. As a result we need to replace all the scalar subqueries in the + // grouping expressions by their result. + val newGrouping = grouping.map { e => + subqueries.find(_.semanticEquals(e)).map(_.query.output.head).getOrElse(e) + } + Aggregate(newGrouping, newExpressions, constructLeftJoins(child, subqueries)) + } else { + a + } + case p @ Project(expressions, child) => + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) + if (subqueries.nonEmpty) { + Project(newExpressions, constructLeftJoins(child, subqueries)) + } else { + p + } + case f @ Filter(condition, child) => + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val newCondition = extractCorrelatedScalarSubqueries(condition, subqueries) + if (subqueries.nonEmpty) { + Project(f.output, Filter(newCondition, constructLeftJoins(child, subqueries))) + } else { + f + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 830a7ac77dd6..7b4615db0661 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -109,7 +109,7 @@ case class Filter(condition: Expression, child: LogicalPlan) override protected def validConstraints: Set[Expression] = { val predicates = splitConjunctivePredicates(condition) - .filterNot(PredicateSubquery.hasPredicateSubquery) + .filterNot(SubqueryExpression.hasCorrelatedSubquery) child.constraints.union(predicates.toSet) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 10bff3d6d82e..2e88f61d491c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -111,7 +111,8 @@ class AnalysisErrorSuite extends AnalysisTest { "scalar subquery with 2 columns", testRelation.select( (ScalarSubquery(testRelation.select('a, dateLit.as('b))) + Literal(1)).as('a)), - "Scalar subquery must return only one column, but got 2" :: Nil) + "The number of columns in the subquery (2)" :: + "does not match the required number of columns (1)":: Nil) errorTest( "scalar subquery with no column", @@ -499,12 +500,4 @@ class AnalysisErrorSuite extends AnalysisTest { LocalRelation(a)) assertAnalysisError(plan3, "Accessing outer query column is not allowed in" :: Nil) } - - test("Correlated Scalar Subquery") { - val a = AttributeReference("a", IntegerType)() - val b = AttributeReference("b", IntegerType)() - val sub = Project(Seq(b), Filter(EqualTo(UnresolvedAttribute("a"), b), LocalRelation(b))) - val plan = Project(Seq(a, Alias(ScalarSubquery(sub), "b")()), LocalRelation(a)) - assertAnalysisError(plan, "Correlated scalar subqueries are not supported." :: Nil) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index ff3f9bb33f9a..80bb4e05385f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -234,4 +234,51 @@ class SubquerySuite extends QueryTest with SharedSQLContext { sql("select a from l group by 1 having exists (select 1 from r where d < min(b))"), Row(null) :: Row(1) :: Row(3) :: Nil) } + + test("correlated scalar subquery in where") { + checkAnswer( + sql("select * from l where b < (select max(d) from r where a = c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Nil) + } + + test("correlated scalar subquery in select") { + checkAnswer( + sql("select a, (select sum(b) from l l2 where l2.a = l1.a) sum_b from l l1"), + Row(1, 4.0) :: Row(1, 4.0) :: Row(2, 2.0) :: Row(2, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, null) :: Row(6, null) :: Nil) + } + + test("correlated scalar subquery in select (null safe)") { + checkAnswer( + sql("select a, (select sum(b) from l l2 where l2.a <=> l1.a) sum_b from l l1"), + Row(1, 4.0) :: Row(1, 4.0) :: Row(2, 2.0) :: Row(2, 2.0) :: Row(3, 3.0) :: + Row(null, 5.0) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + + test("correlated scalar subquery in aggregate") { + checkAnswer( + sql("select a, (select sum(d) from r where a = c) sum_d from l l1 group by 1, 2"), + Row(1, null) :: Row(2, 6.0) :: Row(3, 2.0) :: Row(null, null) :: Row(6, null) :: Nil) + } + + test("non-aggregated correlated scalar subquery") { + val msg1 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1") + } + assert(msg1.getMessage.contains("Correlated scalar subqueries must be Aggregated")) + + val msg2 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1") + } + assert(msg2.getMessage.contains( + "The output of a correlated scalar subquery must be aggregated")) + } + + test("non-equal correlated scalar subquery") { + val msg1 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a < l1.a) sum_b from l l1") + } + assert(msg1.getMessage.contains( + "The correlated scalar subquery can only contain equality predicates")) + } } From 65b94f46021577288ef6c88e00b5b4ed28da33b8 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Mon, 2 May 2016 16:48:20 -0700 Subject: [PATCH 015/313] [SPARK-14747][SQL] Add assertStreaming/assertNoneStreaming checks in DataFrameWriter ## Problem If an end user happens to write code mixed with continuous-query-oriented methods and non-continuous-query-oriented methods: ```scala ctx.read .format("text") .stream("...") // continuous query .write .text("...") // non-continuous query; should be startStream() here ``` He/she would get this somehow confusing exception: > Exception in thread "main" java.lang.AssertionError: assertion failed: No plan for FileSource[./continuous_query_test_input] at scala.Predef$.assert(Predef.scala:170) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at ... ## What changes were proposed in this pull request? This PR adds checks for continuous-query-oriented methods and non-continuous-query-oriented methods in `DataFrameWriter`:
    can be called on continuous query? can be called on non-continuous query?
    mode yes
    trigger yes
    format yes yes
    option/options yes yes
    partitionBy yes yes
    bucketBy yes
    sortBy yes
    save yes
    queryName yes
    startStream yes
    insertInto yes
    saveAsTable yes
    jdbc yes
    json yes
    parquet yes
    orc yes
    text yes
    csv yes
    After this PR's change, the friendly exception would be: > Exception in thread "main" org.apache.spark.sql.AnalysisException: text() can only be called on non-continuous queries; at org.apache.spark.sql.DataFrameWriter.assertNotStreaming(DataFrameWriter.scala:678) at org.apache.spark.sql.DataFrameWriter.text(DataFrameWriter.scala:629) at ss.SSDemo$.main(SSDemo.scala:47) ## How was this patch tested? dedicated unit tests were added Author: Liwei Lin Closes #12521 from lw-lin/dataframe-writer-check. (cherry picked from commit 35d9c8aa69c650f33037813607dc939922c5fc27) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/DataFrameWriter.scala | 59 ++++++- .../DataFrameReaderWriterSuite.scala | 156 ++++++++++++++++++ 2 files changed, 210 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index a57d47d28ceb..a8f96a9b45b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -53,6 +53,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def mode(saveMode: SaveMode): DataFrameWriter = { + // mode() is used for non-continuous queries + // outputMode() is used for continuous queries + assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode this } @@ -67,6 +70,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def mode(saveMode: String): DataFrameWriter = { + // mode() is used for non-continuous queries + // outputMode() is used for continuous queries + assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode.toLowerCase match { case "overwrite" => SaveMode.Overwrite case "append" => SaveMode.Append @@ -103,6 +109,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { */ @Experimental def trigger(trigger: Trigger): DataFrameWriter = { + assertStreaming("trigger() can only be called on continuous queries") this.trigger = trigger this } @@ -236,6 +243,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { */ def save(): Unit = { assertNotBucketed() + assertNotStreaming("save() can only be called on non-continuous queries") val dataSource = DataSource( df.sparkSession, className = source, @@ -253,6 +261,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ def queryName(queryName: String): DataFrameWriter = { + assertStreaming("queryName() can only be called on continuous queries") this.extraOptions += ("queryName" -> queryName) this } @@ -276,6 +285,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ def startStream(): ContinuousQuery = { + assertNotBucketed + assertStreaming("startStream() can only be called on continuous queries") + if (source == "memory") { val queryName = extraOptions.getOrElse( @@ -348,6 +360,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { private def insertInto(tableIdent: TableIdentifier): Unit = { assertNotBucketed() + assertNotStreaming("insertInto() can only be called on non-continuous queries") val partitions = normalizedParCols.map(_.map(col => col -> (None: Option[String])).toMap) val overwrite = mode == SaveMode.Overwrite @@ -446,6 +459,8 @@ final class DataFrameWriter private[sql](df: DataFrame) { } private def saveAsTable(tableIdent: TableIdentifier): Unit = { + assertNotStreaming("saveAsTable() can only be called on non-continuous queries") + val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) (tableExists, mode) match { @@ -486,6 +501,8 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { + assertNotStreaming("jdbc() can only be called on non-continuous queries") + val props = new Properties() extraOptions.foreach { case (key, value) => props.put(key, value) @@ -542,7 +559,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def json(path: String): Unit = format("json").save(path) + def json(path: String): Unit = { + assertNotStreaming("json() can only be called on non-continuous queries") + format("json").save(path) + } /** * Saves the content of the [[DataFrame]] in Parquet format at the specified path. @@ -558,7 +578,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def parquet(path: String): Unit = format("parquet").save(path) + def parquet(path: String): Unit = { + assertNotStreaming("parquet() can only be called on non-continuous queries") + format("parquet").save(path) + } /** * Saves the content of the [[DataFrame]] in ORC format at the specified path. @@ -575,7 +598,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.5.0 * @note Currently, this method can only be used together with `HiveContext`. */ - def orc(path: String): Unit = format("orc").save(path) + def orc(path: String): Unit = { + assertNotStreaming("orc() can only be called on non-continuous queries") + format("orc").save(path) + } /** * Saves the content of the [[DataFrame]] in a text file at the specified path. @@ -596,7 +622,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.6.0 */ - def text(path: String): Unit = format("text").save(path) + def text(path: String): Unit = { + assertNotStreaming("text() can only be called on non-continuous queries") + format("text").save(path) + } /** * Saves the content of the [[DataFrame]] in CSV format at the specified path. @@ -620,7 +649,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 2.0.0 */ - def csv(path: String): Unit = format("csv").save(path) + def csv(path: String): Unit = { + assertNotStreaming("csv() can only be called on non-continuous queries") + format("csv").save(path) + } /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options @@ -641,4 +673,21 @@ final class DataFrameWriter private[sql](df: DataFrame) { private var numBuckets: Option[Int] = None private var sortColumnNames: Option[Seq[String]] = None + + /////////////////////////////////////////////////////////////////////////////////////// + // Helper functions + /////////////////////////////////////////////////////////////////////////////////////// + + private def assertNotStreaming(errMsg: String): Unit = { + if (df.isStreaming) { + throw new AnalysisException(errMsg) + } + } + + private def assertStreaming(errMsg: String): Unit = { + if (!df.isStreaming) { + throw new AnalysisException(errMsg) + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala index 00efe21d39de..c7b2b9982261 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala @@ -368,4 +368,160 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B "org.apache.spark.sql.streaming.test", Map.empty) } + + private def newTextInput = Utils.createTempDir(namePrefix = "text").getCanonicalPath + + test("check trigger() can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.trigger(ProcessingTime("10 seconds"))) + assert(e.getMessage == "trigger() can only be called on continuous queries;") + } + + test("check queryName() can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.queryName("queryName")) + assert(e.getMessage == "queryName() can only be called on continuous queries;") + } + + test("check startStream() can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.startStream()) + assert(e.getMessage == "startStream() can only be called on continuous queries;") + } + + test("check startStream(path) can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.startStream("non_exist_path")) + assert(e.getMessage == "startStream() can only be called on continuous queries;") + } + + test("check mode(SaveMode) can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.mode(SaveMode.Append)) + assert(e.getMessage == "mode() can only be called on non-continuous queries;") + } + + test("check mode(string) can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.mode("append")) + assert(e.getMessage == "mode() can only be called on non-continuous queries;") + } + + test("check bucketBy() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[IllegalArgumentException](w.bucketBy(1, "text").startStream()) + assert(e.getMessage == "Currently we don't support writing bucketed data to this data source.") + } + + test("check sortBy() can only be called on non-continuous queries;") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[IllegalArgumentException](w.sortBy("text").startStream()) + assert(e.getMessage == "Currently we don't support writing bucketed data to this data source.") + } + + test("check save(path) can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.save("non_exist_path")) + assert(e.getMessage == "save() can only be called on non-continuous queries;") + } + + test("check save() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.save()) + assert(e.getMessage == "save() can only be called on non-continuous queries;") + } + + test("check insertInto() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.insertInto("non_exsit_table")) + assert(e.getMessage == "insertInto() can only be called on non-continuous queries;") + } + + test("check saveAsTable() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.saveAsTable("non_exsit_table")) + assert(e.getMessage == "saveAsTable() can only be called on non-continuous queries;") + } + + test("check jdbc() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.jdbc(null, null, null)) + assert(e.getMessage == "jdbc() can only be called on non-continuous queries;") + } + + test("check json() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.json("non_exist_path")) + assert(e.getMessage == "json() can only be called on non-continuous queries;") + } + + test("check parquet() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.parquet("non_exist_path")) + assert(e.getMessage == "parquet() can only be called on non-continuous queries;") + } + + test("check orc() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.orc("non_exist_path")) + assert(e.getMessage == "orc() can only be called on non-continuous queries;") + } + + test("check text() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.text("non_exist_path")) + assert(e.getMessage == "text() can only be called on non-continuous queries;") + } + + test("check csv() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.csv("non_exist_path")) + assert(e.getMessage == "csv() can only be called on non-continuous queries;") + } } From a79797149423568128301507026d7675a6aa6ecb Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 2 May 2016 17:50:40 -0700 Subject: [PATCH 016/313] [SPARK-15050][SQL] Put CSV and JSON options as Python csv and json function parameters ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-15050 This PR adds function parameters for Python API for reading and writing `csv()`. ## How was this patch tested? This was tested by `./dev/run_tests`. Author: hyukjinkwon Author: Hyukjin Kwon Closes #12834 from HyukjinKwon/SPARK-15050. (cherry picked from commit d37c7f7f042f7943b5b684e53cf4284c601fb347) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 232 +++++++++++++++++++++---------- 1 file changed, 155 insertions(+), 77 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index cc5e93dcadf4..e2ee9db04948 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -166,7 +166,10 @@ def stream(self, path=None, format=None, schema=None, **options): return self._df(self._jreader.stream()) @since(1.4) - def json(self, path, schema=None): + def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, + allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, + allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, + mode=None, columnNameOfCorruptRecord=None): """ Loads a JSON file (one object per line) or an RDD of Strings storing JSON objects (one object per record) and returns the result as a :class`DataFrame`. @@ -177,31 +180,36 @@ def json(self, path, schema=None): :param path: string represents path to the JSON dataset, or RDD of Strings storing JSON objects. :param schema: an optional :class:`StructType` for the input schema. + :param primitivesAsString: infers all primitive values as a string type. If None is set, + it uses the default value, ``false``. + :param prefersDecimal: infers all floating-point values as a decimal type. If the values + do not fit in decimal, then it infers them as doubles. If None is + set, it uses the default value, ``false``. + :param allowComments: ignores Java/C++ style comment in JSON records. If None is set, + it uses the default value, ``false``. + :param allowUnquotedFieldNames: allows unquoted JSON field names. If None is set, + it uses the default value, ``false``. + :param allowSingleQuotes: allows single quotes in addition to double quotes. If None is + set, it uses the default value, ``true``. + :param allowNumericLeadingZero: allows leading zeros in numbers (e.g. 00012). If None is + set, it uses the default value, ``false``. + :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character + using backslash quoting mechanism. If None is + set, it uses the default value, ``false``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. - You can set the following JSON-specific options to deal with non-standard JSON files: - * ``primitivesAsString`` (default ``false``): infers all primitive values as a string \ - type - * `prefersDecimal` (default `false`): infers all floating-point values as a decimal \ - type. If the values do not fit in decimal, then it infers them as doubles. - * ``allowComments`` (default ``false``): ignores Java/C++ style comment in JSON records - * ``allowUnquotedFieldNames`` (default ``false``): allows unquoted JSON field names - * ``allowSingleQuotes`` (default ``true``): allows single quotes in addition to double \ - quotes - * ``allowNumericLeadingZeros`` (default ``false``): allows leading zeros in numbers \ - (e.g. 00012) - * ``allowBackslashEscapingAnyCharacter`` (default ``false``): allows accepting quoting \ - of all character using backslash quoting mechanism - * ``mode`` (default ``PERMISSIVE``): allows a mode for dealing with corrupt records \ - during parsing. * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted \ record and puts the malformed string into a new field configured by \ ``columnNameOfCorruptRecord``. When a schema is set by user, it sets \ ``null`` for extra fields. * ``DROPMALFORMED`` : ignores the whole corrupted records. * ``FAILFAST`` : throws an exception when it meets corrupted records. - * ``columnNameOfCorruptRecord`` (default ``_corrupt_record``): allows renaming the \ - new field having malformed string created by ``PERMISSIVE`` mode. \ - This overrides ``spark.sql.columnNameOfCorruptRecord``. + + :param columnNameOfCorruptRecord: allows renaming the new field having malformed string + created by ``PERMISSIVE`` mode. This overrides + ``spark.sql.columnNameOfCorruptRecord``. If None is set, + it uses the default value ``_corrupt_record``. >>> df1 = sqlContext.read.json('python/test_support/sql/people.json') >>> df1.dtypes @@ -214,6 +222,24 @@ def json(self, path, schema=None): """ if schema is not None: self.schema(schema) + if primitivesAsString is not None: + self.option("primitivesAsString", primitivesAsString) + if prefersDecimal is not None: + self.option("prefersDecimal", prefersDecimal) + if allowComments is not None: + self.option("allowComments", allowComments) + if allowUnquotedFieldNames is not None: + self.option("allowUnquotedFieldNames", allowUnquotedFieldNames) + if allowSingleQuotes is not None: + self.option("allowSingleQuotes", allowSingleQuotes) + if allowNumericLeadingZero is not None: + self.option("allowNumericLeadingZero", allowNumericLeadingZero) + if allowBackslashEscapingAnyCharacter is not None: + self.option("allowBackslashEscapingAnyCharacter", allowBackslashEscapingAnyCharacter) + if mode is not None: + self.option("mode", mode) + if columnNameOfCorruptRecord is not None: + self.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) if isinstance(path, basestring): return self._df(self._jreader.json(path)) elif type(path) == list: @@ -270,53 +296,62 @@ def text(self, paths): [Row(value=u'hello'), Row(value=u'this')] """ if isinstance(paths, basestring): - paths = [paths] - return self._df(self._jreader.text(self._sqlContext._sc._jvm.PythonUtils.toSeq(paths))) + path = [paths] + return self._df(self._jreader.text(self._sqlContext._sc._jvm.PythonUtils.toSeq(path))) @since(2.0) - def csv(self, paths): + def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, + comment=None, header=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, + nullValue=None, nanValue=None, positiveInf=None, negativeInf=None, dateFormat=None, + maxColumns=None, maxCharsPerColumn=None, mode=None): """Loads a CSV file and returns the result as a [[DataFrame]]. This function goes through the input once to determine the input schema. To avoid going through the entire data once, specify the schema explicitly using [[schema]]. - :param paths: string, or list of strings, for input path(s). - - You can set the following CSV-specific options to deal with CSV files: - * ``sep`` (default ``,``): sets the single character as a separator \ - for each field and value. - * ``charset`` (default ``UTF-8``): decodes the CSV files by the given \ - encoding type. - * ``quote`` (default ``"``): sets the single character used for escaping \ - quoted values where the separator can be part of the value. - * ``escape`` (default ``\``): sets the single character used for escaping quotes \ - inside an already quoted value. - * ``comment`` (default empty string): sets the single character used for skipping \ - lines beginning with this character. By default, it is disabled. - * ``header`` (default ``false``): uses the first line as names of columns. - * ``ignoreLeadingWhiteSpace`` (default ``false``): defines whether or not leading \ - whitespaces from values being read should be skipped. - * ``ignoreTrailingWhiteSpace`` (default ``false``): defines whether or not trailing \ - whitespaces from values being read should be skipped. - * ``nullValue`` (default empty string): sets the string representation of a null value. - * ``nanValue`` (default ``NaN``): sets the string representation of a non-number \ - value. - * ``positiveInf`` (default ``Inf``): sets the string representation of a positive \ - infinity value. - * ``negativeInf`` (default ``-Inf``): sets the string representation of a negative \ - infinity value. - * ``dateFormat`` (default ``None``): sets the string that indicates a date format. \ - Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This \ - applies to both date type and timestamp type. By default, it is None which means \ - trying to parse times and date by ``java.sql.Timestamp.valueOf()`` and \ - ``java.sql.Date.valueOf()``. - * ``maxColumns`` (default ``20480``): defines a hard limit of how many columns \ - a record can have. - * ``maxCharsPerColumn`` (default ``1000000``): defines the maximum number of \ - characters allowed for any given value being read. - * ``mode`` (default ``PERMISSIVE``): allows a mode for dealing with corrupt records \ - during parsing. - * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. \ + :param path: string, or list of strings, for input path(s). + :param schema: an optional :class:`StructType` for the input schema. + :param sep: sets the single character as a separator for each field and value. + If None is set, it uses the default value, ``,``. + :param encoding: decodes the CSV files by the given encoding type. If None is set, + it uses the default value, ``UTF-8``. + :param quote: sets the single character used for escaping quoted values where the + separator can be part of the value. If None is set, it uses the default + value, ``"``. + :param escape: sets the single character used for escaping quotes inside an already + quoted value. If None is set, it uses the default value, ``\``. + :param comment: sets the single character used for skipping lines beginning with this + character. By default (None), it is disabled. + :param header: uses the first line as names of columns. If None is set, it uses the + default value, ``false``. + :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param ignoreTrailingWhiteSpace: defines whether or not trailing whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. + :param nanValue: sets the string representation of a non-number value. If None is set, it + uses the default value, ``NaN``. + :param positiveInf: sets the string representation of a positive infinity value. If None + is set, it uses the default value, ``Inf``. + :param negativeInf: sets the string representation of a negative infinity value. If None + is set, it uses the default value, ``Inf``. + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to both date type and timestamp type. By default, it is None + which means trying to parse times and date by + ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. + :param maxColumns: defines a hard limit of how many columns a record can have. If None is + set, it uses the default value, ``20480``. + :param maxCharsPerColumn: defines the maximum number of characters allowed for any given + value being read. If None is set, it uses the default value, + ``1000000``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. + + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. When a schema is set by user, it sets ``null`` for extra fields. * ``DROPMALFORMED`` : ignores the whole corrupted records. * ``FAILFAST`` : throws an exception when it meets corrupted records. @@ -325,9 +360,43 @@ def csv(self, paths): >>> df.dtypes [('C0', 'string'), ('C1', 'string')] """ - if isinstance(paths, basestring): - paths = [paths] - return self._df(self._jreader.csv(self._sqlContext._sc._jvm.PythonUtils.toSeq(paths))) + if schema is not None: + self.schema(schema) + if sep is not None: + self.option("sep", sep) + if encoding is not None: + self.option("encoding", encoding) + if quote is not None: + self.option("quote", quote) + if escape is not None: + self.option("escape", escape) + if comment is not None: + self.option("comment", comment) + if header is not None: + self.option("header", header) + if ignoreLeadingWhiteSpace is not None: + self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) + if ignoreTrailingWhiteSpace is not None: + self.option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) + if nullValue is not None: + self.option("nullValue", nullValue) + if nanValue is not None: + self.option("nanValue", nanValue) + if positiveInf is not None: + self.option("positiveInf", positiveInf) + if negativeInf is not None: + self.option("negativeInf", negativeInf) + if dateFormat is not None: + self.option("dateFormat", dateFormat) + if maxColumns is not None: + self.option("maxColumns", maxColumns) + if maxCharsPerColumn is not None: + self.option("maxCharsPerColumn", maxCharsPerColumn) + if mode is not None: + self.option("mode", mode) + if isinstance(path, basestring): + path = [path] + return self._df(self._jreader.csv(self._sqlContext._sc._jvm.PythonUtils.toSeq(path))) @since(1.5) def orc(self, path): @@ -687,7 +756,8 @@ def text(self, path, compression=None): self._jwrite.text(path) @since(2.0) - def csv(self, path, mode=None, compression=None): + def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, + header=None, nullValue=None): """Saves the content of the [[DataFrame]] in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -701,25 +771,33 @@ def csv(self, path, mode=None, compression=None): :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). - - You can set the following CSV-specific options to deal with CSV files: - * ``sep`` (default ``,``): sets the single character as a separator \ - for each field and value. - * ``quote`` (default ``"``): sets the single character used for escaping \ - quoted values where the separator can be part of the value. - * ``escape`` (default ``\``): sets the single character used for escaping quotes \ - inside an already quoted value. - * ``header`` (default ``false``): writes the names of columns as the first line. - * ``nullValue`` (default empty string): sets the string representation of a null value. - * ``compression``: compression codec to use when saving to file. This can be one of \ - the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and \ - deflate). + :param sep: sets the single character as a separator for each field and value. If None is + set, it uses the default value, ``,``. + :param quote: sets the single character used for escaping quoted values where the + separator can be part of the value. If None is set, it uses the default + value, ``"``. + :param escape: sets the single character used for escaping quotes inside an already + quoted value. If None is set, it uses the default value, ``\`` + :param header: writes the names of columns as the first line. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) if compression is not None: self.option("compression", compression) + if sep is not None: + self.option("sep", sep) + if quote is not None: + self.option("quote", quote) + if escape is not None: + self.option("escape", escape) + if header is not None: + self.option("header", header) + if nullValue is not None: + self.option("nullValue", nullValue) self._jwrite.csv(path) @since(1.5) @@ -797,7 +875,7 @@ def _test(): globs['sqlContext'] = SQLContext(sc) globs['hiveContext'] = HiveContext._createForTesting(sc) globs['df'] = globs['sqlContext'].read.parquet('python/test_support/sql/parquet_partitioned') - globs['sdf'] =\ + globs['sdf'] = \ globs['sqlContext'].read.format('text').stream('python/test_support/sql/streaming') (failure_count, test_count) = doctest.testmod( From 86167968f7dea8a44fae2d7bdb0bfe8d735e5004 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 2 May 2016 18:12:31 -0700 Subject: [PATCH 017/313] [SPARK-15047][SQL] Cleanup SQL Parser ## What changes were proposed in this pull request? This PR addresses a few minor issues in SQL parser: - Removes some unused rules and keywords in the grammar. - Removes code path for fallback SQL parsing (was needed for Hive native parsing). - Use `UnresolvedGenerator` instead of hard-coding `Explode` & `JsonTuple`. - Adds a more generic way of creating error messages for unsupported Hive features. - Use `visitFunctionName` as much as possible. - Interpret a `CatalogColumn`'s `DataType` directly instead of parsing it again. ## How was this patch tested? Existing tests. Author: Herman van Hovell Closes #12826 from hvanhovell/SPARK-15047. (cherry picked from commit 1c19c2769edecaefabc2cd67b3b32f901feb3f59) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 35 +++---------------- .../sql/catalyst/parser/AstBuilder.scala | 31 +++------------- .../sql/catalyst/parser/ParseDriver.scala | 10 ++---- .../sql/catalyst/parser/PlanParserSuite.scala | 15 ++++---- .../spark/sql/execution/SparkSqlParser.scala | 12 ++++--- .../spark/sql/hive/HiveDDLCommandSuite.scala | 5 +-- 6 files changed, 31 insertions(+), 77 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4d5d125ecdd7..cc4e5c853e67 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -121,17 +121,13 @@ statement | UNCACHE TABLE identifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE - tableIdentifier partitionSpec? #loadData + tableIdentifier partitionSpec? #loadData + | TRUNCATE TABLE tableIdentifier partitionSpec? + (COLUMNS identifierList)? #truncateTable | ADD identifier .*? #addResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration - | kws=unsupportedHiveNativeCommands .*? #failNativeCommand - | hiveNativeCommands #executeNativeCommand - ; - -hiveNativeCommands - : TRUNCATE TABLE tableIdentifier partitionSpec? - (COLUMNS identifierList)? + | unsupportedHiveNativeCommands .*? #failNativeCommand ; unsupportedHiveNativeCommands @@ -267,14 +263,6 @@ nestedConstantList : '(' constantList (',' constantList)* ')' ; -skewedLocation - : (constant | constantList) EQ STRING - ; - -skewedLocationList - : '(' skewedLocation (',' skewedLocation)* ')' - ; - createFileFormat : STORED AS fileFormat | STORED BY storageHandler @@ -609,11 +597,6 @@ explainOption : LOGICAL | FORMATTED | EXTENDED | CODEGEN ; -transactionMode - : ISOLATION LEVEL SNAPSHOT #isolationLevel - | READ accessMode=(ONLY | WRITE) #transactionAccessMode - ; - qualifiedName : identifier ('.' identifier)* ; @@ -661,8 +644,7 @@ nonReserved | VIEW | REPLACE | IF | NO | DATA - | START | TRANSACTION | COMMIT | ROLLBACK | WORK | ISOLATION | LEVEL - | SNAPSHOT | READ | WRITE | ONLY + | START | TRANSACTION | COMMIT | ROLLBACK | SORT | CLUSTER | DISTRIBUTE | UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE | FIRST | AFTER | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT @@ -778,13 +760,6 @@ START: 'START'; TRANSACTION: 'TRANSACTION'; COMMIT: 'COMMIT'; ROLLBACK: 'ROLLBACK'; -WORK: 'WORK'; -ISOLATION: 'ISOLATION'; -LEVEL: 'LEVEL'; -SNAPSHOT: 'SNAPSHOT'; -READ: 'READ'; -WRITE: 'WRITE'; -ONLY: 'ONLY'; MACRO: 'MACRO'; IF: 'IF'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1f923f47dd0b..c3974625aa22 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -81,26 +81,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * ******************************************************************************************** */ protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree) - /** - * Make sure we do not try to create a plan for a native command. - */ - override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null - /** * Create a plan for a SHOW FUNCTIONS command. */ override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { import ctx._ if (qualifiedName != null) { - val names = qualifiedName().identifier().asScala.map(_.getText).toList - names match { - case db :: name :: Nil => - ShowFunctions(Some(db), Some(name)) - case name :: Nil => - ShowFunctions(None, Some(name)) - case _ => - throw new ParseException("SHOW FUNCTIONS unsupported name", ctx) - } + val name = visitFunctionName(qualifiedName) + ShowFunctions(name.database, Some(name.funcName)) } else if (pattern != null) { ShowFunctions(None, Some(string(pattern))) } else { @@ -117,7 +105,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { if (describeFuncName.STRING() != null) { string(describeFuncName.STRING()) } else if (describeFuncName.qualifiedName() != null) { - describeFuncName.qualifiedName().identifier().asScala.map(_.getText).mkString(".") + visitFunctionName(describeFuncName.qualifiedName).unquotedString } else { describeFuncName.getText } @@ -554,19 +542,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { query: LogicalPlan, ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) { val expressions = expressionList(ctx.expression) - - // Create the generator. - val generator = ctx.qualifiedName.getText.toLowerCase match { - case "explode" if expressions.size == 1 => - Explode(expressions.head) - case "json_tuple" => - JsonTuple(expressions) - case name => - UnresolvedGenerator(visitFunctionName(ctx.qualifiedName), expressions) - } - Generate( - generator, + UnresolvedGenerator(visitFunctionName(ctx.qualifiedName), expressions), join = true, outer = ctx.OUTER != null, Some(ctx.tblName.getText.toLowerCase), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index d0132529f18e..d042e191a9e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -53,19 +53,15 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser => astBuilder.visitSingleStatement(parser.singleStatement()) match { case plan: LogicalPlan => plan - case _ => nativeCommand(sqlText) + case _ => + val position = Origin(None, None) + throw new ParseException(Option(sqlText), "Unsupported SQL statement", position, position) } } /** Get the builder (visitor) which converts a ParseTree into a AST. */ protected def astBuilder: AstBuilder - /** Create a native command, or fail when this is not supported. */ - protected def nativeCommand(sqlText: String): LogicalPlan = { - val position = Origin(None, None) - throw new ParseException(Option(sqlText), "Unsupported SQL statement", position, position) - } - protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = { logInfo(s"Parsing command: $command") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 5e896a33bd3b..b7af2ceda643 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -53,7 +53,7 @@ class PlanParserSuite extends PlanTest { assertEqual("show functions foo", ShowFunctions(None, Some("foo"))) assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar"))) assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*"))) - intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name") + intercept("show functions foo.bar.baz", "Unsupported function name") } test("describe function") { @@ -263,11 +263,14 @@ class PlanParserSuite extends PlanTest { } test("lateral view") { + val explode = UnresolvedGenerator(FunctionIdentifier("explode"), Seq('x)) + val jsonTuple = UnresolvedGenerator(FunctionIdentifier("json_tuple"), Seq('x, 'y)) + // Single lateral view assertEqual( "select * from t lateral view explode(x) expl as x", table("t") - .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x")) + .generate(explode, join = true, outer = false, Some("expl"), Seq("x")) .select(star())) // Multiple lateral views @@ -277,12 +280,12 @@ class PlanParserSuite extends PlanTest { |lateral view explode(x) expl |lateral view outer json_tuple(x, y) jtup q, z""".stripMargin, table("t") - .generate(Explode('x), join = true, outer = false, Some("expl"), Seq.empty) - .generate(JsonTuple(Seq('x, 'y)), join = true, outer = true, Some("jtup"), Seq("q", "z")) + .generate(explode, join = true, outer = false, Some("expl"), Seq.empty) + .generate(jsonTuple, join = true, outer = true, Some("jtup"), Seq("q", "z")) .select(star())) // Multi-Insert lateral views. - val from = table("t1").generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x")) + val from = table("t1").generate(explode, join = true, outer = false, Some("expl"), Seq("x")) assertEqual( """from t1 |lateral view explode(x) expl as x @@ -294,7 +297,7 @@ class PlanParserSuite extends PlanTest { |where s < 10 """.stripMargin, Union(from - .generate(JsonTuple(Seq('x, 'y)), join = true, outer = false, Some("jtup"), Seq("q", "z")) + .generate(jsonTuple, join = true, outer = false, Some("jtup"), Seq("q", "z")) .select(star()) .insertInto("t2"), from.where('s < 10).select(star()).insertInto("t3"))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8128a6efe3cc..dfc56a7d98ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import scala.util.Try import org.antlr.v4.runtime.{ParserRuleContext, Token} +import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier @@ -31,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} - +import org.apache.spark.sql.types.DataType /** * Concrete parser for Spark SQL statements. @@ -780,9 +781,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitFailNativeCommand( ctx: FailNativeCommandContext): LogicalPlan = withOrigin(ctx) { - val keywords = if (ctx.kws != null) { - Seq(ctx.kws.kw1, ctx.kws.kw2, ctx.kws.kw3, ctx.kws.kw4, ctx.kws.kw5, ctx.kws.kw6) - .filter(_ != null).map(_.getText).mkString(" ") + val keywords = if (ctx.unsupportedHiveNativeCommands != null) { + ctx.unsupportedHiveNativeCommands.children.asScala.collect { + case n: TerminalNode => n.getText + }.mkString(" ") } else { // SET ROLE is the exception to the rule, because we handle this before other SET commands. "SET ROLE" @@ -1109,7 +1111,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // just convert the whole type string to lower case, otherwise the struct field names // will no longer be case sensitive. Instead, we rely on our parser to get the proper // case before passing it to Hive. - CatalystSqlParser.parseDataType(col.dataType.getText).catalogString, + typedVisit[DataType](col.dataType).catalogString, nullable = true, Option(col.STRING).map(string)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index c97c28c40c96..8dc3c6435327 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -254,12 +254,13 @@ class HiveDDLCommandSuite extends PlanTest { } test("use native json_tuple instead of hive's UDTF in LATERAL VIEW") { - val plan = parser.parsePlan( + val analyzer = TestHive.sparkSession.sessionState.analyzer + val plan = analyzer.execute(parser.parsePlan( """ |SELECT * |FROM (SELECT '{"f1": "value1", "f2": 12}' json) test |LATERAL VIEW json_tuple(json, 'f1', 'f2') jt AS a, b - """.stripMargin) + """.stripMargin)) assert(plan.children.head.asInstanceOf[Generate].generator.isInstanceOf[JsonTuple]) } From 733cbaa3c0ff617a630a9d6937699db37ad2943b Mon Sep 17 00:00:00 2001 From: bomeng Date: Mon, 2 May 2016 18:20:29 -0700 Subject: [PATCH 018/313] [SPARK-15062][SQL] fix list type infer serializer issue ## What changes were proposed in this pull request? Make serializer correctly inferred if the input type is `List[_]`, since `List[_]` is type of `Seq[_]`, before it was matched to different case (`case t if definedByConstructorParams(t)`). ## How was this patch tested? New test case was added. Author: bomeng Closes #12849 from bomeng/SPARK-15062. (cherry picked from commit 0fd95be3cd815154a11ce7d6998311e7c86bc6b9) Signed-off-by: Michael Armbrust --- .../spark/sql/catalyst/ScalaReflection.scala | 11 +++++++---- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 15 ++++++++++++++- 2 files changed, 21 insertions(+), 5 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 be0d75a8304e..d158a64a85bc 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 @@ -509,6 +509,13 @@ object ScalaReflection extends ScalaReflection { serializerFor(unwrapped, optType, newPath)) } + // Since List[_] also belongs to localTypeOf[Product], we put this case before + // "case t if definedByConstructorParams(t)" to make sure it will match to the + // case "localTypeOf[Seq[_]]" + case t if t <:< localTypeOf[Seq[_]] => + val TypeRef(_, _, Seq(elementType)) = t + toCatalystArray(inputObject, elementType) + case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => @@ -524,10 +531,6 @@ object ScalaReflection extends ScalaReflection { val TypeRef(_, _, Seq(elementType)) = t toCatalystArray(inputObject, elementType) - case t if t <:< localTypeOf[Seq[_]] => - val TypeRef(_, _, Seq(elementType)) = t - toCatalystArray(inputObject, elementType) - case t if t <:< localTypeOf[Map[_, _]] => val TypeRef(_, _, Seq(keyType, valueType)) = t diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 0672551b2972..c00e9c7e3956 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -23,8 +23,9 @@ import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.typeOf import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{BoundReference, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, NewInstance, SpecificMutableRow} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils case class PrimitiveData( @@ -277,6 +278,18 @@ class ScalaReflectionSuite extends SparkFunSuite { assert(anyTypes === Seq(classOf[java.lang.Object], classOf[java.lang.Object])) } + test("SPARK-15062: Get correct serializer for List[_]") { + val list = List(1, 2, 3) + val serializer = serializerFor[List[Int]](BoundReference( + 0, ObjectType(list.getClass), nullable = false)) + assert(serializer.children.size == 2) + assert(serializer.children.head.isInstanceOf[Literal]) + assert(serializer.children.head.asInstanceOf[Literal].value === UTF8String.fromString("value")) + assert(serializer.children.last.isInstanceOf[NewInstance]) + assert(serializer.children.last.asInstanceOf[NewInstance] + .cls.isInstanceOf[Class[org.apache.spark.sql.catalyst.util.GenericArrayData]]) + } + private val dataTypeForComplexData = dataTypeFor[ComplexData] private val typeOfComplexData = typeOf[ComplexData] From dcce0aaafedc496e3e69c02c51ad31f01de05287 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 2 May 2016 18:27:49 -0700 Subject: [PATCH 019/313] [SPARK-15077][SQL] Use a fair lock to avoid thread starvation in StreamExecution ## What changes were proposed in this pull request? Right now `StreamExecution.awaitBatchLock` uses an unfair lock. `StreamExecution.awaitOffset` may run too long and fail some test because `StreamExecution.constructNextBatch` keeps getting the lock. See: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/865/testReport/junit/org.apache.spark.sql.streaming/FileStreamSourceStressTestSuite/file_source_stress_test/ This PR uses a fair ReentrantLock to resolve the thread starvation issue. ## How was this patch tested? Modified `FileStreamSourceStressTestSuite.test("file source stress test")` to run the test codes 100 times locally. It always fails because of timeout without this patch. Author: Shixiong Zhu Closes #12852 from zsxwing/SPARK-15077. (cherry picked from commit 4e3685ae5e5826e63bfcd7c3729e3b9cbab484b5) Signed-off-by: Michael Armbrust --- .../execution/streaming/StreamExecution.scala | 79 ++++++++++++------- 1 file changed, 52 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ce68c0968fb6..310834691380 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal @@ -53,8 +54,12 @@ class StreamExecution( val trigger: Trigger) extends ContinuousQuery with Logging { - /** An monitor used to wait/notify when batches complete. */ - private val awaitBatchLock = new Object + /** + * A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation. + */ + private val awaitBatchLock = new ReentrantLock(true) + private val awaitBatchLockCondition = awaitBatchLock.newCondition() + private val startLatch = new CountDownLatch(1) private val terminationLatch = new CountDownLatch(1) @@ -242,17 +247,22 @@ class StreamExecution( // method. See SPARK-14131. // // Check to see what new data is available. - val hasNewData = awaitBatchLock.synchronized { - val newData = microBatchThread.runUninterruptibly { - uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - } - availableOffsets ++= newData + val hasNewData = { + awaitBatchLock.lock() + try { + val newData = microBatchThread.runUninterruptibly { + uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) + } + availableOffsets ++= newData - if (dataAvailable) { - true - } else { - noNewData = true - false + if (dataAvailable) { + true + } else { + noNewData = true + false + } + } finally { + awaitBatchLock.unlock() } } if (hasNewData) { @@ -269,9 +279,12 @@ class StreamExecution( currentBatchId += 1 logInfo(s"Committed offsets for batch $currentBatchId.") } else { - awaitBatchLock.synchronized { + awaitBatchLock.lock() + try { // Wake up any threads that are waiting for the stream to progress. - awaitBatchLock.notifyAll() + awaitBatchLockCondition.signalAll() + } finally { + awaitBatchLock.unlock() } } } @@ -332,9 +345,12 @@ class StreamExecution( new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) sink.addBatch(currentBatchId - 1, nextBatch) - awaitBatchLock.synchronized { + awaitBatchLock.lock() + try { // Wake up any threads that are waiting for the stream to progress. - awaitBatchLock.notifyAll() + awaitBatchLockCondition.signalAll() + } finally { + awaitBatchLock.unlock() } val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 @@ -374,8 +390,12 @@ class StreamExecution( } while (notDone) { - logInfo(s"Waiting until $newOffset at $source") - awaitBatchLock.synchronized { awaitBatchLock.wait(100) } + awaitBatchLock.lock() + try { + awaitBatchLockCondition.await(100, TimeUnit.MILLISECONDS) + } finally { + awaitBatchLock.unlock() + } } logDebug(s"Unblocked at $newOffset for $source") } @@ -383,16 +403,21 @@ class StreamExecution( /** A flag to indicate that a batch has completed with no new data available. */ @volatile private var noNewData = false - override def processAllAvailable(): Unit = awaitBatchLock.synchronized { - noNewData = false - while (true) { - awaitBatchLock.wait(10000) - if (streamDeathCause != null) { - throw streamDeathCause - } - if (noNewData) { - return + override def processAllAvailable(): Unit = { + awaitBatchLock.lock() + try { + noNewData = false + while (true) { + awaitBatchLockCondition.await(10000, TimeUnit.MILLISECONDS) + if (streamDeathCause != null) { + throw streamDeathCause + } + if (noNewData) { + return + } } + } finally { + awaitBatchLock.unlock() } } From 435d903d3f3d26514d7d9b986ec88a3bd69a4df3 Mon Sep 17 00:00:00 2001 From: Marcin Tustin Date: Mon, 2 May 2016 19:37:57 -0700 Subject: [PATCH 020/313] [SPARK-14685][CORE] Document heritability of localProperties ## What changes were proposed in this pull request? This updates the java-/scala- doc for setLocalProperty to document heritability of localProperties. This also adds tests for that behaviour. ## How was this patch tested? Tests pass. New tests were added. Author: Marcin Tustin Closes #12455 from marcintustin/SPARK-14685. (cherry picked from commit 8028f3a0b4003af15ed44d9ef4727b56f4b10534) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/SparkContext.scala | 5 ++++ .../spark/api/java/JavaSparkContext.scala | 9 ++++-- .../org/apache/spark/SparkContextSuite.scala | 28 +++++++++++++++++++ 3 files changed, 40 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 302dec25c66b..58618b41920a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -608,6 +608,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * scheduler pool. User-defined properties may also be set here. These properties are propagated * through to worker tasks and can be accessed there via * [[org.apache.spark.TaskContext#getLocalProperty]]. + * + * These properties are inherited by child threads spawned from this thread. This + * may have unexpected consequences when working with thread pools. The standard java + * implementation of thread pools have worker threads spawn other worker threads. + * As a result, local properties may propagate unpredictably. */ def setLocalProperty(key: String, value: String) { if (value == null) { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index dfd91ae338e8..fb6323413e3e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -712,8 +712,13 @@ class JavaSparkContext(val sc: SparkContext) } /** - * Set a local property that affects jobs submitted from this thread, such as the - * Spark fair scheduler pool. + * Set a local property that affects jobs submitted from this thread, and all child + * threads, such as the Spark fair scheduler pool. + * + * These properties are inherited by child threads spawned from this thread. This + * may have unexpected consequences when working with thread pools. The standard java + * implementation of thread pools have worker threads spawn other worker threads. + * As a result, local properties may propagate unpredictably. */ def setLocalProperty(key: String, value: String): Unit = sc.setLocalProperty(key, value) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a759f364fe05..63987084ffc0 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -323,4 +323,32 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { assert(sc.getConf.getInt("spark.executor.instances", 0) === 6) } } + + + test("localProperties are inherited by spawned threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.setLocalProperty("testProperty", "testValue") + var result = "unset"; + val thread = new Thread() { override def run() = {result = sc.getLocalProperty("testProperty")}} + thread.start() + thread.join() + sc.stop() + assert(result == "testValue") + } + + test("localProperties do not cross-talk between threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + var result = "unset"; + val thread1 = new Thread() { + override def run() = {sc.setLocalProperty("testProperty", "testValue")}} + // testProperty should be unset and thus return null + val thread2 = new Thread() { + override def run() = {result = sc.getLocalProperty("testProperty")}} + thread1.start() + thread1.join() + thread2.start() + thread2.join() + sc.stop() + assert(result == null) + } } From a7e8cfa64de26be2e517e2eda237a9e8a58008c5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 May 2016 21:12:48 -0700 Subject: [PATCH 021/313] [SPARK-15079] Support average/count/sum in Long/DoubleAccumulator ## What changes were proposed in this pull request? This patch removes AverageAccumulator and adds the ability to compute average to LongAccumulator and DoubleAccumulator. The patch also improves documentation for the two accumulators. ## How was this patch tested? Added unit tests for this. Author: Reynold Xin Closes #12858 from rxin/SPARK-15079. (cherry picked from commit bb9ab56b960153d374d7e8838f62a18e7e45481e) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/Accumulator.scala | 17 --- .../org/apache/spark/AccumulatorV2.scala | 137 ++++++++++++------ .../scala/org/apache/spark/SparkContext.scala | 22 --- .../org/apache/spark/AccumulatorSuite.scala | 17 +-- .../spark/util/AccumulatorV2Suite.scala | 89 ++++++++++++ 5 files changed, 181 insertions(+), 101 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index e52d36b7b564..23245043e246 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -17,9 +17,6 @@ package org.apache.spark -import org.apache.spark.storage.{BlockId, BlockStatus} - - /** * A simpler value of [[Accumulable]] where the result type being accumulated is the same * as the types of elements being merged, i.e. variables that are only "added" to through an @@ -117,18 +114,4 @@ object AccumulatorParam { def addInPlace(t1: String, t2: String): String = t2 def zero(initialValue: String): String = "" } - - // Note: this is expensive as it makes a copy of the list every time the caller adds an item. - // A better way to use this is to first accumulate the values yourself then them all at once. - @deprecated("use AccumulatorV2", "2.0.0") - private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { - def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 - def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] - } - - // For the internal metric that records what blocks are updated in a particular task - @deprecated("use AccumulatorV2", "2.0.0") - private[spark] object UpdatedBlockStatusesAccumulatorParam - extends ListAccumulatorParam[(BlockId, BlockStatus)] - } diff --git a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala index c65108a55eba..a6c64fd68057 100644 --- a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala @@ -257,23 +257,66 @@ private[spark] object AccumulatorContext { } +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for 64-bit integers. + * + * @since 2.0.0 + */ class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { private[this] var _sum = 0L + private[this] var _count = 0L - override def isZero: Boolean = _sum == 0 + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def isZero: Boolean = _count == 0L override def copyAndReset(): LongAccumulator = new LongAccumulator - override def add(v: jl.Long): Unit = _sum += v + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def add(v: jl.Long): Unit = { + _sum += v + _count += 1 + } + + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Long): Unit = { + _sum += v + _count += 1 + } - def add(v: Long): Unit = _sum += v + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ def sum: Long = _sum + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum.toDouble / _count + override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match { - case o: LongAccumulator => _sum += o.sum - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + case o: LongAccumulator => + _sum += o.sum + _count += o.count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } private[spark] def setValue(newValue: Long): Unit = _sum = newValue @@ -282,66 +325,68 @@ class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { } +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for double precision + * floating numbers. + * + * @since 2.0.0 + */ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[this] var _sum = 0.0 - - override def isZero: Boolean = _sum == 0.0 - - override def copyAndReset(): DoubleAccumulator = new DoubleAccumulator - - override def add(v: jl.Double): Unit = _sum += v - - def add(v: Double): Unit = _sum += v - - def sum: Double = _sum - - override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { - case o: DoubleAccumulator => _sum += o.sum - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - - private[spark] def setValue(newValue: Double): Unit = _sum = newValue - - override def localValue: jl.Double = _sum -} - - -class AverageAccumulator extends AccumulatorV2[jl.Double, jl.Double] { - private[this] var _sum = 0.0 private[this] var _count = 0L - override def isZero: Boolean = _sum == 0.0 && _count == 0 + override def isZero: Boolean = _count == 0L - override def copyAndReset(): AverageAccumulator = new AverageAccumulator + override def copyAndReset(): DoubleAccumulator = new DoubleAccumulator + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ override def add(v: jl.Double): Unit = { _sum += v _count += 1 } - def add(d: Double): Unit = { - _sum += d + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Double): Unit = { + _sum += v _count += 1 } + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count + + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ + def sum: Double = _sum + + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum / _count + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { - case o: AverageAccumulator => + case o: DoubleAccumulator => _sum += o.sum _count += o.count - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - - override def localValue: jl.Double = if (_count == 0) { - Double.NaN - } else { - _sum / _count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - def sum: Double = _sum + private[spark] def setValue(newValue: Double): Unit = _sum = newValue - def count: Long = _count + override def localValue: jl.Double = _sum } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 58618b41920a..e39159933607 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1340,28 +1340,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli acc } - /** - * Create and register an average accumulator, which accumulates double inputs by recording the - * total sum and total count, and produce the output by sum / total. Note that Double.NaN will be - * returned if no input is added. - */ - def averageAccumulator: AverageAccumulator = { - val acc = new AverageAccumulator - register(acc) - acc - } - - /** - * Create and register an average accumulator, which accumulates double inputs by recording the - * total sum and total count, and produce the output by sum / total. Note that Double.NaN will be - * returned if no input is added. - */ - def averageAccumulator(name: String): AverageAccumulator = { - val acc = new AverageAccumulator - register(acc, name) - acc - } - /** * Create and register a list accumulator, which starts with empty list and accumulates inputs * by adding them into the inner list. diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 09eb9c1dbdc6..00200962549e 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException -import org.apache.spark.AccumulatorParam.{ListAccumulatorParam, StringAccumulatorParam} +import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer @@ -234,21 +234,6 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex acc.merge("kindness") assert(acc.value === "kindness") } - - test("list accumulator param") { - val acc = new Accumulator(Seq.empty[Int], new ListAccumulatorParam[Int], Some("numbers")) - assert(acc.value === Seq.empty[Int]) - acc.add(Seq(1, 2)) - assert(acc.value === Seq(1, 2)) - acc += Seq(3, 4) - assert(acc.value === Seq(1, 2, 3, 4)) - acc ++= Seq(5, 6) - assert(acc.value === Seq(1, 2, 3, 4, 5, 6)) - acc.merge(Seq(7, 8)) - assert(acc.value === Seq(1, 2, 3, 4, 5, 6, 7, 8)) - acc.setValue(Seq(9, 10)) - assert(acc.value === Seq(9, 10)) - } } private[spark] object AccumulatorSuite { diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala new file mode 100644 index 000000000000..41cdd0249226 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -0,0 +1,89 @@ +/* + * 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.util + +import org.apache.spark.{DoubleAccumulator, LongAccumulator, SparkFunSuite} + +class AccumulatorV2Suite extends SparkFunSuite { + + test("LongAccumulator add/avg/sum/count/isZero") { + val acc = new LongAccumulator + assert(acc.isZero) + assert(acc.count == 0) + assert(acc.sum == 0) + assert(acc.avg.isNaN) + + acc.add(0) + assert(!acc.isZero) + assert(acc.count == 1) + assert(acc.sum == 0) + assert(acc.avg == 0.0) + + acc.add(1) + assert(acc.count == 2) + assert(acc.sum == 1) + assert(acc.avg == 0.5) + + // Also test add using non-specialized add function + acc.add(new java.lang.Long(2)) + assert(acc.count == 3) + assert(acc.sum == 3) + assert(acc.avg == 1.0) + + // Test merging + val acc2 = new LongAccumulator + acc2.add(2) + acc.merge(acc2) + assert(acc.count == 4) + assert(acc.sum == 5) + assert(acc.avg == 1.25) + } + + test("DoubleAccumulator add/avg/sum/count/isZero") { + val acc = new DoubleAccumulator + assert(acc.isZero) + assert(acc.count == 0) + assert(acc.sum == 0.0) + assert(acc.avg.isNaN) + + acc.add(0.0) + assert(!acc.isZero) + assert(acc.count == 1) + assert(acc.sum == 0.0) + assert(acc.avg == 0.0) + + acc.add(1.0) + assert(acc.count == 2) + assert(acc.sum == 1.0) + assert(acc.avg == 0.5) + + // Also test add using non-specialized add function + acc.add(new java.lang.Double(2.0)) + assert(acc.count == 3) + assert(acc.sum == 3.0) + assert(acc.avg == 1.0) + + // Test merging + val acc2 = new DoubleAccumulator + acc2.add(2.0) + acc.merge(acc2) + assert(acc.count == 4) + assert(acc.sum == 5.0) + assert(acc.avg == 1.25) + } +} From 52308103ee9bfb12a505505f6d38f1d09a05208f Mon Sep 17 00:00:00 2001 From: Andrew Ray Date: Mon, 2 May 2016 22:47:32 -0700 Subject: [PATCH 022/313] [SPARK-13749][SQL][FOLLOW-UP] Faster pivot implementation for many distinct values with two phase aggregation ## What changes were proposed in this pull request? This is a follow up PR for #11583. It makes 3 lazy vals into just vals and adds unit test coverage. ## How was this patch tested? Existing unit tests and additional unit tests. Author: Andrew Ray Closes #12861 from aray/fast-pivot-follow-up. (cherry picked from commit d8f528ceb61e3c2ac7ac97cd8147dafbb625932f) Signed-off-by: Yin Huai --- .../expressions/aggregate/PivotFirst.scala | 6 +++--- .../apache/spark/sql/DataFramePivotSuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala index 9154e96e34e9..9ead571c5374 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala @@ -141,12 +141,12 @@ case class PivotFirst( copy(mutableAggBufferOffset = newMutableAggBufferOffset) - override lazy val aggBufferAttributes: Seq[AttributeReference] = + override val aggBufferAttributes: Seq[AttributeReference] = pivotIndex.toList.sortBy(_._2).map(kv => AttributeReference(kv._1.toString, valueDataType)()) - override lazy val aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + override val aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) - override lazy val inputAggBufferAttributes: Seq[AttributeReference] = + override val inputAggBufferAttributes: Seq[AttributeReference] = aggBufferAttributes.map(_.newInstance()) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index b17284aa94d2..c6d67519b0e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -180,4 +180,21 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ ) } + test("pivot with datatype not supported by PivotFirst") { + checkAnswer( + complexData.groupBy().pivot("b", Seq(true, false)).agg(max("a")), + Row(Seq(1, 1, 1), Seq(2, 2, 2)) :: Nil + ) + } + + test("pivot with datatype not supported by PivotFirst 2") { + checkAnswer( + courseSales.withColumn("e", expr("array(earnings, 7.0d)")) + .groupBy("year") + .pivot("course", Seq("dotNET", "Java")) + .agg(min($"e")), + Row(2012, Seq(5000.0, 7.0), Seq(20000.0, 7.0)) :: + Row(2013, Seq(48000.0, 7.0), Seq(30000.0, 7.0)) :: Nil + ) + } } From 27efd92e3683f88233ebe755855dac337069246f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 3 May 2016 00:18:10 -0700 Subject: [PATCH 023/313] [SPARK-6717][ML] Clear shuffle files after checkpointing in ALS ## What changes were proposed in this pull request? When ALS is run with a checkpoint interval, during the checkpoint materialize the current state and cleanup the previous shuffles (non-blocking). ## How was this patch tested? Existing ALS unit tests, new ALS checkpoint cleanup unit tests added & shuffle files checked after ALS w/checkpointing run. Author: Holden Karau Author: Holden Karau Closes #11919 from holdenk/SPARK-6717-clear-shuffle-files-after-checkpointing-in-ALS. --- .../org/apache/spark/ContextCleaner.scala | 2 +- .../apache/spark/ml/recommendation/ALS.scala | 35 ++- .../spark/ml/recommendation/ALSSuite.scala | 203 ++++++++++++++---- 3 files changed, 195 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index a51338c01707..c895fb32069c 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -212,7 +212,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } - /** Perform shuffle cleanup, asynchronously. */ + /** Perform shuffle cleanup. */ def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning shuffle " + shuffleId) diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 541923048a3f..509c944fed74 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.DefaultFormats import org.json4s.JsonDSL._ -import org.apache.spark.Partitioner +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} @@ -706,13 +706,15 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) // TODO: Generalize PeriodicGraphCheckpointer and use it here. + val deps = itemFactors.dependencies if (shouldCheckpoint(iter)) { - itemFactors.checkpoint() // itemFactors gets materialized in computeFactors. + itemFactors.checkpoint() // itemFactors gets materialized in computeFactors } val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) if (shouldCheckpoint(iter)) { + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -723,8 +725,10 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, userLocalIndexEncoder, solver = solver) if (shouldCheckpoint(iter)) { + val deps = itemFactors.dependencies itemFactors.checkpoint() itemFactors.count() // checkpoint item factors and cut lineage + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -1355,4 +1359,31 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * satisfies this requirement, we simply use a type alias here. */ private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner + + /** + * Private function to clean up all of the shuffles files from the dependencies and their parents. + */ + private[spark] def cleanShuffleDependencies[T]( + sc: SparkContext, + deps: Seq[Dependency[_]], + blocking: Boolean = false): Unit = { + // If there is no reference tracking we skip clean up. + sc.cleaner.foreach { cleaner => + /** + * Clean the shuffles & all of its parents. + */ + def cleanEagerly(dep: Dependency[_]): Unit = { + if (dep.isInstanceOf[ShuffleDependency[_, _, _]]) { + val shuffleId = dep.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId + cleaner.doCleanupShuffle(shuffleId, blocking) + } + val rdd = dep.rdd + val rddDeps = rdd.dependencies + if (rdd.getStorageLevel == StorageLevel.NONE && rddDeps != null) { + rddDeps.foreach(cleanEagerly) + } + } + deps.foreach(cleanEagerly) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 4c4eb72cd16e..170403739578 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -17,15 +17,19 @@ package org.apache.spark.ml.recommendation +import java.io.File import java.util.Random import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.ml.recommendation.ALS._ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} @@ -34,8 +38,9 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { @@ -255,37 +260,7 @@ class ALSSuite rank: Int, noiseStd: Double = 0.0, seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { - // The assumption of the implicit feedback model is that unobserved ratings are more likely to - // be negatives. - val positiveFraction = 0.8 - val negativeFraction = 1.0 - positiveFraction - val trainingFraction = 0.6 - val testFraction = 0.3 - val totalFraction = trainingFraction + testFraction - val random = new Random(seed) - val userFactors = genFactors(numUsers, rank, random) - val itemFactors = genFactors(numItems, rank, random) - val training = ArrayBuffer.empty[Rating[Int]] - val test = ArrayBuffer.empty[Rating[Int]] - for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { - val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) - val threshold = if (rating > 0) positiveFraction else negativeFraction - val observed = random.nextDouble() < threshold - if (observed) { - val x = random.nextDouble() - if (x < totalFraction) { - if (x < trainingFraction) { - val noise = noiseStd * random.nextGaussian() - training += Rating(userId, itemId, rating + noise.toFloat) - } else { - test += Rating(userId, itemId, rating) - } - } - } - } - logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + - s"and ${test.size} for test.") - (sc.parallelize(training, 2), sc.parallelize(test, 2)) + ALSSuite.genImplicitTestData(sc, numUsers, numItems, rank, noiseStd, seed) } /** @@ -304,14 +279,7 @@ class ALSSuite random: Random, a: Float = -1.0f, b: Float = 1.0f): Seq[(Int, Array[Float])] = { - require(size > 0 && size < Int.MaxValue / 3) - require(b > a) - val ids = mutable.Set.empty[Int] - while (ids.size < size) { - ids += random.nextInt() - } - val width = b - a - ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + ALSSuite.genFactors(size, rank, random, a, b) } /** @@ -520,6 +488,79 @@ class ALSSuite } } +class ALSCleanerSuite extends SparkFunSuite { + test("ALS shuffle cleanup standalone") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Test checkpoint and clean parents + val input = sc.parallelize(1 to 1000) + val keyed = input.map(x => (x % 20, 1)) + val shuffled = keyed.reduceByKey(_ + _) + val keysOnly = shuffled.keys + val deps = keysOnly.dependencies + keysOnly.count() + ALS.cleanShuffleDependencies(sc, deps, true) + val resultingFiles = getAllFiles + assert(resultingFiles === Set()) + // Ensure running count again works fine even if we kill the shuffle files. + keysOnly.count() + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } + + test("ALS shuffle cleanup in algorithm") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Generate test data + val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0) + // Implicitly test the cleaning of parents during ALS training + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + val als = new ALS() + .setRank(1) + .setRegParam(1e-5) + .setSeed(0) + .setCheckpointInterval(1) + .setMaxIter(7) + val model = als.fit(training.toDF()) + val resultingFiles = getAllFiles + // We expect the last shuffles files, block ratings, user factors, and item factors to be + // around but no more. + val pattern = "shuffle_(\\d+)_.+\\.data".r + val rddIds = resultingFiles.flatMap { f => + pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } + assert(rddIds.toSet.size === 4) + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } +} + class ALSStorageSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { @@ -591,7 +632,7 @@ private class IntermediateRDDStorageListener extends SparkListener { } -object ALSSuite { +object ALSSuite extends Logging { /** * Mapping from all Params to valid settings which differ from the defaults. @@ -620,4 +661,82 @@ object ALSSuite { "intermediateStorageLevel" -> "MEMORY_ONLY", "finalStorageLevel" -> "MEMORY_AND_DISK_SER" ) + + // Helper functions to generate test data we share between ALS test suites + + /** + * Generates random user/item factors, with i.i.d. values drawn from U(a, b). + * @param size number of users/items + * @param rank number of features + * @param random random number generator + * @param a min value of the support (default: -1) + * @param b max value of the support (default: 1) + * @return a sequence of (ID, factors) pairs + */ + private def genFactors( + size: Int, + rank: Int, + random: Random, + a: Float = -1.0f, + b: Float = 1.0f): Seq[(Int, Array[Float])] = { + require(size > 0 && size < Int.MaxValue / 3) + require(b > a) + val ids = mutable.Set.empty[Int] + while (ids.size < size) { + ids += random.nextInt() + } + val width = b - a + ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + } + + /** + * Generates an implicit feedback dataset for testing ALS. + * + * @param sc SparkContext + * @param numUsers number of users + * @param numItems number of items + * @param rank rank + * @param noiseStd the standard deviation of additive Gaussian noise on training data + * @param seed random seed + * @return (training, test) + */ + def genImplicitTestData( + sc: SparkContext, + numUsers: Int, + numItems: Int, + rank: Int, + noiseStd: Double = 0.0, + seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { + // The assumption of the implicit feedback model is that unobserved ratings are more likely to + // be negatives. + val positiveFraction = 0.8 + val negativeFraction = 1.0 - positiveFraction + val trainingFraction = 0.6 + val testFraction = 0.3 + val totalFraction = trainingFraction + testFraction + val random = new Random(seed) + val userFactors = genFactors(numUsers, rank, random) + val itemFactors = genFactors(numItems, rank, random) + val training = ArrayBuffer.empty[Rating[Int]] + val test = ArrayBuffer.empty[Rating[Int]] + for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { + val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) + val threshold = if (rating > 0) positiveFraction else negativeFraction + val observed = random.nextDouble() < threshold + if (observed) { + val x = random.nextDouble() + if (x < totalFraction) { + if (x < trainingFraction) { + val noise = noiseStd * random.nextGaussian() + training += Rating(userId, itemId, rating + noise.toFloat) + } else { + test += Rating(userId, itemId, rating) + } + } + } + } + logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + + s"and ${test.size} for test.") + (sc.parallelize(training, 2), sc.parallelize(test, 2)) + } } From 07a02e8bb6a2a32508627d4a0cb487b38d595184 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 12:38:21 +0100 Subject: [PATCH 024/313] [MINOR][DOCS] Fix type Information in Quick Start and Programming Guide Author: Sandeep Singh Closes #12841 from techaddict/improve_docs_1. (cherry picked from commit dfd9723dd3b3ff5d47a7f04a4330bf33ffe353ac) Signed-off-by: Sean Owen --- docs/programming-guide.md | 2 +- docs/quick-start.md | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index cf6f1d89147f..d375926a910e 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -328,7 +328,7 @@ Text file RDDs can be created using `SparkContext`'s `textFile` method. This met {% highlight scala %} scala> val distFile = sc.textFile("data.txt") -distFile: RDD[String] = MappedRDD@1d4cee08 +distFile: org.apache.spark.rdd.RDD[String] = data.txt MapPartitionsRDD[10] at textFile at :26 {% endhighlight %} Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(s => s.length).reduce((a, b) => a + b)`. diff --git a/docs/quick-start.md b/docs/quick-start.md index d481fe0ea6d7..72372a6bc854 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -33,7 +33,7 @@ Spark's primary abstraction is a distributed collection of items called a Resili {% highlight scala %} scala> val textFile = sc.textFile("README.md") -textFile: spark.RDD[String] = spark.MappedRDD@2ee9b6e3 +textFile: org.apache.spark.rdd.RDD[String] = README.md MapPartitionsRDD[1] at textFile at :25 {% endhighlight %} RDDs have _[actions](programming-guide.html#actions)_, which return values, and _[transformations](programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions: @@ -50,7 +50,7 @@ Now let's use a transformation. We will use the [`filter`](programming-guide.htm {% highlight scala %} scala> val linesWithSpark = textFile.filter(line => line.contains("Spark")) -linesWithSpark: spark.RDD[String] = spark.FilteredRDD@7dd4af09 +linesWithSpark: org.apache.spark.rdd.RDD[String] = MapPartitionsRDD[2] at filter at :27 {% endhighlight %} We can chain together transformations and actions: @@ -123,7 +123,7 @@ One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can i {% highlight scala %} scala> val wordCounts = textFile.flatMap(line => line.split(" ")).map(word => (word, 1)).reduceByKey((a, b) => a + b) -wordCounts: spark.RDD[(String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 +wordCounts: org.apache.spark.rdd.RDD[(String, Int)] = ShuffledRDD[8] at reduceByKey at :28 {% endhighlight %} Here, we combined the [`flatMap`](programming-guide.html#transformations), [`map`](programming-guide.html#transformations), and [`reduceByKey`](programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (String, Int) pairs. To collect the word counts in our shell, we can use the [`collect`](programming-guide.html#actions) action: @@ -181,7 +181,7 @@ Spark also supports pulling data sets into a cluster-wide in-memory cache. This {% highlight scala %} scala> linesWithSpark.cache() -res7: spark.RDD[String] = spark.FilteredRDD@17e51082 +res7: linesWithSpark.type = MapPartitionsRDD[2] at filter at :27 scala> linesWithSpark.count() res8: Long = 19 From 38f6e66afdc92865628238e53ccc156fef976770 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 May 2016 12:39:37 +0100 Subject: [PATCH 025/313] [SPARK-15053][BUILD] Fix Java Lint errors on Hive-Thriftserver module ## What changes were proposed in this pull request? This issue fixes or hides 181 Java linter errors introduced by SPARK-14987 which copied hive service code from Hive. We had better clean up these errors before releasing Spark 2.0. - Fix UnusedImports (15 lines), RedundantModifier (14 lines), SeparatorWrap (9 lines), MethodParamPad (6 lines), FileTabCharacter (5 lines), ArrayTypeStyle (3 lines), ModifierOrder (3 lines), RedundantImport (1 line), CommentsIndentation (1 line), UpperEll (1 line), FallThrough (1 line), OneStatementPerLine (1 line), NewlineAtEndOfFile (1 line) errors. - Ignore `LineLength` errors under `hive/service/*` (118 lines). - Ignore `MethodName` error in `PasswdAuthenticationProvider.java` (1 line). - Ignore `NoFinalizer` error in `ThreadWithGarbageCleanup.java` (1 line). ## How was this patch tested? After passing Jenkins building, run `dev/lint-java` manually. ```bash $ dev/lint-java Checkstyle checks passed. ``` Author: Dongjoon Hyun Closes #12831 from dongjoon-hyun/SPARK-15053. (cherry picked from commit a7444570764b0a08b7e908dc7931744f9dbdf3c6) Signed-off-by: Sean Owen --- dev/checkstyle-suppressions.xml | 6 ++++++ .../main/java/org/apache/hive/service/Service.java | 2 +- .../java/org/apache/hive/service/ServiceUtils.java | 2 +- .../org/apache/hive/service/auth/HttpAuthUtils.java | 10 +++++----- .../apache/hive/service/auth/KerberosSaslHelper.java | 2 +- .../java/org/apache/hive/service/auth/SaslQOP.java | 10 ++++++---- .../java/org/apache/hive/service/cli/CLIService.java | 7 +++---- .../apache/hive/service/cli/HiveSQLException.java | 9 +++++---- .../org/apache/hive/service/cli/OperationState.java | 1 + .../org/apache/hive/service/cli/RowBasedSet.java | 4 ++-- .../cli/operation/ClassicTableTypeMapping.java | 2 +- .../service/cli/operation/GetColumnsOperation.java | 2 -- .../service/cli/operation/GetFunctionsOperation.java | 3 +-- .../service/cli/operation/GetSchemasOperation.java | 8 -------- .../cli/operation/GetTableTypesOperation.java | 4 ++-- .../service/cli/operation/GetTablesOperation.java | 4 ++-- .../service/cli/operation/GetTypeInfoOperation.java | 2 +- .../service/cli/operation/LogDivertAppender.java | 12 ++++++------ .../service/cli/operation/MetadataOperation.java | 1 - .../hive/service/cli/operation/SQLOperation.java | 2 +- .../hive/service/cli/operation/TableTypeMapping.java | 6 +++--- .../hive/service/cli/session/HiveSessionBase.java | 3 --- .../hive/service/cli/session/HiveSessionHook.java | 2 +- .../service/cli/session/HiveSessionHookContext.java | 6 +++--- .../service/cli/session/HiveSessionImplwithUGI.java | 3 +-- .../hive/service/cli/session/HiveSessionProxy.java | 2 +- .../hive/service/cli/session/SessionManager.java | 2 +- .../hive/service/cli/thrift/ThriftCLIService.java | 4 ++-- .../hive/service/cli/thrift/ThriftHttpServlet.java | 9 ++++----- .../org/apache/hive/service/server/HiveServer2.java | 4 ++-- 30 files changed, 63 insertions(+), 71 deletions(-) diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml index a1a88ac8cdac..bfc2e734e517 100644 --- a/dev/checkstyle-suppressions.xml +++ b/dev/checkstyle-suppressions.xml @@ -36,4 +36,10 @@ files="src/test/java/org/apache/spark/sql/hive/test/Complex.java"/> + + + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java index 2111837cac8d..b95077cd6218 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java @@ -29,7 +29,7 @@ public interface Service { /** * Service states */ - public enum STATE { + enum STATE { /** Constructed but not initialized */ NOTINITED, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java index e712aaf2348f..edb5eff9615b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java @@ -41,4 +41,4 @@ public static int indexOfDomainMatch(String userName) { } return endIdx; } -} \ No newline at end of file +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java index 3ef55779a6bd..502152829968 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -56,7 +56,7 @@ public final class HttpAuthUtils { private static final String COOKIE_CLIENT_USER_NAME = "cu"; private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; - private final static Set COOKIE_ATTRIBUTES = + private static final Set COOKIE_ATTRIBUTES = new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); /** @@ -93,10 +93,10 @@ public static String getKerberosServiceTicket(String principal, String host, */ public static String createCookieToken(String clientUserName) { StringBuffer sb = new StringBuffer(); - sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName). - append(COOKIE_ATTR_SEPARATOR); - sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR). - append((new Random(System.currentTimeMillis())).nextLong()); + sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName) + .append(COOKIE_ATTR_SEPARATOR); + sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR) + .append((new Random(System.currentTimeMillis())).nextLong()); return sb.toString(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java index 11d26699fe78..52eb752f1e02 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -96,7 +96,7 @@ private static class CLIServiceProcessorFactory extends TProcessorFactory { private final ThriftCLIService service; private final Server saslServer; - public CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { + CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { super(null); this.service = service; this.saslServer = saslServer; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java index 479ebf32cec3..ab3ac6285aa0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java @@ -25,10 +25,12 @@ * Possible values of SASL quality-of-protection value. */ public enum SaslQOP { - AUTH("auth"), // Authentication only. - AUTH_INT("auth-int"), // Authentication and integrity checking by using signatures. - AUTH_CONF("auth-conf"); // Authentication, integrity and confidentiality checking - // by using signatures and encryption. + // Authentication only. + AUTH("auth"), + // Authentication and integrity checking by using signatures. + AUTH_INT("auth-int"), + // Authentication, integrity and confidentiality checking by using signatures and encryption. + AUTH_CONF("auth-conf"); public final String saslQop; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java index a3af7b2d662b..791ddcbd2c5b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java @@ -481,8 +481,8 @@ public synchronized String getDelegationTokenFromMetaStore(String owner) @Override public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String owner, String renewer) throws HiveSQLException { - String delegationToken = sessionManager.getSession(sessionHandle). - getDelegationToken(authFactory, owner, renewer); + String delegationToken = sessionManager.getSession(sessionHandle) + .getDelegationToken(authFactory, owner, renewer); LOG.info(sessionHandle + ": getDelegationToken()"); return delegationToken; } @@ -490,8 +490,7 @@ public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory au @Override public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { - sessionManager.getSession(sessionHandle). - cancelDelegationToken(authFactory, tokenStr); + sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); LOG.info(sessionHandle + ": cancelDelegationToken()"); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java index 1334dde66375..86e57fbf31fe 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java @@ -111,7 +111,7 @@ public HiveSQLException(TStatus status) { /** * Converts current object to a {@link TStatus} object - * @return a {@link TStatus} object + * @return a {@link TStatus} object */ public TStatus toTStatus() { // TODO: convert sqlState, etc. @@ -125,8 +125,8 @@ public TStatus toTStatus() { /** * Converts the specified {@link Exception} object into a {@link TStatus} object - * @param e a {@link Exception} object - * @return a {@link TStatus} object + * @param e a {@link Exception} object + * @return a {@link TStatus} object */ public static TStatus toTStatus(Exception e) { if (e instanceof HiveSQLException) { @@ -155,7 +155,8 @@ private static List toString(Throwable cause, StackTraceElement[] parent if (parent != null) { int n = parent.length - 1; while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { - m--; n--; + m--; + n--; } } List detail = enroll(cause, trace, m); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java index 51ffb40369b2..116518011841 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java @@ -85,6 +85,7 @@ public static void validateTransition(OperationState oldState, if (OperationState.CLOSED.equals(newState)) { return; } + break; default: // fall-through } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java index a0ee2109dc54..7452137f077d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java @@ -130,8 +130,8 @@ public void remove() { } private static class RemovableList extends ArrayList { - public RemovableList() { super(); } - public RemovableList(List rows) { super(rows); } + RemovableList() { super(); } + RemovableList(List rows) { super(rows); } @Override public void removeRange(int fromIndex, int toIndex) { super.removeRange(fromIndex, toIndex); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java index 87ac39b05186..05a6bf938404 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java @@ -42,7 +42,7 @@ public enum ClassicTableTypes { private final Map hiveToClientMap = new HashMap(); private final Map clientToHiveMap = new HashMap(); - public ClassicTableTypeMapping () { + public ClassicTableTypeMapping() { hiveToClientMap.put(TableType.MANAGED_TABLE.toString(), ClassicTableTypes.TABLE.toString()); hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(), diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java index 309f10f640f9..5efb0759383a 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -27,10 +27,8 @@ import java.util.Map.Entry; import java.util.regex.Pattern; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java index 6df1e8a227f3..5273c386b83d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -23,7 +23,6 @@ import java.util.Set; import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; @@ -103,7 +102,7 @@ public void runInternal() throws HiveSQLException { .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); for (String functionName : functionNames) { FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); - Object rowData[] = new Object[] { + Object[] rowData = new Object[] { null, // FUNCTION_CAT null, // FUNCTION_SCHEM functionInfo.getDisplayName(), // FUNCTION_NAME diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index e56686abb7c5..d6f6280f1c39 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -18,16 +18,8 @@ package org.apache.hive.service.cli.operation; -import java.util.ArrayList; -import java.util.List; - import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.service.cli.FetchOrientation; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.OperationState; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java index a09b39a4e085..3ae012a72764 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -44,8 +44,8 @@ public class GetTableTypesOperation extends MetadataOperation { protected GetTableTypesOperation(HiveSession parentSession) { super(parentSession, OperationType.GET_TABLE_TYPES); - String tableMappingStr = getParentSession().getHiveConf(). - getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); tableTypeMapping = TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java index 0e2fdc657c4f..1a7ca79163d7 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -64,8 +64,8 @@ protected GetTablesOperation(HiveSession parentSession, this.catalogName = catalogName; this.schemaName = schemaName; this.tableName = tableName; - String tableMappingStr = getParentSession().getHiveConf(). - getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); tableTypeMapping = TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); if (tableTypes != null) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java index 2a0fec27715d..0f72071d7e7d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -35,7 +35,7 @@ */ public class GetTypeInfoOperation extends MetadataOperation { - private final static TableSchema RESULT_SET_SCHEMA = new TableSchema() + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, "Type name") .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java index 70340bd13cbc..cb804318ace9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java @@ -60,15 +60,15 @@ private static class NameFilter extends Filter { /* Patterns that are excluded in verbose logging level. * Filter out messages coming from log processing classes, or we'll run an infinite loop. */ - private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|"). - join(new String[] {LOG.getName(), OperationLog.class.getName(), + private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {LOG.getName(), OperationLog.class.getName(), OperationManager.class.getName()})); /* Patterns that are included in execution logging level. * In execution mode, show only select logger messages. */ - private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|"). - join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", + private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", "org.apache.hadoop.mapreduce.Job", "SessionState", Task.class.getName(), "org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor"})); @@ -88,7 +88,7 @@ private void setCurrentNamePattern(OperationLog.LoggingLevel mode) { } } - public NameFilter( + NameFilter( OperationLog.LoggingLevel loggingMode, OperationManager op) { this.operationManager = op; this.loggingMode = loggingMode; @@ -131,7 +131,7 @@ public int decide(LoggingEvent ev) { /** This is where the log message will go to */ private final CharArrayWriter writer = new CharArrayWriter(); - private void setLayout (boolean isVerbose, Layout lo) { + private void setLayout(boolean isVerbose, Layout lo) { if (isVerbose) { if (lo == null) { lo = CLIServiceUtils.verboseLayout; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java index 4595ef56fcee..6c819876a556 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -18,7 +18,6 @@ package org.apache.hive.service.cli.operation; -import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hive.conf.HiveConf; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java index 33ee16b80beb..a35405484a76 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -326,7 +326,7 @@ public TableSchema getResultSetSchema() throws HiveSQLException { return resultSchema; } - private transient final List convey = new ArrayList(); + private final transient List convey = new ArrayList(); @Override public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java index 3a8a07f44f20..e392c459cf58 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java @@ -27,18 +27,18 @@ public interface TableTypeMapping { * @param clientTypeName * @return */ - public String mapToHiveType (String clientTypeName); + String mapToHiveType(String clientTypeName); /** * Map hive's table type name to client's table type * @param clientTypeName * @return */ - public String mapToClientType (String hiveTypeName); + String mapToClientType(String hiveTypeName); /** * Get all the table types of this mapping * @return */ - public Set getTableTypeNames(); + Set getTableTypeNames(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java index 9b04d679df1c..b72c18b2b213 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java @@ -18,8 +18,6 @@ package org.apache.hive.service.cli.session; -import java.util.Map; - import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.service.cli.SessionHandle; @@ -27,7 +25,6 @@ import org.apache.hive.service.cli.thrift.TProtocolVersion; import java.io.File; -import java.util.Map; /** * Methods that don't need to be executed under a doAs diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java index 06388cc795b9..7e4f2b0478a5 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java @@ -33,5 +33,5 @@ public interface HiveSessionHook extends Hook { * @param sessionHookContext context * @throws HiveSQLException */ - public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; + void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java index 156c8147f9d9..0a10dba8b479 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java @@ -30,17 +30,17 @@ public interface HiveSessionHookContext { * Retrieve session conf * @return */ - public HiveConf getSessionConf(); + HiveConf getSessionConf(); /** * The get the username starting the session * @return */ - public String getSessionUser(); + String getSessionUser(); /** * Retrieve handle for the session * @return */ - public String getSessionHandle(); + String getSessionHandle(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java index a29e5d1d81c1..762dbb2faade 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hive.service.auth.HiveAuthFactory; @@ -83,7 +82,7 @@ public UserGroupInformation getSessionUgi() { return this.sessionUgi; } - public String getDelegationToken () { + public String getDelegationToken() { return this.delegationTokenStr; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java index 5b10521febfc..8e539512f741 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java @@ -57,7 +57,7 @@ public Object invoke(Object arg0, final Method method, final Object[] args) return invoke(method, args); } return ugi.doAs( - new PrivilegedExceptionAction () { + new PrivilegedExceptionAction() { @Override public Object run() throws HiveSQLException { return invoke(method, args); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index e31570bdfba5..c6d177798f55 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -151,7 +151,7 @@ public synchronized void start() { } private void startTimeoutChecker() { - final long interval = Math.max(checkInterval, 3000l); // minimum 3 seconds + final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds Runnable timeoutChecker = new Runnable() { @Override public void run() { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 5a0f1c83c70f..866beb19f5f9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -722,8 +722,8 @@ private String getProxyUser(String realUser, Map sessionConf, } // If there's no authentication, then directly substitute the user - if (HiveAuthFactory.AuthTypes.NONE.toString(). - equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { + if (HiveAuthFactory.AuthTypes.NONE.toString() + .equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { return proxyUser; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java index 56c8cb6e5459..e15d2d0566d2 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.security.PrivilegedExceptionAction; -import java.util.Arrays; import java.util.Map; import java.util.Random; import java.util.Set; @@ -241,9 +240,9 @@ private String getClientNameFromCookie(Cookie[] cookies) { * Each cookie is of the format [key]=[value] */ private String toCookieStr(Cookie[] cookies) { - String cookieStr = ""; + String cookieStr = ""; - for (Cookie c : cookies) { + for (Cookie c : cookies) { cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; } return cookieStr; @@ -458,7 +457,7 @@ private String getPrincipalWithoutRealmAndHost(String fullPrincipal) private String getUsername(HttpServletRequest request, String authType) throws HttpAuthenticationException { - String creds[] = getAuthHeaderTokens(request, authType); + String[] creds = getAuthHeaderTokens(request, authType); // Username must be present if (creds[0] == null || creds[0].isEmpty()) { throw new HttpAuthenticationException("Authorization header received " + @@ -469,7 +468,7 @@ private String getUsername(HttpServletRequest request, String authType) private String getPassword(HttpServletRequest request, String authType) throws HttpAuthenticationException { - String creds[] = getAuthHeaderTokens(request, authType); + String[] creds = getAuthHeaderTokens(request, authType); // Password must be present if (creds[1] == null || creds[1].isEmpty()) { throw new HttpAuthenticationException("Authorization header received " + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java index 1500e537cef5..9bf96cff572e 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java @@ -236,8 +236,8 @@ ServerOptionsExecutor getServerOptionsExecutor() { /** * The executor interface for running the appropriate HiveServer2 command based on parsed options */ - static interface ServerOptionsExecutor { - public void execute(); + interface ServerOptionsExecutor { + void execute(); } /** From b802979ad42fd58ed1d8c6e23629169bc2891cbe Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 May 2016 19:45:12 +0800 Subject: [PATCH 026/313] [SPARK-15081] Move AccumulatorV2 and subclasses into util package ## What changes were proposed in this pull request? This patch moves AccumulatorV2 and subclasses into util package. ## How was this patch tested? Updated relevant tests. Author: Reynold Xin Closes #12863 from rxin/SPARK-15081. (cherry picked from commit d557a5e01e8f819d3bd9e6e43d2df733f390d764) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/Accumulable.scala | 1 + .../org/apache/spark/ContextCleaner.scala | 2 +- .../org/apache/spark/HeartbeatReceiver.scala | 2 +- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../scala/org/apache/spark/TaskEndReason.scala | 2 +- .../apache/spark/executor/InputMetrics.scala | 2 +- .../apache/spark/executor/OutputMetrics.scala | 2 +- .../spark/executor/ShuffleReadMetrics.scala | 2 +- .../spark/executor/ShuffleWriteMetrics.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 1 + .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../apache/spark/scheduler/TaskResult.scala | 4 ++-- .../spark/scheduler/TaskResultGetter.scala | 2 +- .../apache/spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../spark/{ => util}/AccumulatorV2.scala | 18 +++++++++--------- .../org/apache/spark/AccumulatorSuite.scala | 1 + .../spark/InternalAccumulatorSuite.scala | 1 + .../scala/org/apache/spark/SparkFunSuite.scala | 1 + .../spark/executor/TaskMetricsSuite.scala | 1 + .../spark/scheduler/DAGSchedulerSuite.scala | 3 +-- .../ExternalClusterManagerSuite.scala | 3 ++- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../apache/spark/util/AccumulatorV2Suite.scala | 2 +- .../columnar/InMemoryTableScanExec.scala | 4 +++- .../sql/execution/metric/SQLMetrics.scala | 4 ++-- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 ++ 30 files changed, 44 insertions(+), 34 deletions(-) rename core/src/main/scala/org/apache/spark/{ => util}/AccumulatorV2.scala (95%) diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index 799c7e4fd500..5c6761eb764b 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, LegacyAccumulatorWrapper} /** diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index c895fb32069c..5678d790e9e7 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, ThreadUtils, Utils} /** * Classes that represent cleaning tasks. diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 29018c75b922..73495a8d7d7f 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} +import org.apache.spark.util._ /** * A heartbeat from executors to the driver. This is a shared message used by several internal diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 1a8f8cf11c1b..27abccf5ac2a 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.Source -import org.apache.spark.util.{TaskCompletionListener, TaskFailureListener} +import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener} object TaskContext { diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index ef333e397f64..42690844f961 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} // ============================================================================================== // NOTE: new task end reasons MUST be accompanied with serialization logic in util.JsonProtocol! diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 6f7160ac0d3a..3d15f3a0396e 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index db3924cb6937..dada9697c1cf 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index fa962108c306..f7a991770d40 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 0e70a4f52284..ada2e1bc0859 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 56d034fd033e..085aa7fbd63d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 0a2c2dc03937..8c761124824a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,7 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.CallSite +import org.apache.spark.util.{AccumulatorV2, CallSite} /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 362f8e51ce05..95bcc7bc9653 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -28,7 +28,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} +import org.apache.spark.util.{AccumulatorV2, ByteBufferInputStream, ByteBufferOutputStream, Utils} /** * A unit of execution. We have two kinds of Task's in Spark: diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 69ce00f30d9a..80f2bf41224b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -22,9 +22,9 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{AccumulatorV2, SparkEnv} +import org.apache.spark.SparkEnv import org.apache.spark.storage.BlockId -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} // Task result. Also contains updates to accumulator variables. private[spark] sealed trait TaskResult[T] diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index b438c285fdf1..685ef55c6687 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -27,7 +27,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.Logging import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{LongAccumulator, ThreadUtils, Utils} /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 9881a1018cf5..cd13eebe74a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,9 +17,9 @@ package org.apache.spark.scheduler -import org.apache.spark.AccumulatorV2 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 /** * Low-level task scheduler interface, currently implemented exclusively by diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 666b6365585c..393680f4c154 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b724050f5bb8..cd634bbf6fe7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -32,7 +32,7 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} /** * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of diff --git a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/AccumulatorV2.scala rename to core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index a6c64fd68057..0e280f6f6a29 100644 --- a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import java.{lang => jl} import java.io.ObjectInputStream import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong +import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext} import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.util.Utils private[spark] case class AccumulatorMetadata( @@ -204,8 +204,8 @@ private[spark] object AccumulatorContext { private[this] val nextId = new AtomicLong(0L) /** - * Returns a globally unique ID for a new [[Accumulator]]. - * Note: Once you copy the [[Accumulator]] the ID is no longer unique. + * Returns a globally unique ID for a new [[AccumulatorV2]]. + * Note: Once you copy the [[AccumulatorV2]] the ID is no longer unique. */ def newId(): Long = nextId.getAndIncrement @@ -213,14 +213,14 @@ private[spark] object AccumulatorContext { def numAccums: Int = originals.size /** - * Registers an [[Accumulator]] created on the driver such that it can be used on the executors. + * Registers an [[AccumulatorV2]] created on the driver such that it can be used on the executors. * * All accumulators registered here can later be used as a container for accumulating partial * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. * Note: if an accumulator is registered here, it should also be registered with the active * context cleaner for cleanup so as to avoid memory leaks. * - * If an [[Accumulator]] with the same ID was already registered, this does nothing instead + * If an [[AccumulatorV2]] with the same ID was already registered, this does nothing instead * of overwriting it. We will never register same accumulator twice, this is just a sanity check. */ def register(a: AccumulatorV2[_, _]): Unit = { @@ -228,14 +228,14 @@ private[spark] object AccumulatorContext { } /** - * Unregisters the [[Accumulator]] with the given ID, if any. + * Unregisters the [[AccumulatorV2]] with the given ID, if any. */ def remove(id: Long): Unit = { originals.remove(id) } /** - * Returns the [[Accumulator]] registered with the given ID, if any. + * Returns the [[AccumulatorV2]] registered with the given ID, if any. */ def get(id: Long): Option[AccumulatorV2[_, _]] = { Option(originals.get(id)).map { ref => @@ -249,7 +249,7 @@ private[spark] object AccumulatorContext { } /** - * Clears all registered [[Accumulator]]s. For testing only. + * Clears all registered [[AccumulatorV2]]s. For testing only. */ def clear(): Unit = { originals.clear() diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 00200962549e..cade67b1d2ba 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 25977a466024..840f55ce2f6e 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 4aae2c9b4a8e..0081bca63959 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -21,6 +21,7 @@ package org.apache.spark import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} import org.apache.spark.internal.Logging +import org.apache.spark.util.AccumulatorContext /** * Base abstract class for all unit tests in Spark for handling common functionality. diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 27a1e7bb35ba..eae26fa742a2 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.Assertions import org.apache.spark._ import org.apache.spark.storage.{BlockStatus, StorageLevel, TestBlockId} +import org.apache.spark.util.AccumulatorV2 class TaskMetricsSuite extends SparkFunSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 5a5c3a0cd193..844c780a3fdd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -28,11 +28,10 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.{CallSite, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 72ac848f12b4..59c1b359a780 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.scheduler -import org.apache.spark.{AccumulatorV2, LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { test("launch of backend and scheduler") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 122a3ecb4954..9b7b945bf367 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{AccumulatorV2, ManualClock} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala index 41cdd0249226..815b13488401 100644 --- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import org.apache.spark.{DoubleAccumulator, LongAccumulator, SparkFunSuite} +import org.apache.spark.SparkFunSuite class AccumulatorV2Suite extends SparkFunSuite { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 94b87a5812e4..a36071a97bcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Accumulable, Accumulator, AccumulatorContext} +import org.apache.spark.{Accumulable, Accumulator} import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -33,6 +33,8 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.AccumulatorContext + private[sql] object InMemoryRelation { def apply( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 40c00ee1881f..0f68aaaee1f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.metric import java.text.NumberFormat -import org.apache.spark.{AccumulatorV2, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} class SQLMetric(val metricType: String, initValue: Long = 0L) extends AccumulatorV2[Long, Long] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 12dbed89d561..5ef20267f874 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,13 +22,13 @@ import scala.language.postfixOps import org.scalatest.concurrent.Eventually._ -import org.apache.spark.AccumulatorContext import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.storage.{RDDBlockId, StorageLevel} +import org.apache.spark.util.AccumulatorContext private case class BigData(s: String) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 8572ed16aa26..964787015a87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -30,6 +30,8 @@ import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.ui.SparkUI +import org.apache.spark.util.{AccumulatorMetadata, LongAccumulator} + class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ From f03bf7eacb834d2eaeba197ccf704bb721f0b4af Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 3 May 2016 13:13:35 +0100 Subject: [PATCH 027/313] [SPARK-14897][CORE] Upgrade Jetty to latest version of 8 ## What changes were proposed in this pull request? Update Jetty 8.1 to the latest 2016/02 release, from a 2013/10 release, for security and bug fixes. This does not resolve the JIRA necessarily, as it's still worth considering an update to 9.3. ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #12842 from srowen/SPARK-14897. (cherry picked from commit 57ac7c182465e1653e74a8ad6c826b2cf56a0ad8) Signed-off-by: Sean Owen --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 66f1d8ea902a..1dce91b7dfe4 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ 10.10.1.1 1.7.0 1.6.0 - 8.1.14.v20131031 + 8.1.19.v20160209 3.0.0.v201112011016 0.8.0 2.4.0 From 932e1b5b2ad19153b5a5aa2255837569406486dd Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 3 May 2016 16:46:13 +0200 Subject: [PATCH 028/313] [SPARK-14971][ML][PYSPARK] PySpark ML Params setter code clean up ## What changes were proposed in this pull request? PySpark ML Params setter code clean up. For examples, ```setInputCol``` can be simplified from ``` self._set(inputCol=value) return self ``` to: ``` return self._set(inputCol=value) ``` This is a pretty big sweeps, and we cleaned wherever possible. ## How was this patch tested? Exist unit tests. Author: Yanbo Liang Closes #12749 from yanboliang/spark-14971. (cherry picked from commit d26f7cb0121767da678bbbbf3a0e31c63d5e3159) Signed-off-by: Nick Pentreath --- python/pyspark/ml/classification.py | 21 ++-- python/pyspark/ml/clustering.py | 18 +-- python/pyspark/ml/evaluation.py | 9 +- python/pyspark/ml/feature.py | 111 ++++++------------ .../ml/param/_shared_params_code_gen.py | 3 +- python/pyspark/ml/param/shared.py | 87 +++++--------- python/pyspark/ml/pipeline.py | 3 +- python/pyspark/ml/recommendation.py | 35 ++---- python/pyspark/ml/regression.py | 36 ++---- python/pyspark/ml/tuning.py | 6 +- 10 files changed, 110 insertions(+), 219 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4331f73b7325..f03296333446 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -464,8 +464,7 @@ def setImpurity(self, value): """ Sets the value of :py:attr:`impurity`. """ - self._set(impurity=value) - return self + return self._set(impurity=value) @since("1.6.0") def getImpurity(self): @@ -826,8 +825,7 @@ def setLossType(self, value): """ Sets the value of :py:attr:`lossType`. """ - self._set(lossType=value) - return self + return self._set(lossType=value) @since("1.4.0") def getLossType(self): @@ -956,8 +954,7 @@ def setSmoothing(self, value): """ Sets the value of :py:attr:`smoothing`. """ - self._set(smoothing=value) - return self + return self._set(smoothing=value) @since("1.5.0") def getSmoothing(self): @@ -971,8 +968,7 @@ def setModelType(self, value): """ Sets the value of :py:attr:`modelType`. """ - self._set(modelType=value) - return self + return self._set(modelType=value) @since("1.5.0") def getModelType(self): @@ -1098,8 +1094,7 @@ def setLayers(self, value): """ Sets the value of :py:attr:`layers`. """ - self._set(layers=value) - return self + return self._set(layers=value) @since("1.6.0") def getLayers(self): @@ -1113,8 +1108,7 @@ def setBlockSize(self, value): """ Sets the value of :py:attr:`blockSize`. """ - self._set(blockSize=value) - return self + return self._set(blockSize=value) @since("1.6.0") def getBlockSize(self): @@ -1162,8 +1156,7 @@ def setClassifier(self, value): .. note:: Only LogisticRegression and NaiveBayes are supported now. """ - self._set(classifier=value) - return self + return self._set(classifier=value) @since("2.0.0") def getClassifier(self): diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 50ebf4fde1cf..b90daf16e2f7 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -156,8 +156,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("2.0.0") def getK(self): @@ -272,8 +271,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("1.5.0") def getK(self): @@ -287,8 +285,7 @@ def setInitMode(self, value): """ Sets the value of :py:attr:`initMode`. """ - self._set(initMode=value) - return self + return self._set(initMode=value) @since("1.5.0") def getInitMode(self): @@ -302,8 +299,7 @@ def setInitSteps(self, value): """ Sets the value of :py:attr:`initSteps`. """ - self._set(initSteps=value) - return self + return self._set(initSteps=value) @since("1.5.0") def getInitSteps(self): @@ -422,8 +418,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("2.0.0") def getK(self): @@ -437,8 +432,7 @@ def setMinDivisibleClusterSize(self, value): """ Sets the value of :py:attr:`minDivisibleClusterSize`. """ - self._set(minDivisibleClusterSize=value) - return self + return self._set(minDivisibleClusterSize=value) @since("2.0.0") def getMinDivisibleClusterSize(self): diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 455795f9a083..2a41678741ea 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -147,8 +147,7 @@ def setMetricName(self, value): """ Sets the value of :py:attr:`metricName`. """ - self._set(metricName=value) - return self + return self._set(metricName=value) @since("1.4.0") def getMetricName(self): @@ -217,8 +216,7 @@ def setMetricName(self, value): """ Sets the value of :py:attr:`metricName`. """ - self._set(metricName=value) - return self + return self._set(metricName=value) @since("1.4.0") def getMetricName(self): @@ -284,8 +282,7 @@ def setMetricName(self, value): """ Sets the value of :py:attr:`metricName`. """ - self._set(metricName=value) - return self + return self._set(metricName=value) @since("1.5.0") def getMetricName(self): diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index b95d288198b5..f21e3062ef4f 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -114,8 +114,7 @@ def setThreshold(self, value): """ Sets the value of :py:attr:`threshold`. """ - self._set(threshold=value) - return self + return self._set(threshold=value) @since("1.4.0") def getThreshold(self): @@ -190,8 +189,7 @@ def setSplits(self, value): """ Sets the value of :py:attr:`splits`. """ - self._set(splits=value) - return self + return self._set(splits=value) @since("1.4.0") def getSplits(self): @@ -295,8 +293,7 @@ def setMinTF(self, value): """ Sets the value of :py:attr:`minTF`. """ - self._set(minTF=value) - return self + return self._set(minTF=value) @since("1.6.0") def getMinTF(self): @@ -310,8 +307,7 @@ def setMinDF(self, value): """ Sets the value of :py:attr:`minDF`. """ - self._set(minDF=value) - return self + return self._set(minDF=value) @since("1.6.0") def getMinDF(self): @@ -325,8 +321,7 @@ def setVocabSize(self, value): """ Sets the value of :py:attr:`vocabSize`. """ - self._set(vocabSize=value) - return self + return self._set(vocabSize=value) @since("1.6.0") def getVocabSize(self): @@ -340,8 +335,7 @@ def setBinary(self, value): """ Sets the value of :py:attr:`binary`. """ - self._set(binary=value) - return self + return self._set(binary=value) @since("2.0.0") def getBinary(self): @@ -433,8 +427,7 @@ def setInverse(self, value): """ Sets the value of :py:attr:`inverse`. """ - self._set(inverse=value) - return self + return self._set(inverse=value) @since("1.6.0") def getInverse(self): @@ -500,8 +493,7 @@ def setScalingVec(self, value): """ Sets the value of :py:attr:`scalingVec`. """ - self._set(scalingVec=value) - return self + return self._set(scalingVec=value) @since("1.5.0") def getScalingVec(self): @@ -573,8 +565,7 @@ def setBinary(self, value): """ Sets the value of :py:attr:`binary`. """ - self._set(binary=value) - return self + return self._set(binary=value) @since("2.0.0") def getBinary(self): @@ -647,8 +638,7 @@ def setMinDocFreq(self, value): """ Sets the value of :py:attr:`minDocFreq`. """ - self._set(minDocFreq=value) - return self + return self._set(minDocFreq=value) @since("1.4.0") def getMinDocFreq(self): @@ -832,8 +822,7 @@ def setMin(self, value): """ Sets the value of :py:attr:`min`. """ - self._set(min=value) - return self + return self._set(min=value) @since("1.6.0") def getMin(self): @@ -847,8 +836,7 @@ def setMax(self, value): """ Sets the value of :py:attr:`max`. """ - self._set(max=value) - return self + return self._set(max=value) @since("1.6.0") def getMax(self): @@ -956,8 +944,7 @@ def setN(self, value): """ Sets the value of :py:attr:`n`. """ - self._set(n=value) - return self + return self._set(n=value) @since("1.5.0") def getN(self): @@ -1023,8 +1010,7 @@ def setP(self, value): """ Sets the value of :py:attr:`p`. """ - self._set(p=value) - return self + return self._set(p=value) @since("1.4.0") def getP(self): @@ -1106,8 +1092,7 @@ def setDropLast(self, value): """ Sets the value of :py:attr:`dropLast`. """ - self._set(dropLast=value) - return self + return self._set(dropLast=value) @since("1.4.0") def getDropLast(self): @@ -1175,8 +1160,7 @@ def setDegree(self, value): """ Sets the value of :py:attr:`degree`. """ - self._set(degree=value) - return self + return self._set(degree=value) @since("1.4.0") def getDegree(self): @@ -1257,8 +1241,7 @@ def setNumBuckets(self, value): """ Sets the value of :py:attr:`numBuckets`. """ - self._set(numBuckets=value) - return self + return self._set(numBuckets=value) @since("2.0.0") def getNumBuckets(self): @@ -1355,8 +1338,7 @@ def setMinTokenLength(self, value): """ Sets the value of :py:attr:`minTokenLength`. """ - self._set(minTokenLength=value) - return self + return self._set(minTokenLength=value) @since("1.4.0") def getMinTokenLength(self): @@ -1370,8 +1352,7 @@ def setGaps(self, value): """ Sets the value of :py:attr:`gaps`. """ - self._set(gaps=value) - return self + return self._set(gaps=value) @since("1.4.0") def getGaps(self): @@ -1385,8 +1366,7 @@ def setPattern(self, value): """ Sets the value of :py:attr:`pattern`. """ - self._set(pattern=value) - return self + return self._set(pattern=value) @since("1.4.0") def getPattern(self): @@ -1400,8 +1380,7 @@ def setToLowercase(self, value): """ Sets the value of :py:attr:`toLowercase`. """ - self._set(toLowercase=value) - return self + return self._set(toLowercase=value) @since("2.0.0") def getToLowercase(self): @@ -1462,8 +1441,7 @@ def setStatement(self, value): """ Sets the value of :py:attr:`statement`. """ - self._set(statement=value) - return self + return self._set(statement=value) @since("1.6.0") def getStatement(self): @@ -1540,8 +1518,7 @@ def setWithMean(self, value): """ Sets the value of :py:attr:`withMean`. """ - self._set(withMean=value) - return self + return self._set(withMean=value) @since("1.4.0") def getWithMean(self): @@ -1555,8 +1532,7 @@ def setWithStd(self, value): """ Sets the value of :py:attr:`withStd`. """ - self._set(withStd=value) - return self + return self._set(withStd=value) @since("1.4.0") def getWithStd(self): @@ -1724,8 +1700,7 @@ def setLabels(self, value): """ Sets the value of :py:attr:`labels`. """ - self._set(labels=value) - return self + return self._set(labels=value) @since("1.6.0") def getLabels(self): @@ -1795,8 +1770,7 @@ def setStopWords(self, value): """ Specify the stopwords to be filtered. """ - self._set(stopWords=value) - return self + return self._set(stopWords=value) @since("1.6.0") def getStopWords(self): @@ -1810,8 +1784,7 @@ def setCaseSensitive(self, value): """ Set whether to do a case sensitive comparison over the stop words """ - self._set(caseSensitive=value) - return self + return self._set(caseSensitive=value) @since("1.6.0") def getCaseSensitive(self): @@ -2027,8 +2000,7 @@ def setMaxCategories(self, value): """ Sets the value of :py:attr:`maxCategories`. """ - self._set(maxCategories=value) - return self + return self._set(maxCategories=value) @since("1.4.0") def getMaxCategories(self): @@ -2137,8 +2109,7 @@ def setIndices(self, value): """ Sets the value of :py:attr:`indices`. """ - self._set(indices=value) - return self + return self._set(indices=value) @since("1.6.0") def getIndices(self): @@ -2152,8 +2123,7 @@ def setNames(self, value): """ Sets the value of :py:attr:`names`. """ - self._set(names=value) - return self + return self._set(names=value) @since("1.6.0") def getNames(self): @@ -2261,8 +2231,7 @@ def setVectorSize(self, value): """ Sets the value of :py:attr:`vectorSize`. """ - self._set(vectorSize=value) - return self + return self._set(vectorSize=value) @since("1.4.0") def getVectorSize(self): @@ -2276,8 +2245,7 @@ def setNumPartitions(self, value): """ Sets the value of :py:attr:`numPartitions`. """ - self._set(numPartitions=value) - return self + return self._set(numPartitions=value) @since("1.4.0") def getNumPartitions(self): @@ -2291,8 +2259,7 @@ def setMinCount(self, value): """ Sets the value of :py:attr:`minCount`. """ - self._set(minCount=value) - return self + return self._set(minCount=value) @since("1.4.0") def getMinCount(self): @@ -2306,8 +2273,7 @@ def setWindowSize(self, value): """ Sets the value of :py:attr:`windowSize`. """ - self._set(windowSize=value) - return self + return self._set(windowSize=value) @since("2.0.0") def getWindowSize(self): @@ -2412,8 +2378,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("1.5.0") def getK(self): @@ -2545,8 +2510,7 @@ def setFormula(self, value): """ Sets the value of :py:attr:`formula`. """ - self._set(formula=value) - return self + return self._set(formula=value) @since("1.5.0") def getFormula(self): @@ -2638,8 +2602,7 @@ def setNumTopFeatures(self, value): """ Sets the value of :py:attr:`numTopFeatures`. """ - self._set(numTopFeatures=value) - return self + return self._set(numTopFeatures=value) @since("2.0.0") def getNumTopFeatures(self): diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index a2acf956bc2a..12f14534f4b8 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -85,8 +85,7 @@ def set$Name(self, value): """ Sets the value of :py:attr:`$name`. """ - self._set($name=value) - return self + return self._set($name=value) def get$Name(self): """ diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 538c0b718ad9..9ed63e48ae40 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -34,8 +34,7 @@ def setMaxIter(self, value): """ Sets the value of :py:attr:`maxIter`. """ - self._set(maxIter=value) - return self + return self._set(maxIter=value) def getMaxIter(self): """ @@ -58,8 +57,7 @@ def setRegParam(self, value): """ Sets the value of :py:attr:`regParam`. """ - self._set(regParam=value) - return self + return self._set(regParam=value) def getRegParam(self): """ @@ -83,8 +81,7 @@ def setFeaturesCol(self, value): """ Sets the value of :py:attr:`featuresCol`. """ - self._set(featuresCol=value) - return self + return self._set(featuresCol=value) def getFeaturesCol(self): """ @@ -108,8 +105,7 @@ def setLabelCol(self, value): """ Sets the value of :py:attr:`labelCol`. """ - self._set(labelCol=value) - return self + return self._set(labelCol=value) def getLabelCol(self): """ @@ -133,8 +129,7 @@ def setPredictionCol(self, value): """ Sets the value of :py:attr:`predictionCol`. """ - self._set(predictionCol=value) - return self + return self._set(predictionCol=value) def getPredictionCol(self): """ @@ -158,8 +153,7 @@ def setProbabilityCol(self, value): """ Sets the value of :py:attr:`probabilityCol`. """ - self._set(probabilityCol=value) - return self + return self._set(probabilityCol=value) def getProbabilityCol(self): """ @@ -183,8 +177,7 @@ def setRawPredictionCol(self, value): """ Sets the value of :py:attr:`rawPredictionCol`. """ - self._set(rawPredictionCol=value) - return self + return self._set(rawPredictionCol=value) def getRawPredictionCol(self): """ @@ -207,8 +200,7 @@ def setInputCol(self, value): """ Sets the value of :py:attr:`inputCol`. """ - self._set(inputCol=value) - return self + return self._set(inputCol=value) def getInputCol(self): """ @@ -231,8 +223,7 @@ def setInputCols(self, value): """ Sets the value of :py:attr:`inputCols`. """ - self._set(inputCols=value) - return self + return self._set(inputCols=value) def getInputCols(self): """ @@ -256,8 +247,7 @@ def setOutputCol(self, value): """ Sets the value of :py:attr:`outputCol`. """ - self._set(outputCol=value) - return self + return self._set(outputCol=value) def getOutputCol(self): """ @@ -280,8 +270,7 @@ def setNumFeatures(self, value): """ Sets the value of :py:attr:`numFeatures`. """ - self._set(numFeatures=value) - return self + return self._set(numFeatures=value) def getNumFeatures(self): """ @@ -304,8 +293,7 @@ def setCheckpointInterval(self, value): """ Sets the value of :py:attr:`checkpointInterval`. """ - self._set(checkpointInterval=value) - return self + return self._set(checkpointInterval=value) def getCheckpointInterval(self): """ @@ -329,8 +317,7 @@ def setSeed(self, value): """ Sets the value of :py:attr:`seed`. """ - self._set(seed=value) - return self + return self._set(seed=value) def getSeed(self): """ @@ -353,8 +340,7 @@ def setTol(self, value): """ Sets the value of :py:attr:`tol`. """ - self._set(tol=value) - return self + return self._set(tol=value) def getTol(self): """ @@ -377,8 +363,7 @@ def setStepSize(self, value): """ Sets the value of :py:attr:`stepSize`. """ - self._set(stepSize=value) - return self + return self._set(stepSize=value) def getStepSize(self): """ @@ -401,8 +386,7 @@ def setHandleInvalid(self, value): """ Sets the value of :py:attr:`handleInvalid`. """ - self._set(handleInvalid=value) - return self + return self._set(handleInvalid=value) def getHandleInvalid(self): """ @@ -426,8 +410,7 @@ def setElasticNetParam(self, value): """ Sets the value of :py:attr:`elasticNetParam`. """ - self._set(elasticNetParam=value) - return self + return self._set(elasticNetParam=value) def getElasticNetParam(self): """ @@ -451,8 +434,7 @@ def setFitIntercept(self, value): """ Sets the value of :py:attr:`fitIntercept`. """ - self._set(fitIntercept=value) - return self + return self._set(fitIntercept=value) def getFitIntercept(self): """ @@ -476,8 +458,7 @@ def setStandardization(self, value): """ Sets the value of :py:attr:`standardization`. """ - self._set(standardization=value) - return self + return self._set(standardization=value) def getStandardization(self): """ @@ -500,8 +481,7 @@ def setThresholds(self, value): """ Sets the value of :py:attr:`thresholds`. """ - self._set(thresholds=value) - return self + return self._set(thresholds=value) def getThresholds(self): """ @@ -524,8 +504,7 @@ def setWeightCol(self, value): """ Sets the value of :py:attr:`weightCol`. """ - self._set(weightCol=value) - return self + return self._set(weightCol=value) def getWeightCol(self): """ @@ -549,8 +528,7 @@ def setSolver(self, value): """ Sets the value of :py:attr:`solver`. """ - self._set(solver=value) - return self + return self._set(solver=value) def getSolver(self): """ @@ -573,8 +551,7 @@ def setVarianceCol(self, value): """ Sets the value of :py:attr:`varianceCol`. """ - self._set(varianceCol=value) - return self + return self._set(varianceCol=value) def getVarianceCol(self): """ @@ -603,8 +580,7 @@ def setMaxDepth(self, value): """ Sets the value of :py:attr:`maxDepth`. """ - self._set(maxDepth=value) - return self + return self._set(maxDepth=value) def getMaxDepth(self): """ @@ -616,8 +592,7 @@ def setMaxBins(self, value): """ Sets the value of :py:attr:`maxBins`. """ - self._set(maxBins=value) - return self + return self._set(maxBins=value) def getMaxBins(self): """ @@ -629,8 +604,7 @@ def setMinInstancesPerNode(self, value): """ Sets the value of :py:attr:`minInstancesPerNode`. """ - self._set(minInstancesPerNode=value) - return self + return self._set(minInstancesPerNode=value) def getMinInstancesPerNode(self): """ @@ -642,8 +616,7 @@ def setMinInfoGain(self, value): """ Sets the value of :py:attr:`minInfoGain`. """ - self._set(minInfoGain=value) - return self + return self._set(minInfoGain=value) def getMinInfoGain(self): """ @@ -655,8 +628,7 @@ def setMaxMemoryInMB(self, value): """ Sets the value of :py:attr:`maxMemoryInMB`. """ - self._set(maxMemoryInMB=value) - return self + return self._set(maxMemoryInMB=value) def getMaxMemoryInMB(self): """ @@ -668,8 +640,7 @@ def setCacheNodeIds(self, value): """ Sets the value of :py:attr:`cacheNodeIds`. """ - self._set(cacheNodeIds=value) - return self + return self._set(cacheNodeIds=value) def getCacheNodeIds(self): """ diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 146e403a8f97..eb6d3310d693 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -71,8 +71,7 @@ def setStages(self, value): :param value: a list of transformers or estimators :return: the pipeline instance """ - self._set(stages=value) - return self + return self._set(stages=value) @since("1.3.0") def getStages(self): diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 08770d998127..db02684262cc 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -177,8 +177,7 @@ def setRank(self, value): """ Sets the value of :py:attr:`rank`. """ - self._set(rank=value) - return self + return self._set(rank=value) @since("1.4.0") def getRank(self): @@ -192,8 +191,7 @@ def setNumUserBlocks(self, value): """ Sets the value of :py:attr:`numUserBlocks`. """ - self._set(numUserBlocks=value) - return self + return self._set(numUserBlocks=value) @since("1.4.0") def getNumUserBlocks(self): @@ -207,8 +205,7 @@ def setNumItemBlocks(self, value): """ Sets the value of :py:attr:`numItemBlocks`. """ - self._set(numItemBlocks=value) - return self + return self._set(numItemBlocks=value) @since("1.4.0") def getNumItemBlocks(self): @@ -223,15 +220,14 @@ def setNumBlocks(self, value): Sets both :py:attr:`numUserBlocks` and :py:attr:`numItemBlocks` to the specific value. """ self._set(numUserBlocks=value) - self._set(numItemBlocks=value) + return self._set(numItemBlocks=value) @since("1.4.0") def setImplicitPrefs(self, value): """ Sets the value of :py:attr:`implicitPrefs`. """ - self._set(implicitPrefs=value) - return self + return self._set(implicitPrefs=value) @since("1.4.0") def getImplicitPrefs(self): @@ -245,8 +241,7 @@ def setAlpha(self, value): """ Sets the value of :py:attr:`alpha`. """ - self._set(alpha=value) - return self + return self._set(alpha=value) @since("1.4.0") def getAlpha(self): @@ -260,8 +255,7 @@ def setUserCol(self, value): """ Sets the value of :py:attr:`userCol`. """ - self._set(userCol=value) - return self + return self._set(userCol=value) @since("1.4.0") def getUserCol(self): @@ -275,8 +269,7 @@ def setItemCol(self, value): """ Sets the value of :py:attr:`itemCol`. """ - self._set(itemCol=value) - return self + return self._set(itemCol=value) @since("1.4.0") def getItemCol(self): @@ -290,8 +283,7 @@ def setRatingCol(self, value): """ Sets the value of :py:attr:`ratingCol`. """ - self._set(ratingCol=value) - return self + return self._set(ratingCol=value) @since("1.4.0") def getRatingCol(self): @@ -305,8 +297,7 @@ def setNonnegative(self, value): """ Sets the value of :py:attr:`nonnegative`. """ - self._set(nonnegative=value) - return self + return self._set(nonnegative=value) @since("1.4.0") def getNonnegative(self): @@ -320,8 +311,7 @@ def setIntermediateStorageLevel(self, value): """ Sets the value of :py:attr:`intermediateStorageLevel`. """ - self._set(intermediateStorageLevel=value) - return self + return self._set(intermediateStorageLevel=value) @since("2.0.0") def getIntermediateStorageLevel(self): @@ -335,8 +325,7 @@ def setFinalStorageLevel(self, value): """ Sets the value of :py:attr:`finalStorageLevel`. """ - self._set(finalStorageLevel=value) - return self + return self._set(finalStorageLevel=value) @since("2.0.0") def getFinalStorageLevel(self): diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 0f08f9b9737e..04f566dfecd6 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -469,8 +469,7 @@ def setIsotonic(self, value): """ Sets the value of :py:attr:`isotonic`. """ - self._set(isotonic=value) - return self + return self._set(isotonic=value) def getIsotonic(self): """ @@ -482,8 +481,7 @@ def setFeatureIndex(self, value): """ Sets the value of :py:attr:`featureIndex`. """ - self._set(featureIndex=value) - return self + return self._set(featureIndex=value) def getFeatureIndex(self): """ @@ -532,8 +530,7 @@ def setSubsamplingRate(self, value): """ Sets the value of :py:attr:`subsamplingRate`. """ - self._set(subsamplingRate=value) - return self + return self._set(subsamplingRate=value) @since("1.4.0") def getSubsamplingRate(self): @@ -562,8 +559,7 @@ def setImpurity(self, value): """ Sets the value of :py:attr:`impurity`. """ - self._set(impurity=value) - return self + return self._set(impurity=value) @since("1.4.0") def getImpurity(self): @@ -595,8 +591,7 @@ def setNumTrees(self, value): """ Sets the value of :py:attr:`numTrees`. """ - self._set(numTrees=value) - return self + return self._set(numTrees=value) @since("1.4.0") def getNumTrees(self): @@ -610,8 +605,7 @@ def setFeatureSubsetStrategy(self, value): """ Sets the value of :py:attr:`featureSubsetStrategy`. """ - self._set(featureSubsetStrategy=value) - return self + return self._set(featureSubsetStrategy=value) @since("1.4.0") def getFeatureSubsetStrategy(self): @@ -982,8 +976,7 @@ def setLossType(self, value): """ Sets the value of :py:attr:`lossType`. """ - self._set(lossType=value) - return self + return self._set(lossType=value) @since("1.4.0") def getLossType(self): @@ -1120,8 +1113,7 @@ def setCensorCol(self, value): """ Sets the value of :py:attr:`censorCol`. """ - self._set(censorCol=value) - return self + return self._set(censorCol=value) @since("1.6.0") def getCensorCol(self): @@ -1135,8 +1127,7 @@ def setQuantileProbabilities(self, value): """ Sets the value of :py:attr:`quantileProbabilities`. """ - self._set(quantileProbabilities=value) - return self + return self._set(quantileProbabilities=value) @since("1.6.0") def getQuantileProbabilities(self): @@ -1150,8 +1141,7 @@ def setQuantilesCol(self, value): """ Sets the value of :py:attr:`quantilesCol`. """ - self._set(quantilesCol=value) - return self + return self._set(quantilesCol=value) @since("1.6.0") def getQuantilesCol(self): @@ -1300,8 +1290,7 @@ def setFamily(self, value): """ Sets the value of :py:attr:`family`. """ - self._set(family=value) - return self + return self._set(family=value) @since("2.0.0") def getFamily(self): @@ -1315,8 +1304,7 @@ def setLink(self, value): """ Sets the value of :py:attr:`link`. """ - self._set(link=value) - return self + return self._set(link=value) @since("2.0.0") def getLink(self): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index eb1f029ebb4e..b21cf9255952 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -198,8 +198,7 @@ def setNumFolds(self, value): """ Sets the value of :py:attr:`numFolds`. """ - self._set(numFolds=value) - return self + return self._set(numFolds=value) @since("1.4.0") def getNumFolds(self): @@ -350,8 +349,7 @@ def setTrainRatio(self, value): """ Sets the value of :py:attr:`trainRatio`. """ - self._set(trainRatio=value) - return self + return self._set(trainRatio=value) @since("2.0.0") def getTrainRatio(self): From a373c39a98a395e78ac4c0116c47a9eec39ac3e6 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Tue, 3 May 2016 09:29:49 -0700 Subject: [PATCH 029/313] [SPARK-15091][SPARKR] Fix warnings and a failure in SparkR test cases with testthat version 1.0.1 ## What changes were proposed in this pull request? Fix warnings and a failure in SparkR test cases with testthat version 1.0.1 ## How was this patch tested? SparkR unit test cases. Author: Sun Rui Closes #12867 from sun-rui/SPARK-15091. (cherry picked from commit 8b6491fc0b49b4e363887ae4b452ba69fe0290d5) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_client.R | 2 +- R/pkg/inst/tests/testthat/test_context.R | 2 +- R/pkg/inst/tests/testthat/test_mllib.R | 4 ++-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 11 +++++------ 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_client.R b/R/pkg/inst/tests/testthat/test_client.R index a0664f32f31c..28276a020df9 100644 --- a/R/pkg/inst/tests/testthat/test_client.R +++ b/R/pkg/inst/tests/testthat/test_client.R @@ -32,7 +32,7 @@ test_that("no package specified doesn't add packages flag", { }) test_that("multiple packages don't produce a warning", { - expect_that(generateSparkSubmitArgs("", "", "", "", c("A", "B")), not(gives_warning())) + expect_warning(generateSparkSubmitArgs("", "", "", "", c("A", "B")), NA) }) test_that("sparkJars sparkPackages as character vectors", { diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index ca04342cd512..0e5e15c0a96c 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -138,7 +138,7 @@ test_that("sparkJars sparkPackages as comma-separated strings", { # check normalizePath f <- dir()[[1]] - expect_that(processSparkJars(f), not(gives_warning())) + expect_warning(processSparkJars(f), NA) expect_match(processSparkJars(f), f) }) diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index 37d87aa8a046..5f8a27d4e094 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -450,9 +450,9 @@ test_that("spark.survreg", { if (requireNamespace("survival", quietly = TRUE)) { rData <- list(time = c(4, 3, 1, 1, 2, 2, 3), status = c(1, 1, 1, 0, 1, 1, 0), x = c(0, 2, 1, 1, 1, 0, 0), sex = c(0, 0, 0, 0, 1, 1, 1)) - expect_that( + expect_error( model <- survival::survreg(formula = survival::Surv(time, status) ~ x + sex, data = rData), - not(throws_error())) + NA) expect_equal(predict(model, rData)[[1]], 3.724591, tolerance = 1e-4) } }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 5cf9dc405b16..081f7b166327 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1196,9 +1196,9 @@ test_that("date functions on a DataFrame", { c(as.POSIXlt("2012-12-13 21:34:00 UTC"), as.POSIXlt("2014-12-15 10:24:34 UTC"))) expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], c(as.POSIXlt("2012-12-13 03:34:00 UTC"), as.POSIXlt("2014-12-14 16:24:34 UTC"))) - expect_more_than(collect(select(df2, unix_timestamp()))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) l3 <- list(list(a = 1000), list(a = -1000)) df3 <- createDataFrame(sqlContext, l3) @@ -1502,7 +1502,6 @@ test_that("toJSON() returns an RDD of the correct values", { test_that("showDF()", { df <- read.json(sqlContext, jsonPath) - s <- capture.output(showDF(df)) expected <- paste("+----+-------+\n", "| age| name|\n", "+----+-------+\n", @@ -1510,7 +1509,7 @@ test_that("showDF()", { "| 30| Andy|\n", "| 19| Justin|\n", "+----+-------+\n", sep = "") - expect_output(s, expected) + expect_output(showDF(df), expected) }) test_that("isLocal()", { @@ -1896,7 +1895,7 @@ test_that("Method as.data.frame as a synonym for collect()", { expect_equal(as.data.frame(irisDF2), collect(irisDF2)) # Make sure as.data.frame in the R base package is not covered - expect_that(as.data.frame(c(1, 2)), not(throws_error())) + expect_error(as.data.frame(c(1, 2)), NA) }) test_that("attach() on a DataFrame", { From 17996e7d02b6566d21c352c37ea0ed3e543ded59 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 May 2016 09:43:47 -0700 Subject: [PATCH 030/313] [SPARK-15088] [SQL] Remove SparkSqlSerializer ## What changes were proposed in this pull request? This patch removes SparkSqlSerializer. I believe this is now dead code. ## How was this patch tested? Removed a test case related to it. Author: Reynold Xin Closes #12864 from rxin/SPARK-15088. (cherry picked from commit 5503e453ba00676925531f91f66c0108ac6b1fca) Signed-off-by: Davies Liu --- .../sql/execution/SparkSqlSerializer.scala | 108 ------------------ .../scala/org/apache/spark/sql/RowSuite.scala | 10 -- 2 files changed, 118 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala deleted file mode 100644 index c590f7c6c3e8..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution - -import java.nio.ByteBuffer -import java.util.{HashMap => JavaHashMap} - -import scala.reflect.ClassTag - -import com.esotericsoftware.kryo.{Kryo, Serializer} -import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.ResourcePool - -import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.serializer.{KryoSerializer, SerializerInstance} -import org.apache.spark.sql.types.Decimal -import org.apache.spark.util.MutablePair - -private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { - override def newKryo(): Kryo = { - val kryo = super.newKryo() - kryo.setRegistrationRequired(false) - kryo.register(classOf[MutablePair[_, _]]) - kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) - kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericInternalRow]) - kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) - kryo.register(classOf[java.math.BigDecimal], new JavaBigDecimalSerializer) - kryo.register(classOf[BigDecimal], new ScalaBigDecimalSerializer) - - kryo.register(classOf[Decimal]) - kryo.register(classOf[JavaHashMap[_, _]]) - - kryo.setReferences(false) - kryo - } -} - -private[execution] class KryoResourcePool(size: Int) - extends ResourcePool[SerializerInstance](size) { - - val ser: SparkSqlSerializer = { - val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) - new SparkSqlSerializer(sparkConf) - } - - def newInstance(): SerializerInstance = ser.newInstance() -} - -private[sql] object SparkSqlSerializer { - @transient lazy val resourcePool = new KryoResourcePool(30) - - private[this] def acquireRelease[O](fn: SerializerInstance => O): O = { - val kryo = resourcePool.borrow - try { - fn(kryo) - } finally { - resourcePool.release(kryo) - } - } - - def serialize[T: ClassTag](o: T): Array[Byte] = - acquireRelease { k => - JavaUtils.bufferToArray(k.serialize(o)) - } - - def deserialize[T: ClassTag](bytes: Array[Byte]): T = - acquireRelease { k => - k.deserialize[T](ByteBuffer.wrap(bytes)) - } -} - -private[sql] class JavaBigDecimalSerializer extends Serializer[java.math.BigDecimal] { - def write(kryo: Kryo, output: Output, bd: java.math.BigDecimal) { - // TODO: There are probably more efficient representations than strings... - output.writeString(bd.toString) - } - - def read(kryo: Kryo, input: Input, tpe: Class[java.math.BigDecimal]): java.math.BigDecimal = { - new java.math.BigDecimal(input.readString()) - } -} - -private[sql] class ScalaBigDecimalSerializer extends Serializer[BigDecimal] { - def write(kryo: Kryo, output: Output, bd: BigDecimal) { - // TODO: There are probably more efficient representations than strings... - output.writeString(bd.toString) - } - - def read(kryo: Kryo, input: Input, tpe: Class[BigDecimal]): BigDecimal = { - new java.math.BigDecimal(input.readString()) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 4552eb6ce00a..34936b38fb5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} -import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -55,15 +54,6 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { assert(row.isNullAt(0)) } - test("serialize w/ kryo") { - val row = Seq((1, Seq(1), Map(1 -> 1), BigDecimal(1))).toDF().first() - val serializer = new SparkSqlSerializer(sparkContext.getConf) - val instance = serializer.newInstance() - val ser = instance.serialize(row) - val de = instance.deserialize(ser).asInstanceOf[Row] - assert(de === row) - } - test("get values by field name on Row created via .toDF") { val row = Seq((1, Seq(1))).toDF("a", "b").first() assert(row.getAs[Int]("a") === 1) From 45bc65519d702147d9fb4169efd235199407fa14 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Tue, 3 May 2016 10:10:25 -0700 Subject: [PATCH 031/313] [SPARK-14884][SQL][STREAMING][WEBUI] Fix call site for continuous queries ## What changes were proposed in this pull request? Since we've been processing continuous queries in separate threads, the call sites are then `run at :0`. It's not wrong but provides very little information; in addition, we can not distinguish two queries only from their call sites. This patch fixes this. ### Before [Jobs Tab] ![s1a](https://cloud.githubusercontent.com/assets/15843379/14766101/a47246b2-0a30-11e6-8d81-06a9a600113b.png) [SQL Tab] ![s1b](https://cloud.githubusercontent.com/assets/15843379/14766102/a4750226-0a30-11e6-9ada-773d977d902b.png) ### After [Jobs Tab] ![s2a](https://cloud.githubusercontent.com/assets/15843379/14766104/a89705b6-0a30-11e6-9830-0d40ec68527b.png) [SQL Tab] ![s2b](https://cloud.githubusercontent.com/assets/15843379/14766103/a8966728-0a30-11e6-8e4d-c2e326400478.png) ## How was this patch tested? Manually checks - see screenshots above. Author: Liwei Lin Closes #12650 from lw-lin/fix-call-site. (cherry picked from commit 5bd9a2f697dac44a4777e24321a2eb4a3d54e24b) Signed-off-by: Shixiong Zhu --- .../apache/spark/sql/execution/SQLExecution.scala | 7 +++++-- .../sql/execution/streaming/StreamExecution.scala | 12 ++++++++++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 397d66b31153..31c9f1aef2f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -23,7 +23,6 @@ import org.apache.spark.SparkContext import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} -import org.apache.spark.util.Utils private[sql] object SQLExecution { @@ -46,7 +45,11 @@ private[sql] object SQLExecution { val executionId = SQLExecution.nextExecutionId sc.setLocalProperty(EXECUTION_ID_KEY, executionId.toString) val r = try { - val callSite = Utils.getCallSite() + // sparkContext.getCallSite() would first try to pick up any call site that was previously + // set, then fall back to Utils.getCallSite(); call Utils.getCallSite() directly on + // continuous queries would give us call site like "run at :0" + val callSite = sparkSession.sparkContext.getCallSite() + sparkSession.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart( executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 310834691380..3c5ced2af73f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.util.ContinuousQueryListener import org.apache.spark.sql.util.ContinuousQueryListener._ -import org.apache.spark.util.UninterruptibleThread +import org.apache.spark.util.{UninterruptibleThread, Utils} /** * Manages the execution of a streaming Spark SQL query that is occurring in a separate thread. @@ -101,10 +101,18 @@ class StreamExecution( @volatile private[sql] var streamDeathCause: ContinuousQueryException = null + /* Get the call site in the caller thread; will pass this into the micro batch thread */ + private val callSite = Utils.getCallSite() + /** The thread that runs the micro-batches of this stream. */ private[sql] val microBatchThread = new UninterruptibleThread(s"stream execution thread for $name") { - override def run(): Unit = { runBatches() } + override def run(): Unit = { + // To fix call site like "run at :0", we bridge the call site from the caller + // thread to this micro batch thread + sparkSession.sparkContext.setCallSite(callSite) + runBatches() + } } /** From c5b7e1f70424b176b655936dbcace93e4d4a7210 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 3 May 2016 10:58:26 -0700 Subject: [PATCH 032/313] [SPARK-14716][SQL] Added support for partitioning in FileStreamSink # What changes were proposed in this pull request? Support partitioning in the file stream sink. This is implemented using a new, but simpler code path for writing parquet files - both unpartitioned and partitioned. This new code path does not use Output Committers, as we will eventually write the file names to the metadata log for "committing" them. This patch duplicates < 100 LOC from the WriterContainer. But its far simpler that WriterContainer as it does not involve output committing. In addition, it introduces the new APIs in FileFormat and OutputWriterFactory in an attempt to simplify the APIs (not have Job in the `FileFormat` API, not have bucket and other stuff in the `OutputWriterFactory.newInstance()` ). # Tests - New unit tests to test the FileStreamSinkWriter for partitioned and unpartitioned files - New unit test to partially test the FileStreamSink for partitioned files (does not test recovery of partition column data, as that requires change in the StreamFileCatalog, future PR). - Updated FileStressSuite to test number of records read from partitioned output files. Author: Tathagata Das Closes #12409 from tdas/streaming-partitioned-parquet. (cherry picked from commit 4ad492c40358d0104db508db98ce0971114b6817) Signed-off-by: Tathagata Das --- python/pyspark/sql/tests.py | 4 +- .../execution/datasources/DataSource.scala | 5 +- .../datasources/fileSourceInterfaces.scala | 28 +++ .../datasources/parquet/ParquetRelation.scala | 116 ++++++++- .../execution/streaming/FileStreamSink.scala | 230 ++++++++++++++++-- .../streaming/FileStreamSinkLog.scala | 13 + .../execution/streaming/HDFSMetadataLog.scala | 5 +- .../sql/streaming/FileStreamSinkSuite.scala | 218 +++++++++++++++-- .../spark/sql/streaming/FileStressSuite.scala | 40 ++- 9 files changed, 605 insertions(+), 54 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 4995b263e193..cd5c4a7b3e9f 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -940,7 +940,7 @@ def test_stream_save_options(self): cq.processAllAvailable() output_files = [] for _, _, files in os.walk(out): - output_files.extend([f for f in files if 'parquet' in f and not f.startswith('.')]) + output_files.extend([f for f in files if not f.startswith('.')]) self.assertTrue(len(output_files) > 0) self.assertTrue(len(os.listdir(chk)) > 0) finally: @@ -967,7 +967,7 @@ def test_stream_save_options_overwrite(self): cq.processAllAvailable() output_files = [] for _, _, files in os.walk(out): - output_files.extend([f for f in files if 'parquet' in f and not f.startswith('.')]) + output_files.extend([f for f in files if not f.startswith('.')]) self.assertTrue(len(output_files) > 0) self.assertTrue(len(os.listdir(chk)) > 0) self.assertFalse(os.path.isdir(fake1)) # should not have been created diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 63dc1fd71e6d..6114142cefea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -203,13 +203,14 @@ case class DataSource( def createSink(): Sink = { providingClass.newInstance() match { case s: StreamSinkProvider => s.createSink(sparkSession.wrapped, options, partitionColumns) - case format: FileFormat => + + case parquet: parquet.DefaultSource => val caseInsensitiveOptions = new CaseInsensitiveMap(options) val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") }) + new FileStreamSink(sparkSession, path, parquet, partitionColumns, options) - new FileStreamSink(sparkSession, path, format) case _ => throw new UnsupportedOperationException( s"Data source $className does not support streamed writing") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 25f88d9c3948..0a3461151c62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -64,6 +64,20 @@ abstract class OutputWriterFactory extends Serializable { bucketId: Option[Int], // TODO: This doesn't belong here... dataSchema: StructType, context: TaskAttemptContext): OutputWriter + + /** + * Returns a new instance of [[OutputWriter]] that will write data to the given path. + * This method gets called by each task on executor to write [[InternalRow]]s to + * format-specific files. Compared to the other `newInstance()`, this is a newer API that + * passes only the path that the writer must write to. The writer must write to the exact path + * and not modify it (do not add subdirectories, extensions, etc.). All other + * file-format-specific information needed to create the writer must be passed + * through the [[OutputWriterFactory]] implementation. + * @since 2.0.0 + */ + private[sql] def newWriter(path: String): OutputWriter = { + throw new UnsupportedOperationException("newInstance with just path not supported") + } } /** @@ -223,6 +237,20 @@ trait FileFormat { // Until then we guard in [[FileSourceStrategy]] to only call this method on supported formats. throw new UnsupportedOperationException(s"buildReader is not supported for $this") } + + /** + * Returns a [[OutputWriterFactory]] for generating output writers that can write data. + * This method is current used only by FileStreamSinkWriter to generate output writers that + * does not use output committers to write data. The OutputWriter generated by the returned + * [[OutputWriterFactory]] must implement the method `newWriter(path)`.. + */ + def buildWriter( + sqlContext: SQLContext, + dataSchema: StructType, + options: Map[String, String]): OutputWriterFactory = { + // TODO: Remove this default implementation when the other formats have been ported + throw new UnsupportedOperationException(s"buildWriter is not supported for $this") + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index b1513bbe945b..79185df67325 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -41,13 +41,13 @@ import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{AtomicType, DataType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration private[sql] class DefaultSource @@ -372,8 +372,120 @@ private[sql] class DefaultSource } } } + + override def buildWriter( + sqlContext: SQLContext, + dataSchema: StructType, + options: Map[String, String]): OutputWriterFactory = { + new ParquetOutputWriterFactory( + sqlContext.conf, + dataSchema, + sqlContext.sparkContext.hadoopConfiguration, + options) + } } +/** + * A factory for generating OutputWriters for writing parquet files. This implemented is different + * from the [[ParquetOutputWriter]] as this does not use any [[OutputCommitter]]. It simply + * writes the data to the path used to generate the output writer. Callers of this factory + * has to ensure which files are to be considered as committed. + */ +private[sql] class ParquetOutputWriterFactory( + sqlConf: SQLConf, + dataSchema: StructType, + hadoopConf: Configuration, + options: Map[String, String]) extends OutputWriterFactory { + + private val serializableConf: SerializableConfiguration = { + val job = Job.getInstance(hadoopConf) + val conf = ContextUtil.getConfiguration(job) + val parquetOptions = new ParquetOptions(options, sqlConf) + + // We're not really using `ParquetOutputFormat[Row]` for writing data here, because we override + // it in `ParquetOutputWriter` to support appending and dynamic partitioning. The reason why + // we set it here is to setup the output committer class to `ParquetOutputCommitter`, which is + // bundled with `ParquetOutputFormat[Row]`. + job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) + + ParquetOutputFormat.setWriteSupportClass(job, classOf[CatalystWriteSupport]) + + // We want to clear this temporary metadata from saving into Parquet file. + // This metadata is only useful for detecting optional columns when pushdowning filters. + val dataSchemaToWrite = StructType.removeMetadata( + StructType.metadataKeyForOptionalField, + dataSchema).asInstanceOf[StructType] + CatalystWriteSupport.setSchema(dataSchemaToWrite, conf) + + // Sets flags for `CatalystSchemaConverter` (which converts Catalyst schema to Parquet schema) + // and `CatalystWriteSupport` (writing actual rows to Parquet files). + conf.set( + SQLConf.PARQUET_BINARY_AS_STRING.key, + sqlConf.isParquetBinaryAsString.toString) + + conf.set( + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, + sqlConf.isParquetINT96AsTimestamp.toString) + + conf.set( + SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, + sqlConf.writeLegacyParquetFormat.toString) + + // Sets compression scheme + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + new SerializableConfiguration(conf) + } + + /** + * Returns a [[OutputWriter]] that writes data to the give path without using + * [[OutputCommitter]]. + */ + override private[sql] def newWriter(path: String): OutputWriter = new OutputWriter { + + // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter + private val hadoopTaskAttempId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) + private val hadoopAttemptContext = new TaskAttemptContextImpl( + serializableConf.value, hadoopTaskAttempId) + + // Instance of ParquetRecordWriter that does not use OutputCommitter + private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) + + override def write(row: Row): Unit = { + throw new UnsupportedOperationException("call writeInternal") + } + + protected[sql] override def writeInternal(row: InternalRow): Unit = { + recordWriter.write(null, row) + } + + override def close(): Unit = recordWriter.close(hadoopAttemptContext) + } + + /** Create a [[ParquetRecordWriter]] that writes the given path without using OutputCommitter */ + private def createNoCommitterRecordWriter( + path: String, + hadoopAttemptContext: TaskAttemptContext): RecordWriter[Void, InternalRow] = { + // Custom ParquetOutputFormat that disable use of committer and writes to the given path + val outputFormat = new ParquetOutputFormat[InternalRow]() { + override def getOutputCommitter(c: TaskAttemptContext): OutputCommitter = { null } + override def getDefaultWorkFile(c: TaskAttemptContext, ext: String): Path = { new Path(path) } + } + outputFormat.getRecordWriter(hadoopAttemptContext) + } + + /** Disable the use of the older API. */ + def newInstance( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + throw new UnsupportedOperationException( + "this verison of newInstance not supported for " + + "ParquetOutputWriterFactory") + } +} + + // NOTE: This class is instantiated and used on executor side only, no need to be serializable. private[sql] class ParquetOutputWriter( path: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index 70aea7fa49a1..e19101032967 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -19,11 +19,20 @@ package org.apache.spark.sql.execution.streaming import java.util.UUID +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.{SparkEnv, SparkException, TaskContext, TaskContextImpl} import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.UnsafeKVExternalSorter +import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, PartitioningUtils} +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.SerializableConfiguration object FileStreamSink { // The name of the subdirectory that is used to store metadata about which files are valid. @@ -40,28 +49,24 @@ object FileStreamSink { class FileStreamSink( sparkSession: SparkSession, path: String, - fileFormat: FileFormat) extends Sink with Logging { + fileFormat: FileFormat, + partitionColumnNames: Seq[String], + options: Map[String, String]) extends Sink with Logging { private val basePath = new Path(path) private val logPath = new Path(basePath, FileStreamSink.metadataDir) private val fileLog = new FileStreamSinkLog(sparkSession, logPath.toUri.toString) - private val fs = basePath.getFileSystem(sparkSession.sessionState.newHadoopConf()) + private val hadoopConf = sparkSession.sessionState.newHadoopConf() + private val fs = basePath.getFileSystem(hadoopConf) override def addBatch(batchId: Long, data: DataFrame): Unit = { if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) { logInfo(s"Skipping already committed batch $batchId") } else { - val files = fs.listStatus(writeFiles(data)).map { f => - SinkFileStatus( - path = f.getPath.toUri.toString, - size = f.getLen, - isDir = f.isDirectory, - modificationTime = f.getModificationTime, - blockReplication = f.getReplication, - blockSize = f.getBlockSize, - action = FileStreamSinkLog.ADD_ACTION) - } - if (fileLog.add(batchId, files)) { + val writer = new FileStreamSinkWriter( + data, fileFormat, path, partitionColumnNames, hadoopConf, options) + val fileStatuses = writer.write() + if (fileLog.add(batchId, fileStatuses)) { logInfo(s"Committed batch $batchId") } else { throw new IllegalStateException(s"Race while writing batch $batchId") @@ -69,17 +74,192 @@ class FileStreamSink( } } - /** Writes the [[DataFrame]] to a UUID-named dir, returning the list of files paths. */ - private def writeFiles(data: DataFrame): Array[Path] = { - val file = new Path(basePath, UUID.randomUUID().toString).toUri.toString - data.write.parquet(file) - sparkSession.read - .schema(data.schema) - .parquet(file) - .inputFiles - .map(new Path(_)) - .filterNot(_.getName.startsWith("_")) + override def toString: String = s"FileSink[$path]" +} + + +/** + * Writes data given to a [[FileStreamSink]] to the given `basePath` in the given `fileFormat`, + * partitioned by the given `partitionColumnNames`. This writer always appends data to the + * directory if it already has data. + */ +class FileStreamSinkWriter( + data: DataFrame, + fileFormat: FileFormat, + basePath: String, + partitionColumnNames: Seq[String], + hadoopConf: Configuration, + options: Map[String, String]) extends Serializable with Logging { + + PartitioningUtils.validatePartitionColumnDataTypes( + data.schema, partitionColumnNames, data.sqlContext.conf.caseSensitiveAnalysis) + + private val serializableConf = new SerializableConfiguration(hadoopConf) + private val dataSchema = data.schema + private val dataColumns = data.logicalPlan.output + + // Get the actual partition columns as attributes after matching them by name with + // the given columns names. + private val partitionColumns = partitionColumnNames.map { col => + val nameEquality = if (data.sparkSession.sessionState.conf.caseSensitiveAnalysis) { + org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + } else { + org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution + } + data.logicalPlan.output.find(f => nameEquality(f.name, col)).getOrElse { + throw new RuntimeException(s"Partition column $col not found in schema $dataSchema") + } + } + + // Columns that are to be written to the files. If there are partitioning columns, then + // those will not be written to the files. + private val writeColumns = { + val partitionSet = AttributeSet(partitionColumns) + dataColumns.filterNot(partitionSet.contains) } - override def toString: String = s"FileSink[$path]" + // An OutputWriterFactory for generating writers in the executors for writing the files. + private val outputWriterFactory = + fileFormat.buildWriter(data.sqlContext, writeColumns.toStructType, options) + + /** Expressions that given a partition key build a string like: col1=val/col2=val/... */ + private def partitionStringExpression: Seq[Expression] = { + partitionColumns.zipWithIndex.flatMap { case (c, i) => + val escaped = + ScalaUDF( + PartitioningUtils.escapePathName _, + StringType, + Seq(Cast(c, StringType)), + Seq(StringType)) + val str = If(IsNull(c), Literal(PartitioningUtils.DEFAULT_PARTITION_NAME), escaped) + val partitionName = Literal(c.name + "=") :: str :: Nil + if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName + } + } + + /** Generate a new output writer from the writer factory */ + private def newOutputWriter(path: Path): OutputWriter = { + val newWriter = outputWriterFactory.newWriter(path.toString) + newWriter.initConverter(dataSchema) + newWriter + } + + /** Write the dataframe to files. This gets called in the driver by the [[FileStreamSink]]. */ + def write(): Array[SinkFileStatus] = { + data.sqlContext.sparkContext.runJob( + data.queryExecution.toRdd, + (taskContext: TaskContext, iterator: Iterator[InternalRow]) => { + if (partitionColumns.isEmpty) { + Seq(writePartitionToSingleFile(iterator)) + } else { + writePartitionToPartitionedFiles(iterator) + } + }).flatten + } + + /** + * Writes a RDD partition to a single file without dynamic partitioning. + * This gets called in the executor, and it uses a [[OutputWriter]] to write the data. + */ + def writePartitionToSingleFile(iterator: Iterator[InternalRow]): SinkFileStatus = { + var writer: OutputWriter = null + try { + val path = new Path(basePath, UUID.randomUUID.toString) + val fs = path.getFileSystem(serializableConf.value) + writer = newOutputWriter(path) + while (iterator.hasNext) { + writer.writeInternal(iterator.next) + } + writer.close() + writer = null + SinkFileStatus(fs.getFileStatus(path)) + } catch { + case cause: Throwable => + logError("Aborting task.", cause) + // call failure callbacks first, so we could have a chance to cleanup the writer. + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(cause) + throw new SparkException("Task failed while writing rows.", cause) + } finally { + if (writer != null) { + writer.close() + } + } + } + + /** + * Writes a RDD partition to multiple dynamically partitioned files. + * This gets called in the executor. It first sorts the data based on the partitioning columns + * and then writes the data of each key to separate files using [[OutputWriter]]s. + */ + def writePartitionToPartitionedFiles(iterator: Iterator[InternalRow]): Seq[SinkFileStatus] = { + + // Returns the partitioning columns for sorting + val getSortingKey = UnsafeProjection.create(partitionColumns, dataColumns) + + // Returns the data columns to be written given an input row + val getOutputRow = UnsafeProjection.create(writeColumns, dataColumns) + + // Returns the partition path given a partition key + val getPartitionString = + UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) + + // Sort the data before write, so that we only need one writer at the same time. + val sorter = new UnsafeKVExternalSorter( + partitionColumns.toStructType, + StructType.fromAttributes(writeColumns), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + logDebug(s"Sorting complete. Writing out partition files one at a time.") + + val sortedIterator = sorter.sortedIterator() + val paths = new ArrayBuffer[Path] + + // Write the sorted data to partitioned files, one for each unique key + var currentWriter: OutputWriter = null + try { + var currentKey: UnsafeRow = null + while (sortedIterator.next()) { + val nextKey = sortedIterator.getKey + + // If key changes, close current writer, and open a new writer to a new partitioned file + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + currentKey = nextKey.copy() + val partitionPath = getPartitionString(currentKey).getString(0) + val path = new Path(new Path(basePath, partitionPath), UUID.randomUUID.toString) + paths += path + currentWriter = newOutputWriter(path) + logInfo(s"Writing partition $currentKey to $path") + } + currentWriter.writeInternal(sortedIterator.getValue) + } + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + if (paths.nonEmpty) { + val fs = paths.head.getFileSystem(serializableConf.value) + paths.map(p => SinkFileStatus(fs.getFileStatus(p))) + } else Seq.empty + } catch { + case cause: Throwable => + logError("Aborting task.", cause) + // call failure callbacks first, so we could have a chance to cleanup the writer. + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(cause) + throw new SparkException("Task failed while writing rows.", cause) + } finally { + if (currentWriter != null) { + currentWriter.close() + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala index b694b6155ad9..4254df44c97a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala @@ -54,6 +54,19 @@ case class SinkFileStatus( } } +object SinkFileStatus { + def apply(f: FileStatus): SinkFileStatus = { + SinkFileStatus( + path = f.getPath.toUri.toString, + size = f.getLen, + isDir = f.isDirectory, + modificationTime = f.getModificationTime, + blockReplication = f.getReplication, + blockSize = f.getBlockSize, + action = FileStreamSinkLog.ADD_ACTION) + } +} + /** * A special log for [[FileStreamSink]]. It will write one log file for each batch. The first line * of the log file is the version number, and there are multiple JSON lines following. Each JSON diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 9fe06a6c36cb..fca3d51535a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -216,8 +216,9 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) new FileContextManager(metadataPath, hadoopConf) } catch { case e: UnsupportedFileSystemException => - logWarning("Could not use FileContext API for managing metadata log file. The log may be" + - "inconsistent under failures.", e) + logWarning("Could not use FileContext API for managing metadata log files at path " + + s"$metadataPath. Using FileSystem API instead for managing log files. The log may be " + + s"inconsistent under failures.") new FileSystemManager(metadataPath, hadoopConf) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 8cf5dedabcee..609ca976a016 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -17,33 +17,223 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql.StreamTest -import org.apache.spark.sql.execution.streaming.MemoryStream +import java.io.File + +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.{DirectoryFileFilter, RegexFileFilter} + +import org.apache.spark.sql.{ContinuousQuery, Row, StreamTest} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.execution.datasources.parquet +import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils class FileStreamSinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ - test("unpartitioned writing") { + + test("FileStreamSinkWriter - unpartitioned data") { + val path = Utils.createTempDir() + path.delete() + + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val fileFormat = new parquet.DefaultSource() + + def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { + val df = sqlContext + .range(start, end, 1, numPartitions) + .select($"id", lit(100).as("data")) + val writer = new FileStreamSinkWriter( + df, fileFormat, path.toString, partitionColumnNames = Nil, hadoopConf, Map.empty) + writer.write().map(_.path.stripPrefix("file://")) + } + + // Write and check whether new files are written correctly + val files1 = writeRange(0, 10, 2) + assert(files1.size === 2, s"unexpected number of files: $files1") + checkFilesExist(path, files1, "file not written") + checkAnswer(sqlContext.read.load(path.getCanonicalPath), (0 until 10).map(Row(_, 100))) + + // Append and check whether new files are written correctly and old files still exist + val files2 = writeRange(10, 20, 3) + assert(files2.size === 3, s"unexpected number of files: $files2") + assert(files2.intersect(files1).isEmpty, "old files returned") + checkFilesExist(path, files2, s"New file not written") + checkFilesExist(path, files1, s"Old file not found") + checkAnswer(sqlContext.read.load(path.getCanonicalPath), (0 until 20).map(Row(_, 100))) + } + + test("FileStreamSinkWriter - partitioned data") { + implicit val e = ExpressionEncoder[java.lang.Long] + val path = Utils.createTempDir() + path.delete() + + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val fileFormat = new parquet.DefaultSource() + + def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { + val df = sqlContext + .range(start, end, 1, numPartitions) + .flatMap(x => Iterator(x, x, x)).toDF("id") + .select($"id", lit(100).as("data1"), lit(1000).as("data2")) + + require(df.rdd.partitions.size === numPartitions) + val writer = new FileStreamSinkWriter( + df, fileFormat, path.toString, partitionColumnNames = Seq("id"), hadoopConf, Map.empty) + writer.write().map(_.path.stripPrefix("file://")) + } + + def checkOneFileWrittenPerKey(keys: Seq[Int], filesWritten: Seq[String]): Unit = { + keys.foreach { id => + assert( + filesWritten.count(_.contains(s"/id=$id/")) == 1, + s"no file for id=$id. all files: \n\t${filesWritten.mkString("\n\t")}" + ) + } + } + + // Write and check whether new files are written correctly + val files1 = writeRange(0, 10, 2) + assert(files1.size === 10, s"unexpected number of files:\n${files1.mkString("\n")}") + checkFilesExist(path, files1, "file not written") + checkOneFileWrittenPerKey(0 until 10, files1) + + val answer1 = (0 until 10).flatMap(x => Iterator(x, x, x)).map(Row(100, 1000, _)) + checkAnswer(sqlContext.read.load(path.getCanonicalPath), answer1) + + // Append and check whether new files are written correctly and old files still exist + val files2 = writeRange(0, 20, 3) + assert(files2.size === 20, s"unexpected number of files:\n${files2.mkString("\n")}") + assert(files2.intersect(files1).isEmpty, "old files returned") + checkFilesExist(path, files2, s"New file not written") + checkFilesExist(path, files1, s"Old file not found") + checkOneFileWrittenPerKey(0 until 20, files2) + + val answer2 = (0 until 20).flatMap(x => Iterator(x, x, x)).map(Row(100, 1000, _)) + checkAnswer(sqlContext.read.load(path.getCanonicalPath), answer1 ++ answer2) + } + + test("FileStreamSink - unpartitioned writing and batch reading") { val inputData = MemoryStream[Int] val df = inputData.toDF() val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath - val query = - df.write - .format("parquet") - .option("checkpointLocation", checkpointDir) - .startStream(outputDir) + var query: ContinuousQuery = null + + try { + query = + df.write + .format("parquet") + .option("checkpointLocation", checkpointDir) + .startStream(outputDir) + + inputData.addData(1, 2, 3) + + failAfter(streamingTimeout) { + query.processAllAvailable() + } - inputData.addData(1, 2, 3) - failAfter(streamingTimeout) { query.processAllAvailable() } + val outputDf = sqlContext.read.parquet(outputDir).as[Int] + checkDataset(outputDf, 1, 2, 3) - val outputDf = sqlContext.read.parquet(outputDir).as[Int] - checkDataset( - outputDf, - 1, 2, 3) + } finally { + if (query != null) { + query.stop() + } + } } + + test("FileStreamSink - partitioned writing and batch reading [IGNORES PARTITION COLUMN]") { + val inputData = MemoryStream[Int] + val ds = inputData.toDS() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: ContinuousQuery = null + + try { + query = + ds.map(i => (i, i * 1000)) + .toDF("id", "value") + .write + .format("parquet") + .partitionBy("id") + .option("checkpointLocation", checkpointDir) + .startStream(outputDir) + + inputData.addData(1, 2, 3) + failAfter(streamingTimeout) { + query.processAllAvailable() + } + + // TODO (tdas): Test partition column can be read or not + val outputDf = sqlContext.read.parquet(outputDir) + checkDataset( + outputDf.as[Int], + 1000, 2000, 3000) + + } finally { + if (query != null) { + query.stop() + } + } + } + + test("FileStreamSink - supported formats") { + def testFormat(format: Option[String]): Unit = { + val inputData = MemoryStream[Int] + val ds = inputData.toDS() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: ContinuousQuery = null + + try { + val writer = + ds.map(i => (i, i * 1000)) + .toDF("id", "value") + .write + if (format.nonEmpty) { + writer.format(format.get) + } + query = writer + .option("checkpointLocation", checkpointDir) + .startStream(outputDir) + } finally { + if (query != null) { + query.stop() + } + } + } + + testFormat(None) // should not throw error as default format parquet when not specified + testFormat(Some("parquet")) + val e = intercept[UnsupportedOperationException] { + testFormat(Some("text")) + } + Seq("text", "not support", "stream").foreach { s => + assert(e.getMessage.contains(s)) + } + } + + private def checkFilesExist(dir: File, expectedFiles: Seq[String], msg: String): Unit = { + import scala.collection.JavaConverters._ + val files = + FileUtils.listFiles(dir, new RegexFileFilter("[^.]+"), DirectoryFileFilter.DIRECTORY) + .asScala + .map(_.getCanonicalPath) + .toSet + + expectedFiles.foreach { f => + assert(files.contains(f), + s"\n$msg\nexpected file:\n\t$f\nfound files:\n${files.mkString("\n\t")}") + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala index 5b49a0a86a04..50703e532fa4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala @@ -41,7 +41,15 @@ import org.apache.spark.util.Utils class FileStressSuite extends StreamTest with SharedSQLContext { import testImplicits._ - test("fault tolerance stress test") { + testQuietly("fault tolerance stress test - unpartitioned output") { + stressTest(partitionWrites = false) + } + + testQuietly("fault tolerance stress test - partitioned output") { + stressTest(partitionWrites = true) + } + + def stressTest(partitionWrites: Boolean): Unit = { val numRecords = 10000 val inputDir = Utils.createTempDir(namePrefix = "stream.input").getCanonicalPath val stagingDir = Utils.createTempDir(namePrefix = "stream.staging").getCanonicalPath @@ -93,18 +101,36 @@ class FileStressSuite extends StreamTest with SharedSQLContext { writer.start() val input = sqlContext.read.format("text").stream(inputDir) - def startStream(): ContinuousQuery = input + + def startStream(): ContinuousQuery = { + val output = input .repartition(5) .as[String] .mapPartitions { iter => val rand = Random.nextInt(100) - if (rand < 5) { sys.error("failure") } + if (rand < 10) { + sys.error("failure") + } iter.map(_.toLong) } - .write - .format("parquet") - .option("checkpointLocation", checkpoint) - .startStream(outputDir) + .map(x => (x % 400, x.toString)) + .toDF("id", "data") + + if (partitionWrites) { + output + .write + .partitionBy("id") + .format("parquet") + .option("checkpointLocation", checkpoint) + .startStream(outputDir) + } else { + output + .write + .format("parquet") + .option("checkpointLocation", checkpoint) + .startStream(outputDir) + } + } var failures = 0 val streamThread = new Thread("stream runner") { From 31e5a2a76115e681a2f8ebef06352d417113ae05 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 3 May 2016 11:16:55 -0700 Subject: [PATCH 033/313] [SPARK-14860][TESTS] Create a new Waiter in reset to bypass an issue of ScalaTest's Waiter.wait ## What changes were proposed in this pull request? This PR updates `QueryStatusCollector.reset` to create Waiter instead of calling `await(1 milliseconds)` to bypass an ScalaTest's issue that Waiter.await may block forever. ## How was this patch tested? I created a local stress test to call codes in `test("event ordering")` 100 times. It cannot pass without this patch. Author: Shixiong Zhu Closes #12623 from zsxwing/flaky-test. (cherry picked from commit b545d752195f6dcba4c512b8a1d5bf5b74279dc8) Signed-off-by: Tathagata Das --- .../sql/util/ContinuousQueryListenerSuite.scala | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala index 3498fe83d02e..2596231a1244 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.util import java.util.concurrent.ConcurrentLinkedQueue -import scala.util.control.NonFatal - import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ import org.scalatest.concurrent.AsyncAssertions.Waiter @@ -164,8 +162,8 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with } class QueryStatusCollector extends ContinuousQueryListener { - - private val asyncTestWaiter = new Waiter // to catch errors in the async listener events + // to catch errors in the async listener events + @volatile private var asyncTestWaiter = new Waiter @volatile var startStatus: QueryStatus = null @volatile var terminationStatus: QueryStatus = null @@ -175,11 +173,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with startStatus = null terminationStatus = null progressStatuses.clear() - - // To reset the waiter - try asyncTestWaiter.await(timeout(1 milliseconds)) catch { - case NonFatal(e) => - } + asyncTestWaiter = new Waiter } def checkAsyncErrors(): Unit = { From 2b715251d3bc9fa7de282356e00cd8bf0fcbcdee Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 11:38:43 -0700 Subject: [PATCH 034/313] [SPARK-15087][CORE][SQL] Remove AccumulatorV2.localValue and keep only value ## What changes were proposed in this pull request? Remove AccumulatorV2.localValue and keep only value ## How was this patch tested? existing tests Author: Sandeep Singh Closes #12865 from techaddict/SPARK-15087. (cherry picked from commit ca813330c716bed76ac0034c12f56665960a1105) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/Accumulable.scala | 10 ++++-- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 6 ++-- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../org/apache/spark/util/AccumulatorV2.scala | 32 ++++++------------- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 4 +-- .../sql/execution/metric/SQLMetrics.scala | 4 +-- .../spark/sql/execution/ui/SQLListener.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 2 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- 11 files changed, 30 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index 5c6761eb764b..812145aaeedb 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -110,7 +110,13 @@ class Accumulable[R, T] private ( /** * Access the accumulator's current value; only allowed on driver. */ - def value: R = newAcc.value + def value: R = { + if (newAcc.isAtDriverSide) { + newAcc.value + } else { + throw new UnsupportedOperationException("Can't read accumulator value in task") + } + } /** * Get the current value of this accumulator from within a task. @@ -121,7 +127,7 @@ class Accumulable[R, T] private ( * The typical use of this method is to directly mutate the local value, eg., to add * an element to a Set. */ - def localValue: R = newAcc.localValue + def localValue: R = newAcc.value /** * Set the accumulator's value; only allowed on driver. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4f74dc92d7df..64e87a95d034 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -362,7 +362,7 @@ private[spark] class Executor( Seq.empty } - val accUpdates = accums.map(acc => acc.toInfo(Some(acc.localValue), None)) + val accUpdates = accums.map(acc => acc.toInfo(Some(acc.value), None)) val serializedTaskEndReason = { try { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 085aa7fbd63d..7f4652c2dd76 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -99,7 +99,7 @@ class TaskMetrics private[spark] () extends Serializable { /** * Storage statuses of any blocks that have been updated as a result of this task. */ - def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.value // Setters and increment-ers private[spark] def setExecutorDeserializeTime(v: Long): Unit = @@ -301,12 +301,12 @@ private[spark] class BlockStatusesAccumulator override def merge(other: AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) : Unit = other match { - case o: BlockStatusesAccumulator => _seq ++= o.localValue + case o: BlockStatusesAccumulator => _seq ++= o.value case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: Seq[(BlockId, BlockStatus)] = _seq + override def value: Seq[(BlockId, BlockStatus)] = _seq def setValue(newValue: Seq[(BlockId, BlockStatus)]): Unit = { _seq.clear() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 393680f4c154..8ce8fb15efbc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -394,7 +394,7 @@ private[spark] class TaskSchedulerImpl( // deserialized. This brings trouble to the accumulator framework, which depends on // serialization to set the `atDriverSide` flag. Here we call `acc.localValue` instead to // be more robust about this issue. - val accInfos = updates.map(acc => acc.toInfo(Some(acc.localValue), None)) + val accInfos = updates.map(acc => acc.toInfo(Some(acc.value), None)) taskIdToTaskSetManager.get(id).map { taskSetMgr => (id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, accInfos) } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 0e280f6f6a29..d8f380e1230e 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -126,23 +126,9 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { def merge(other: AccumulatorV2[IN, OUT]): Unit /** - * Access this accumulator's current value; only allowed on driver. + * Defines the current value of this accumulator */ - final def value: OUT = { - if (atDriverSide) { - localValue - } else { - throw new UnsupportedOperationException("Can't read accumulator value in task") - } - } - - /** - * Defines the current value of this accumulator. - * - * This is NOT the global value of the accumulator. To get the global value after a - * completed operation on the dataset, call `value`. - */ - def localValue: OUT + def value: OUT // Called by Java when serializing an object final protected def writeReplace(): Any = { @@ -182,7 +168,7 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { if (metadata == null) { "Un-registered Accumulator: " + getClass.getSimpleName } else { - getClass.getSimpleName + s"(id: $id, name: $name, value: $localValue)" + getClass.getSimpleName + s"(id: $id, name: $name, value: $value)" } } } @@ -321,7 +307,7 @@ class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { private[spark] def setValue(newValue: Long): Unit = _sum = newValue - override def localValue: jl.Long = _sum + override def value: jl.Long = _sum } @@ -386,7 +372,7 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[spark] def setValue(newValue: Double): Unit = _sum = newValue - override def localValue: jl.Double = _sum + override def value: jl.Double = _sum } @@ -400,12 +386,12 @@ class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { override def add(v: T): Unit = _list.add(v) override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match { - case o: ListAccumulator[T] => _list.addAll(o.localValue) + case o: ListAccumulator[T] => _list.addAll(o.value) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: java.util.List[T] = java.util.Collections.unmodifiableList(_list) + override def value: java.util.List[T] = java.util.Collections.unmodifiableList(_list) private[spark] def setValue(newValue: java.util.List[T]): Unit = { _list.clear() @@ -430,10 +416,10 @@ class LegacyAccumulatorWrapper[R, T]( override def add(v: T): Unit = _value = param.addAccumulator(_value, v) override def merge(other: AccumulatorV2[T, R]): Unit = other match { - case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.localValue) + case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.value) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: R = _value + override def value: R = _value } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index aeab71d9df60..18547d459eb5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -841,7 +841,7 @@ private[spark] object JsonProtocol { val accumUpdates = Utils.jsonOption(json \ "Accumulator Updates") .map(_.extract[List[JValue]].map(accumulableInfoFromJson)) .getOrElse(taskMetricsFromJson(json \ "Metrics").accumulators().map(acc => { - acc.toInfo(Some(acc.localValue), None) + acc.toInfo(Some(acc.value), None) })) ExceptionFailure(className, description, stackTrace, fullStackTrace, None, accumUpdates) case `taskResultLost` => TaskResultLost diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index cade67b1d2ba..6cbd5ae5d428 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -70,7 +70,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex // serialize and de-serialize it, to simulate sending accumulator to executor. val acc2 = ser.deserialize[LongAccumulator](ser.serialize(acc)) // value is reset on the executors - assert(acc2.localValue == 0) + assert(acc2.value == 0) assert(!acc2.isAtDriverSide) acc2.add(10) @@ -259,7 +259,7 @@ private[spark] object AccumulatorSuite { * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the * info as an accumulator update. */ - def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.localValue), None) + def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None) /** * Run one or more Spark jobs and verify that in at least one job the peak execution memory diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 0f68aaaee1f2..f82e0b8bca77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -34,7 +34,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato override def copyAndReset(): SQLMetric = new SQLMetric(metricType, initValue) override def merge(other: AccumulatorV2[Long, Long]): Unit = other match { - case o: SQLMetric => _value += o.localValue + case o: SQLMetric => _value += o.value case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } @@ -45,7 +45,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato def +=(v: Long): Unit = _value += v - override def localValue: Long = _value + override def value: Long = _value // Provide special identifier as metadata so we can tell that this is a `SQLMetric` later private[spark] override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 9118593c0e4c..29c54111ea7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -164,7 +164,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi taskEnd.taskInfo.taskId, taskEnd.stageId, taskEnd.stageAttemptId, - taskEnd.taskMetrics.accumulators().map(a => a.toInfo(Some(a.localValue), None)), + taskEnd.taskMetrics.accumulators().map(a => a.toInfo(Some(a.value), None)), finishTask = true) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 8de4d8bbd4e0..d41e88a0aa85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -302,7 +302,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("metrics can be loaded by history server") { val metric = SQLMetrics.createMetric(sparkContext, "zanzibar") metric += 10L - val metricInfo = metric.toInfo(Some(metric.localValue), None) + val metricInfo = metric.toInfo(Some(metric.value), None) metricInfo.update match { case Some(v: Long) => assert(v === 10L) case Some(v) => fail(s"metric value was not a Long: ${v.getClass.getName}") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 964787015a87..5e08658e5efa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -366,7 +366,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { // The listener should only track the ones that are actually SQL metrics. val sqlMetric = SQLMetrics.createMetric(sparkContext, "beach umbrella") val nonSqlMetric = sparkContext.accumulator[Int](0, "baseball") - val sqlMetricInfo = sqlMetric.toInfo(Some(sqlMetric.localValue), None) + val sqlMetricInfo = sqlMetric.toInfo(Some(sqlMetric.value), None) val nonSqlMetricInfo = nonSqlMetric.toInfo(Some(nonSqlMetric.localValue), None) val taskInfo = createTaskInfo(0, 0) taskInfo.accumulables ++= Seq(sqlMetricInfo, nonSqlMetricInfo) From 69e030226b9546bb411280635087fbafaea3848b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Tue, 3 May 2016 11:42:47 -0700 Subject: [PATCH 035/313] [SPARK-9819][STREAMING][DOCUMENTATION] Clarify doc for invReduceFunc in incremental versions of reduceByWindow MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - that reduceFunc and invReduceFunc should be associative - that the intermediate result in iterated applications of inverseReduceFunc is its first argument Author: François Garillot Closes #8103 from huitseeker/issue/invReduceFuncDoc. (cherry picked from commit 439e361010e51d2213c92ccabed5093be92a72ee) Signed-off-by: Shixiong Zhu --- python/pyspark/streaming/dstream.py | 4 +++- .../org/apache/spark/streaming/api/java/JavaDStreamLike.scala | 3 ++- .../org/apache/spark/streaming/api/java/JavaPairDStream.scala | 3 ++- .../scala/org/apache/spark/streaming/dstream/DStream.scala | 3 ++- .../apache/spark/streaming/dstream/PairDStreamFunctions.scala | 3 ++- 5 files changed, 11 insertions(+), 5 deletions(-) diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 205666387219..67a0819601ef 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -454,7 +454,9 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio This is more efficient than `invReduceFunc` is None. @param reduceFunc: associative and commutative reduce function - @param invReduceFunc: inverse reduce function of `reduceFunc` + @param invReduceFunc: inverse reduce function of `reduceFunc`; such that for all y, + and invertible x: + `invReduceFunc(reduceFunc(x, y), x) = y` @param windowDuration: width of the window; must be a multiple of this DStream's batching interval @param slideDuration: sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 43632f37ccb1..a0a40fcee26d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -240,7 +240,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2a80cf446658..dec983165fb3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -336,7 +336,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse function + * @param invReduceFunc inverse function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 583f5a48d1a6..01dcfcf24b0f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -793,7 +793,8 @@ abstract class DStream[T: ClassTag] ( * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index b6394e36b515..2f2a6d13dd79 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -290,7 +290,8 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which From a08d2c9541d0962a6344b0f4e7000b86a753de87 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 11:45:51 -0700 Subject: [PATCH 036/313] [SPARK-15082][CORE] Improve unit test coverage for AccumulatorV2 ## What changes were proposed in this pull request? Added tests for ListAccumulator and LegacyAccumulatorWrapper, test for ListAccumulator is one similar to old Collection Accumulators ## How was this patch tested? Ran tests locally. cc rxin Author: Sandeep Singh Closes #12862 from techaddict/SPARK-15082. (cherry picked from commit 84b3a4a873de8444eda1bd162ed52f79941ec864) Signed-off-by: Reynold Xin --- .../spark/util/AccumulatorV2Suite.scala | 61 ++++++++++++++++++- 1 file changed, 60 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala index 815b13488401..ecaf4f0c6436 100644 --- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import org.apache.spark.SparkFunSuite +import org.apache.spark._ class AccumulatorV2Suite extends SparkFunSuite { @@ -86,4 +86,63 @@ class AccumulatorV2Suite extends SparkFunSuite { assert(acc.sum == 5.0) assert(acc.avg == 1.25) } + + test("ListAccumulator") { + val acc = new ListAccumulator[Double] + assert(acc.value.isEmpty) + assert(acc.isZero) + + acc.add(0.0) + assert(acc.value.contains(0.0)) + assert(!acc.isZero) + + acc.add(new java.lang.Double(1.0)) + + val acc2 = acc.copyAndReset() + assert(acc2.value.isEmpty) + assert(acc2.isZero) + + assert(acc.value.contains(1.0)) + assert(!acc.isZero) + assert(acc.value.size() === 2) + + acc2.add(2.0) + assert(acc2.value.contains(2.0)) + assert(!acc2.isZero) + assert(acc2.value.size() === 1) + + // Test merging + acc.merge(acc2) + assert(acc.value.contains(2.0)) + assert(!acc.isZero) + assert(acc.value.size() === 3) + } + + test("LegacyAccumulatorWrapper") { + val acc = new LegacyAccumulatorWrapper("default", AccumulatorParam.StringAccumulatorParam) + assert(acc.value === "default") + assert(!acc.isZero) + + acc.add("foo") + assert(acc.value === "foo") + assert(!acc.isZero) + + acc.add(new java.lang.String("bar")) + + val acc2 = acc.copyAndReset() + assert(acc2.value === "") + assert(acc2.isZero) + + assert(acc.value === "bar") + assert(!acc.isZero) + + acc2.add("baz") + assert(acc2.value === "baz") + assert(!acc2.isZero) + + // Test merging + acc.merge(acc2) + assert(acc.value === "baz") + assert(!acc.isZero) + } } From 50adfcdce394f49057f768aa4af5164f96703ed9 Mon Sep 17 00:00:00 2001 From: Zheng Tan Date: Tue, 3 May 2016 12:22:52 -0700 Subject: [PATCH 037/313] [SPARK-15059][CORE] Remove fine-grained lock in ChildFirstURLClassLoader to avoid dead lock ## What changes were proposed in this pull request? In some cases, fine-grained lock have race condition with class-loader lock and have caused dead lock issue. It is safe to drop this fine grained lock and load all classes by single class-loader lock. Author: Zheng Tan Closes #12857 from tankkyo/master. (cherry picked from commit f5623b460224ce363316c63f5d28947215078fc5) Signed-off-by: Shixiong Zhu --- .../spark/util/MutableURLClassLoader.scala | 31 +++---------------- 1 file changed, 5 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala index 0a3180da8798..034826c57ef1 100644 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -19,7 +19,6 @@ package org.apache.spark.util import java.net.{URL, URLClassLoader} import java.util.Enumeration -import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -48,32 +47,12 @@ private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoa private val parentClassLoader = new ParentClassLoader(parent) - /** - * Used to implement fine-grained class loading locks similar to what is done by Java 7. This - * prevents deadlock issues when using non-hierarchical class loaders. - * - * Note that due to some issues with implementing class loaders in - * Scala, Java 7's `ClassLoader.registerAsParallelCapable` method is not called. - */ - private val locks = new ConcurrentHashMap[String, Object]() - override def loadClass(name: String, resolve: Boolean): Class[_] = { - var lock = locks.get(name) - if (lock == null) { - val newLock = new Object() - lock = locks.putIfAbsent(name, newLock) - if (lock == null) { - lock = newLock - } - } - - lock.synchronized { - try { - super.loadClass(name, resolve) - } catch { - case e: ClassNotFoundException => - parentClassLoader.loadClass(name, resolve) - } + try { + super.loadClass(name, resolve) + } catch { + case e: ClassNotFoundException => + parentClassLoader.loadClass(name, resolve) } } From 4bacdebe95c380a281bb2fc02270dc181e26aa14 Mon Sep 17 00:00:00 2001 From: Devaraj K Date: Tue, 3 May 2016 13:25:28 -0700 Subject: [PATCH 038/313] [SPARK-14234][CORE] Executor crashes for TaskRunner thread interruption ## What changes were proposed in this pull request? Resetting the task interruption status before updating the task status. ## How was this patch tested? I have verified it manually by running multiple applications, Executor doesn't crash and updates the status to the driver without any exceptions with the patch changes. Author: Devaraj K Closes #12031 from devaraj-kavali/SPARK-14234. (cherry picked from commit 659f635d3bd0c0d025bf514dfb1747ed7386ba45) Signed-off-by: Shixiong Zhu --- .../org/apache/spark/executor/Executor.scala | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 64e87a95d034..b695aecc13ea 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,6 +23,7 @@ import java.net.URL import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -194,6 +195,10 @@ private[spark] class Executor( /** Whether this task has been killed. */ @volatile private var killed = false + /** Whether this task has been finished. */ + @GuardedBy("TaskRunner.this") + private var finished = false + /** How much the JVM process has spent in GC when the task starts to run. */ @volatile var startGCTime: Long = _ @@ -207,10 +212,25 @@ private[spark] class Executor( logInfo(s"Executor is trying to kill $taskName (TID $taskId)") killed = true if (task != null) { - task.kill(interruptThread) + synchronized { + if (!finished) { + task.kill(interruptThread) + } + } } } + /** + * Set the finished flag to true and clear the current thread's interrupt status + */ + private def setTaskFinishedAndClearInterruptStatus(): Unit = synchronized { + this.finished = true + // SPARK-14234 - Reset the interrupted status of the thread to avoid the + // ClosedByInterruptException during execBackend.statusUpdate which causes + // Executor to crash + Thread.interrupted() + } + override def run(): Unit = { val taskMemoryManager = new TaskMemoryManager(env.memoryManager, taskId) val deserializeStartTime = System.currentTimeMillis() @@ -336,14 +356,17 @@ private[spark] class Executor( } catch { case ffe: FetchFailedException => val reason = ffe.toTaskEndReason + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) case _: TaskKilledException | _: InterruptedException if task.killed => logInfo(s"Executor killed $taskName (TID $taskId)") + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) case CausedBy(cDE: CommitDeniedException) => val reason = cDE.toTaskEndReason + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) case t: Throwable => @@ -373,6 +396,7 @@ private[spark] class Executor( ser.serialize(new ExceptionFailure(t, accUpdates, false).withAccums(accums)) } } + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) // Don't forcibly exit unless the exception was inherently fatal, to avoid From a2369c63f905e31c78eb64e547b279b33e30a07c Mon Sep 17 00:00:00 2001 From: yzhou2001 Date: Tue, 3 May 2016 13:41:04 -0700 Subject: [PATCH 039/313] [SPARK-14521] [SQL] StackOverflowError in Kryo when executing TPC-DS ## What changes were proposed in this pull request? Observed stackOverflowError in Kryo when executing TPC-DS Query27. Spark thrift server disables kryo reference tracking (if not specified in conf). When "spark.kryo.referenceTracking" is set to true explicitly in spark-defaults.conf, query executes successfully. The root cause is that the TaskMemoryManager inside MemoryConsumer and LongToUnsafeRowMap were not transient and thus were serialized and broadcast around from within LongHashedRelation, which could potentially cause circular reference inside Kryo. But the TaskMemoryManager is per task and should not be passed around at the first place. This fix makes it transient. ## How was this patch tested? core/test, hive/test, sql/test, catalyst/test, dev/lint-scala, org.apache.spark.sql.hive.execution.HiveCompatibilitySuite, dev/scalastyle, manual test of TBC-DS Query 27 with 1GB data but without the "limit 100" which would cause a NPE due to SPARK-14752. Author: yzhou2001 Closes #12598 from yzhou2001/master. (cherry picked from commit a4aed71719b4fc728de93afc623aef05d27bc89a) Signed-off-by: Davies Liu --- .../sql/execution/joins/HashedRelation.scala | 136 ++++++++++++------ .../execution/joins/HashedRelationSuite.scala | 35 +++++ 2 files changed, 129 insertions(+), 42 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index b280c76c70a6..315ef6a8796f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.execution.joins -import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import java.io._ + +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.{SparkConf, SparkEnv, SparkException} import org.apache.spark.memory.{MemoryConsumer, MemoryMode, StaticMemoryManager, TaskMemoryManager} @@ -116,7 +119,7 @@ private[execution] object HashedRelation { private[joins] class UnsafeHashedRelation( private var numFields: Int, private var binaryMap: BytesToBytesMap) - extends HashedRelation with Externalizable { + extends HashedRelation with Externalizable with KryoSerializable { private[joins] def this() = this(0, null) // Needed for serialization @@ -171,10 +174,21 @@ private[joins] class UnsafeHashedRelation( } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - out.writeInt(numFields) + write(out.writeInt, out.writeLong, out.write) + } + + override def write(kryo: Kryo, out: Output): Unit = Utils.tryOrIOException { + write(out.writeInt, out.writeLong, out.write) + } + + private def write( + writeInt: (Int) => Unit, + writeLong: (Long) => Unit, + writeBuffer: (Array[Byte], Int, Int) => Unit) : Unit = { + writeInt(numFields) // TODO: move these into BytesToBytesMap - out.writeLong(binaryMap.numKeys()) - out.writeLong(binaryMap.numValues()) + writeLong(binaryMap.numKeys()) + writeLong(binaryMap.numValues()) var buffer = new Array[Byte](64) def write(base: Object, offset: Long, length: Int): Unit = { @@ -182,25 +196,32 @@ private[joins] class UnsafeHashedRelation( buffer = new Array[Byte](length) } Platform.copyMemory(base, offset, buffer, Platform.BYTE_ARRAY_OFFSET, length) - out.write(buffer, 0, length) + writeBuffer(buffer, 0, length) } val iter = binaryMap.iterator() while (iter.hasNext) { val loc = iter.next() // [key size] [values size] [key bytes] [value bytes] - out.writeInt(loc.getKeyLength) - out.writeInt(loc.getValueLength) + writeInt(loc.getKeyLength) + writeInt(loc.getValueLength) write(loc.getKeyBase, loc.getKeyOffset, loc.getKeyLength) write(loc.getValueBase, loc.getValueOffset, loc.getValueLength) } } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - numFields = in.readInt() + read(in.readInt, in.readLong, in.readFully) + } + + private def read( + readInt: () => Int, + readLong: () => Long, + readBuffer: (Array[Byte], Int, Int) => Unit): Unit = { + numFields = readInt() resultRow = new UnsafeRow(numFields) - val nKeys = in.readLong() - val nValues = in.readLong() + val nKeys = readLong() + val nValues = readLong() // This is used in Broadcast, shared by multiple tasks, so we use on-heap memory // TODO(josh): This needs to be revisited before we merge this patch; making this change now // so that tests compile: @@ -227,16 +248,16 @@ private[joins] class UnsafeHashedRelation( var keyBuffer = new Array[Byte](1024) var valuesBuffer = new Array[Byte](1024) while (i < nValues) { - val keySize = in.readInt() - val valuesSize = in.readInt() + val keySize = readInt() + val valuesSize = readInt() if (keySize > keyBuffer.length) { keyBuffer = new Array[Byte](keySize) } - in.readFully(keyBuffer, 0, keySize) + readBuffer(keyBuffer, 0, keySize) if (valuesSize > valuesBuffer.length) { valuesBuffer = new Array[Byte](valuesSize) } - in.readFully(valuesBuffer, 0, valuesSize) + readBuffer(valuesBuffer, 0, valuesSize) val loc = binaryMap.lookup(keyBuffer, Platform.BYTE_ARRAY_OFFSET, keySize) val putSuceeded = loc.append(keyBuffer, Platform.BYTE_ARRAY_OFFSET, keySize, @@ -248,6 +269,10 @@ private[joins] class UnsafeHashedRelation( i += 1 } } + + override def read(kryo: Kryo, in: Input): Unit = Utils.tryOrIOException { + read(in.readInt, in.readLong, in.readBytes) + } } private[joins] object UnsafeHashedRelation { @@ -324,8 +349,8 @@ private[joins] object UnsafeHashedRelation { * * see http://java-performance.info/implementing-world-fastest-java-int-to-int-hash-map/ */ -private[execution] final class LongToUnsafeRowMap(var mm: TaskMemoryManager, capacity: Int) - extends MemoryConsumer(mm) with Externalizable { +private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, capacity: Int) + extends MemoryConsumer(mm) with Externalizable with KryoSerializable { // Whether the keys are stored in dense mode or not. private var isDense = false @@ -624,58 +649,85 @@ private[execution] final class LongToUnsafeRowMap(var mm: TaskMemoryManager, cap } } - private def writeLongArray(out: ObjectOutput, arr: Array[Long], len: Int): Unit = { + private def writeLongArray( + writeBuffer: (Array[Byte], Int, Int) => Unit, + arr: Array[Long], + len: Int): Unit = { val buffer = new Array[Byte](4 << 10) var offset: Long = Platform.LONG_ARRAY_OFFSET val end = len * 8L + Platform.LONG_ARRAY_OFFSET while (offset < end) { val size = Math.min(buffer.length, (end - offset).toInt) Platform.copyMemory(arr, offset, buffer, Platform.BYTE_ARRAY_OFFSET, size) - out.write(buffer, 0, size) + writeBuffer(buffer, 0, size) offset += size } } - override def writeExternal(out: ObjectOutput): Unit = { - out.writeBoolean(isDense) - out.writeLong(minKey) - out.writeLong(maxKey) - out.writeLong(numKeys) - out.writeLong(numValues) - - out.writeLong(array.length) - writeLongArray(out, array, array.length) + private def write( + writeBoolean: (Boolean) => Unit, + writeLong: (Long) => Unit, + writeBuffer: (Array[Byte], Int, Int) => Unit): Unit = { + writeBoolean(isDense) + writeLong(minKey) + writeLong(maxKey) + writeLong(numKeys) + writeLong(numValues) + + writeLong(array.length) + writeLongArray(writeBuffer, array, array.length) val used = ((cursor - Platform.LONG_ARRAY_OFFSET) / 8).toInt - out.writeLong(used) - writeLongArray(out, page, used) + writeLong(used) + writeLongArray(writeBuffer, page, used) } - private def readLongArray(in: ObjectInput, length: Int): Array[Long] = { + override def writeExternal(output: ObjectOutput): Unit = { + write(output.writeBoolean, output.writeLong, output.write) + } + + override def write(kryo: Kryo, out: Output): Unit = { + write(out.writeBoolean, out.writeLong, out.write) + } + + private def readLongArray( + readBuffer: (Array[Byte], Int, Int) => Unit, + length: Int): Array[Long] = { val array = new Array[Long](length) val buffer = new Array[Byte](4 << 10) var offset: Long = Platform.LONG_ARRAY_OFFSET val end = length * 8L + Platform.LONG_ARRAY_OFFSET while (offset < end) { val size = Math.min(buffer.length, (end - offset).toInt) - in.readFully(buffer, 0, size) + readBuffer(buffer, 0, size) Platform.copyMemory(buffer, Platform.BYTE_ARRAY_OFFSET, array, offset, size) offset += size } array } - override def readExternal(in: ObjectInput): Unit = { - isDense = in.readBoolean() - minKey = in.readLong() - maxKey = in.readLong() - numKeys = in.readLong - numValues = in.readLong() + private def read( + readBoolean: () => Boolean, + readLong: () => Long, + readBuffer: (Array[Byte], Int, Int) => Unit): Unit = { + isDense = readBoolean() + minKey = readLong() + maxKey = readLong() + numKeys = readLong() + numValues = readLong() - val length = in.readLong().toInt + val length = readLong().toInt mask = length - 2 - array = readLongArray(in, length) - val pageLength = in.readLong().toInt - page = readLongArray(in, pageLength) + array = readLongArray(readBuffer, length) + val pageLength = readLong().toInt + page = readLongArray(readBuffer, pageLength) + } + + override def readExternal(in: ObjectInput): Unit = { + read(in.readBoolean, in.readLong, in.readFully) + } + + override def read(kryo: Kryo, in: Input): Unit = { + read(in.readBoolean, in.readLong, in.readBytes) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 3ee25c099603..9826a64fe280 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -21,6 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.SharedSQLContext @@ -151,6 +152,40 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { } } + test("Spark-14521") { + val ser = new KryoSerializer( + (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() + val key = Seq(BoundReference(0, IntegerType, false)) + + // Testing Kryo serialization of HashedRelation + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, IntegerType, false), BoundReference(1, IntegerType, true))) + val rows = (0 until 100).map(i => unsafeProj(InternalRow(i, i + 1)).copy()) + val longRelation = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) + val longRelation2 = ser.deserialize[LongHashedRelation](ser.serialize(longRelation)) + (0 until 100).foreach { i => + val rows = longRelation2.get(i).toArray + assert(rows.length === 2) + assert(rows(0).getInt(0) === i) + assert(rows(0).getInt(1) === i + 1) + assert(rows(1).getInt(0) === i) + assert(rows(1).getInt(1) === i + 1) + } + + // Testing Kryo serialization of UnsafeHashedRelation + val unsafeHashed = UnsafeHashedRelation(rows.iterator, key, 1, mm) + val os = new ByteArrayOutputStream() + val out = new ObjectOutputStream(os) + unsafeHashed.asInstanceOf[UnsafeHashedRelation].writeExternal(out) + out.flush() + val unsafeHashed2 = ser.deserialize[UnsafeHashedRelation](ser.serialize(unsafeHashed)) + val os2 = new ByteArrayOutputStream() + val out2 = new ObjectOutputStream(os2) + unsafeHashed2.writeExternal(out2) + out2.flush() + assert(java.util.Arrays.equals(os.toByteArray, os2.toByteArray)) + } + // This test require 4G heap to run, should run it manually ignore("build HashedRelation that is larger than 1G") { val unsafeProj = UnsafeProjection.create( From b67668bc4f6d79aeeccc63578e6f0f7a7febd48d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 3 May 2016 13:43:20 -0700 Subject: [PATCH 040/313] [SPARK-11316] coalesce doesn't handle UnionRDD with partial locality properly ## What changes were proposed in this pull request? coalesce doesn't handle UnionRDD with partial locality properly. I had a user who had a UnionRDD that was made up of mapPartitionRDD without preferred locations and a checkpointedRDD with preferred locations (getting from hdfs). It took the driver over 20 minutes to setup the groups and put the partitions into those groups before it even started any tasks. Even perhaps worse is it didn't end up with the number of partitions he was asking for because it didn't put a partition in each of the groups properly. The changes in this patch get rid of a n^2 while loop that was causing the 20 minutes, it properly distributes the partitions to have at least one per group, and it changes from using the rotation iterator which got the preferred locations many times to get all the preferred locations once up front. Note that the n^2 while loop that I removed in setupGroups took so long because all of the partitions with preferred locations were already assigned to group, so it basically looped through every single one and wasn't ever able to assign it. At the time I had 960 partitions with preferred locations and 1020 without and did the outer while loop 319 times because that is the # of groups left to create. Note that each of those times through the inner while loop is going off to hdfs to get the block locations, so this is extremely inefficient. ## How was the this patch tested? Added unit tests for this case and ran existing ones that applied to make sure no regressions. Also manually tested on the users production job to make sure it fixed their issue. It created the proper number of partitions and now it takes about 6 seconds rather then 20 minutes. I did also run some basic manual tests with spark-shell doing coalesced to smaller number, same number, and then greater with shuffle. Author: Thomas Graves Closes #11327 from tgravescs/SPARK-11316. (cherry picked from commit 83ee92f60345f016a390d61a82f1d924f64ddf90) Signed-off-by: Davies Liu --- .../org/apache/spark/rdd/CoalescedRDD.scala | 158 +++++++++++------- .../scala/org/apache/spark/rdd/RDDSuite.scala | 69 ++++++++ 2 files changed, 165 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index e75f1dbf8107..c19ed1529bbf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -169,42 +169,37 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) var noLocality = true // if true if no preferredLocations exists for parent RDD - // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currPrefLocs(part: Partition, prev: RDD[_]): Seq[String] = { - prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) - } - - // this class just keeps iterating and rotating infinitely over the partitions of the RDD - // next() returns the next preferred machine that a partition is replicated on - // the rotator first goes through the first replica copy of each partition, then second, third - // the iterators return type is a tuple: (replicaString, partition) - class LocationIterator(prev: RDD[_]) extends Iterator[(String, Partition)] { - - var it: Iterator[(String, Partition)] = resetIterator() - - override val isEmpty = !it.hasNext - - // initializes/resets to start iterating from the beginning - def resetIterator(): Iterator[(String, Partition)] = { - val iterators = (0 to 2).map { x => - prev.partitions.iterator.flatMap { p => - if (currPrefLocs(p, prev).size > x) Some((currPrefLocs(p, prev)(x), p)) else None + class PartitionLocations(prev: RDD[_]) { + + // contains all the partitions from the previous RDD that don't have preferred locations + val partsWithoutLocs = ArrayBuffer[Partition]() + // contains all the partitions from the previous RDD that have preferred locations + val partsWithLocs = ArrayBuffer[(String, Partition)]() + + getAllPrefLocs(prev) + + // gets all the preffered locations of the previous RDD and splits them into partitions + // with preferred locations and ones without + def getAllPrefLocs(prev: RDD[_]) { + val tmpPartsWithLocs = mutable.LinkedHashMap[Partition, Seq[String]]() + // first get the locations for each partition, only do this once since it can be expensive + prev.partitions.foreach(p => { + val locs = prev.context.getPreferredLocs(prev, p.index).map(tl => tl.host) + if (locs.size > 0) { + tmpPartsWithLocs.put(p, locs) + } else { + partsWithoutLocs += p + } } - } - iterators.reduceLeft((x, y) => x ++ y) - } - - // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD - override def hasNext: Boolean = { !isEmpty } - - // return the next preferredLocation of some partition of the RDD - override def next(): (String, Partition) = { - if (it.hasNext) { - it.next() - } else { - it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning - it.next() - } + ) + // convert it into an array of host to partition + (0 to 2).map(x => + tmpPartsWithLocs.foreach(parts => { + val p = parts._1 + val locs = parts._2 + if (locs.size > x) partsWithLocs += ((locs(x), p)) + } ) + ) } } @@ -228,33 +223,32 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } /** - * Initializes targetLen partition groups and assigns a preferredLocation - * This uses coupon collector to estimate how many preferredLocations it must rotate through - * until it has seen most of the preferred locations (2 * n log(n)) + * Initializes targetLen partition groups. If there are preferred locations, each group + * is assigned a preferredLocation. This uses coupon collector to estimate how many + * preferredLocations it must rotate through until it has seen most of the preferred + * locations (2 * n log(n)) * @param targetLen */ - def setupGroups(targetLen: Int, prev: RDD[_]) { - val rotIt = new LocationIterator(prev) - + def setupGroups(targetLen: Int, partitionLocs: PartitionLocations) { // deal with empty case, just create targetLen partition groups with no preferred location - if (!rotIt.hasNext) { + if (partitionLocs.partsWithLocs.isEmpty) { (1 to targetLen).foreach(x => groupArr += new PartitionGroup()) return } noLocality = false - // number of iterations needed to be certain that we've seen most preferred locations val expectedCoupons2 = 2 * (math.log(targetLen)*targetLen + targetLen + 0.5).toInt var numCreated = 0 var tries = 0 // rotate through until either targetLen unique/distinct preferred locations have been created - // OR we've rotated expectedCoupons2, in which case we have likely seen all preferred locations, - // i.e. likely targetLen >> number of preferred locations (more buckets than there are machines) - while (numCreated < targetLen && tries < expectedCoupons2) { + // OR (we have went through either all partitions OR we've rotated expectedCoupons2 - in + // which case we have likely seen all preferred locations) + val numPartsToLookAt = math.min(expectedCoupons2, partitionLocs.partsWithLocs.length) + while (numCreated < targetLen && tries < numPartsToLookAt) { + val (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) tries += 1 - val (nxt_replica, nxt_part) = rotIt.next() if (!groupHash.contains(nxt_replica)) { val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup @@ -263,20 +257,18 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) numCreated += 1 } } - - while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates - var (nxt_replica, nxt_part) = rotIt.next() + tries = 0 + // if we don't have enough partition groups, create duplicates + while (numCreated < targetLen) { + var (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) + tries += 1 val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup groupHash.getOrElseUpdate(nxt_replica, ArrayBuffer()) += pgroup - var tries = 0 - while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure at least one part - nxt_part = rotIt.next()._2 - tries += 1 - } + addPartToPGroup(nxt_part, pgroup) numCreated += 1 + if (tries >= partitionLocs.partsWithLocs.length) tries = 0 } - } /** @@ -289,10 +281,15 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) * imbalance in favor of locality * @return partition group (bin to be put in) */ - def pickBin(p: Partition, prev: RDD[_], balanceSlack: Double): PartitionGroup = { + def pickBin( + p: Partition, + prev: RDD[_], + balanceSlack: Double, + partitionLocs: PartitionLocations): PartitionGroup = { val slack = (balanceSlack * prev.partitions.length).toInt + val preflocs = partitionLocs.partsWithLocs.filter(_._2 == p).map(_._1).toSeq // least loaded pref locs - val pref = currPrefLocs(p, prev).map(getLeastGroupHash(_)).sortWith(compare) + val pref = preflocs.map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) @@ -320,7 +317,10 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } } - def throwBalls(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { + def throwBalls( + maxPartitions: Int, + prev: RDD[_], + balanceSlack: Double, partitionLocs: PartitionLocations) { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p, i) <- prev.partitions.zipWithIndex) { @@ -334,8 +334,39 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } } } else { + // It is possible to have unionRDD where one rdd has preferred locations and another rdd + // that doesn't. To make sure we end up with the requested number of partitions, + // make sure to put a partition in every group. + + // if we don't have a partition assigned to every group first try to fill them + // with the partitions with preferred locations + val partIter = partitionLocs.partsWithLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partIter.hasNext && pg.numPartitions == 0) { + var (nxt_replica, nxt_part) = partIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // if we didn't get one partitions per group from partitions with preferred locations + // use partitions without preferred locations + val partNoLocIter = partitionLocs.partsWithoutLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partNoLocIter.hasNext && pg.numPartitions == 0) { + var nxt_part = partNoLocIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // finally pick bin for the rest for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group - pickBin(p, prev, balanceSlack).partitions += p + pickBin(p, prev, balanceSlack, partitionLocs).partitions += p } } } @@ -349,8 +380,11 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) * @return array of partition groups */ def coalesce(maxPartitions: Int, prev: RDD[_]): Array[PartitionGroup] = { - setupGroups(math.min(prev.partitions.length, maxPartitions), prev) // setup the groups (bins) - throwBalls(maxPartitions, prev, balanceSlack) // assign partitions (balls) to each group (bins) + val partitionLocs = new PartitionLocations(prev) + // setup the groups (bins) + setupGroups(math.min(prev.partitions.length, maxPartitions), partitionLocs) + // assign partitions (balls) to each group (bins) + throwBalls(maxPartitions, prev, balanceSlack, partitionLocs) getPartitions } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 8dc463d56d18..a663dab772bf 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -377,6 +377,33 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") } + test("coalesced RDDs with partial locality") { + // Make an RDD that has some locality preferences and some without. This can happen + // with UnionRDD + val data = sc.makeRDD((1 to 9).map(i => { + if (i > 4) { + (i, (i to (i + 2)).map { j => "m" + (j % 6) }) + } else { + (i, Vector()) + } + })) + val coalesced1 = data.coalesce(3) + assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") + + val splits = coalesced1.glom().collect().map(_.toList).toList + assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) + + assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") + + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = data.coalesce(20) + val listOfLists = coalesced4.glom().collect().map(_.toList).toList + val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) } + assert(sortedList === (1 to 9). + map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") + } + test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable @@ -418,6 +445,48 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("coalesced RDDs with partial locality, large scale (10K partitions)") { + // large scale experiment + import collection.mutable + val halfpartitions = 5000 + val partitions = 10000 + val numMachines = 50 + val machines = mutable.ListBuffer[String]() + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) + + val firstBlocks = (1 to halfpartitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } + val blocksNoLocality = (halfpartitions + 1 to partitions).map { i => + (i, List()) + } + val blocks = firstBlocks ++ blocksNoLocality + + val data2 = sc.makeRDD(blocks) + + // first try going to same number of partitions + val coalesced2 = data2.coalesce(partitions) + + // test that we have 10000 partitions + assert(coalesced2.partitions.size == 10000, "Expected 10000 partitions, but got " + + coalesced2.partitions.size) + + // test that we have 100 partitions + val coalesced3 = data2.coalesce(numMachines * 2) + assert(coalesced3.partitions.size == 100, "Expected 100 partitions, but got " + + coalesced3.partitions.size) + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance3 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance3 <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance3) + } + } + // Test for SPARK-2412 -- ensure that the second pass of the algorithm does not throw an exception test("coalesced RDDs with locality, fail first pass") { val initialPartitions = 1000 From ca9917160849597a839a8b6b39321435734d936b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 May 2016 13:47:58 -0700 Subject: [PATCH 041/313] [SPARK-15073][SQL] Hide SparkSession constructor from the public ## What changes were proposed in this pull request? Users should use the builder pattern instead. ## How was this patch tested? Jenks. Author: Andrew Or Closes #12873 from andrewor14/spark-session-constructor. (cherry picked from commit 588cac414a9cf1e0f40a82cc6a78f77e26825f29) Signed-off-by: Reynold Xin --- .../apache/spark/examples/sql/RDDRelation.scala | 9 +++------ .../scala/org/apache/spark/repl/SparkILoop.scala | 4 ++-- .../main/scala/org/apache/spark/repl/Main.scala | 4 ++-- .../scala/org/apache/spark/sql/SparkSession.scala | 14 ++++++++++++-- 4 files changed, 19 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 8ce4427c5399..b4118b16e291 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.sql -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.{SaveMode, SparkSession} // One method for defining the schema of an RDD is to make a case class with the desired column @@ -27,14 +26,12 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("RDDRelation") - val sc = new SparkContext(sparkConf) - val spark = new SparkSession(sc) + val spark = SparkSession.builder.appName("RDDRelation").getOrCreate() // Importing the SparkSession gives access to all the SQL functions and implicit conversions. import spark.implicits._ - val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF() + val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) // Any RDD containing case classes can be registered as a table. The schema of the table is // automatically inferred using scala reflection. df.registerTempTable("records") @@ -70,7 +67,7 @@ object RDDRelation { parquetFile.registerTempTable("parquetFile") spark.sql("SELECT * FROM parquetFile").collect().foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 6a811adcf9b7..c4f64505a216 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1030,10 +1030,10 @@ class SparkILoop( def createSparkSession(): SparkSession = { if (SparkSession.hiveClassesArePresent) { logInfo("Creating Spark session with Hive support") - SparkSession.withHiveSupport(sparkContext) + SparkSession.builder.enableHiveSupport().getOrCreate() } else { logInfo("Creating Spark session") - new SparkSession(sparkContext) + SparkSession.builder.getOrCreate() } } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 8e381ff6ae5a..a17175980925 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -94,10 +94,10 @@ object Main extends Logging { def createSparkSession(): SparkSession = { if (SparkSession.hiveClassesArePresent) { - sparkSession = SparkSession.withHiveSupport(sparkContext) + sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate() logInfo("Created Spark session with Hive support") } else { - sparkSession = new SparkSession(sparkContext) + sparkSession = SparkSession.builder.getOrCreate() logInfo("Created Spark session") } sparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 3836ce2daa56..aa7c335c53d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -54,6 +54,7 @@ import org.apache.spark.util.Utils * {{{ * SparkSession.builder() * .master("local") + * .appName("Word Count") * .config("spark.some.config.option", "some-value"). * .getOrCreate() * }}} @@ -63,7 +64,7 @@ class SparkSession private( @transient private val existingSharedState: Option[SharedState]) extends Serializable with Logging { self => - def this(sc: SparkContext) { + private[sql] def this(sc: SparkContext) { this(sc, None) } @@ -573,7 +574,7 @@ class SparkSession private( * common Scala objects into [[DataFrame]]s. * * {{{ - * val sparkSession = new SparkSession(sc) + * val sparkSession = SparkSession.builder.getOrCreate() * import sparkSession.implicits._ * }}} * @@ -586,6 +587,15 @@ class SparkSession private( } // scalastyle:on + /** + * Stop the underlying [[SparkContext]]. + * + * @since 2.0.0 + */ + def stop(): Unit = { + sparkContext.stop() + } + protected[sql] def parseSql(sql: String): LogicalPlan = { sessionState.sqlParser.parsePlan(sql) } From c50746a4a32565f181422aa888eaa4f03f94a25f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 May 2016 14:15:25 -0700 Subject: [PATCH 042/313] [SPARK-15095][SQL] drop binary mode in ThriftServer ## What changes were proposed in this pull request? This PR drop the support for binary mode in ThriftServer, only HTTP mode is supported now, to reduce the maintain burden. The code to support binary mode is still kept, just in case if we want it in future. ## How was this patch tested? Updated tests to use HTTP mode. Author: Davies Liu Closes #12876 from davies/hide_binary. (cherry picked from commit d6c7b2a5cc11a82e5137ee86350550e06e81f609) Signed-off-by: Reynold Xin --- .../hive/thriftserver/HiveThriftServer2.scala | 18 ++++---- .../HiveThriftServer2Suites.scala | 41 +++---------------- .../hive/thriftserver/UISeleniumSuite.scala | 2 +- 3 files changed, 14 insertions(+), 47 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 94b1ced9908d..cef5912c6240 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} +import org.apache.hive.service.cli.thrift.ThriftHttpCLIService import org.apache.hive.service.server.HiveServer2 import org.apache.spark.SparkContext @@ -34,7 +34,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.{HiveSharedState, HiveUtils} +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab import org.apache.spark.sql.internal.SQLConf @@ -271,7 +271,7 @@ object HiveThriftServer2 extends Logging { private[hive] class HiveThriftServer2(sqlContext: SQLContext) extends HiveServer2 - with ReflectedCompositeService { + with ReflectedCompositeService with Logging { // state is tracked internally so that the server only attempts to shut down if it successfully // started, and then once only. private val started = new AtomicBoolean(false) @@ -281,20 +281,18 @@ private[hive] class HiveThriftServer2(sqlContext: SQLContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) - val thriftCliService = if (isHTTPTransportMode(hiveConf)) { - new ThriftHttpCLIService(sparkSqlCliService) - } else { - new ThriftBinaryCLIService(sparkSqlCliService) + if (isBinaryTransportMode(hiveConf)) { + logWarning("Binary mode is not supported, use HTTP mode instead") } - + val thriftCliService = new ThriftHttpCLIService(sparkSqlCliService) setSuperField(this, "thriftCLIService", thriftCliService) addService(thriftCliService) initCompositeService(hiveConf) } - private def isHTTPTransportMode(hiveConf: HiveConf): Boolean = { + private def isBinaryTransportMode(hiveConf: HiveConf): Boolean = { val transportMode = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE) - transportMode.toLowerCase(Locale.ENGLISH).equals("http") + transportMode.toLowerCase(Locale.ENGLISH).equals("binary") } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 55a93ea06ba5..1a894ae85729 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -55,8 +55,8 @@ object TestData { val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") } -class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.binary +class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { + override def mode: ServerMode.Value = ServerMode.http private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { // Transport creation logic below mimics HiveConnection.createBinaryTransport @@ -70,7 +70,8 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { try f(client) finally transport.close() } - test("GetInfo Thrift API") { + // TODO: update this test to work in HTTP mode + ignore("GetInfo Thrift API") { withCLIServiceClient { client => val user = System.getProperty("user.name") val sessionHandle = client.openSession(user, "") @@ -566,7 +567,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } class SingleSessionSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.http override protected def extraConf: Seq[String] = "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil @@ -616,38 +617,6 @@ class SingleSessionSuite extends HiveThriftJdbcTest { } } -class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.http - - test("JDBC query execution") { - withJdbcStatement { statement => - val queries = Seq( - "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", - "CACHE TABLE test") - - queries.foreach(statement.execute) - - assertResult(5, "Row count mismatch") { - val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") - resultSet.next() - resultSet.getInt(1) - } - } - } - - test("Checks Hive version") { - withJdbcStatement { statement => - val resultSet = statement.executeQuery("SET spark.sql.hive.version") - resultSet.next() - assert(resultSet.getString(1) === "spark.sql.hive.version") - assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) - } - } -} - object ServerMode extends Enumeration { val binary, http = Value } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index bf431cd6b026..b6b9de1ba663 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -36,7 +36,7 @@ class UISeleniumSuite implicit var webDriver: WebDriver = _ var server: HiveThriftServer2 = _ val uiPort = 20000 + Random.nextInt(10000) - override def mode: ServerMode.Value = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.http override def beforeAll(): Unit = { webDriver = new HtmlUnitDriver { From f21eddb2901405cfd9c5fb85a1f73d48e3edb515 Mon Sep 17 00:00:00 2001 From: yinxusen Date: Tue, 3 May 2016 14:19:13 -0700 Subject: [PATCH 043/313] [SPARK-14973][ML] The CrossValidator and TrainValidationSplit miss the seed when saving and loading ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14973 Add seed support when saving/loading of CrossValidator and TrainValidationSplit. ## How was this patch tested? Spark unit test. Author: yinxusen Closes #12825 from yinxusen/SPARK-14973. (cherry picked from commit 2e2a6211c4391d67edb2a252f26647fb059bc18b) Signed-off-by: Joseph K. Bradley --- .../apache/spark/ml/tuning/CrossValidator.scala | 17 ++++++++++------- .../spark/ml/tuning/TrainValidationSplit.scala | 17 ++++++++++------- .../spark/ml/tuning/ValidatorParams.scala | 9 +++++---- .../spark/ml/tuning/CrossValidatorSuite.scala | 3 +++ .../ml/tuning/TrainValidationSplitSuite.scala | 2 ++ 5 files changed, 30 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index a41d02cde755..7d42da4a2ffa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -30,7 +30,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.HasSeed import org.apache.spark.ml.util._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, Dataset} @@ -39,7 +38,7 @@ import org.apache.spark.sql.types.StructType /** * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ -private[ml] trait CrossValidatorParams extends ValidatorParams with HasSeed { +private[ml] trait CrossValidatorParams extends ValidatorParams { /** * Param for number of folds for cross validation. Must be >= 2. * Default: 3 @@ -179,11 +178,13 @@ object CrossValidator extends MLReadable[CrossValidator] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val numFolds = (metadata.params \ "numFolds").extract[Int] + val seed = (metadata.params \ "seed").extract[Long] new CrossValidator(metadata.uid) .setEstimator(estimator) .setEvaluator(evaluator) .setEstimatorParamMaps(estimatorParamMaps) .setNumFolds(numFolds) + .setSeed(seed) } } } @@ -267,14 +268,16 @@ object CrossValidatorModel extends MLReadable[CrossValidatorModel] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val numFolds = (metadata.params \ "numFolds").extract[Int] + val seed = (metadata.params \ "seed").extract[Long] val bestModelPath = new Path(path, "bestModel").toString val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) val avgMetrics = (metadata.metadata \ "avgMetrics").extract[Seq[Double]].toArray - val cv = new CrossValidatorModel(metadata.uid, bestModel, avgMetrics) - cv.set(cv.estimator, estimator) - .set(cv.evaluator, evaluator) - .set(cv.estimatorParamMaps, estimatorParamMaps) - .set(cv.numFolds, numFolds) + val model = new CrossValidatorModel(metadata.uid, bestModel, avgMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + .set(model.numFolds, numFolds) + .set(model.seed, seed) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index f2b7badbe513..f6f2bad401a1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -30,7 +30,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{DoubleParam, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.HasSeed import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -38,7 +37,7 @@ import org.apache.spark.sql.types.StructType /** * Params for [[TrainValidationSplit]] and [[TrainValidationSplitModel]]. */ -private[ml] trait TrainValidationSplitParams extends ValidatorParams with HasSeed { +private[ml] trait TrainValidationSplitParams extends ValidatorParams { /** * Param for ratio between train and validation data. Must be between 0 and 1. * Default: 0.75 @@ -177,11 +176,13 @@ object TrainValidationSplit extends MLReadable[TrainValidationSplit] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val trainRatio = (metadata.params \ "trainRatio").extract[Double] + val seed = (metadata.params \ "seed").extract[Long] new TrainValidationSplit(metadata.uid) .setEstimator(estimator) .setEvaluator(evaluator) .setEstimatorParamMaps(estimatorParamMaps) .setTrainRatio(trainRatio) + .setSeed(seed) } } } @@ -265,14 +266,16 @@ object TrainValidationSplitModel extends MLReadable[TrainValidationSplitModel] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val trainRatio = (metadata.params \ "trainRatio").extract[Double] + val seed = (metadata.params \ "seed").extract[Long] val bestModelPath = new Path(path, "bestModel").toString val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) val validationMetrics = (metadata.metadata \ "validationMetrics").extract[Seq[Double]].toArray - val tvs = new TrainValidationSplitModel(metadata.uid, bestModel, validationMetrics) - tvs.set(tvs.estimator, estimator) - .set(tvs.evaluator, evaluator) - .set(tvs.estimatorParamMaps, estimatorParamMaps) - .set(tvs.trainRatio, trainRatio) + val model = new TrainValidationSplitModel(metadata.uid, bestModel, validationMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + .set(model.trainRatio, trainRatio) + .set(model.seed, seed) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala index 7a4e106aeb99..26fd73814d70 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala @@ -25,15 +25,15 @@ import org.apache.spark.SparkContext import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} -import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter, MetaAlgorithmReadWrite, - MLWritable} +import org.apache.spark.ml.param.shared.HasSeed +import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter, MetaAlgorithmReadWrite, MLWritable} import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.sql.types.StructType /** * Common params for [[TrainValidationSplitParams]] and [[CrossValidatorParams]]. */ -private[ml] trait ValidatorParams extends Params { +private[ml] trait ValidatorParams extends HasSeed with Params { /** * param for the estimator to be validated @@ -137,7 +137,8 @@ private[ml] object ValidatorParams { } val jsonParams = validatorSpecificParams ++ List( - "estimatorParamMaps" -> parse(estimatorParamMapsJson)) + "estimatorParamMaps" -> parse(estimatorParamMapsJson), + "seed" -> parse(instance.seed.jsonEncode(instance.getSeed))) DefaultParamsWriter.saveMetadata(instance, path, sc, extraMetadata, Some(jsonParams)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index 3e734aabc554..061d04c93215 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -136,6 +136,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] @@ -186,6 +187,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) assert(cv.getEvaluator.uid === cv2.getEvaluator.uid) @@ -259,6 +261,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index dbee47c8475d..df9ba418b897 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -127,6 +127,7 @@ class TrainValidationSplitSuite val tvs2 = testDefaultReadWrite(tvs, testParams = false) assert(tvs.getTrainRatio === tvs2.getTrainRatio) + assert(tvs.getSeed === tvs2.getSeed) } test("read/write: TrainValidationSplitModel") { @@ -149,6 +150,7 @@ class TrainValidationSplitSuite assert(tvs.getTrainRatio === tvs2.getTrainRatio) assert(tvs.validationMetrics === tvs2.validationMetrics) + assert(tvs.getSeed === tvs2.getSeed) } } From 70c5ca220c7548a7ebaba16e5dbf7e9ed47b5338 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 3 May 2016 23:20:18 +0200 Subject: [PATCH 044/313] [SPARK-15056][SQL] Parse Unsupported Sampling Syntax and Issue Better Exceptions #### What changes were proposed in this pull request? Compared with the current Spark parser, there are two extra syntax are supported in Hive for sampling - In `On` clauses, `rand()` is used for indicating sampling on the entire row instead of an individual column. For example, ```SQL SELECT * FROM source TABLESAMPLE(BUCKET 3 OUT OF 32 ON rand()) s; ``` - Users can specify the total length to be read. For example, ```SQL SELECT * FROM source TABLESAMPLE(100M) s; ``` Below is the link for references: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Sampling This PR is to parse and capture these two extra syntax, and issue a better error message. #### How was this patch tested? Added test cases to verify the thrown exceptions Author: gatorsmile Closes #12838 from gatorsmile/bucketOnRand. (cherry picked from commit 71296c041e59159bd7c5836cf652c02843974077) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 7 ++++++- .../spark/sql/catalyst/parser/AstBuilder.scala | 12 +++++++++++- .../spark/sql/catalyst/parser/PlanParserSuite.scala | 6 +++++- 3 files changed, 22 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index cc4e5c853e67..3ab448dd9e25 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -389,7 +389,8 @@ sample : TABLESAMPLE '(' ( (percentage=(INTEGER_VALUE | DECIMAL_VALUE) sampleType=PERCENTLIT) | (expression sampleType=ROWS) - | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON identifier)?)) + | sampleType=BYTELENGTH_LITERAL + | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON (identifier | qualifiedName '(' ')'))?)) ')' ; @@ -895,6 +896,10 @@ TINYINT_LITERAL : DIGIT+ 'Y' ; +BYTELENGTH_LITERAL + : DIGIT+ ('B' | 'K' | 'M' | 'G') + ; + INTEGER_VALUE : DIGIT+ ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c3974625aa22..1d4e1ec3b894 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -632,8 +632,18 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val fraction = ctx.percentage.getText.toDouble sample(fraction / 100.0d) + case SqlBaseParser.BYTELENGTH_LITERAL => + throw new ParseException( + "TABLESAMPLE(byteLengthLiteral) is not supported", ctx) + case SqlBaseParser.BUCKET if ctx.ON != null => - throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx) + if (ctx.identifier != null) { + throw new ParseException( + "TABLESAMPLE(BUCKET x OUT OF y ON colname) is not supported", ctx) + } else { + throw new ParseException( + "TABLESAMPLE(BUCKET x OUT OF y ON function) is not supported", ctx) + } case SqlBaseParser.BUCKET => sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index b7af2ceda643..aaf84268afbe 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -372,9 +372,13 @@ class PlanParserSuite extends PlanTest { assertEqual(s"$sql tablesample(bucket 4 out of 10) as x", Sample(0, .4d, withReplacement = false, 10L, table("t").as("x"))(true).select(star())) intercept(s"$sql tablesample(bucket 4 out of 10 on x) as x", - "TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported") + "TABLESAMPLE(BUCKET x OUT OF y ON colname) is not supported") intercept(s"$sql tablesample(bucket 11 out of 10) as x", s"Sampling fraction (${11.0/10.0}) must be on interval [0, 1]") + intercept("SELECT * FROM parquet_t0 TABLESAMPLE(300M) s", + "TABLESAMPLE(byteLengthLiteral) is not supported") + intercept("SELECT * FROM parquet_t0 TABLESAMPLE(BUCKET 3 OUT OF 32 ON rand()) s", + "TABLESAMPLE(BUCKET x OUT OF y ON function) is not supported") } test("sub-query") { From 51bb0bcc85519be6608cd41a9be4801419314550 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 May 2016 14:40:47 -0700 Subject: [PATCH 045/313] [SQL-15102][SQL] remove delegation token support from ThriftServer ## What changes were proposed in this pull request? These API is only useful for Hadoop, may not work for Spark SQL. The APIs is kept for source compatibility. ## How was this patch tested? No unit tests needed. Author: Davies Liu Closes #12878 from davies/remove_delegate. (cherry picked from commit 028c6a5dba01e5d82c34701f40d15916c9d3e9d0) Signed-off-by: Reynold Xin --- .../service/cli/thrift/ThriftCLIService.java | 65 ++----------------- 1 file changed, 7 insertions(+), 58 deletions(-) diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 866beb19f5f9..ad7a9a238f8a 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -18,6 +18,7 @@ package org.apache.hive.service.cli.thrift; +import javax.security.auth.login.LoginException; import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; @@ -25,8 +26,6 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import javax.security.auth.login.LoginException; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; @@ -36,17 +35,7 @@ import org.apache.hive.service.ServiceUtils; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.auth.TSetIpAddressProcessor; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.FetchType; -import org.apache.hive.service.cli.GetInfoType; -import org.apache.hive.service.cli.GetInfoValue; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.OperationStatus; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.*; import org.apache.hive.service.cli.session.SessionManager; import org.apache.hive.service.server.HiveServer2; import org.apache.thrift.TException; @@ -223,23 +212,7 @@ public InetAddress getServerIPAddress() { public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws TException { TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); - - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - String token = cliService.getDelegationToken( - new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getOwner(), req.getRenewer()); - resp.setDelegationToken(token); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error obtaining delegation token", e); - TStatus tokenErrorStatus = HiveSQLException.toTStatus(e); - tokenErrorStatus.setSqlState("42000"); - resp.setStatus(tokenErrorStatus); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } @@ -247,19 +220,7 @@ public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws TException { TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); - - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - cliService.cancelDelegationToken(new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getDelegationToken()); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error canceling delegation token", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } @@ -267,25 +228,13 @@ public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenRe public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws TException { TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - cliService.renewDelegationToken(new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getDelegationToken()); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error obtaining renewing token", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } - private TStatus unsecureTokenErrorStatus() { + private TStatus notSupportTokenErrorStatus() { TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); - errorStatus.setErrorMessage("Delegation token only supported over remote " + - "client with kerberos authentication"); + errorStatus.setErrorMessage("Delegation token is not supported"); return errorStatus; } From c212307b9a187c626c7162838e34551f6700e611 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Tue, 3 May 2016 14:54:43 -0700 Subject: [PATCH 046/313] [SPARK-15104] Fix spacing in log line Otherwise get logs that look like this (note no space before NODE_LOCAL) ``` INFO [2016-05-03 21:18:51,477] org.apache.spark.scheduler.TaskSetManager: Starting task 0.0 in stage 101.0 (TID 7029, localhost, partition 0,NODE_LOCAL, 1894 bytes) ``` Author: Andrew Ash Closes #12880 from ash211/patch-7. (cherry picked from commit dbacd99983a81d35476618be1615807ebba88f05) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index cd634bbf6fe7..bfa1e86749a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -479,7 +479,7 @@ private[spark] class TaskSetManager( // val timeTaken = clock.getTime() - startTime val taskName = s"task ${info.id} in stage ${taskSet.id}" logInfo(s"Starting $taskName (TID $taskId, $host, partition ${task.partitionId}," + - s"$taskLocality, ${serializedTask.limit} bytes)") + s" $taskLocality, ${serializedTask.limit} bytes)") sched.dagScheduler.taskStarted(task, info) return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, From 0d16b7f3a5fbb2eb047edca838bcbde9037227a3 Mon Sep 17 00:00:00 2001 From: Shuai Lin Date: Tue, 3 May 2016 18:02:12 -0700 Subject: [PATCH 047/313] [MINOR][DOC] Fixed some python snippets in mllib data types documentation. ## What changes were proposed in this pull request? Some python snippets is using scala imports and comments. ## How was this patch tested? Generated the docs locally with `SKIP_API=1 jekyll build` and viewed the changes in the browser. Author: Shuai Lin Closes #12869 from lins05/fix-mllib-python-snippets. (cherry picked from commit c4e0fde876fff259308d1d58ab51ae2697ae31f1) Signed-off-by: Andrew Or --- docs/mllib-data-types.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index 5e3ee472a72c..2ffe0f1c2b31 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -314,12 +314,12 @@ matrices. Remember, local matrices in MLlib are stored in column-major order. Refer to the [`Matrix` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrix) and [`Matrices` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrices) for more details on the API. {% highlight python %} -import org.apache.spark.mllib.linalg.{Matrix, Matrices} +from pyspark.mllib.linalg import Matrix, Matrices -// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +# Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) dm2 = Matrices.dense(3, 2, [1, 2, 3, 4, 5, 6]) -// Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) +# Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) sm = Matrices.sparse(3, 2, [0, 1, 3], [0, 2, 1], [9, 6, 8]) {% endhighlight %} From 5625b037a0c952b97e1afa6a44443113c0847ade Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 18:02:57 -0700 Subject: [PATCH 048/313] [SPARK-14422][SQL] Improve handling of optional configs in SQLConf ## What changes were proposed in this pull request? Create a new API for handling Optional Configs in SQLConf. Right now `getConf` for `OptionalConfigEntry[T]` returns value of type `T`, if doesn't exist throws an exception. Add new method `getOptionalConf`(suggestions on naming) which will now returns value of type `Option[T]`(so if doesn't exist it returns `None`). ## How was this patch tested? Add test and ran tests locally. Author: Sandeep Singh Closes #12846 from techaddict/SPARK-14422. (cherry picked from commit a8d56f538878443da6eae69449858ad4e2274151) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/sql/DataFrameWriter.scala | 9 +++++---- .../scala/org/apache/spark/sql/RuntimeConfig.scala | 6 +++++- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 9 ++++----- .../apache/spark/sql/internal/SQLConfEntrySuite.scala | 11 +++++++++++ 4 files changed, 25 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index a8f96a9b45b2..0793b62faefb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -296,7 +296,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { new Path(userSpecified).toUri.toString }.orElse { val checkpointConfig: Option[String] = - df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION, None) + df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION) checkpointConfig.map { location => new Path(location, queryName).toUri.toString @@ -334,9 +334,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { partitionColumns = normalizedParCols.getOrElse(Nil)) val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) - val checkpointLocation = extraOptions.getOrElse("checkpointLocation", { - new Path(df.sparkSession.sessionState.conf.checkpointLocation, queryName).toUri.toString - }) + val checkpointLocation = extraOptions.getOrElse("checkpointLocation", + new Path(df.sparkSession.sessionState.conf.checkpointLocation.get, queryName).toUri.toString + ) + df.sparkSession.sessionState.continuousQueryManager.startQuery( queryName, checkpointLocation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 670288b23400..4fd6e42640d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.internal.config.ConfigEntry +import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} import org.apache.spark.sql.internal.SQLConf @@ -86,6 +86,10 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { sqlConf.getConf(entry) } + protected[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T] = { + sqlConf.getConf(entry) + } + /** * Returns the value of Spark runtime configuration property for the given key. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0bcf0f817a1d..5e19984debaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -546,7 +546,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def optimizerInSetConversionThreshold: Int = getConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD) - def checkpointLocation: String = getConf(CHECKPOINT_LOCATION) + def checkpointLocation: Option[String] = getConf(CHECKPOINT_LOCATION) def filesMaxPartitionBytes: Long = getConf(FILES_MAX_PARTITION_BYTES) @@ -717,12 +717,11 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { /** * Return the value of an optional Spark SQL configuration property for the given key. If the key - * is not set yet, throw an exception. + * is not set yet, returns None. */ - def getConf[T](entry: OptionalConfigEntry[T]): T = { + def getConf[T](entry: OptionalConfigEntry[T]): Option[T] = { require(sqlConfEntries.get(entry.key) == entry, s"$entry is not registered") - Option(settings.get(entry.key)).map(entry.rawValueConverter). - getOrElse(throw new NoSuchElementException(entry.key)) + Option(settings.get(entry.key)).map(entry.rawValueConverter) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala index cc6919913948..95bfd05c1f26 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala @@ -153,6 +153,17 @@ class SQLConfEntrySuite extends SparkFunSuite { assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c", "d", "e")) } + test("optionalConf") { + val key = "spark.sql.SQLConfEntrySuite.optional" + val confEntry = SQLConfigBuilder(key) + .stringConf + .createOptional + + assert(conf.getConf(confEntry) === None) + conf.setConfString(key, "a") + assert(conf.getConf(confEntry) === Some("a")) + } + test("duplicate entry") { val key = "spark.sql.SQLConfEntrySuite.duplicate" SQLConfigBuilder(key).stringConf.createOptional From 4c7f5a74d702c3bed0b07cfa498cbe7e653fce03 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 3 May 2016 18:04:04 -0700 Subject: [PATCH 049/313] [SPARK-14645][MESOS] Fix python running on cluster mode mesos to have non local uris ## What changes were proposed in this pull request? Fix SparkSubmit to allow non-local python uris ## How was this patch tested? Manually tested with mesos-spark-dispatcher Author: Timothy Chen Closes #12403 from tnachen/enable_remote_python. (cherry picked from commit c1839c9911e37488230a68dec9041eb5958b6f1c) Signed-off-by: Andrew Or --- .../src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 926e1ff7a874..755c4b6ec11d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -305,8 +305,9 @@ object SparkSubmit { } // Require all python files to be local, so we can add them to the PYTHONPATH - // In YARN cluster mode, python files are distributed as regular files, which can be non-local - if (args.isPython && !isYarnCluster) { + // In YARN cluster mode, python files are distributed as regular files, which can be non-local. + // In Mesos cluster mode, non-local python files are automatically downloaded by Mesos. + if (args.isPython && !isYarnCluster && !isMesosCluster) { if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { printErrorAndExit(s"Only local python files are supported: $args.primaryResource") } From 5e15615d1258db8c31a8c0c9f9f33965bc14a910 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 May 2016 18:05:40 -0700 Subject: [PATCH 050/313] [SPARK-15084][PYTHON][SQL] Use builder pattern to create SparkSession in PySpark. ## What changes were proposed in this pull request? This is a python port of corresponding Scala builder pattern code. `sql.py` is modified as a target example case. ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #12860 from dongjoon-hyun/SPARK-15084. (cherry picked from commit 0903a185c7ebc57c75301a27d215b08efd347f99) Signed-off-by: Andrew Or --- examples/src/main/python/sql.py | 35 ++++++------- python/pyspark/sql/session.py | 91 ++++++++++++++++++++++++++++++++- 2 files changed, 105 insertions(+), 21 deletions(-) diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 2c188759328f..ea6a22dbfe82 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -20,33 +20,28 @@ import os import sys -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType if __name__ == "__main__": - sc = SparkContext(appName="PythonSQL") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PythonSQL").getOrCreate() - # RDD is created from a list of rows - some_rdd = sc.parallelize([Row(name="John", age=19), - Row(name="Smith", age=23), - Row(name="Sarah", age=18)]) - # Infer schema from the first row, create a DataFrame and print the schema - some_df = sqlContext.createDataFrame(some_rdd) + # A list of Rows. Infer schema from the first row, create a DataFrame and print the schema + rows = [Row(name="John", age=19), Row(name="Smith", age=23), Row(name="Sarah", age=18)] + some_df = spark.createDataFrame(rows) some_df.printSchema() - # Another RDD is created from a list of tuples - another_rdd = sc.parallelize([("John", 19), ("Smith", 23), ("Sarah", 18)]) + # A list of tuples + tuples = [("John", 19), ("Smith", 23), ("Sarah", 18)] # Schema with two fields - person_name and person_age schema = StructType([StructField("person_name", StringType(), False), StructField("person_age", IntegerType(), False)]) # Create a DataFrame by applying the schema to the RDD and print the schema - another_df = sqlContext.createDataFrame(another_rdd, schema) + another_df = spark.createDataFrame(tuples, schema) another_df.printSchema() # root - # |-- age: integer (nullable = true) + # |-- age: long (nullable = true) # |-- name: string (nullable = true) # A JSON dataset is pointed to by path. @@ -57,7 +52,7 @@ else: path = sys.argv[1] # Create a DataFrame from the file(s) pointed to by path - people = sqlContext.jsonFile(path) + people = spark.read.json(path) # root # |-- person_name: string (nullable = false) # |-- person_age: integer (nullable = false) @@ -65,16 +60,16 @@ # The inferred schema can be visualized using the printSchema() method. people.printSchema() # root - # |-- age: IntegerType - # |-- name: StringType + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) # Register this DataFrame as a table. - people.registerAsTable("people") + people.registerTempTable("people") # SQL statements can be run by using the sql methods provided by sqlContext - teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") for each in teenagers.collect(): print(each[0]) - sc.stop() + spark.stop() diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 35c36b493509..fb3e318163e8 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -19,6 +19,7 @@ import sys import warnings from functools import reduce +from threading import RLock if sys.version >= '3': basestring = unicode = str @@ -58,16 +59,98 @@ def toDF(self, schema=None, sampleRatio=None): class SparkSession(object): - """Main entry point for Spark SQL functionality. + """The entry point to programming Spark with the Dataset and DataFrame API. A SparkSession can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. + To create a SparkSession, use the following builder pattern: + + >>> spark = SparkSession.builder \ + .master("local") \ + .appName("Word Count") \ + .config("spark.some.config.option", "some-value") \ + .getOrCreate() :param sparkContext: The :class:`SparkContext` backing this SparkSession. :param jsparkSession: An optional JVM Scala SparkSession. If set, we do not instantiate a new SparkSession in the JVM, instead we make all calls to this object. """ + class Builder(object): + """Builder for :class:`SparkSession`. + """ + + _lock = RLock() + _options = {} + + @since(2.0) + def config(self, key=None, value=None, conf=None): + """Sets a config option. Options set using this method are automatically propagated to + both :class:`SparkConf` and :class:`SparkSession`'s own configuration. + + For an existing SparkConf, use `conf` parameter. + >>> from pyspark.conf import SparkConf + >>> SparkSession.builder.config(conf=SparkConf()) + >> SparkSession.builder.config("spark.some.config.option", "some-value") + Date: Tue, 3 May 2016 18:06:35 -0700 Subject: [PATCH 051/313] [SPARK-15097][SQL] make Dataset.sqlContext a stable identifier for imports ## What changes were proposed in this pull request? Make Dataset.sqlContext a lazy val so that its a stable identifier and can be used for imports. Now this works again: import someDataset.sqlContext.implicits._ ## How was this patch tested? Add unit test to DatasetSuite that uses the import show above. Author: Koert Kuipers Closes #12877 from koertkuipers/feat-sqlcontext-stable-import. (cherry picked from commit 9e4928b7e00788913553c1cb0722048001b91601) Signed-off-by: Andrew Or --- .../main/scala/org/apache/spark/sql/Dataset.scala | 3 ++- .../scala/org/apache/spark/sql/DatasetSuite.scala | 13 +++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 08be94e8d4f1..1bea72c4711f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -211,7 +211,8 @@ class Dataset[T] private[sql]( private implicit def classTag = unresolvedTEncoder.clsTag - def sqlContext: SQLContext = sparkSession.wrapped + // sqlContext must be val because a stable identifier is expected when you import implicits + @transient lazy val sqlContext: SQLContext = sparkSession.wrapped protected[sql] def resolve(colName: String): NamedExpression = { queryExecution.analyzed.resolveQuoted(colName, sparkSession.sessionState.analyzer.resolver) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index eee21acf7510..68a12b062249 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -653,6 +653,11 @@ class DatasetSuite extends QueryTest with SharedSQLContext { dataset.join(actual, dataset("user") === actual("id")).collect() } + + test("SPARK-15097: implicits on dataset's sqlContext can be imported") { + val dataset = Seq(1, 2, 3).toDS() + checkDataset(DatasetTransform.addOne(dataset), 2, 3, 4) + } } case class OtherTuple(_1: String, _2: Int) @@ -713,3 +718,11 @@ class JavaData(val a: Int) extends Serializable { object JavaData { def apply(a: Int): JavaData = new JavaData(a) } + +/** Used to test importing dataset.sqlContext.implicits._ */ +object DatasetTransform { + def addOne(ds: Dataset[Int]): Dataset[Int] = { + import ds.sqlContext.implicits._ + ds.map(_ + 1) + } +} From 940b8f60b90d0acf6910abfd368af25cefdf4ffa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 May 2016 18:07:53 -0700 Subject: [PATCH 052/313] [SPARK-14414][SQL] Make DDL exceptions more consistent ## What changes were proposed in this pull request? Just a bunch of small tweaks on DDL exception messages. ## How was this patch tested? `DDLCommandSuite` et al. Author: Andrew Or Closes #12853 from andrewor14/make-exceptions-consistent. (cherry picked from commit 6ba17cd147277a20a7fbb244c040e694de486c36) Signed-off-by: Andrew Or --- .../spark/sql/catalyst/parser/SqlBase.g4 | 26 +-- .../analysis/NoSuchItemException.scala | 14 +- .../catalyst/catalog/InMemoryCatalog.scala | 10 +- .../sql/catalyst/catalog/SessionCatalog.scala | 18 +- .../sql/catalyst/parser/ParserUtils.scala | 2 +- .../catalog/SessionCatalogSuite.scala | 6 +- .../spark/sql/execution/SparkSqlParser.scala | 152 +++------------ .../sql/execution/command/AnalyzeTable.scala | 6 +- .../spark/sql/execution/command/ddl.scala | 70 +------ .../sql/execution/command/functions.scala | 10 +- .../spark/sql/execution/command/tables.scala | 39 ++-- .../spark/sql/execution/command/views.scala | 3 +- .../execution/command/DDLCommandSuite.scala | 174 +++--------------- .../sources/CreateTableAsSelectSuite.scala | 13 +- .../execution/HiveCompatibilitySuite.scala | 10 +- .../spark/sql/hive/HiveExternalCatalog.scala | 2 +- .../sql/hive/client/HiveClientImpl.scala | 12 +- .../sql/hive/execution/HiveCommandSuite.scala | 4 +- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- .../sql/hive/execution/SQLViewSuite.scala | 3 +- 20 files changed, 141 insertions(+), 435 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 3ab448dd9e25..273ad9289169 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -81,18 +81,8 @@ statement DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions | ALTER VIEW tableIdentifier DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions - | ALTER TABLE tableIdentifier partitionSpec? - SET FILEFORMAT fileFormat #setTableFileFormat | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation - | ALTER TABLE tableIdentifier partitionSpec? - CHANGE COLUMN? oldName=identifier colType - (FIRST | AFTER after=identifier)? (CASCADE | RESTRICT)? #changeColumn - | ALTER TABLE tableIdentifier partitionSpec? - ADD COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)? #addColumns - | ALTER TABLE tableIdentifier partitionSpec? - REPLACE COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)? #replaceColumns - | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? - (FOR METADATA? REPLICATION '(' STRING ')')? #dropTable + | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? tableIdentifier #dropTable | CREATE (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier identifierCommentList? (COMMENT STRING)? @@ -170,6 +160,10 @@ unsupportedHiveNativeCommands | kw1=ALTER kw2=TABLE tableIdentifier kw3=TOUCH | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=COMPACT | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CONCATENATE + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=SET kw4=FILEFORMAT + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=ADD kw4=COLUMNS + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CHANGE kw4=COLUMNS? + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=REPLACE kw4=COLUMNS | kw1=START kw2=TRANSACTION | kw1=COMMIT | kw1=ROLLBACK @@ -647,9 +641,9 @@ nonReserved | NO | DATA | START | TRANSACTION | COMMIT | ROLLBACK | SORT | CLUSTER | DISTRIBUTE | UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION - | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE | FIRST - | AFTER | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT - | DBPROPERTIES | DFS | TRUNCATE | METADATA | REPLICATION | COMPUTE + | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE + | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT + | DBPROPERTIES | DFS | TRUNCATE | COMPUTE | STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER | REVOKE | GRANT | LOCK | UNLOCK | MSCK | REPAIR | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE | ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEX | INDEXES | LOCKS | OPTION | LOCAL | INPATH @@ -836,8 +830,6 @@ TOUCH: 'TOUCH'; COMPACT: 'COMPACT'; CONCATENATE: 'CONCATENATE'; CHANGE: 'CHANGE'; -FIRST: 'FIRST'; -AFTER: 'AFTER'; CASCADE: 'CASCADE'; RESTRICT: 'RESTRICT'; CLUSTERED: 'CLUSTERED'; @@ -849,8 +841,6 @@ DATABASE: 'DATABASE' | 'SCHEMA'; DATABASES: 'DATABASES' | 'SCHEMAS'; DFS: 'DFS'; TRUNCATE: 'TRUNCATE'; -METADATA: 'METADATA'; -REPLICATION: 'REPLICATION'; ANALYZE: 'ANALYZE'; COMPUTE: 'COMPUTE'; STATISTICS: 'STATISTICS'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index 11ef9e1160a9..2412ec46c783 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -25,13 +25,17 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ -class NoSuchDatabaseException(db: String) extends AnalysisException(s"Database $db not found") +class NoSuchDatabaseException(db: String) extends AnalysisException(s"Database '$db' not found") class NoSuchTableException(db: String, table: String) - extends AnalysisException(s"Table or View $table not found in database $db") + extends AnalysisException(s"Table or view '$table' not found in database '$db'") -class NoSuchPartitionException(db: String, table: String, spec: TablePartitionSpec) extends - AnalysisException(s"Partition not found in table $table database $db:\n" + spec.mkString("\n")) +class NoSuchPartitionException( + db: String, + table: String, + spec: TablePartitionSpec) + extends AnalysisException( + s"Partition not found in table '$table' database '$db':\n" + spec.mkString("\n")) class NoSuchFunctionException(db: String, func: String) - extends AnalysisException(s"Function $func not found in database $db") + extends AnalysisException(s"Function '$func' not found in database '$db'") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 60eb7329f71d..1d2ca2863fae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -164,7 +164,7 @@ class InMemoryCatalog extends ExternalCatalog { catalog(db).tables.remove(table) } else { if (!ignoreIfNotExists) { - throw new AnalysisException(s"Table or View '$table' does not exist in database '$db'") + throw new AnalysisException(s"Table or view '$table' does not exist in database '$db'") } } } @@ -211,7 +211,7 @@ class InMemoryCatalog extends ExternalCatalog { loadPath: String, isOverwrite: Boolean, holdDDLTime: Boolean): Unit = { - throw new AnalysisException("loadTable is not implemented for InMemoryCatalog.") + throw new UnsupportedOperationException("loadTable is not implemented") } override def loadPartition( @@ -223,7 +223,7 @@ class InMemoryCatalog extends ExternalCatalog { holdDDLTime: Boolean, inheritTableSpecs: Boolean, isSkewedStoreAsSubdir: Boolean): Unit = { - throw new AnalysisException("loadPartition is not implemented for InMemoryCatalog.") + throw new UnsupportedOperationException("loadPartition is not implemented.") } // -------------------------------------------------------------------------- @@ -304,8 +304,8 @@ class InMemoryCatalog extends ExternalCatalog { partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = synchronized { requireTableExists(db, table) if (partialSpec.nonEmpty) { - throw new AnalysisException("listPartition does not support partition spec in " + - "InMemoryCatalog.") + throw new UnsupportedOperationException( + "listPartition with partial partition spec is not implemented") } catalog(db).tables(table).partitions.values.toSeq } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index a445a253ff9b..ff6303471e14 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.catalog -import java.io.File - import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -284,10 +282,12 @@ class SessionCatalog( * This assumes the database specified in `oldName` matches the one specified in `newName`. */ def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = { - if (oldName.database != newName.database) { - throw new AnalysisException("rename does not support moving tables across databases") - } val db = oldName.database.getOrElse(currentDb) + val newDb = newName.database.getOrElse(currentDb) + if (db != newDb) { + throw new AnalysisException( + s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") + } val oldTableName = formatTableName(oldName.table) val newTableName = formatTableName(newName.table) if (oldName.database.isDefined || !tempTables.contains(oldTableName)) { @@ -315,7 +315,7 @@ class SessionCatalog( if (externalCatalog.tableExists(db, table)) { externalCatalog.dropTable(db, table, ignoreIfNotExists = true) } else if (!ignoreIfNotExists) { - logError(s"Table or View '${name.quotedString}' does not exist") + throw new AnalysisException(s"Table or view '${name.quotedString}' does not exist") } } else { tempTables.remove(table) @@ -534,7 +534,7 @@ class SessionCatalog( if (!functionExists(identifier)) { externalCatalog.createFunction(db, newFuncDefinition) } else if (!ignoreIfExists) { - throw new AnalysisException(s"function '$identifier' already exists in database '$db'") + throw new AnalysisException(s"Function '$identifier' already exists in database '$db'") } } @@ -632,9 +632,9 @@ class SessionCatalog( } protected def failFunctionLookup(name: String): Nothing = { - throw new AnalysisException(s"Undefined function: $name. This function is " + + throw new AnalysisException(s"Undefined function: '$name'. This function is " + s"neither a registered temporary function nor " + - s"a permanent function registered in the database $currentDb.") + s"a permanent function registered in the database '$currentDb'.") } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 64713cddf4e0..58e2bdb6e24f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -39,7 +39,7 @@ object ParserUtils { stream.getText(Interval.of(0, stream.size())) } - def parseException(message: String, ctx: ParserRuleContext): ParseException = { + def operationNotAllowed(message: String, ctx: ParserRuleContext): ParseException = { new ParseException(s"Operation not allowed: $message", ctx) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index ba5d8ce0f48f..a704ca78f99d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -233,9 +233,9 @@ class SessionCatalogSuite extends SparkFunSuite { intercept[AnalysisException] { catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = true) } - // If the table does not exist, we do not issue an exception. Instead, we output an error log - // message to console when ignoreIfNotExists is set to false. - catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false) + intercept[AnalysisException] { + catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false) + } catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = true) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index dfc56a7d98ba..b000cc99533e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -142,7 +142,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * A command for users to list the columm names for a table. + * A command for users to list the column names for a table. * This function creates a [[ShowColumnsCommand]] logical plan. * * The syntax of using this command in SQL is: @@ -155,8 +155,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val lookupTable = Option(ctx.db) match { case None => table - case Some(db) if table.database.isDefined => - throw new ParseException("Duplicates the declaration for database", ctx) + case Some(db) if table.database.exists(_ != db) => + throw operationNotAllowed( + s"SHOW COLUMNS with conflicting databases: '$db' != '${table.database.get}'", + ctx) case Some(db) => TableIdentifier(table.identifier, Some(db.getText)) } ShowColumnsCommand(lookupTable) @@ -214,7 +216,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitExplain(ctx: ExplainContext): LogicalPlan = withOrigin(ctx) { val options = ctx.explainOption.asScala if (options.exists(_.FORMATTED != null)) { - logWarning("Unsupported operation: EXPLAIN FORMATTED option") + throw operationNotAllowed("EXPLAIN FORMATTED", ctx) } // Create the explain comment. @@ -260,9 +262,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: CreateTableHeaderContext): TableHeader = withOrigin(ctx) { val temporary = ctx.TEMPORARY != null val ifNotExists = ctx.EXISTS != null - assert(!temporary || !ifNotExists, - "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.", - ctx) + if (temporary && ifNotExists) { + throw operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) + } (visitTableIdentifier(ctx.tableIdentifier), temporary, ifNotExists, ctx.EXTERNAL != null) } @@ -274,7 +276,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitCreateTableUsing(ctx: CreateTableUsingContext): LogicalPlan = withOrigin(ctx) { val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) if (external) { - throw new ParseException("Unsupported operation: EXTERNAL option", ctx) + throw operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } val options = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) val provider = ctx.tableProvider.qualifiedName.getText @@ -423,7 +425,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case "jar" | "file" | "archive" => resourceType -> string(resource.STRING) case other => - throw new ParseException(s"Resource Type '$resourceType' is not supported.", ctx) + throw operationNotAllowed(s"CREATE FUNCTION with resource type '$resourceType'", ctx) } } @@ -459,10 +461,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { if (ctx.PURGE != null) { - throw new ParseException("Unsupported operation: PURGE option", ctx) - } - if (ctx.REPLICATION != null) { - throw new ParseException("Unsupported operation: REPLICATION clause", ctx) + throw operationNotAllowed("DROP TABLE ... PURGE", ctx) } DropTable( visitTableIdentifier(ctx.tableIdentifier), @@ -554,7 +553,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitAddTablePartition( ctx: AddTablePartitionContext): LogicalPlan = withOrigin(ctx) { if (ctx.VIEW != null) { - throw new ParseException(s"Operation not allowed: partitioned views", ctx) + throw operationNotAllowed("ALTER VIEW ... ADD PARTITION", ctx) } // Create partition spec to location mapping. val specsAndLocs = if (ctx.partitionSpec.isEmpty) { @@ -605,10 +604,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitDropTablePartitions( ctx: DropTablePartitionsContext): LogicalPlan = withOrigin(ctx) { if (ctx.VIEW != null) { - throw new ParseException(s"Operation not allowed: partitioned views", ctx) + throw operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx) } if (ctx.PURGE != null) { - throw new ParseException(s"Operation not allowed: PURGE", ctx) + throw operationNotAllowed("ALTER TABLE ... DROP PARTITION ... PURGE", ctx) } AlterTableDropPartition( visitTableIdentifier(ctx.tableIdentifier), @@ -616,35 +615,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.EXISTS != null) } - /** - * Create an [[AlterTableSetFileFormat]] command - * - * For example: - * {{{ - * ALTER TABLE table [PARTITION spec] SET FILEFORMAT file_format; - * }}} - */ - override def visitSetTableFileFormat( - ctx: SetTableFileFormatContext): LogicalPlan = withOrigin(ctx) { - // AlterTableSetFileFormat currently takes both a GenericFileFormat and a - // TableFileFormatContext. This is a bit weird because it should only take one. It also should - // use a CatalogFileFormat instead of either a String or a Sequence of Strings. We will address - // this in a follow-up PR. - val (fileFormat, genericFormat) = ctx.fileFormat match { - case s: GenericFileFormatContext => - (Seq.empty[String], Option(s.identifier.getText)) - case s: TableFileFormatContext => - val elements = Seq(s.inFmt, s.outFmt) ++ Option(s.serdeCls).toSeq - (elements.map(string), None) - } - AlterTableSetFileFormat( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - fileFormat, - genericFormat)( - parseException("ALTER TABLE SET FILEFORMAT", ctx)) - } - /** * Create an [[AlterTableSetLocation]] command * @@ -660,79 +630,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { visitLocationSpec(ctx.locationSpec)) } - /** - * Create an [[AlterTableChangeCol]] command - * - * For example: - * {{{ - * ALTER TABLE tableIdentifier [PARTITION spec] - * CHANGE [COLUMN] col_old_name col_new_name column_type [COMMENT col_comment] - * [FIRST|AFTER column_name] [CASCADE|RESTRICT]; - * }}} - */ - override def visitChangeColumn(ctx: ChangeColumnContext): LogicalPlan = withOrigin(ctx) { - val col = visitColType(ctx.colType()) - val comment = if (col.metadata.contains("comment")) { - Option(col.metadata.getString("comment")) - } else { - None - } - - AlterTableChangeCol( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - ctx.oldName.getText, - // We could also pass in a struct field - seems easier. - col.name, - col.dataType, - comment, - Option(ctx.after).map(_.getText), - // Note that Restrict and Cascade are mutually exclusive. - ctx.RESTRICT != null, - ctx.CASCADE != null)( - parseException("ALTER TABLE CHANGE COLUMN", ctx)) - } - - /** - * Create an [[AlterTableAddCol]] command - * - * For example: - * {{{ - * ALTER TABLE tableIdentifier [PARTITION spec] - * ADD COLUMNS (name type [COMMENT comment], ...) [CASCADE|RESTRICT] - * }}} - */ - override def visitAddColumns(ctx: AddColumnsContext): LogicalPlan = withOrigin(ctx) { - AlterTableAddCol( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - createStructType(ctx.colTypeList), - // Note that Restrict and Cascade are mutually exclusive. - ctx.RESTRICT != null, - ctx.CASCADE != null)( - parseException("ALTER TABLE ADD COLUMNS", ctx)) - } - - /** - * Create an [[AlterTableReplaceCol]] command - * - * For example: - * {{{ - * ALTER TABLE tableIdentifier [PARTITION spec] - * REPLACE COLUMNS (name type [COMMENT comment], ...) [CASCADE|RESTRICT] - * }}} - */ - override def visitReplaceColumns(ctx: ReplaceColumnsContext): LogicalPlan = withOrigin(ctx) { - AlterTableReplaceCol( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - createStructType(ctx.colTypeList), - // Note that Restrict and Cascade are mutually exclusive. - ctx.RESTRICT != null, - ctx.CASCADE != null)( - parseException("ALTER TABLE REPLACE COLUMNS", ctx)) - } - /** * Create location string. */ @@ -753,7 +650,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { .map { orderedIdCtx => Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => if (dir.toLowerCase != "asc") { - throw parseException("Only ASC ordering is supported for sorting columns", ctx) + throw operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) } } @@ -789,7 +686,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // SET ROLE is the exception to the rule, because we handle this before other SET commands. "SET ROLE" } - throw parseException(keywords, ctx) + throw operationNotAllowed(keywords, ctx) } /** @@ -799,7 +696,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.identifier.getText.toLowerCase match { case "file" => AddFile(remainder(ctx.identifier).trim) case "jar" => AddJar(remainder(ctx.identifier).trim) - case other => throw new ParseException(s"Unsupported resource type '$other'.", ctx) + case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) } } @@ -836,10 +733,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { "Please use registerTempTable as an alternative.", ctx) } if (ctx.skewSpec != null) { - throw new ParseException("Operation not allowed: CREATE TABLE ... SKEWED BY ...", ctx) + throw operationNotAllowed("CREATE TABLE ... SKEWED BY", ctx) } if (ctx.bucketSpec != null) { - throw new ParseException("Operation not allowed: CREATE TABLE ... CLUSTERED BY ...", ctx) + throw operationNotAllowed("CREATE TABLE ... CLUSTERED BY", ctx) } val tableType = if (external) { CatalogTableType.EXTERNAL @@ -926,9 +823,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case (c: GenericFileFormatContext, null) => visitGenericFileFormat(c) case (null, storageHandler) => - throw new ParseException("Operation not allowed: ... STORED BY storage_handler ...", ctx) + throw operationNotAllowed("STORED BY", ctx) case _ => - throw new ParseException("expected either STORED AS or STORED BY, not both", ctx) + throw new ParseException("Expected either STORED AS or STORED BY, not both", ctx) } } @@ -960,7 +857,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { outputFormat = s.outputFormat, serde = s.serde) case None => - throw new ParseException(s"Unrecognized file format in STORED AS clause: $source", ctx) + throw operationNotAllowed(s"STORED AS with file format '$source'", ctx) } } @@ -1041,7 +938,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) { if (ctx.identifierList != null) { - throw new ParseException(s"Operation not allowed: partitioned views", ctx) + throw operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx) } else { val identifiers = Option(ctx.identifierCommentList).toSeq.flatMap(_.identifierComment.asScala) val schema = identifiers.map { ic => @@ -1128,6 +1025,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { recordReader: Token, schemaLess: Boolean): ScriptInputOutputSchema = { if (recordWriter != null || recordReader != null) { + // TODO: what does this message mean? throw new ParseException( "Unsupported operation: Used defined record reader/writer classes.", ctx) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala index 54ff5ae7d9d0..de2db44b0e1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala @@ -21,7 +21,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} @@ -99,8 +99,8 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { } case otherRelation => - throw new UnsupportedOperationException( - s"Analyze only works for Hive tables, but $tableName is a ${otherRelation.nodeName}") + throw new AnalysisException(s"ANALYZE TABLE is only supported for Hive tables, " + + s"but '${tableIdent.unquotedString}' is a ${otherRelation.nodeName}.") } Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 44647116b488..aa06c014fb0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -30,30 +30,9 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.types._ - // Note: The definition of these commands are based on the ones described in // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL -/** - * A DDL command that is not supported right now. Since we have already implemented - * the parsing rules for some commands that are not allowed, we use this as the base class - * of those commands. - */ -abstract class UnsupportedCommand(exception: ParseException) extends RunnableCommand { - - // Throws the ParseException when we create this command. - throw exception - - override def run(sparkSession: SparkSession): Seq[Row] = { - Seq.empty[Row] - } - - override val output: Seq[Attribute] = { - Seq(AttributeReference("result", StringType, nullable = false)()) - } - -} - /** * A command for users to create a new database. * @@ -251,8 +230,8 @@ case class AlterTableSetProperties( val table = catalog.getTableMetadata(tableName) val newProperties = table.properties ++ properties if (DDLUtils.isDatasourceTable(newProperties)) { - throw new AnalysisException( - "alter table properties is not supported for tables defined using the datasource API") + throw new AnalysisException("ALTER TABLE SET TBLPROPERTIES is not supported for " + + "tables defined using the datasource API") } val newTable = table.copy(properties = newProperties) catalog.alterTable(newTable) @@ -319,15 +298,14 @@ case class AlterTableSerDeProperties( // should never happen if we parsed things correctly require(serdeClassName.isDefined || serdeProperties.isDefined, - "alter table attempted to set neither serde class name nor serde properties") + "ALTER TABLE attempted to set neither serde class name nor serde properties") override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) // Do not support setting serde for datasource tables if (serdeClassName.isDefined && DDLUtils.isDatasourceTable(table)) { - throw new AnalysisException( - "alter table serde is not supported for datasource tables") + throw new AnalysisException("ALTER TABLE SET SERDE is not supported for datasource tables") } val newTable = table.withNewStorage( serde = serdeClassName.orElse(table.storage.serde), @@ -361,7 +339,7 @@ case class AlterTableAddPartition( val table = catalog.getTableMetadata(tableName) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - "alter table add partition is not allowed for tables defined using the datasource API") + "ALTER TABLE ADD PARTITION is not allowed for tables defined using the datasource API") } val parts = partitionSpecsAndLocs.map { case (spec, location) => // inherit table storage format (possibly except for location) @@ -420,7 +398,7 @@ case class AlterTableDropPartition( val table = catalog.getTableMetadata(tableName) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - "alter table drop partition is not allowed for tables defined using the datasource API") + "ALTER TABLE DROP PARTITIONS is not allowed for tables defined using the datasource API") } catalog.dropPartitions(tableName, specs, ignoreIfNotExists = ifExists) Seq.empty[Row] @@ -428,12 +406,6 @@ case class AlterTableDropPartition( } -case class AlterTableSetFileFormat( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - fileFormat: Seq[String], - genericFormat: Option[String])(exception: ParseException) - extends UnsupportedCommand(exception) with Logging /** * A command that sets the location of a table or a partition. @@ -462,7 +434,7 @@ case class AlterTableSetLocation( val newPart = if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - "alter table set location for partition is not allowed for tables defined " + + "ALTER TABLE SET LOCATION for partition is not allowed for tables defined " + "using the datasource API") } else { part.copy(storage = part.storage.copy(locationUri = Some(location))) @@ -485,34 +457,6 @@ case class AlterTableSetLocation( } -case class AlterTableChangeCol( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - oldColName: String, - newColName: String, - dataType: DataType, - comment: Option[String], - afterColName: Option[String], - restrict: Boolean, - cascade: Boolean)(exception: ParseException) - extends UnsupportedCommand(exception) with Logging - -case class AlterTableAddCol( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - columns: StructType, - restrict: Boolean, - cascade: Boolean)(exception: ParseException) - extends UnsupportedCommand(exception) with Logging - -case class AlterTableReplaceCol( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - columns: StructType, - restrict: Boolean, - cascade: Boolean)(exception: ParseException) - extends UnsupportedCommand(exception) with Logging - private[sql] object DDLUtils { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 5aa779ddeb6c..73c1ef70a2d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -51,9 +51,8 @@ case class CreateFunction( val catalog = sparkSession.sessionState.catalog if (isTemp) { if (databaseName.isDefined) { - throw new AnalysisException( - s"It is not allowed to provide database name when defining a temporary function. " + - s"However, database name ${databaseName.get} is provided.") + throw new AnalysisException(s"Specifying a database in CREATE TEMPORARY FUNCTION " + + s"is not allowed: '${databaseName.get}'") } // We first load resources and then put the builder in the function registry. // Please note that it is allowed to overwrite an existing temp function. @@ -153,9 +152,8 @@ case class DropFunction( val catalog = sparkSession.sessionState.catalog if (isTemp) { if (databaseName.isDefined) { - throw new AnalysisException( - s"It is not allowed to provide database name when dropping a temporary function. " + - s"However, database name ${databaseName.get} is provided.") + throw new AnalysisException(s"Specifying a database in DROP TEMPORARY FUNCTION " + + s"is not allowed: '${databaseName.get}'") } catalog.dropTempFunction(functionName, ifExists) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 6078918316d9..489c980c1654 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -162,37 +162,36 @@ case class LoadData( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog if (!catalog.tableExists(table)) { - throw new AnalysisException( - s"Table in LOAD DATA does not exist: '$table'") + throw new AnalysisException(s"Target table in LOAD DATA does not exist: '$table'") } - val targetTable = catalog.getTableMetadataOption(table).getOrElse { - throw new AnalysisException( - s"Table in LOAD DATA cannot be temporary: '$table'") + throw new AnalysisException(s"Target table in LOAD DATA cannot be temporary: '$table'") } - if (DDLUtils.isDatasourceTable(targetTable)) { - throw new AnalysisException( - "LOAD DATA is not supported for datasource tables") + throw new AnalysisException(s"LOAD DATA is not supported for datasource tables: '$table'") } - if (targetTable.partitionColumnNames.nonEmpty) { - if (partition.isEmpty || targetTable.partitionColumnNames.size != partition.get.size) { - throw new AnalysisException( - "LOAD DATA to partitioned table must specify a specific partition of " + - "the table by specifying values for all of the partitioning columns.") + if (partition.isEmpty) { + throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + s"but no partition spec is provided") + } + if (targetTable.partitionColumnNames.size != partition.get.size) { + throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + s"but number of columns in provided partition spec (${partition.get.size}) " + + s"do not match number of partitioned columns in table " + + s"(s${targetTable.partitionColumnNames.size})") } - partition.get.keys.foreach { colName => if (!targetTable.partitionColumnNames.contains(colName)) { - throw new AnalysisException( - s"LOAD DATA to partitioned table specifies a non-existing partition column: '$colName'") + throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + s"but the specified partition spec refers to a column that is not partitioned: " + + s"'$colName'") } } } else { if (partition.nonEmpty) { - throw new AnalysisException( - "LOAD DATA to non-partitioned table cannot specify partition.") + throw new AnalysisException(s"LOAD DATA target table '$table' is not partitioned, " + + s"but a partition spec was provided.") } } @@ -200,7 +199,7 @@ case class LoadData( if (isLocal) { val uri = Utils.resolveURI(path) if (!new File(uri.getPath()).exists()) { - throw new AnalysisException(s"LOAD DATA with non-existing path: $path") + throw new AnalysisException(s"LOAD DATA input path does not exist: $path") } uri } else { @@ -231,7 +230,7 @@ case class LoadData( if (scheme == null) { throw new AnalysisException( - "LOAD DATA with non-local path must specify URI Scheme.") + s"LOAD DATA: URI scheme is required for non-local input paths: '$path'") } // Follow Hive's behavior: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 1641780db8bc..0f656ef53e39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -58,8 +58,7 @@ case class CreateViewCommand( private val tableIdentifier = tableDesc.identifier if (allowExisting && replace) { - throw new AnalysisException( - "It is not allowed to define a view with both IF NOT EXISTS and OR REPLACE.") + throw new AnalysisException("CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed.") } override def run(sparkSession: SparkSession): Seq[Row] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index be0f4d78a523..bd428a06f509 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -23,17 +23,18 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ + // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { private val parser = new SparkSqlParser(new SQLConf) - private def assertUnsupported(sql: String): Unit = { + private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { val e = intercept[ParseException] { parser.parsePlan(sql) } assert(e.getMessage.toLowerCase.contains("operation not allowed")) + containsThesePhrases.foreach { p => assert(e.getMessage.toLowerCase.contains(p)) } } test("create database") { @@ -347,27 +348,13 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed2, expected2) } - // ALTER VIEW view_name ADD [IF NOT EXISTS] PARTITION partition_spec PARTITION partition_spec ...; - test("alter view: add partition") { - val sql1 = + test("alter view: add partition (not supported)") { + assertUnsupported( """ |ALTER VIEW view_name ADD IF NOT EXISTS PARTITION |(dt='2008-08-08', country='us') PARTITION |(dt='2009-09-09', country='uk') - """.stripMargin - // different constant types in partitioning spec - val sql2 = - """ - |ALTER VIEW view_name ADD PARTITION - |(col1=NULL, cOL2='f', col3=5, COL4=true) - """.stripMargin - - intercept[ParseException] { - parser.parsePlan(sql1) - } - intercept[ParseException] { - parser.parsePlan(sql2) - } + """.stripMargin) } test("alter table: rename partition") { @@ -392,7 +379,7 @@ class DDLCommandSuite extends PlanTest { """.stripMargin) } - // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE] + // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] // ALTER VIEW table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] test("alter table/view: drop partitions") { val sql1_table = @@ -403,24 +390,17 @@ class DDLCommandSuite extends PlanTest { val sql2_table = """ |ALTER TABLE table_name DROP PARTITION - |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') PURGE + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') """.stripMargin val sql1_view = sql1_table.replace("TABLE", "VIEW") - // Note: ALTER VIEW DROP PARTITION does not support PURGE - val sql2_view = sql2_table.replace("TABLE", "VIEW").replace("PURGE", "") + val sql2_view = sql2_table.replace("TABLE", "VIEW") val parsed1_table = parser.parsePlan(sql1_table) - val e = intercept[ParseException] { - parser.parsePlan(sql2_table) - } - assert(e.getMessage.contains("Operation not allowed")) - - intercept[ParseException] { - parser.parsePlan(sql1_view) - } - intercept[ParseException] { - parser.parsePlan(sql2_view) - } + val parsed2_table = parser.parsePlan(sql2_table) + assertUnsupported(sql1_table + " PURGE") + assertUnsupported(sql2_table + " PURGE") + assertUnsupported(sql1_view) + assertUnsupported(sql2_view) val tableIdent = TableIdentifier("table_name", None) val expected1_table = AlterTableDropPartition( @@ -429,8 +409,10 @@ class DDLCommandSuite extends PlanTest { Map("dt" -> "2008-08-08", "country" -> "us"), Map("dt" -> "2009-09-09", "country" -> "uk")), ifExists = true) + val expected2_table = expected1_table.copy(ifExists = false) comparePlans(parsed1_table, expected1_table) + comparePlans(parsed2_table, expected2_table) } test("alter table: archive partition (not supported)") { @@ -441,29 +423,6 @@ class DDLCommandSuite extends PlanTest { assertUnsupported("ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')") } - /* - test("alter table: set file format") { - val sql1 = "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + - "OUTPUTFORMAT 'test' SERDE 'test'" - val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "SET FILEFORMAT PARQUET" - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableSetFileFormat( - tableIdent, - None, - List("test", "test", "test"), - None)(sql1) - val expected2 = AlterTableSetFileFormat( - tableIdent, - Some(Map("dt" -> "2008-08-08", "country" -> "us")), - Seq(), - Some("PARQUET"))(sql2) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } */ - test("alter table: set file format (not allowed)") { assertUnsupported( "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + @@ -527,58 +486,6 @@ class DDLCommandSuite extends PlanTest { assertUnsupported("ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES") } - /* - test("alter table: change column name/type/position/comment") { - val sql1 = "ALTER TABLE table_name CHANGE col_old_name col_new_name INT" - val sql2 = - """ - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' FIRST CASCADE - """.stripMargin - val sql3 = - """ - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' AFTER column_name RESTRICT - """.stripMargin - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val parsed3 = parser.parsePlan(sql3) - val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableChangeCol( - tableName = tableIdent, - partitionSpec = None, - oldColName = "col_old_name", - newColName = "col_new_name", - dataType = IntegerType, - comment = None, - afterColName = None, - restrict = false, - cascade = false)(sql1) - val expected2 = AlterTableChangeCol( - tableName = tableIdent, - partitionSpec = None, - oldColName = "col_old_name", - newColName = "col_new_name", - dataType = IntegerType, - comment = Some("col_comment"), - afterColName = None, - restrict = false, - cascade = true)(sql2) - val expected3 = AlterTableChangeCol( - tableName = tableIdent, - partitionSpec = None, - oldColName = "col_old_name", - newColName = "col_new_name", - dataType = IntegerType, - comment = Some("col_comment"), - afterColName = Some("column_name"), - restrict = true, - cascade = false)(sql3) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected3) - } */ - test("alter table: change column name/type/position/comment (not allowed)") { assertUnsupported("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") assertUnsupported( @@ -592,44 +499,6 @@ class DDLCommandSuite extends PlanTest { """.stripMargin) } - /* - test("alter table: add/replace columns") { - val sql1 = - """ - |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') - |ADD COLUMNS (new_col1 INT COMMENT 'test_comment', new_col2 LONG - |COMMENT 'test_comment2') CASCADE - """.stripMargin - val sql2 = - """ - |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT - |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT - """.stripMargin - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val meta1 = new MetadataBuilder().putString("comment", "test_comment").build() - val meta2 = new MetadataBuilder().putString("comment", "test_comment2").build() - val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableAddCol( - tableIdent, - Some(Map("dt" -> "2008-08-08", "country" -> "us")), - StructType(Seq( - StructField("new_col1", IntegerType, nullable = true, meta1), - StructField("new_col2", LongType, nullable = true, meta2))), - restrict = false, - cascade = true)(sql1) - val expected2 = AlterTableReplaceCol( - tableIdent, - None, - StructType(Seq( - StructField("new_col1", IntegerType, nullable = true, meta1), - StructField("new_col2", LongType, nullable = true, meta2))), - restrict = true, - cascade = false)(sql2) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } */ - test("alter table: add/replace columns (not allowed)") { assertUnsupported( """ @@ -678,6 +547,7 @@ class DDLCommandSuite extends PlanTest { val parsed2 = parser.parsePlan(s"DROP TABLE IF EXISTS $tableName1") val parsed3 = parser.parsePlan(s"DROP TABLE $tableName2") val parsed4 = parser.parsePlan(s"DROP TABLE IF EXISTS $tableName2") + assertUnsupported(s"DROP TABLE IF EXISTS $tableName2 PURGE") val expected1 = DropTable(TableIdentifier("tab", Option("db")), ifExists = false, isView = false) @@ -722,20 +592,20 @@ class DDLCommandSuite extends PlanTest { val sql1 = "SHOW COLUMNS FROM t1" val sql2 = "SHOW COLUMNS IN db1.t1" val sql3 = "SHOW COLUMNS FROM t1 IN db1" - val sql4 = "SHOW COLUMNS FROM db1.t1 IN db2" + val sql4 = "SHOW COLUMNS FROM db1.t1 IN db1" + val sql5 = "SHOW COLUMNS FROM db1.t1 IN db2" val parsed1 = parser.parsePlan(sql1) val expected1 = ShowColumnsCommand(TableIdentifier("t1", None)) val parsed2 = parser.parsePlan(sql2) val expected2 = ShowColumnsCommand(TableIdentifier("t1", Some("db1"))) val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql3) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected2) - val message = intercept[ParseException] { - parser.parsePlan(sql4) - }.getMessage - assert(message.contains("Duplicates the declaration for database")) + comparePlans(parsed4, expected2) + assertUnsupported(sql5) } test("show partitions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index cb88a1c83c99..c1dc9b983407 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -22,6 +22,7 @@ import java.io.{File, IOException} import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils @@ -104,7 +105,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with sql("SELECT a, b FROM jsonTable"), sql("SELECT a, b FROM jt").collect()) - val message = intercept[AnalysisException]{ + val message = intercept[ParseException]{ sql( s""" |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable @@ -115,9 +116,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with |SELECT a * 4 FROM jt """.stripMargin) }.getMessage - assert( - message.contains(s"a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause."), - "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") + assert(message.toLowerCase.contains("operation not allowed")) // Overwrite the temporary table. sql( @@ -155,7 +154,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with } test("CREATE TEMPORARY TABLE AS SELECT with IF NOT EXISTS is not allowed") { - val message = intercept[AnalysisException]{ + val message = intercept[ParseException]{ sql( s""" |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable @@ -166,9 +165,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with |SELECT b FROM jt """.stripMargin) }.getMessage - assert( - message.contains("a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause."), - "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") + assert(message.toLowerCase.contains("operation not allowed")) } test("a CTAS statement with column definitions is not allowed") { 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 f082035852cc..1eebecaf654a 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 @@ -503,7 +503,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // We have converted the useful parts of these tests to tests // in org.apache.spark.sql.hive.execution.SQLQuerySuite. "drop_database_removes_partition_dirs", - "drop_table_removes_partition_dirs" + "drop_table_removes_partition_dirs", + + // These tests use EXPLAIN FORMATTED, which is not supported + "input4", + "join0", + "plan_json" ) /** @@ -699,7 +704,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input26", "input28", "input2_limit", - "input4", "input40", "input41", "input49", @@ -728,7 +732,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "insert1_overwrite_partitions", "insert2_overwrite_partitions", "insert_compressed", - "join0", "join1", "join10", "join11", @@ -866,7 +869,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "partition_type_check", "partition_varchar1", "partition_wise_fileformat9", - "plan_json", "ppd1", "ppd2", "ppd_clusterby", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index ee048b258830..5ffd8ef149a1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -75,7 +75,7 @@ private[spark] class HiveExternalCatalog(client: HiveClient) extends ExternalCat private def requireDbMatches(db: String, table: CatalogTable): Unit = { if (table.identifier.database != Some(db)) { throw new AnalysisException( - s"Provided database $db does not match the one specified in the " + + s"Provided database '$db' does not match the one specified in the " + s"table definition (${table.identifier.database.getOrElse("n/a")})") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index cdfadfaaeac7..47d9546c4f51 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -26,9 +26,12 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{PartitionDropOptions, TableType => HiveTableType} -import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema, Function => HiveFunction, FunctionType, PrincipalType, ResourceType, ResourceUri} +import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} +import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType} +import org.apache.hadoop.hive.metastore.api.{NoSuchObjectException, PrincipalType} +import org.apache.hadoop.hive.metastore.api.{ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState @@ -43,7 +46,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.util.{CircularBuffer, Utils} +import org.apache.spark.util.{CausedBy, CircularBuffer, Utils} /** * A class that wraps the HiveClient and converts its responses to externally visible classes. @@ -616,7 +619,8 @@ private[hive] class HiveClientImpl( try { Option(client.getFunction(db, name)).map(fromHiveFunction) } catch { - case he: HiveException => None + case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + None } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 8b3f2d1a0cd0..b8fef23f5464 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -125,7 +125,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message1 = intercept[AnalysisException] { sql("SHOW TBLPROPERTIES badtable") }.getMessage - assert(message1.contains("Table or View badtable not found in database default")) + assert(message1.contains("'badtable' not found in database 'default'")) // When key is not found, a row containing the error is returned. checkAnswer( @@ -289,7 +289,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message = intercept[NoSuchTableException] { sql("SHOW COLUMNS IN badtable FROM default") }.getMessage - assert(message.contains("badtable not found in database")) + assert(message.contains("'badtable' not found in database")) } test("show partitions - show everything") { 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 f20ab36efbf0..f7da9e73f6cb 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 @@ -1502,7 +1502,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } - assert(cause.getMessage.contains("Only ASC ordering is supported for sorting columns")) + assert(cause.getMessage.contains("Column ordering must be ASC, was 'DESC'")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 0d88b3b87f50..51848470502a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -105,7 +105,8 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val e = intercept[AnalysisException] { sql("CREATE OR REPLACE VIEW IF NOT EXISTS testView AS SELECT id FROM jt") } - assert(e.message.contains("not allowed to define a view")) + assert(e.message.contains( + "CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed")) } } } From fd3accca6c2b192fe193cb71c44b4db70ddca1cc Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 May 2016 21:59:03 -0700 Subject: [PATCH 053/313] [SPARK-15095][SQL] remove HiveSessionHook from ThriftServer ## What changes were proposed in this pull request? Remove HiveSessionHook ## How was this patch tested? No tests needed. Author: Davies Liu Closes #12881 from davies/remove_hooks. (cherry picked from commit 348c1389842c4d9a8807a41cf22caaa82f81d7ab) Signed-off-by: Reynold Xin --- .../service/cli/session/HiveSessionHook.java | 37 ------------------- .../service/cli/session/SessionManager.java | 20 ---------- 2 files changed, 57 deletions(-) delete mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java deleted file mode 100644 index 7e4f2b0478a5..000000000000 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.session; - -import org.apache.hadoop.hive.ql.hooks.Hook; -import org.apache.hive.service.cli.HiveSQLException; - -/** - * HiveSessionHook. - * HiveServer2 session level Hook interface. The run method is executed - * when session manager starts a new session - * - */ -public interface HiveSessionHook extends Hook { - - /** - * @param sessionHookContext context - * @throws HiveSQLException - */ - void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; -} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index c6d177798f55..0457b3781ebf 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -268,17 +268,6 @@ public SessionHandle openSession(TProtocolVersion protocol, String username, Str if (isOperationLogEnabled) { session.setOperationLogSessionDir(operationLogRootDir); } - try { - executeSessionHooks(session); - } catch (Exception e) { - try { - session.close(); - } catch (Throwable t) { - LOG.warn("Error closing session", t); - } - session = null; - throw new HiveSQLException("Failed to execute session hooks", e); - } handleToSession.put(session.getSessionHandle(), session); return session.getSessionHandle(); } @@ -361,15 +350,6 @@ public static void clearProxyUserName() { threadLocalProxyUserName.remove(); } - // execute session hooks - private void executeSessionHooks(HiveSession session) throws Exception { - List sessionHooks = HookUtils.getHooks(hiveConf, - HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, HiveSessionHook.class); - for (HiveSessionHook sessionHook : sessionHooks) { - sessionHook.run(new HiveSessionHookContextImpl(session)); - } - } - public Future submitBackgroundOperation(Runnable r) { return backgroundOperationPool.submit(r); } From 2ff6676062edf6b4d463c30aeba623869887d21a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 May 2016 22:56:40 -0700 Subject: [PATCH 054/313] [SPARK-15107][SQL] Allow varying # iterations by test case in Benchmark This patch changes our micro-benchmark util to allow setting different iteration numbers for different test cases. For some of our benchmarks, turning off whole-stage codegen can make the runtime 20X slower, making it very difficult to run a large number of times without substantially shortening the input cardinality. With this change, I set the default num iterations to 2 for whole stage codegen off, and 5 for whole stage codegen on. I also updated some results. N/A - this is a test util. Author: Reynold Xin Closes #12884 from rxin/SPARK-15107. (cherry picked from commit 695f0e9195209c75bfc62fc70bfc6d7d9f1047b3) Signed-off-by: Reynold Xin --- .../org/apache/spark/util/Benchmark.scala | 21 ++- .../unsafe/sort/RadixSortSuite.scala | 2 +- .../BenchmarkWholeStageCodegen.scala | 137 +++++++++++------- 3 files changed, 93 insertions(+), 67 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 1fc0ad7a4d6d..0c685b1918a5 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -38,7 +38,7 @@ import org.apache.commons.lang3.SystemUtils private[spark] class Benchmark( name: String, valuesPerIteration: Long, - iters: Int = 5, + defaultNumIters: Int = 5, outputPerIteration: Boolean = false) { val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] @@ -46,8 +46,8 @@ private[spark] class Benchmark( * Adds a case to run when run() is called. The given function will be run for several * iterations to collect timing statistics. */ - def addCase(name: String)(f: Int => Unit): Unit = { - addTimerCase(name) { timer => + def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { + addTimerCase(name, numIters) { timer => timer.startTiming() f(timer.iteration) timer.stopTiming() @@ -59,8 +59,8 @@ private[spark] class Benchmark( * until timer.startTiming() is called within the given function. The corresponding * timer.stopTiming() method must be called before the function returns. */ - def addTimerCase(name: String)(f: Benchmark.Timer => Unit): Unit = { - benchmarks += Benchmark.Case(name, f) + def addTimerCase(name: String, numIters: Int = 0)(f: Benchmark.Timer => Unit): Unit = { + benchmarks += Benchmark.Case(name, f, if (numIters == 0) defaultNumIters else numIters) } /** @@ -75,7 +75,7 @@ private[spark] class Benchmark( val results = benchmarks.map { c => println(" Running case: " + c.name) - Benchmark.measure(valuesPerIteration, iters, outputPerIteration)(c.fn) + Benchmark.measure(valuesPerIteration, c.numIters, outputPerIteration)(c.fn) } println @@ -83,12 +83,11 @@ private[spark] class Benchmark( // The results are going to be processor specific so it is useful to include that. println(Benchmark.getJVMOSInfo()) println(Benchmark.getProcessorName()) - printf("%-35s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", "Per Row(ns)", "Relative") - println("-----------------------------------------------------------------------------------" + - "--------") + println("-" * 96) results.zip(benchmarks).foreach { case (result, benchmark) => - printf("%-35s %16s %12s %13s %10s\n", + printf("%-40s %16s %12s %13s %10s\n", benchmark.name, "%5.0f / %4.0f" format (result.bestMs, result.avgMs), "%10.1f" format result.bestRate, @@ -128,7 +127,7 @@ private[spark] object Benchmark { } } - case class Case(name: String, fn: Timer => Unit) + case class Case(name: String, fn: Timer => Unit, numIters: Int) case class Result(avgMs: Double, bestRate: Double, bestMs: Double) /** diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 52428634e520..b03df1a94d84 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -244,7 +244,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) timer.stopTiming() } - benchmark.run + benchmark.run() /** Running benchmark: radix sort 25000000 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala index 841263d3dab9..7ca4b75f480b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala @@ -36,6 +36,8 @@ import org.apache.spark.util.Benchmark * Benchmark to measure whole stage codegen performance. * To run this: * build/sbt "sql/test-only *BenchmarkWholeStageCodegen" + * + * Benchmarks in this file are skipped in normal builds. */ class BenchmarkWholeStageCodegen extends SparkFunSuite { lazy val conf = new SparkConf().setMaster("local[1]").setAppName("benchmark") @@ -44,31 +46,50 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { lazy val sc = SparkContext.getOrCreate(conf) lazy val sqlContext = SQLContext.getOrCreate(sc) - def runBenchmark(name: String, values: Long)(f: => Unit): Unit = { - val benchmark = new Benchmark(name, values) + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality) - Seq(false, true).foreach { enabled => - benchmark.addCase(s"$name codegen=$enabled") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", enabled.toString) - f - } + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + f + } + + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + sqlContext.setConf("spark.sql.codegen.wholeStage", "true") + f } benchmark.run() } - // These benchmark are skipped in normal build - ignore("range/filter/sum") { - val N = 500L << 20 - runBenchmark("rang/filter/sum", N) { + ignore("aggregate without grouping") { + val N = 500L << 22 + val benchmark = new Benchmark("agg without grouping", N) + runBenchmark("agg w/o group", N) { + sqlContext.range(N).selectExpr("sum(id)").collect() + } + /* + agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + agg w/o group wholestage off 30136 / 31885 69.6 14.4 1.0X + agg w/o group wholestage on 1851 / 1860 1132.9 0.9 16.3X + */ + } + + ignore("filter & aggregate without group") { + val N = 500L << 22 + runBenchmark("range/filter/sum", N) { sqlContext.range(N).filter("(id & 1) = 1").groupBy().sum().collect() } /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - rang/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - rang/filter/sum codegen=false 14332 / 16646 36.0 27.8 1.0X - rang/filter/sum codegen=true 897 / 1022 584.6 1.7 16.4X + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X + range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X */ } @@ -86,28 +107,32 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { */ } - ignore("range/sample/sum") { - val N = 500 << 20 - runBenchmark("range/sample/sum", N) { - sqlContext.range(N).sample(true, 0.01).groupBy().sum().collect() + ignore("sample") { + val N = 500 << 18 + runBenchmark("sample with replacement", N) { + sqlContext.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() } /* - Westmere E56xx/L56xx/X56xx (Nehalem-C) - range/sample/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - range/sample/sum codegen=false 53888 / 56592 9.7 102.8 1.0X - range/sample/sum codegen=true 41614 / 42607 12.6 79.4 1.3X + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X + sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X */ - runBenchmark("range/sample/sum", N) { - sqlContext.range(N).sample(false, 0.01).groupBy().sum().collect() + runBenchmark("sample without replacement", N) { + sqlContext.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() } /* - Westmere E56xx/L56xx/X56xx (Nehalem-C) - range/sample/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - range/sample/sum codegen=false 12982 / 13384 40.4 24.8 1.0X - range/sample/sum codegen=true 7074 / 7383 74.1 13.5 1.8X + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X + sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X */ } @@ -151,23 +176,23 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { } ignore("aggregate with linear keys") { - val N = 20 << 20 + val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) def f(): Unit = sqlContext.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() - benchmark.addCase(s"codegen = F") { iter => + benchmark.addCase(s"codegen = F", numIters = 2) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "false") f() } - benchmark.addCase(s"codegen = T hashmap = F") { iter => + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") f() } - benchmark.addCase(s"codegen = T hashmap = T") { iter => + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") f() @@ -176,36 +201,37 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { benchmark.run() /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - codegen = F 2067 / 2166 10.1 98.6 1.0X - codegen = T hashmap = F 1149 / 1321 18.3 54.8 1.8X - codegen = T hashmap = T 388 / 475 54.0 18.5 5.3X + + Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + codegen = F 6619 / 6780 12.7 78.9 1.0X + codegen = T hashmap = F 3935 / 4059 21.3 46.9 1.7X + codegen = T hashmap = T 897 / 971 93.5 10.7 7.4X */ } ignore("aggregate with randomized keys") { - val N = 20 << 20 + val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) sqlContext.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") def f(): Unit = sqlContext.sql("select k, k, sum(id) from test group by k, k").collect() - benchmark.addCase(s"codegen = F") { iter => + benchmark.addCase(s"codegen = F", numIters = 2) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "false") f() } - benchmark.addCase(s"codegen = T hashmap = F") { iter => + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") f() } - benchmark.addCase(s"codegen = T hashmap = T") { iter => + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") f() @@ -214,13 +240,14 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { benchmark.run() /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - codegen = F 2517 / 2608 8.3 120.0 1.0X - codegen = T hashmap = F 1484 / 1560 14.1 70.8 1.7X - codegen = T hashmap = T 794 / 908 26.4 37.9 3.2X + + Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + codegen = F 7445 / 7517 11.3 88.7 1.0X + codegen = T hashmap = F 4672 / 4703 18.0 55.7 1.6X + codegen = T hashmap = T 1764 / 1958 47.6 21.0 4.2X */ } @@ -231,18 +258,18 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { def f(): Unit = sqlContext.range(N).selectExpr("id", "cast(id & 1023 as string) as k") .groupBy("k").count().collect() - benchmark.addCase(s"codegen = F") { iter => + benchmark.addCase(s"codegen = F", numIters = 2) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "false") f() } - benchmark.addCase(s"codegen = T hashmap = F") { iter => + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") f() } - benchmark.addCase(s"codegen = T hashmap = T") { iter => + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") f() From 64ad9ba27aafa333a6897861fe2aeb4bb83506fc Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 4 May 2016 14:16:57 +0800 Subject: [PATCH 055/313] [SPARK-14237][SQL] De-duplicate partition value appending logic in various buildReader() implementations ## What changes were proposed in this pull request? Currently, various `FileFormat` data sources share approximately the same code for partition value appending. This PR tries to eliminate this duplication. A new method `buildReaderWithPartitionValues()` is added to `FileFormat` with a default implementation that appends partition values to `InternalRow`s produced by the reader function returned by `buildReader()`. Special data sources like Parquet, which implements partition value appending inside `buildReader()` because of the vectorized reader, and the Text data source, which doesn't support partitioning, override `buildReaderWithPartitionValues()` and simply delegate to `buildReader()`. This PR brings two benefits: 1. Apparently, it de-duplicates partition value appending logic 2. Now the reader function returned by `buildReader()` is only required to produce `InternalRow`s rather than `UnsafeRow`s if the data source doesn't override `buildReaderWithPartitionValues()`. Because the safe-to-unsafe conversion is also performed while appending partition values. This makes 3rd-party data sources (e.g. spark-avro) easier to implement since they no longer need to access private APIs involving `UnsafeRow`. ## How was this patch tested? Existing tests should do the work. Author: Cheng Lian Closes #12866 from liancheng/spark-14237-simplify-partition-values-appending. (cherry picked from commit bc3760d405cc8c3ffcd957b188afa8b7e3b1f824) Signed-off-by: Cheng Lian --- .../ml/source/libsvm/LibSVMRelation.scala | 17 +------- .../datasources/FileSourceStrategy.scala | 2 +- .../datasources/csv/DefaultSource.scala | 17 ++------ .../datasources/fileSourceInterfaces.scala | 40 +++++++++++++++++++ .../datasources/json/JSONRelation.scala | 10 +---- .../datasources/parquet/ParquetRelation.scala | 14 +++++++ .../datasources/text/DefaultSource.scala | 13 ++++++ .../execution/datasources/csv/CSVSuite.scala | 3 -- .../spark/sql/hive/orc/OrcRelation.scala | 11 +---- 9 files changed, 74 insertions(+), 53 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index ba2e1e2bc269..5f78fab4ddba 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -204,25 +204,10 @@ class DefaultSource extends FileFormat with DataSourceRegister { val converter = RowEncoder(dataSchema) - val unsafeRowIterator = points.map { pt => + points.map { pt => val features = if (sparse) pt.features.toSparse else pt.features.toDense converter.toRow(Row(pt.label, features)) } - - def toAttribute(f: StructField): AttributeReference = - AttributeReference(f.name, f.dataType, f.nullable, f.metadata)() - - // Appends partition values - val fullOutput = (dataSchema ++ partitionSchema).map(toAttribute) - val requiredOutput = fullOutput.filter { a => - requiredSchema.fieldNames.contains(a.name) || partitionSchema.fieldNames.contains(a.name) - } - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput) - - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) - } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 615906a52e8d..8a93c6ff9a4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -106,7 +106,7 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}") - val readFile = files.fileFormat.buildReader( + val readFile = files.fileFormat.buildReaderWithPartitionValues( sparkSession = files.sparkSession, dataSchema = files.dataSchema, partitionSchema = files.partitionSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala index 75143e609aaf..948fac0d58ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala @@ -117,20 +117,9 @@ class DefaultSource extends FileFormat with DataSourceRegister { CSVRelation.dropHeaderLine(file, lineIterator, csvOptions) - val unsafeRowIterator = { - val tokenizedIterator = new BulkCsvReader(lineIterator, csvOptions, headers) - val parser = CSVRelation.csvParser(dataSchema, requiredSchema.fieldNames, csvOptions) - tokenizedIterator.flatMap(parser(_).toSeq) - } - - // Appends partition values - val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullOutput, fullOutput) - - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) - } + val tokenizedIterator = new BulkCsvReader(lineIterator, csvOptions, headers) + val parser = CSVRelation.csvParser(dataSchema, requiredSchema.fieldNames, csvOptions) + tokenizedIterator.flatMap(parser(_).toSeq) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 0a3461151c62..24e2bf6d136a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.FileRelation import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.{StringType, StructType} @@ -238,6 +239,45 @@ trait FileFormat { throw new UnsupportedOperationException(s"buildReader is not supported for $this") } + /** + * Exactly the same as [[buildReader]] except that the reader function returned by this method + * appends partition values to [[InternalRow]]s produced by the reader function [[buildReader]] + * returns. + */ + private[sql] def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + val dataReader = buildReader( + sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) + + new (PartitionedFile => Iterator[InternalRow]) with Serializable { + private val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + + private val joinedRow = new JoinedRow() + + // Using lazy val to avoid serialization + private lazy val appendPartitionColumns = + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + override def apply(file: PartitionedFile): Iterator[InternalRow] = { + // Using local val to avoid per-row lazy val check (pre-mature optimization?...) + val converter = appendPartitionColumns + + // Note that we have to apply the converter even though `file.partitionValues` is empty. + // This is because the converter is also responsible for converting safe `InternalRow`s into + // `UnsafeRow`s. + dataReader(file).map { dataRow => + converter(joinedRow(dataRow, file.partitionValues)) + } + } + } + } + /** * Returns a [[OutputWriterFactory]] for generating output writers that can write data. * This method is current used only by FileStreamSinkWriter to generate output writers that diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index 62446583a55b..4c97abed53ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -106,22 +106,14 @@ class DefaultSource extends FileFormat with DataSourceRegister { val columnNameOfCorruptRecord = parsedOptions.columnNameOfCorruptRecord .getOrElse(sparkSession.sessionState.conf.columnNameOfCorruptRecord) - val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - (file: PartitionedFile) => { val lines = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map(_.toString) - val rows = JacksonParser.parseJson( + JacksonParser.parseJson( lines, requiredSchema, columnNameOfCorruptRecord, parsedOptions) - - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - rows.map { row => - appendPartitionColumns(joinedRow(row, file.partitionValues)) - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 79185df67325..cf5c8e94f468 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -255,6 +255,20 @@ private[sql] class DefaultSource schema.forall(_.dataType.isInstanceOf[AtomicType]) } + override private[sql] def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + // For Parquet data source, `buildReader` already handles partition values appending. Here we + // simply delegate to `buildReader`. + buildReader( + sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) + } + override def buildReader( sparkSession: SparkSession, dataSchema: StructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index 348edfcf7a85..f22c0241d930 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -83,6 +83,19 @@ class DefaultSource extends FileFormat with DataSourceRegister { } } + override private[sql] def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + // Text data source doesn't support partitioning. Here we simply delegate to `buildReader`. + buildReader( + sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) + } + override def buildReader( sparkSession: SparkSession, dataSchema: StructType, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 07f00a086865..28e59055fa1c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -22,9 +22,6 @@ import java.nio.charset.UnsupportedCharsetException import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import scala.collection.JavaConverters._ - -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index d6a847f3ba3a..89d258e84428 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -157,20 +157,11 @@ private[sql] class DefaultSource } // Unwraps `OrcStruct`s to `UnsafeRow`s - val unsafeRowIterator = OrcRelation.unwrapOrcStructs( + OrcRelation.unwrapOrcStructs( conf, requiredSchema, Some(orcRecordReader.getObjectInspector.asInstanceOf[StructObjectInspector]), new RecordReaderIterator[OrcStruct](orcRecordReader)) - - // Appends partition values - val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullOutput, fullOutput) - - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) - } } } } From b99f715e86773ed8b3173df0121f6e5e34e18978 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 4 May 2016 00:10:20 -0700 Subject: [PATCH 056/313] [SPARK-15029] improve error message for Generate ## What changes were proposed in this pull request? This PR improve the error message for `Generate` in 3 cases: 1. generator is nested in expressions, e.g. `SELECT explode(list) + 1 FROM tbl` 2. generator appears more than one time in SELECT, e.g. `SELECT explode(list), explode(list) FROM tbl` 3. generator appears in other operator which is not project, e.g. `SELECT * FROM tbl SORT BY explode(list)` ## How was this patch tested? new tests in `AnalysisErrorSuite` Author: Wenchen Fan Closes #12810 from cloud-fan/bug. (cherry picked from commit 6c12e801e84565265d2939b920901d1805d5935f) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/Analyzer.scala | 123 ++++++++++++------ .../sql/catalyst/analysis/unresolved.scala | 3 +- .../sql/catalyst/expressions/generators.scala | 17 +-- .../expressions/jsonExpressions.scala | 8 +- .../plans/logical/basicLogicalOperators.scala | 2 +- .../spark/sql/catalyst/util/package.scala | 2 + .../analysis/AnalysisErrorSuite.scala | 20 ++- .../scala/org/apache/spark/sql/Dataset.scala | 13 +- .../spark/sql/execution/GenerateExec.scala | 2 +- .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 7 +- 11 files changed, 126 insertions(+), 73 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 59af5b7095a7..527d5b635a7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef -import org.apache.spark.sql.catalyst.util.usePrettyExpression +import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.types._ /** @@ -91,6 +91,7 @@ class Analyzer( ResolvePivot :: ResolveOrdinalInOrderByAndGroupBy :: ResolveMissingReferences :: + ExtractGenerator :: ResolveGenerate :: ResolveFunctions :: ResolveAliases :: @@ -178,8 +179,8 @@ class Analyzer( case e if !e.resolved => u case g: Generator => MultiAlias(g, Nil) case c @ Cast(ne: NamedExpression, _) => Alias(c, ne.name)() - case e: ExtractValue => Alias(e, usePrettyExpression(e).sql)() - case e => Alias(e, optionalAliasName.getOrElse(usePrettyExpression(e).sql))() + case e: ExtractValue => Alias(e, toPrettySQL(e))() + case e => Alias(e, optionalAliasName.getOrElse(toPrettySQL(e)))() } } }.asInstanceOf[Seq[NamedExpression]] @@ -1278,20 +1279,54 @@ class Analyzer( } /** - * Rewrites table generating expressions that either need one or more of the following in order - * to be resolved: - * - concrete attribute references for their output. - * - to be relocated from a SELECT clause (i.e. from a [[Project]]) into a [[Generate]]). + * Extracts [[Generator]] from the projectList of a [[Project]] operator and create [[Generate]] + * operator under [[Project]]. * - * Names for the output [[Attribute]]s are extracted from [[Alias]] or [[MultiAlias]] expressions - * that wrap the [[Generator]]. If more than one [[Generator]] is found in a Project, an - * [[AnalysisException]] is throw. + * This rule will throw [[AnalysisException]] for following cases: + * 1. [[Generator]] is nested in expressions, e.g. `SELECT explode(list) + 1 FROM tbl` + * 2. more than one [[Generator]] is found in projectList, + * e.g. `SELECT explode(list), explode(list) FROM tbl` + * 3. [[Generator]] is found in other operators that are not [[Project]] or [[Generate]], + * e.g. `SELECT * FROM tbl SORT BY explode(list)` */ - object ResolveGenerate extends Rule[LogicalPlan] { + object ExtractGenerator extends Rule[LogicalPlan] { + private def hasGenerator(expr: Expression): Boolean = { + expr.find(_.isInstanceOf[Generator]).isDefined + } + + private def hasNestedGenerator(expr: NamedExpression): Boolean = expr match { + case UnresolvedAlias(_: Generator, _) => false + case Alias(_: Generator, _) => false + case MultiAlias(_: Generator, _) => false + case other => hasGenerator(other) + } + + private def trimAlias(expr: NamedExpression): Expression = expr match { + case UnresolvedAlias(child, _) => child + case Alias(child, _) => child + case MultiAlias(child, _) => child + case _ => expr + } + + /** Extracts a [[Generator]] expression and any names assigned by aliases to their output. */ + private object AliasedGenerator { + def unapply(e: Expression): Option[(Generator, Seq[String])] = e match { + case Alias(g: Generator, name) if g.resolved => Some((g, name :: Nil)) + case MultiAlias(g: Generator, names) if g.resolved => Some(g, names) + case _ => None + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case p: Generate if !p.child.resolved || !p.generator.resolved => p - case g: Generate if !g.resolved => - g.copy(generatorOutput = makeGeneratorOutput(g.generator, g.generatorOutput.map(_.name))) + case Project(projectList, _) if projectList.exists(hasNestedGenerator) => + val nestedGenerator = projectList.find(hasNestedGenerator).get + throw new AnalysisException("Generators are not supported when it's nested in " + + "expressions, but got: " + toPrettySQL(trimAlias(nestedGenerator))) + + case Project(projectList, _) if projectList.count(hasGenerator) > 1 => + val generators = projectList.filter(hasGenerator).map(trimAlias) + throw new AnalysisException("Only one generator allowed per select clause but found " + + generators.size + ": " + generators.map(toPrettySQL).mkString(", ")) case p @ Project(projectList, child) => // Holds the resolved generator, if one exists in the project list. @@ -1299,11 +1334,9 @@ class Analyzer( val newProjectList = projectList.flatMap { case AliasedGenerator(generator, names) if generator.childrenResolved => - if (resolvedGenerator != null) { - failAnalysis( - s"Only one generator allowed per select but ${resolvedGenerator.nodeName} and " + - s"and ${generator.nodeName} found.") - } + // It's a sanity check, this should not happen as the previous case will throw + // exception earlier. + assert(resolvedGenerator == null, "More than one generator found in SELECT.") resolvedGenerator = Generate( @@ -1311,7 +1344,7 @@ class Analyzer( join = projectList.size > 1, // Only join if there are other expressions in SELECT. outer = false, qualifier = None, - generatorOutput = makeGeneratorOutput(generator, names), + generatorOutput = ResolveGenerate.makeGeneratorOutput(generator, names), child) resolvedGenerator.generatorOutput @@ -1323,44 +1356,50 @@ class Analyzer( } else { p } + + case g: Generate => g + + case p if p.expressions.exists(hasGenerator) => + throw new AnalysisException("Generators are not supported outside the SELECT clause, but " + + "got: " + p.simpleString) } + } - /** Extracts a [[Generator]] expression and any names assigned by aliases to their output. */ - private object AliasedGenerator { - def unapply(e: Expression): Option[(Generator, Seq[String])] = e match { - case Alias(g: Generator, name) if g.resolved && g.elementTypes.size > 1 => - // If not given the default names, and the TGF with multiple output columns - failAnalysis( - s"""Expect multiple names given for ${g.getClass.getName}, - |but only single name '${name}' specified""".stripMargin) - case Alias(g: Generator, name) if g.resolved => Some((g, name :: Nil)) - case MultiAlias(g: Generator, names) if g.resolved => Some(g, names) - case _ => None - } + /** + * Rewrites table generating expressions that either need one or more of the following in order + * to be resolved: + * - concrete attribute references for their output. + * - to be relocated from a SELECT clause (i.e. from a [[Project]]) into a [[Generate]]). + * + * Names for the output [[Attribute]]s are extracted from [[Alias]] or [[MultiAlias]] expressions + * that wrap the [[Generator]]. + */ + object ResolveGenerate extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case g: Generate if !g.child.resolved || !g.generator.resolved => g + case g: Generate if !g.resolved => + g.copy(generatorOutput = makeGeneratorOutput(g.generator, g.generatorOutput.map(_.name))) } /** * Construct the output attributes for a [[Generator]], given a list of names. If the list of * names is empty names are assigned from field names in generator. */ - private def makeGeneratorOutput( + private[sql] def makeGeneratorOutput( generator: Generator, names: Seq[String]): Seq[Attribute] = { - val elementTypes = generator.elementTypes + val elementAttrs = generator.elementSchema.toAttributes - if (names.length == elementTypes.length) { - names.zip(elementTypes).map { - case (name, (t, nullable, _)) => - AttributeReference(name, t, nullable)() + if (names.length == elementAttrs.length) { + names.zip(elementAttrs).map { + case (name, attr) => attr.withName(name) } } else if (names.isEmpty) { - elementTypes.map { - case (t, nullable, name) => AttributeReference(name, t, nullable)() - } + elementAttrs } else { failAnalysis( "The number of aliases supplied in the AS clause does not match the number of columns " + - s"output by the UDTF expected ${elementTypes.size} aliases but got " + + s"output by the UDTF expected ${elementAttrs.size} aliases but got " + s"${names.mkString(",")} ") } } 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 f82b63ad9676..1f1897dc36df 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 @@ -142,8 +142,7 @@ object UnresolvedAttribute { case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expression]) extends Generator { - override def elementTypes: Seq[(DataType, Boolean, String)] = - throw new UnresolvedException(this, "elementTypes") + override def elementSchema: StructType = throw new UnresolvedException(this, "elementTypes") override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 65d7a1d5a090..12c35644e564 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -41,19 +41,16 @@ import org.apache.spark.sql.types._ */ trait Generator extends Expression { - // TODO ideally we should return the type of ArrayType(StructType), - // however, we don't keep the output field names in the Generator. - override def dataType: DataType = throw new UnsupportedOperationException + override def dataType: DataType = ArrayType(elementSchema) override def foldable: Boolean = false override def nullable: Boolean = false /** - * The output element data types in structure of Seq[(DataType, Nullable)] - * TODO we probably need to add more information like metadata etc. + * The output element schema. */ - def elementTypes: Seq[(DataType, Boolean, String)] + def elementSchema: StructType /** Should be implemented by child classes to perform specific Generators. */ override def eval(input: InternalRow): TraversableOnce[InternalRow] @@ -69,7 +66,7 @@ trait Generator extends Expression { * A generator that produces its output using the provided lambda function. */ case class UserDefinedGenerator( - elementTypes: Seq[(DataType, Boolean, String)], + elementSchema: StructType, function: Row => TraversableOnce[InternalRow], children: Seq[Expression]) extends Generator with CodegenFallback { @@ -117,10 +114,10 @@ case class Explode(child: Expression) extends UnaryExpression with Generator wit } // hive-compatible default alias for explode function ("col" for array, "key", "value" for map) - override def elementTypes: Seq[(DataType, Boolean, String)] = child.dataType match { - case ArrayType(et, containsNull) => (et, containsNull, "col") :: Nil + override def elementSchema: StructType = child.dataType match { + case ArrayType(et, containsNull) => new StructType().add("col", et, containsNull) case MapType(kt, vt, valueContainsNull) => - (kt, false, "key") :: (vt, valueContainsNull, "value") :: Nil + new StructType().add("key", kt, false).add("value", vt, valueContainsNull) } override def eval(input: InternalRow): TraversableOnce[InternalRow] = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index ecd09b7083f2..c14a2fb12261 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -26,7 +26,7 @@ import com.fasterxml.jackson.core._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -356,9 +356,9 @@ case class JsonTuple(children: Seq[Expression]) // and count the number of foldable fields, we'll use this later to optimize evaluation @transient private lazy val constantFields: Int = foldableFieldNames.count(_ != null) - override def elementTypes: Seq[(DataType, Boolean, String)] = fieldExpressions.zipWithIndex.map { - case (_, idx) => (StringType, true, s"c$idx") - } + override def elementSchema: StructType = StructType(fieldExpressions.zipWithIndex.map { + case (_, idx) => StructField(s"c$idx", StringType, nullable = true) + }) override def prettyName: String = "json_tuple" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 7b4615db0661..8b438e40e6af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -85,7 +85,7 @@ case class Generate( override lazy val resolved: Boolean = { generator.resolved && childrenResolved && - generator.elementTypes.length == generatorOutput.length && + generator.elementSchema.length == generatorOutput.length && generatorOutput.forall(_.resolved) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index f879b34358a9..3d2a624ba3b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -153,6 +153,8 @@ package object util { "`" + name.replace("`", "``") + "`" } + def toPrettySQL(e: Expression): String = usePrettyExpression(e).sql + /** * Returns the string representation of this expression that is safe to be put in * code comments of generated code. The length is capped at 128 characters. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 2e88f61d491c..a41383fbf656 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count} -import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} @@ -330,6 +329,25 @@ class AnalysisErrorSuite extends AnalysisTest { "The start time" :: "must be greater than or equal to 0." :: Nil ) + errorTest( + "generator nested in expressions", + listRelation.select(Explode('list) + 1), + "Generators are not supported when it's nested in expressions, but got: (explode(list) + 1)" + :: Nil + ) + + errorTest( + "generator appears in operator which is not Project", + listRelation.sortBy(Explode('list).asc), + "Generators are not supported outside the SELECT clause, but got: Sort" :: Nil + ) + + errorTest( + "more than one generators in SELECT", + listRelation.select(Explode('list), Explode('list)), + "Only one generator allowed per select clause but found 2: explode(list), explode(list)" :: Nil + ) + test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) // Since we manually construct the logical plan at here and Sum only accept diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 1bea72c4711f..31dd64e909bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1578,16 +1578,13 @@ class Dataset[T] private[sql]( */ @Experimental def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { - val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] + val elementSchema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] - val elementTypes = schema.toAttributes.map { - attr => (attr.dataType, attr.nullable, attr.name) } - val names = schema.toAttributes.map(_.name) - val convert = CatalystTypeConverters.createToCatalystConverter(schema) + val convert = CatalystTypeConverters.createToCatalystConverter(elementSchema) val rowFunction = f.andThen(_.map(convert(_).asInstanceOf[InternalRow])) - val generator = UserDefinedGenerator(elementTypes, rowFunction, input.map(_.expr)) + val generator = UserDefinedGenerator(elementSchema, rowFunction, input.map(_.expr)) withPlan { Generate(generator, join = true, outer = false, @@ -1614,13 +1611,13 @@ class Dataset[T] private[sql]( val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil // TODO handle the metadata? - val elementTypes = attributes.map { attr => (attr.dataType, attr.nullable, attr.name) } + val elementSchema = attributes.toStructType def rowFunction(row: Row): TraversableOnce[InternalRow] = { val convert = CatalystTypeConverters.createToCatalystConverter(dataType) f(row(0).asInstanceOf[A]).map(o => InternalRow(convert(o))) } - val generator = UserDefinedGenerator(elementTypes, rowFunction, apply(inputColumn).expr :: Nil) + val generator = UserDefinedGenerator(elementSchema, rowFunction, apply(inputColumn).expr :: Nil) withPlan { Generate(generator, join = true, outer = false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 934bc38dc47c..8b62c5507c0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -66,7 +66,7 @@ case class GenerateExec( // boundGenerator.terminate() should be triggered after all of the rows in the partition val rows = if (join) { child.execute().mapPartitionsInternal { iter => - val generatorNullRow = new GenericInternalRow(generator.elementTypes.size) + val generatorNullRow = new GenericInternalRow(generator.elementSchema.length) val joinedRow = new JoinedRow iter.flatMap { row => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index f023edbd96db..3220f143aa23 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -145,7 +145,7 @@ private[sql] class HiveSessionCatalog( udaf } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children) - udtf.elementTypes // Force it to check input data types. + udtf.elementSchema // Force it to check input data types. udtf } else { throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 5aab4132bc4c..c53675694f62 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -202,9 +202,10 @@ private[hive] case class HiveGenericUDTF( @transient protected lazy val collector = new UDTFCollector - override lazy val elementTypes = outputInspector.getAllStructFieldRefs.asScala.map { - field => (inspectorToDataType(field.getFieldObjectInspector), true, field.getFieldName) - } + override lazy val elementSchema = StructType(outputInspector.getAllStructFieldRefs.asScala.map { + field => StructField(field.getFieldName, inspectorToDataType(field.getFieldObjectInspector), + nullable = true) + }) @transient private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray From 1fba7595f2e9cac984692238c6698bd5cfb0a453 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 4 May 2016 16:44:09 +0800 Subject: [PATCH 057/313] [SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] " DDL command ## What changes were proposed in this pull request? This PR implements native `DESC [EXTENDED | FORMATTED]
    ` DDL command. Sample output: ``` scala> spark.sql("desc extended src").show(100, truncate = false) +----------------------------+---------------------------------+-------+ |col_name |data_type |comment| +----------------------------+---------------------------------+-------+ |key |int | | |value |string | | | | | | |# Detailed Table Information|CatalogTable(`default`.`src`, ...| | +----------------------------+---------------------------------+-------+ scala> spark.sql("desc formatted src").show(100, truncate = false) +----------------------------+----------------------------------------------------------+-------+ |col_name |data_type |comment| +----------------------------+----------------------------------------------------------+-------+ |key |int | | |value |string | | | | | | |# Detailed Table Information| | | |Database: |default | | |Owner: |lian | | |Create Time: |Mon Jan 04 17:06:00 CST 2016 | | |Last Access Time: |Thu Jan 01 08:00:00 CST 1970 | | |Location: |hdfs://localhost:9000/user/hive/warehouse_hive121/src | | |Table Type: |MANAGED | | |Table Parameters: | | | | transient_lastDdlTime |1451898360 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.TextInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat| | |Num Buckets: |-1 | | |Bucket Columns: |[] | | |Sort Columns: |[] | | |Storage Desc Parameters: | | | | serialization.format |1 | | +----------------------------+----------------------------------------------------------+-------+ ``` ## How was this patch tested? A test case is added to `HiveDDLSuite` to check command output. Author: Cheng Lian Closes #12844 from liancheng/spark-14127-desc-table. (cherry picked from commit f152fae306dc75565cb4648ee1211416d7c0bb23) Signed-off-by: Cheng Lian --- .../sql/catalyst/catalog/interface.scala | 5 +- .../catalog/ExternalCatalogSuite.scala | 1 + .../spark/sql/execution/SparkSqlParser.scala | 14 ++- .../command/createDataSourceTables.scala | 2 + .../spark/sql/execution/command/tables.scala | 104 ++++++++++++++---- .../sql/execution/command/DDLSuite.scala | 6 +- .../sql/hive/client/HiveClientImpl.scala | 4 +- .../hive/execution/CreateTableAsSelect.scala | 3 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 1 + .../spark/sql/hive/client/VersionsSuite.scala | 1 + .../sql/hive/execution/HiveDDLSuite.scala | 17 +++ 12 files changed, 131 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 3851e4c70674..2c6e9f53b27b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -48,6 +48,7 @@ case class CatalogStorageFormat( inputFormat: Option[String], outputFormat: Option[String], serde: Option[String], + compressed: Boolean, serdeProperties: Map[String, String]) @@ -89,6 +90,7 @@ case class CatalogTable( sortColumnNames: Seq[String] = Seq.empty, bucketColumnNames: Seq[String] = Seq.empty, numBuckets: Int = -1, + owner: String = "", createTime: Long = System.currentTimeMillis, lastAccessTime: Long = -1, properties: Map[String, String] = Map.empty, @@ -123,10 +125,11 @@ case class CatalogTable( locationUri: Option[String] = storage.locationUri, inputFormat: Option[String] = storage.inputFormat, outputFormat: Option[String] = storage.outputFormat, + compressed: Boolean = false, serde: Option[String] = storage.serde, serdeProperties: Map[String, String] = storage.serdeProperties): CatalogTable = { copy(storage = CatalogStorageFormat( - locationUri, inputFormat, outputFormat, serde, serdeProperties)) + locationUri, inputFormat, outputFormat, serde, compressed, serdeProperties)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index d739b177430c..ae7c503e65cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -507,6 +507,7 @@ abstract class CatalogTestUtils { inputFormat = Some(tableInputFormat), outputFormat = Some(tableOutputFormat), serde = None, + compressed = false, serdeProperties = Map.empty) lazy val part1 = CatalogTablePartition(Map("a" -> "1", "b" -> "2"), storageFormat) lazy val part2 = CatalogTablePartition(Map("a" -> "3", "b" -> "4"), storageFormat) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index b000cc99533e..60388df59646 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -243,10 +243,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { // FORMATTED and columns are not supported. Return null and let the parser decide what to do // with this (create an exception or pass it on to a different system). - if (ctx.describeColName != null || ctx.FORMATTED != null || ctx.partitionSpec != null) { + if (ctx.describeColName != null || ctx.partitionSpec != null) { null } else { - DescribeTableCommand(visitTableIdentifier(ctx.tableIdentifier), ctx.EXTENDED != null) + DescribeTableCommand( + visitTableIdentifier(ctx.tableIdentifier), + ctx.EXTENDED != null, + ctx.FORMATTED() != null) } } @@ -766,6 +769,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Note: Keep this unspecified because we use the presence of the serde to decide // whether to convert a table created by CTAS to a datasource table. serde = None, + compressed = false, serdeProperties = Map()) } val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) @@ -777,6 +781,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), + compressed = false, serdeProperties = rowStorage.serdeProperties ++ fileStorage.serdeProperties) // TODO support the sql text - have a proper location for this! @@ -830,7 +835,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** Empty storage format for default values and copies. */ - private val EmptyStorageFormat = CatalogStorageFormat(None, None, None, None, Map.empty) + private val EmptyStorageFormat = CatalogStorageFormat(None, None, None, None, false, Map.empty) /** * Create a [[CatalogStorageFormat]]. @@ -911,6 +916,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { entry("field.delim", ctx.fieldsTerminatedBy) ++ entry("serialization.format", ctx.fieldsTerminatedBy) ++ entry("escape.delim", ctx.escapedBy) ++ + // The following typo is inherited from Hive... entry("colelction.delim", ctx.collectionItemsTerminatedBy) ++ entry("mapkey.delim", ctx.keysTerminatedBy) ++ Option(ctx.linesSeparatedBy).toSeq.map { token => @@ -1051,7 +1057,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case c: RowFormatSerdeContext => // Use a serde format. - val CatalogStorageFormat(None, None, None, Some(name), props) = visitRowFormatSerde(c) + val CatalogStorageFormat(None, None, None, Some(name), _, props) = visitRowFormatSerde(c) // SPARK-10310: Special cases LazySimpleSerDe val recordHandler = if (name == "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index f670f63472bf..e07ab99ef3e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -349,6 +349,7 @@ object CreateDataSourceTableUtils extends Logging { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = options ), properties = tableProperties.toMap) @@ -368,6 +369,7 @@ object CreateDataSourceTableUtils extends Logging { inputFormat = serde.inputFormat, outputFormat = serde.outputFormat, serde = serde.serde, + compressed = false, serdeProperties = options ), schema = relation.schema.map { f => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 489c980c1654..31c804f7a3d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -19,16 +19,17 @@ package org.apache.spark.sql.execution.command import java.io.File import java.net.URI +import java.util.Date import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogRelation, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, UnaryNode} -import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType} +import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType, StructType} import org.apache.spark.util.Utils case class CreateTableAsSelectLogicalPlan( @@ -269,10 +270,10 @@ case class LoadData( /** * Command that looks like * {{{ - * DESCRIBE (EXTENDED) table_name; + * DESCRIBE [EXTENDED|FORMATTED] table_name; * }}} */ -case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean) +case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isFormatted: Boolean) extends RunnableCommand { override val output: Seq[Attribute] = Seq( @@ -289,29 +290,92 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean) val result = new ArrayBuffer[Row] sparkSession.sessionState.catalog.lookupRelation(table) match { case catalogRelation: CatalogRelation => - catalogRelation.catalogTable.schema.foreach { column => - result += Row(column.name, column.dataType, column.comment.orNull) - } - - if (catalogRelation.catalogTable.partitionColumns.nonEmpty) { - result += Row("# Partition Information", "", "") - result += Row(s"# ${output(0).name}", output(1).name, output(2).name) - - catalogRelation.catalogTable.partitionColumns.foreach { col => - result += Row(col.name, col.dataType, col.comment.orNull) - } + if (isExtended) { + describeExtended(catalogRelation, result) + } else if (isFormatted) { + describeFormatted(catalogRelation, result) + } else { + describe(catalogRelation, result) } case relation => - relation.schema.fields.foreach { field => - val comment = - if (field.metadata.contains("comment")) field.metadata.getString("comment") else "" - result += Row(field.name, field.dataType.simpleString, comment) - } + describeSchema(relation.schema, result) } result } + + // Shows data columns and partitioned columns (if any) + private def describe(relation: CatalogRelation, buffer: ArrayBuffer[Row]): Unit = { + describeSchema(relation.catalogTable.schema, buffer) + + if (relation.catalogTable.partitionColumns.nonEmpty) { + append(buffer, "# Partition Information", "", "") + append(buffer, s"# ${output(0).name}", output(1).name, output(2).name) + describeSchema(relation.catalogTable.partitionColumns, buffer) + } + } + + private def describeExtended(relation: CatalogRelation, buffer: ArrayBuffer[Row]): Unit = { + describe(relation, buffer) + + append(buffer, "", "", "") + append(buffer, "# Detailed Table Information", relation.catalogTable.toString, "") + } + + private def describeFormatted(relation: CatalogRelation, buffer: ArrayBuffer[Row]): Unit = { + describe(relation, buffer) + + val table = relation.catalogTable + + append(buffer, "", "", "") + append(buffer, "# Detailed Table Information", "", "") + append(buffer, "Database:", table.database, "") + append(buffer, "Owner:", table.owner, "") + append(buffer, "Create Time:", new Date(table.createTime).toString, "") + append(buffer, "Last Access Time:", new Date(table.lastAccessTime).toString, "") + append(buffer, "Location:", table.storage.locationUri.getOrElse(""), "") + append(buffer, "Table Type:", table.tableType.name, "") + + append(buffer, "Table Parameters:", "", "") + table.properties.foreach { case (key, value) => + append(buffer, s" $key", value, "") + } + + append(buffer, "", "", "") + append(buffer, "# Storage Information", "", "") + table.storage.serde.foreach(serdeLib => append(buffer, "SerDe Library:", serdeLib, "")) + table.storage.inputFormat.foreach(format => append(buffer, "InputFormat:", format, "")) + table.storage.outputFormat.foreach(format => append(buffer, "OutputFormat:", format, "")) + append(buffer, "Compressed:", if (table.storage.compressed) "Yes" else "No", "") + append(buffer, "Num Buckets:", table.numBuckets.toString, "") + append(buffer, "Bucket Columns:", table.bucketColumnNames.mkString("[", ", ", "]"), "") + append(buffer, "Sort Columns:", table.sortColumnNames.mkString("[", ", ", "]"), "") + + append(buffer, "Storage Desc Parameters:", "", "") + table.storage.serdeProperties.foreach { case (key, value) => + append(buffer, s" $key", value, "") + } + } + + private def describeSchema(schema: StructType, buffer: ArrayBuffer[Row]): Unit = { + schema.foreach { column => + val comment = + if (column.metadata.contains("comment")) column.metadata.getString("comment") else "" + append(buffer, column.name, column.dataType.simpleString, comment) + } + } + + private def describeSchema(schema: Seq[CatalogColumn], buffer: ArrayBuffer[Row]): Unit = { + schema.foreach { column => + append(buffer, column.name, column.dataType.toLowerCase, column.comment.orNull) + } + } + + private def append( + buffer: ArrayBuffer[Row], column: String, dataType: String, comment: String): Unit = { + buffer += Row(column, dataType, comment) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 12acb9f2761d..0ae099ecc2bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -76,6 +76,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map()) catalog.createTable(CatalogTable( identifier = name, @@ -89,7 +90,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog: SessionCatalog, spec: TablePartitionSpec, tableName: TableIdentifier): Unit = { - val part = CatalogTablePartition(spec, CatalogStorageFormat(None, None, None, None, Map())) + val part = CatalogTablePartition( + spec, CatalogStorageFormat(None, None, None, None, false, Map())) catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) } @@ -264,6 +266,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map()) val expectedTable = CatalogTable( @@ -288,6 +291,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map()) val expectedTable = CatalogTable( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 47d9546c4f51..cddc0b6e34a4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -351,6 +351,7 @@ private[hive] class HiveClientImpl( sortColumnNames = Seq(), // TODO: populate this bucketColumnNames = h.getBucketCols.asScala, numBuckets = h.getNumBuckets, + owner = h.getOwner, createTime = h.getTTable.getCreateTime.toLong * 1000, lastAccessTime = h.getLastAccessTime.toLong * 1000, storage = CatalogStorageFormat( @@ -358,6 +359,7 @@ private[hive] class HiveClientImpl( inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), serde = Option(h.getSerializationLib), + compressed = h.getTTable.getSd.isCompressed, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap ), properties = h.getParameters.asScala.toMap, @@ -788,7 +790,7 @@ private[hive] class HiveClientImpl( inputFormat = Option(apiPartition.getSd.getInputFormat), outputFormat = Option(apiPartition.getSd.getOutputFormat), serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), + compressed = apiPartition.getSd.isCompressed, serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) } - } 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 08d4b99d30ea..9dfbafae872f 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 @@ -56,7 +56,8 @@ case class CreateTableAsSelect( outputFormat = tableDesc.storage.outputFormat .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), - serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName))) + serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName)), + compressed = tableDesc.storage.compressed) val withSchema = if (withFormat.schema.isEmpty) { // Hive doesn't support specifying the column list for target table in CTAS diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 8dc3c6435327..c4ebc604dc13 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -579,7 +579,7 @@ class HiveDDLCommandSuite extends PlanTest { assert(source2.table == "table2") } - test("load data") { + test("load data") { val v1 = "LOAD DATA INPATH 'path' INTO TABLE table1" val (table, path, isLocal, isOverwrite, partition) = parser.parsePlan(v1).collect { case LoadData(t, path, l, o, partition) => (t, path, l, o, partition) 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 c3a9f2479ce7..4bdcb96feb0a 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 @@ -732,6 +732,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map( "path" -> sessionState.catalog.hiveDefaultTableFilePath(TableIdentifier(tableName))) ), 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 9341b3816fea..a6a5ab3988fc 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 @@ -157,6 +157,7 @@ class VersionsSuite extends SparkFunSuite with Logging { outputFormat = Some( classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), serde = Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName()), + compressed = false, serdeProperties = Map.empty )) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 687a4a7e512a..373d1a1e0ebc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -348,4 +348,21 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } } + + test("desc table") { + withTable("tab1") { + val tabName = "tab1" + sql(s"CREATE TABLE $tabName(c1 int)") + + assert(sql(s"DESC $tabName").collect().length == 1) + + assert( + sql(s"DESC FORMATTED $tabName").collect() + .exists(_.getString(0) == "# Storage Information")) + + assert( + sql(s"DESC EXTENDED $tabName").collect() + .exists(_.getString(0) == "# Detailed Table Information")) + } + } } From dcbe85ff20736915bba6c3269221c3367ee798c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dominik=20Jastrz=C4=99bski?= Date: Wed, 4 May 2016 14:25:51 +0200 Subject: [PATCH 058/313] =?UTF-8?q?[SPARK-14844][ML]=20Add=20setFeaturesCo?= =?UTF-8?q?l=20and=20setPredictionCol=20to=20KMeansM=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Introduction of setFeaturesCol and setPredictionCol methods to KMeansModel in ML library. ## How was this patch tested? By running KMeansSuite. Author: Dominik Jastrzębski Closes #12609 from dominik-jastrzebski/master. (cherry picked from commit abecbcd5e9598471b705a2f701731af1adc9d48b) Signed-off-by: Nick Pentreath --- .../org/apache/spark/ml/clustering/KMeans.scala | 8 ++++++++ .../apache/spark/ml/clustering/KMeansSuite.scala | 15 +++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 7c9ac02521ff..42a25396adb4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -105,6 +105,14 @@ class KMeansModel private[ml] ( copyValues(copied, extra) } + /** @group setParam */ + @Since("2.0.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val predictUDF = udf((vector: Vector) => predict(vector)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 2ca386e4229c..241d21961fae 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -117,6 +117,21 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR assert(clusterSizes.forall(_ >= 0)) } + test("KMeansModel transform with non-default feature and prediction cols") { + val featuresColName = "kmeans_model_features" + val predictionColName = "kmeans_model_prediction" + + val model = new KMeans().setK(k).setSeed(1).fit(dataset) + model.setFeaturesCol(featuresColName).setPredictionCol(predictionColName) + + val transformed = model.transform(dataset.withColumnRenamed("features", featuresColName)) + Seq(featuresColName, predictionColName).foreach { column => + assert(transformed.columns.contains(column)) + } + assert(model.getFeaturesCol == featuresColName) + assert(model.getPredictionCol == predictionColName) + } + test("read/write") { def checkModelData(model: KMeansModel, model2: KMeansModel): Unit = { assert(model.clusterCenters === model2.clusterCenters) From ae79032dcf160796851ca29116cca146c4d86ada Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Wed, 4 May 2016 08:45:43 -0500 Subject: [PATCH 059/313] [SPARK-4224][CORE][YARN] Support group acls ## What changes were proposed in this pull request? Currently only a list of users can be specified for view and modify acls. This change enables a group of admins/devs/users to be provisioned for viewing and modifying Spark jobs. **Changes Proposed in the fix** Three new corresponding config entries have been added where the user can specify the groups to be given access. ``` spark.admin.acls.groups spark.modify.acls.groups spark.ui.view.acls.groups ``` New config entries were added because specifying the users and groups explicitly is a better and cleaner way compared to specifying them in the existing config entry using a delimiter. A generic trait has been introduced to provide the user to group mapping which makes it pluggable to support a variety of mapping protocols - similar to the one used in hadoop. A default unix shell based implementation has been provided. Custom user to group mapping protocol can be specified and configured by the entry ```spark.user.groups.mapping``` **How the patch was Tested** We ran different spark jobs setting the config entries in combinations of admin, modify and ui acls. For modify acls we tried killing the job stages from the ui and using yarn commands. For view acls we tried accessing the UI tabs and the logs. Headless accounts were used to launch these jobs and different users tried to modify and view the jobs to ensure that the groups mapping applied correctly. Additional Unit tests have been added without modifying the existing ones. These test for different ways of setting the acls through configuration and/or API and validate the expected behavior. Author: Dhruve Ashar Closes #12760 from dhruve/impr/SPARK-4224. (cherry picked from commit a45647746d1efb90cb8bc142c2ef110a0db9bc9f) Signed-off-by: Tom Graves --- .../org/apache/spark/SecurityManager.scala | 124 ++++++++--- .../deploy/history/FsHistoryProvider.scala | 2 + .../scheduler/ApplicationEventListener.scala | 4 + .../GroupMappingServiceProvider.scala | 38 ++++ .../ShellBasedGroupsMappingProvider.scala | 45 ++++ .../scala/org/apache/spark/util/Utils.scala | 19 ++ .../apache/spark/SecurityManagerSuite.scala | 198 ++++++++++++++++++ docs/configuration.md | 55 ++++- docs/monitoring.md | 4 +- docs/security.md | 6 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 8 +- 11 files changed, 468 insertions(+), 35 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala create mode 100644 core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index e8f68224d597..f72c7ded5ea5 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -50,17 +50,19 @@ import org.apache.spark.util.Utils * secure the UI if it has data that other users should not be allowed to see. The javax * servlet filter specified by the user can authenticate the user and then once the user * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.acls.enable' and 'spark.ui.view.acls' - * control the behavior of the acls. Note that the person who started the application - * always has view access to the UI. + * authorized to view the UI. The configs 'spark.acls.enable', 'spark.ui.view.acls' and + * 'spark.ui.view.acls.groups' control the behavior of the acls. Note that the person who + * started the application always has view access to the UI. * - * Spark has a set of modify acls (`spark.modify.acls`) that controls which users have permission - * to modify a single application. This would include things like killing the application. By - * default the person who started the application has modify access. For modify access through - * the UI, you must have a filter that does authentication in place for the modify acls to work - * properly. + * Spark has a set of individual and group modify acls (`spark.modify.acls`) and + * (`spark.modify.acls.groups`) that controls which users and groups have permission to + * modify a single application. This would include things like killing the application. + * By default the person who started the application has modify access. For modify access + * through the UI, you must have a filter that does authentication in place for the modify + * acls to work properly. * - * Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators + * Spark also has a set of individual and group admin acls (`spark.admin.acls`) and + * (`spark.admin.acls.groups`) which is a set of users/administrators and admin groups * who always have permission to view or modify the Spark application. * * Starting from version 1.3, Spark has partial support for encrypted connections with SSL. @@ -184,6 +186,9 @@ private[spark] class SecurityManager(sparkConf: SparkConf) import SecurityManager._ + // allow all users/groups to have view/modify permissions + private val WILDCARD_ACL = "*" + private val authOn = sparkConf.getBoolean(SecurityManager.SPARK_AUTH_CONF, false) // keep spark.ui.acls.enable for backwards compatibility with 1.0 private var aclsOn = @@ -193,12 +198,20 @@ private[spark] class SecurityManager(sparkConf: SparkConf) private var adminAcls: Set[String] = stringToSet(sparkConf.get("spark.admin.acls", "")) + // admin group acls should be set before view or modify group acls + private var adminAclsGroups : Set[String] = + stringToSet(sparkConf.get("spark.admin.acls.groups", "")) + private var viewAcls: Set[String] = _ + private var viewAclsGroups: Set[String] = _ + // list of users who have permission to modify the application. This should // apply to both UI and CLI for things like killing the application. private var modifyAcls: Set[String] = _ + private var modifyAclsGroups: Set[String] = _ + // always add the current user and SPARK_USER to the viewAcls private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Utils.getCurrentUserName()) @@ -206,11 +219,16 @@ private[spark] class SecurityManager(sparkConf: SparkConf) setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) + setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", "")); + setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", "")); + private val secretKey = generateSecretKey() logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + "; ui acls " + (if (aclsOn) "enabled" else "disabled") + - "; users with view permissions: " + viewAcls.toString() + - "; users with modify permissions: " + modifyAcls.toString()) + "; users with view permissions: " + viewAcls.toString() + + "; groups with view permissions: " + viewAclsGroups.toString() + + "; users with modify permissions: " + modifyAcls.toString() + + "; groups with modify permissions: " + modifyAclsGroups.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. // This is needed by the HTTP client fetching from the HttpServer. Put here so its @@ -302,17 +320,34 @@ private[spark] class SecurityManager(sparkConf: SparkConf) setViewAcls(Set[String](defaultUser), allowedUsers) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setViewAclsGroups(allowedUserGroups: String) { + viewAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + logInfo("Changing view acls groups to: " + viewAclsGroups.mkString(",")) + } + /** * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" */ def getViewAcls: String = { - if (viewAcls.contains("*")) { - "*" + if (viewAcls.contains(WILDCARD_ACL)) { + WILDCARD_ACL } else { viewAcls.mkString(",") } } + def getViewAclsGroups: String = { + if (viewAclsGroups.contains(WILDCARD_ACL)) { + WILDCARD_ACL + } else { + viewAclsGroups.mkString(",") + } + } + /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. @@ -322,17 +357,34 @@ private[spark] class SecurityManager(sparkConf: SparkConf) logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setModifyAclsGroups(allowedUserGroups: String) { + modifyAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + logInfo("Changing modify acls groups to: " + modifyAclsGroups.mkString(",")) + } + /** * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" */ def getModifyAcls: String = { - if (modifyAcls.contains("*")) { - "*" + if (modifyAcls.contains(WILDCARD_ACL)) { + WILDCARD_ACL } else { modifyAcls.mkString(",") } } + def getModifyAclsGroups: String = { + if (modifyAclsGroups.contains(WILDCARD_ACL)) { + WILDCARD_ACL + } else { + modifyAclsGroups.mkString(",") + } + } + /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. @@ -342,6 +394,15 @@ private[spark] class SecurityManager(sparkConf: SparkConf) logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setAdminAclsGroups(adminUserGroups: String) { + adminAclsGroups = stringToSet(adminUserGroups) + logInfo("Changing admin acls groups to: " + adminAclsGroups.mkString(",")) + } + def setAcls(aclSetting: Boolean) { aclsOn = aclSetting logInfo("Changing acls enabled to: " + aclsOn) @@ -398,36 +459,49 @@ private[spark] class SecurityManager(sparkConf: SparkConf) def aclsEnabled(): Boolean = aclsOn /** - * Checks the given user against the view acl list to see if they have + * Checks the given user against the view acl and groups list to see if they have * authorization to view the UI. If the UI acls are disabled * via spark.acls.enable, all users have view access. If the user is null - * it is assumed authentication is off and all users have access. + * it is assumed authentication is off and all users have access. Also if any one of the + * UI acls or groups specify the WILDCARD(*) then all users have view access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" + - viewAcls.mkString(",")) - !aclsEnabled || user == null || viewAcls.contains(user) || viewAcls.contains("*") + viewAcls.mkString(",") + " viewAclsGroups=" + viewAclsGroups.mkString(",")) + if (!aclsEnabled || user == null || viewAcls.contains(user) || + viewAcls.contains(WILDCARD_ACL) || viewAclsGroups.contains(WILDCARD_ACL)) { + return true + } + val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) + logDebug("userGroups=" + currentUserGroups.mkString(",")) + viewAclsGroups.exists(currentUserGroups.contains(_)) } /** - * Checks the given user against the modify acl list to see if they have - * authorization to modify the application. If the UI acls are disabled + * Checks the given user against the modify acl and groups list to see if they have + * authorization to modify the application. If the modify acls are disabled * via spark.acls.enable, all users have modify access. If the user is null - * it is assumed authentication isn't turned on and all users have access. + * it is assumed authentication isn't turned on and all users have access. Also if any one + * of the modify acls or groups specify the WILDCARD(*) then all users have modify access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkModifyPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" + - modifyAcls.mkString(",")) - !aclsEnabled || user == null || modifyAcls.contains(user) || modifyAcls.contains("*") + modifyAcls.mkString(",") + " modifyAclsGroups=" + modifyAclsGroups.mkString(",")) + if (!aclsEnabled || user == null || modifyAcls.contains(user) || + modifyAcls.contains(WILDCARD_ACL) || modifyAclsGroups.contains(WILDCARD_ACL)) { + return true + } + val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) + logDebug("userGroups=" + currentUserGroups) + modifyAclsGroups.exists(currentUserGroups.contains(_)) } - /** * Check to see if authentication for the Spark communication protocols is enabled * @return true if authentication is enabled, otherwise false diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 07cbcec8e5f0..110d882f0559 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -245,6 +245,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) + ui.getSecurityManager.setAdminAclsGroups(appListener.adminAclsGroups.getOrElse("")) + ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 9f218c64cac2..28c45d800ed0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -32,6 +32,8 @@ private[spark] class ApplicationEventListener extends SparkListener { var endTime: Option[Long] = None var viewAcls: Option[String] = None var adminAcls: Option[String] = None + var viewAclsGroups: Option[String] = None + var adminAclsGroups: Option[String] = None override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = Some(applicationStart.appName) @@ -51,6 +53,8 @@ private[spark] class ApplicationEventListener extends SparkListener { val allProperties = environmentDetails("Spark Properties").toMap viewAcls = allProperties.get("spark.ui.view.acls") adminAcls = allProperties.get("spark.admin.acls") + viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") + adminAclsGroups = allProperties.get("spark.admin.acls.groups") } } } diff --git a/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala new file mode 100644 index 000000000000..ea047a4f75d5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala @@ -0,0 +1,38 @@ +/* + * 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.security + +/** + * This Spark trait is used for mapping a given userName to a set of groups which it belongs to. + * This is useful for specifying a common group of admins/developers to provide them admin, modify + * and/or view access rights. Based on whether access control checks are enabled using + * spark.acls.enable, every time a user tries to access or modify the application, the + * SecurityManager gets the corresponding groups a user belongs to from the instance of the groups + * mapping provider specified by the entry spark.user.groups.mapping. + */ + +trait GroupMappingServiceProvider { + + /** + * Get the groups the user belongs to. + * @param userName User's Name + * @return set of groups that the user belongs to. Empty in case of an invalid user. + */ + def getGroups(userName : String) : Set[String] + +} diff --git a/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala new file mode 100644 index 000000000000..f71dd08246b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.security + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * This class is responsible for getting the groups for a particular user in Unix based + * environments. This implementation uses the Unix Shell based id command to fetch the user groups + * for the specified user. It does not cache the user groups as the invocations are expected + * to be infrequent. + */ + +private[spark] class ShellBasedGroupsMappingProvider extends GroupMappingServiceProvider + with Logging { + + override def getGroups(username: String): Set[String] = { + val userGroups = getUnixGroups(username) + logDebug("User: " + username + " Groups: " + userGroups.mkString(",")) + userGroups + } + + // shells out a "bash -c id -Gn username" to get user groups + private def getUnixGroups(username: String): Set[String] = { + val cmdSeq = Seq("bash", "-c", "id -Gn " + username) + // we need to get rid of the trailing "\n" from the result of command execution + Utils.executeAndGetOutput(cmdSeq).stripLineEnd.split(" ").toSet + } +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index ea49991493fd..a8bb0002a7b2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2181,6 +2181,25 @@ private[spark] object Utils extends Logging { .getOrElse(UserGroupInformation.getCurrentUser().getShortUserName()) } + val EMPTY_USER_GROUPS = Set[String]() + + // Returns the groups to which the current user belongs. + def getCurrentUserGroups(sparkConf: SparkConf, username: String): Set[String] = { + val groupProviderClassName = sparkConf.get("spark.user.groups.mapping", + "org.apache.spark.security.ShellBasedGroupsMappingProvider") + if (groupProviderClassName != "") { + try { + val groupMappingServiceProvider = classForName(groupProviderClassName).newInstance. + asInstanceOf[org.apache.spark.security.GroupMappingServiceProvider] + val currentUserGroups = groupMappingServiceProvider.getGroups(username) + return currentUserGroups + } catch { + case e: Exception => logError(s"Error getting groups for user=$username", e) + } + } + EMPTY_USER_GROUPS + } + /** * Split the comma delimited string of master URLs into a list. * For instance, "spark://abc,def" becomes [spark://abc, spark://def]. diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index 8bdb237c28f6..9801b2638cc1 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -19,8 +19,18 @@ package org.apache.spark import java.io.File +import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} +class DummyGroupMappingServiceProvider extends GroupMappingServiceProvider { + + val userGroups: Set[String] = Set[String]("group1", "group2", "group3") + + override def getGroups(username: String): Set[String] = { + userGroups + } +} + class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf") { @@ -37,6 +47,45 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user3") === false) } + test("set security with conf for groups") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls.groups", "group1,group2") + // default ShellBasedGroupsMappingProvider is used to resolve user groups + val securityManager = new SecurityManager(conf); + // assuming executing user does not belong to group1,group2 + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf2.set("spark.authenticate", "true") + conf2.set("spark.authenticate.secret", "good") + conf2.set("spark.ui.acls.enable", "true") + conf2.set("spark.ui.view.acls.groups", "group1,group2") + // explicitly specify a custom GroupsMappingServiceProvider + conf2.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager2 = new SecurityManager(conf2); + // group4,group5 do not match + assert(securityManager2.checkUIViewPermissions("user1") === true) + assert(securityManager2.checkUIViewPermissions("user2") === true) + + val conf3 = new SparkConf + conf3.set("spark.authenticate", "true") + conf3.set("spark.authenticate.secret", "good") + conf3.set("spark.ui.acls.enable", "true") + conf3.set("spark.ui.view.acls.groups", "group4,group5") + // explicitly specify a bogus GroupsMappingServiceProvider + conf3.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager3 = new SecurityManager(conf3); + // BogusServiceProvider cannot be loaded and an error is logged returning an empty group set + assert(securityManager3.checkUIViewPermissions("user1") === false) + assert(securityManager3.checkUIViewPermissions("user2") === false) + } + test("set security with api") { val conf = new SparkConf conf.set("spark.ui.view.acls", "user1,user2") @@ -60,6 +109,40 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions(null) === true) } + test("set security with api for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setViewAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + + // change groups so they do not match + securityManager.setViewAclsGroups("group4,group5") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager2 = new SecurityManager(conf2) + securityManager2.setAcls(true) + securityManager2.setViewAclsGroups("group1,group2") + + // group1,group2 do not match because of BogusServiceProvider + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + // setting viewAclsGroups to empty should still not match because of BogusServiceProvider + securityManager2.setViewAclsGroups("") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + } + test("set security modify acls") { val conf = new SparkConf conf.set("spark.modify.acls", "user1,user2") @@ -84,6 +167,29 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions(null) === true) } + test("set security modify acls for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setModifyAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // change groups so they do not match + securityManager.setModifyAclsGroups("group4,group5") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // change so they match again + securityManager.setModifyAclsGroups("group2,group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + } + test("set security admin acls") { val conf = new SparkConf conf.set("spark.admin.acls", "user1,user2") @@ -122,7 +228,48 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user3") === false) assert(securityManager.checkUIViewPermissions(null) === true) + } + + test("set security admin acls for groups") { + val conf = new SparkConf + conf.set("spark.admin.acls.groups", "group1") + conf.set("spark.ui.view.acls.groups", "group2") + conf.set("spark.modify.acls.groups", "group3") + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + + // group1,group2,group3 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + // change admin groups so they do not match. view and modify groups are set to admin groups + securityManager.setAdminAclsGroups("group4,group5") + // invoke the set ui and modify to propagate the changes + securityManager.setViewAclsGroups("") + securityManager.setModifyAclsGroups("") + + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change modify groups so they match + securityManager.setModifyAclsGroups("group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change view groups so they match + securityManager.setViewAclsGroups("group2") + securityManager.setModifyAclsGroups("group4") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === true) + + // change modify and view groups so they do not match + securityManager.setViewAclsGroups("group7") + securityManager.setModifyAclsGroups("group8") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) } test("set security with * in acls") { @@ -166,6 +313,57 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user8") === true) } + test("set security with * in acls for groups") { + val conf = new SparkConf + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.admin.acls.groups", "group4,group5") + conf.set("spark.ui.view.acls.groups", "*") + conf.set("spark.modify.acls.groups", "group6") + + val securityManager = new SecurityManager(conf) + assert(securityManager.aclsEnabled() === true) + + // check for viewAclsGroups with * + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // check for modifyAcls with * + securityManager.setModifyAclsGroups("*") + securityManager.setViewAclsGroups("group6") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // check for adminAcls with * + securityManager.setAdminAclsGroups("group9,*") + securityManager.setModifyAclsGroups("group4,group5") + securityManager.setViewAclsGroups("group6,group7") + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === true) + } + + test("security for groups default behavior") { + // no groups or userToGroupsMapper provided + // this will default to the ShellBasedGroupsMappingProvider + val conf = new SparkConf + + val securityManager = new SecurityManager(conf) + securityManager.setAcls(true) + + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + + // set groups only + securityManager.setAdminAclsGroups("group1,group2") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + } + test("ssl on setup") { val conf = SSLSampleConfigs.sparkSSLConfig() val expectedAlgorithms = Set( diff --git a/docs/configuration.md b/docs/configuration.md index 6512e16faf4c..9191570d0762 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1231,7 +1231,7 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + @@ -1305,6 +1330,18 @@ Apart from these, the following properties are also available, and may be useful the list means any user can have access to modify it. + + + + + @@ -1328,6 +1365,18 @@ Apart from these, the following properties are also available, and may be useful have view access to this Spark job. + + + + +
    spark.acls.enable false - Whether Spark acls should are enabled. If enabled, this checks to see if the user has + Whether Spark acls should be enabled. If enabled, this checks to see if the user has access permissions to view or modify the job. Note this requires the user to be known, so if the user comes across as null no checks are done. Filters can be used with the UI to authenticate and set the user. @@ -1243,8 +1243,33 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of users/administrators that have view and modify access to all Spark jobs. This can be used if you run on a shared cluster and have a set of administrators or devs who - help debug when things work. Putting a "*" in the list means any user can have the privilege - of admin. + help debug when things do not work. Putting a "*" in the list means any user can have the + privilege of admin. +
    spark.admin.acls.groupsEmpty + Comma separated list of groups that have view and modify access to all Spark jobs. + This can be used if you have a set of administrators or developers who help maintain and debug + the underlying infrastructure. Putting a "*" in the list means any user in any group can have + the privilege of admin. The user groups are obtained from the instance of the groups mapping + provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. +
    spark.user.groups.mappingorg.apache.spark.security.ShellBasedGroupsMappingProvider + The list of groups for a user are determined by a group mapping service defined by the trait + org.apache.spark.security.GroupMappingServiceProvider which can configured by this property. + A default unix shell based implementation is provided org.apache.spark.security.ShellBasedGroupsMappingProvider + which can be specified to resolve a list of groups for a user. + Note: This implementation supports only a Unix/Linux based environment. Windows environment is + currently not supported. However, a new platform/protocol can be supported by implementing + the trait org.apache.spark.security.GroupMappingServiceProvider.
    spark.modify.acls.groupsEmpty + Comma separated list of groups that have modify access to the Spark job. This can be used if you + have a set of administrators or developers from the same team to have access to control the job. + Putting a "*" in the list means any user in any group has the access to modify the Spark job. + The user groups are obtained from the instance of the groups mapping provider specified by + spark.user.groups.mapping. Check the entry spark.user.groups.mapping + for more details. +
    spark.ui.filters None
    spark.ui.view.acls.groupsEmpty + Comma separated list of groups that have view access to the Spark web ui to view the Spark Job + details. This can be used if you have a set of administrators or developers or users who can + monitor the Spark job submitted. Putting a "*" in the list means any user in any group can view + the Spark job details on the Spark web ui. The user groups are obtained from the instance of the + groups mapping provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. +
    #### Encryption diff --git a/docs/monitoring.md b/docs/monitoring.md index 88002ebdc39c..697962ae3a4c 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -162,8 +162,8 @@ The history server can be configured as follows: If enabled, access control checks are made regardless of what the individual application had set for spark.ui.acls.enable when the application was run. The application owner will always have authorization to view their own application and any users specified via - spark.ui.view.acls when the application was run will also have authorization - to view that application. + spark.ui.view.acls and groups specified via spark.ui.view.acls.groups + when the application was run will also have authorization to view that application. If disabled, no access control checks are made. diff --git a/docs/security.md b/docs/security.md index 32c33d285747..d2708a80703e 100644 --- a/docs/security.md +++ b/docs/security.md @@ -16,10 +16,10 @@ and by using [https/SSL](http://en.wikipedia.org/wiki/HTTPS) via the `spark.ui.h ### Authentication -A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. +A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable`, `spark.ui.view.acls` and `spark.ui.view.acls.groups` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. -Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable` and `spark.modify.acls`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. -Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the config `spark.admin.acls`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. +Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable`, `spark.modify.acls` and `spark.modify.acls.groups`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. +Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the configs `spark.admin.acls` and `spark.admin.acls.groups`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. ## Event Logging diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index ee002f622390..44181610d770 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -464,11 +464,15 @@ object YarnSparkHadoopUtil { } } + // YARN/Hadoop acls are specified as user1,user2 group1,group2 + // Users and groups are separated by a space and hence we need to pass the acls in same format def getApplicationAclsForYarn(securityMgr: SecurityManager) : Map[ApplicationAccessType, String] = { Map[ApplicationAccessType, String] ( - ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, - ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls + ApplicationAccessType.VIEW_APP -> (securityMgr.getViewAcls + " " + + securityMgr.getViewAclsGroups), + ApplicationAccessType.MODIFY_APP -> (securityMgr.getModifyAcls + " " + + securityMgr.getModifyAclsGroups) ) } From 343c2850440279f8447a866459029b5dfc5429d0 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Wed, 4 May 2016 10:25:14 -0700 Subject: [PATCH 060/313] [SPARK-15022][SPARK-15023][SQL][STREAMING] Add support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `ManualClock` ## What changes were proposed in this pull request? Currently in `StreamTest`, we have a `StartStream` which will start a streaming query against trigger `ProcessTime(intervalMS = 0)` and `SystemClock`. We also need to test cases against `ProcessTime(intervalMS > 0)`, which often requires `ManualClock`. This patch: - fixes an issue of `ProcessingTimeExecutor`, where for a batch it should run `batchRunner` only once but might run multiple times under certain conditions; - adds support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `AdvanceManualClock`, by specifying them as fields for `StartStream`, and by adding an `AdvanceClock` action; - adds a test, which takes advantage of the new `StartStream` and `AdvanceManualClock`, to test against [PR#[SPARK-14942] Reduce delay between batch construction and execution ](https://github.com/apache/spark/pull/12725). ## How was this patch tested? N/A Author: Liwei Lin Closes #12797 from lw-lin/add-trigger-test-support. (cherry picked from commit e597ec6f1c8ba1f9c10de06534bda1862b0c59aa) Signed-off-by: Shixiong Zhu --- .../spark/sql/ContinuousQueryManager.scala | 7 ++-- .../execution/streaming/StreamExecution.scala | 9 ++--- .../execution/streaming/TriggerExecutor.scala | 9 +++-- .../org/apache/spark/sql/StreamTest.scala | 33 ++++++++++++++----- .../ProcessingTimeExecutorSuite.scala | 19 +++++++++-- .../sql/streaming/ContinuousQuerySuite.scala | 2 +- .../sql/streaming/FileStreamSourceSuite.scala | 8 ++--- .../spark/sql/streaming/StreamSuite.scala | 24 +++++++++++--- .../streaming/StreamingAggregationSuite.scala | 6 ++-- .../util/ContinuousQueryListenerSuite.scala | 6 ++-- 10 files changed, 89 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala index f82130cfa849..eab557443d1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.ContinuousQueryListener +import org.apache.spark.util.{Clock, SystemClock} /** * :: Experimental :: @@ -175,6 +176,7 @@ class ContinuousQueryManager(sparkSession: SparkSession) { df: DataFrame, sink: Sink, trigger: Trigger = ProcessingTime(0), + triggerClock: Clock = new SystemClock(), outputMode: OutputMode = Append): ContinuousQuery = { activeQueriesLock.synchronized { if (activeQueries.contains(name)) { @@ -206,8 +208,9 @@ class ContinuousQueryManager(sparkSession: SparkSession) { checkpointLocation, logicalPlan, sink, - outputMode, - trigger) + trigger, + triggerClock, + outputMode) query.start() activeQueries.put(name, query) query diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 3c5ced2af73f..ea367b699f56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.util.ContinuousQueryListener import org.apache.spark.sql.util.ContinuousQueryListener._ -import org.apache.spark.util.{UninterruptibleThread, Utils} +import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} /** * Manages the execution of a streaming Spark SQL query that is occurring in a separate thread. @@ -50,8 +50,9 @@ class StreamExecution( checkpointRoot: String, private[sql] val logicalPlan: LogicalPlan, val sink: Sink, - val outputMode: OutputMode, - val trigger: Trigger) + val trigger: Trigger, + private[sql] val triggerClock: Clock, + val outputMode: OutputMode) extends ContinuousQuery with Logging { /** @@ -88,7 +89,7 @@ class StreamExecution( private val uniqueSources = sources.distinct private val triggerExecutor = trigger match { - case t: ProcessingTime => ProcessingTimeExecutor(t) + case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock) } /** Defines the internal state of execution */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala index a1132d510685..569907b369a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala @@ -65,8 +65,13 @@ case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = s"${intervalMs} milliseconds, but spent ${realElapsedTimeMs} milliseconds") } - /** Return the next multiple of intervalMs */ + /** + * Returns the start time in milliseconds for the next batch interval, given the current time. + * Note that a batch interval is inclusive with respect to its start time, and thus calling + * `nextBatchTime` with the result of a previous call should return the next interval. (i.e. given + * an interval of `100 ms`, `nextBatchTime(nextBatchTime(0)) = 200` rather than `0`). + */ def nextBatchTime(now: Long): Long = { - (now - 1) / intervalMs * intervalMs + intervalMs + now / intervalMs * intervalMs + intervalMs } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala index dff6acc94b3f..6fb1aca769e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} /** * A framework for implementing tests for streaming queries and sources. @@ -138,11 +138,17 @@ trait StreamTest extends QueryTest with Timeouts { private def operatorName = if (lastOnly) "CheckLastBatch" else "CheckAnswer" } - /** Stops the stream. It must currently be running. */ + /** Stops the stream. It must currently be running. */ case object StopStream extends StreamAction with StreamMustBeRunning - /** Starts the stream, resuming if data has already been processed. It must not be running. */ - case object StartStream extends StreamAction + /** Starts the stream, resuming if data has already been processed. It must not be running. */ + case class StartStream( + trigger: Trigger = ProcessingTime(0), + triggerClock: Clock = new SystemClock) + extends StreamAction + + /** Advance the trigger clock's time manually. */ + case class AdvanceManualClock(timeToAdd: Long) extends StreamAction /** Signals that a failure is expected and should not kill the test. */ case class ExpectFailure[T <: Throwable : ClassTag]() extends StreamAction { @@ -199,8 +205,8 @@ trait StreamTest extends QueryTest with Timeouts { // If the test doesn't manually start the stream, we do it automatically at the beginning. val startedManually = - actions.takeWhile(!_.isInstanceOf[StreamMustBeRunning]).contains(StartStream) - val startedTest = if (startedManually) actions else StartStream +: actions + actions.takeWhile(!_.isInstanceOf[StreamMustBeRunning]).exists(_.isInstanceOf[StartStream]) + val startedTest = if (startedManually) actions else StartStream() +: actions def testActions = actions.zipWithIndex.map { case (a, i) => @@ -280,7 +286,7 @@ trait StreamTest extends QueryTest with Timeouts { try { startedTest.foreach { action => action match { - case StartStream => + case StartStream(trigger, triggerClock) => verify(currentStream == null, "stream already running") lastStream = currentStream currentStream = @@ -291,6 +297,8 @@ trait StreamTest extends QueryTest with Timeouts { metadataRoot, stream, sink, + trigger, + triggerClock, outputMode = outputMode) .asInstanceOf[StreamExecution] currentStream.microBatchThread.setUncaughtExceptionHandler( @@ -301,6 +309,13 @@ trait StreamTest extends QueryTest with Timeouts { } }) + case AdvanceManualClock(timeToAdd) => + verify(currentStream != null, + "can not advance manual clock when a stream is not running") + verify(currentStream.triggerClock.isInstanceOf[ManualClock], + s"can not advance clock of type ${currentStream.triggerClock.getClass}") + currentStream.triggerClock.asInstanceOf[ManualClock].advance(timeToAdd) + case StopStream => verify(currentStream != null, "can not stop a stream that is not running") try failAfter(streamingTimeout) { @@ -470,7 +485,7 @@ trait StreamTest extends QueryTest with Timeouts { addRandomData() case _ => // StartStream - actions += StartStream + actions += StartStream() running = true } } else { @@ -488,7 +503,7 @@ trait StreamTest extends QueryTest with Timeouts { } } } - if(!running) { actions += StartStream } + if(!running) { actions += StartStream() } addCheck() testStream(ds)(actions: _*) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala index dd5f92248bf5..7f99d303ba08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -21,19 +21,34 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.ProcessingTime -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{Clock, ManualClock, SystemClock} class ProcessingTimeExecutorSuite extends SparkFunSuite { test("nextBatchTime") { val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(100)) + assert(processingTimeExecutor.nextBatchTime(0) === 100) assert(processingTimeExecutor.nextBatchTime(1) === 100) assert(processingTimeExecutor.nextBatchTime(99) === 100) - assert(processingTimeExecutor.nextBatchTime(100) === 100) + assert(processingTimeExecutor.nextBatchTime(100) === 200) assert(processingTimeExecutor.nextBatchTime(101) === 200) assert(processingTimeExecutor.nextBatchTime(150) === 200) } + test("calling nextBatchTime with the result of a previous call should return the next interval") { + val intervalMS = 100 + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMS)) + + val ITERATION = 10 + var nextBatchTime: Long = 0 + for (it <- 1 to ITERATION) { + nextBatchTime = processingTimeExecutor.nextBatchTime(nextBatchTime) + } + + // nextBatchTime should be 1000 + assert(nextBatchTime === intervalMS * ITERATION) + } + private def testBatchTermination(intervalMs: Long): Unit = { var batchCounts = 0 val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMs)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala index 3be0ea481dc5..f469cde6bef8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala @@ -45,7 +45,7 @@ class ContinuousQuerySuite extends StreamTest with SharedSQLContext { TestAwaitTermination(ExpectNotBlocked), TestAwaitTermination(ExpectNotBlocked, timeoutMs = 2000, expectedReturnValue = true), TestAwaitTermination(ExpectNotBlocked, timeoutMs = 10, expectedReturnValue = true), - StartStream, + StartStream(), AssertOnQuery(_.isActive === true), AddData(inputData, 0), ExpectFailure[SparkException], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 6b1ecd08c13c..bc5c0c1f6933 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -268,7 +268,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { CheckAnswer("keep2", "keep3"), StopStream, AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") @@ -292,7 +292,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { "{'value': 'drop4'}\n{'value': 'keep5'}\n{'value': 'keep6'}", src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddTextFileData( "{'value': 'drop7'}\n{'value': 'keep8'}\n{'value': 'keep9'}", @@ -385,7 +385,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { CheckAnswer("keep2", "keep3"), StopStream, AddParquetFileData(Seq("drop4", "keep5", "keep6"), src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddParquetFileData(Seq("drop7", "keep8", "keep9"), src, tmp), CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") @@ -449,7 +449,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { CheckAnswer("keep2", "keep3"), StopStream, AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 6f3149dbc503..bcd3cba55a55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.streaming import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.functions._ import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.util.ManualClock class StreamSuite extends StreamTest with SharedSQLContext { @@ -34,11 +34,11 @@ class StreamSuite extends StreamTest with SharedSQLContext { testStream(mapped)( AddData(inputData, 1, 2, 3), - StartStream, + StartStream(), CheckAnswer(2, 3, 4), StopStream, AddData(inputData, 4, 5, 6), - StartStream, + StartStream(), CheckAnswer(2, 3, 4, 5, 6, 7)) } @@ -70,7 +70,7 @@ class StreamSuite extends StreamTest with SharedSQLContext { CheckAnswer(1, 2, 3, 4, 5, 6), StopStream, AddData(inputData1, 7), - StartStream, + StartStream(), AddData(inputData2, 8), CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8)) } @@ -136,6 +136,22 @@ class StreamSuite extends StreamTest with SharedSQLContext { testStream(ds)() } } + + // This would fail for now -- error is "Timed out waiting for stream" + // Root cause is that data generated in batch 0 may not get processed in batch 1 + // Let's enable this after SPARK-14942: Reduce delay between batch construction and execution + ignore("minimize delay between batch construction and execution") { + val inputData = MemoryStream[Int] + testStream(inputData.toDS())( + StartStream(ProcessingTime("10 seconds"), new ManualClock), + /* -- batch 0 ----------------------- */ + AddData(inputData, 1), + AddData(inputData, 2), + AddData(inputData, 3), + AdvanceManualClock(10 * 1000), // 10 seconds + /* -- batch 1 ----------------------- */ + CheckAnswer(1, 2, 3)) + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index fa3b122f6d2d..bdf40f5cd45d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -50,7 +50,7 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext { AddData(inputData, 3, 2), CheckLastBatch((3, 2), (2, 1)), StopStream, - StartStream, + StartStream(), AddData(inputData, 3, 2, 1), CheckLastBatch((3, 3), (2, 2), (1, 1)), // By default we run in new tuple mode. @@ -113,10 +113,10 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext { .as[(Int, Long)] testStream(aggregated)( - StartStream, + StartStream(), AddData(inputData, 1, 2, 3, 4), ExpectFailure[SparkException](), - StartStream, + StartStream(), CheckLastBatch((1, 1), (2, 1), (3, 1), (4, 1)) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala index 2596231a1244..54acd4db3c71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala @@ -48,7 +48,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with val input = MemoryStream[Int] withListenerAdded(listener) { testStream(input.toDS)( - StartStream, + StartStream(), Assert("Incorrect query status in onQueryStarted") { val status = listener.startStatus assert(status != null) @@ -102,7 +102,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with def isListenerActive(listener: QueryStatusCollector): Boolean = { listener.reset() testStream(MemoryStream[Int].toDS)( - StartStream, + StartStream(), StopStream ) listener.startStatus != null @@ -133,7 +133,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with listener.reset() require(listener.startStatus === null) testStream(MemoryStream[Int].toDS)( - StartStream, + StartStream(), Assert(listener.startStatus !== null, "onQueryStarted not called before query returned"), StopStream, Assert { listener.checkAsyncErrors() } From b3c4912173a5d03f104c748b2bb7ea6b148b43c9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 May 2016 10:38:27 -0700 Subject: [PATCH 061/313] [SPARK-15109][SQL] Accept Dataset[_] in joins ## What changes were proposed in this pull request? This patch changes the join API in Dataset so they can accept any Dataset, rather than just DataFrames. ## How was this patch tested? N/A. Author: Reynold Xin Closes #12886 from rxin/SPARK-15109. (cherry picked from commit d864c55cf8c92466336e796d0c98d83230e330af) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/sql/Dataset.scala | 12 ++++++------ .../main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 31dd64e909bb..c77b13832c8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -564,7 +564,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame): DataFrame = withPlan { + def join(right: Dataset[_]): DataFrame = withPlan { Join(logicalPlan, right.logicalPlan, joinType = Inner, None) } @@ -589,7 +589,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, usingColumn: String): DataFrame = { + def join(right: Dataset[_], usingColumn: String): DataFrame = { join(right, Seq(usingColumn)) } @@ -614,7 +614,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, usingColumns: Seq[String]): DataFrame = { + def join(right: Dataset[_], usingColumns: Seq[String]): DataFrame = { join(right, usingColumns, "inner") } @@ -635,7 +635,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, usingColumns: Seq[String], joinType: String): DataFrame = { + def join(right: Dataset[_], usingColumns: Seq[String], joinType: String): DataFrame = { // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right // by creating a new instance for one of the branch. val joined = sparkSession.executePlan( @@ -663,7 +663,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, joinExprs: Column): DataFrame = join(right, joinExprs, "inner") + def join(right: Dataset[_], joinExprs: Column): DataFrame = join(right, joinExprs, "inner") /** * Join with another [[DataFrame]], using the given join expression. The following performs @@ -686,7 +686,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { + def join(right: Dataset[_], joinExprs: Column, joinType: String): DataFrame = { // Note that in this function, we introduce a hack in the case of self-join to automatically // resolve ambiguous join conditions into ones that might make sense [SPARK-6231]. // Consider this case: df.join(df, df("key") === df("key")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index fe63c80815a9..3e295c20b6d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -931,8 +931,8 @@ object functions { * @group normal_funcs * @since 1.5.0 */ - def broadcast(df: DataFrame): DataFrame = { - Dataset.ofRows(df.sparkSession, BroadcastHint(df.logicalPlan)) + def broadcast[T](df: Dataset[T]): Dataset[T] = { + Dataset[T](df.sparkSession, BroadcastHint(df.logicalPlan))(df.unresolvedTEncoder) } /** From 21e368de4d17a52a380959bb709caca091a3fac7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 4 May 2016 10:54:51 -0700 Subject: [PATCH 062/313] [SPARK-14951] [SQL] Support subexpression elimination in TungstenAggregate ## What changes were proposed in this pull request? We can support subexpression elimination in TungstenAggregate by using current `EquivalentExpressions` which is already used in subexpression elimination for expression codegen. However, in wholestage codegen, we can't wrap the common expression's codes in functions as before, we simply generate the code snippets for common expressions. These code snippets are inserted before the common expressions are actually used in generated java codes. For multiple `TypedAggregateExpression` used in aggregation operator, since their input type should be the same. So their `inputDeserializer` will be the same too. This patch can also reduce redundant input deserialization. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #12729 from viirya/subexpr-elimination-tungstenaggregate. (cherry picked from commit b85d21fb9dc3d498d9a10e065d254abde797efb6) Signed-off-by: Davies Liu --- .../expressions/EquivalentExpressions.scala | 7 +- .../expressions/codegen/CodeGenerator.scala | 74 ++++++++++++++++++- .../aggregate/TungstenAggregate.scala | 31 ++++++-- .../aggregate/TypedAggregateExpression.scala | 38 ++-------- 4 files changed, 109 insertions(+), 41 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala index d0ad7a05a0c3..b8e2b67b2fe9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala @@ -68,7 +68,10 @@ class EquivalentExpressions { * is found. That is, if `expr` has already been added, its children are not added. * If ignoreLeaf is true, leaf nodes are ignored. */ - def addExprTree(root: Expression, ignoreLeaf: Boolean = true): Unit = { + def addExprTree( + root: Expression, + ignoreLeaf: Boolean = true, + skipReferenceToExpressions: Boolean = true): Unit = { val skip = root.isInstanceOf[LeafExpression] && ignoreLeaf // There are some special expressions that we should not recurse into children. // 1. CodegenFallback: it's children will not be used to generate code (call eval() instead) @@ -77,7 +80,7 @@ class EquivalentExpressions { // TODO: some expressions implements `CodegenFallback` but can still do codegen, // e.g. `CaseWhen`, we should support them. case _: CodegenFallback => false - case _: ReferenceToExpressions => false + case _: ReferenceToExpressions if skipReferenceToExpressions => false case _ => true } if (!skip && !addExpr(root) && shouldRecurse) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index e4fa429b3754..67f671926561 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -46,6 +46,25 @@ import org.apache.spark.util.Utils */ case class ExprCode(var code: String, var isNull: String, var value: String) +/** + * State used for subexpression elimination. + * + * @param isNull A term that holds a boolean value representing whether the expression evaluated + * to null. + * @param value A term for a value of a common sub-expression. Not valid if `isNull` + * is set to `true`. + */ +case class SubExprEliminationState(isNull: String, value: String) + +/** + * Codes and common subexpressions mapping used for subexpression elimination. + * + * @param codes Strings representing the codes that evaluate common subexpressions. + * @param states Foreach expression that is participating in subexpression elimination, + * the state to use. + */ +case class SubExprCodes(codes: Seq[String], states: Map[Expression, SubExprEliminationState]) + /** * A context for codegen, tracking a list of objects that could be passed into generated Java * function. @@ -148,9 +167,6 @@ class CodegenContext { */ val equivalentExpressions: EquivalentExpressions = new EquivalentExpressions - // State used for subexpression elimination. - case class SubExprEliminationState(isNull: String, value: String) - // Foreach expression that is participating in subexpression elimination, the state to use. val subExprEliminationExprs = mutable.HashMap.empty[Expression, SubExprEliminationState] @@ -571,6 +587,58 @@ class CodegenContext { } } + /** + * Perform a function which generates a sequence of ExprCodes with a given mapping between + * expressions and common expressions, instead of using the mapping in current context. + */ + def withSubExprEliminationExprs( + newSubExprEliminationExprs: Map[Expression, SubExprEliminationState])( + f: => Seq[ExprCode]): Seq[ExprCode] = { + val oldsubExprEliminationExprs = subExprEliminationExprs + subExprEliminationExprs.clear + newSubExprEliminationExprs.foreach(subExprEliminationExprs += _) + + val genCodes = f + + // Restore previous subExprEliminationExprs + subExprEliminationExprs.clear + oldsubExprEliminationExprs.foreach(subExprEliminationExprs += _) + genCodes + } + + /** + * Checks and sets up the state and codegen for subexpression elimination. This finds the + * common subexpressions, generates the code snippets that evaluate those expressions and + * populates the mapping of common subexpressions to the generated code snippets. The generated + * code snippets will be returned and should be inserted into generated codes before these + * common subexpressions actually are used first time. + */ + def subexpressionEliminationForWholeStageCodegen(expressions: Seq[Expression]): SubExprCodes = { + // Create a clear EquivalentExpressions and SubExprEliminationState mapping + val equivalentExpressions: EquivalentExpressions = new EquivalentExpressions + val subExprEliminationExprs = mutable.HashMap.empty[Expression, SubExprEliminationState] + + // Add each expression tree and compute the common subexpressions. + expressions.foreach(equivalentExpressions.addExprTree(_, true, false)) + + // Get all the expressions that appear at least twice and set up the state for subexpression + // elimination. + val commonExprs = equivalentExpressions.getAllEquivalentExprs.filter(_.size > 1) + val codes = commonExprs.map { e => + val expr = e.head + val fnName = freshName("evalExpr") + val isNull = s"${fnName}IsNull" + val value = s"${fnName}Value" + + // Generate the code for this expression tree. + val code = expr.genCode(this) + val state = SubExprEliminationState(code.isNull, code.value) + e.foreach(subExprEliminationExprs.put(_, state)) + code.code.trim + } + SubExprCodes(codes, subExprEliminationExprs.toMap) + } + /** * Checks and sets up the state and codegen for subexpression elimination. This finds the * common subexpressions, generates the functions that evaluate those expressions and populates diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index d0ba37ee1338..d2dc80a7e42e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -244,8 +244,12 @@ case class TungstenAggregate( } } ctx.currentVars = bufVars ++ input - // TODO: support subexpression elimination - val aggVals = updateExpr.map(BindReferences.bindReference(_, inputAttrs).genCode(ctx)) + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttrs)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val aggVals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } // aggregate buffer should be updated atomic val updates = aggVals.zipWithIndex.map { case (ev, i) => s""" @@ -255,6 +259,9 @@ case class TungstenAggregate( } s""" | // do aggregate + | // common sub-expressions + | $effectiveCodes + | // evaluate aggregate function | ${evaluateVariables(aggVals)} | // update aggregation buffer | ${updates.mkString("\n").trim} @@ -650,8 +657,12 @@ case class TungstenAggregate( val updateRowInVectorizedHashMap: Option[String] = { if (isVectorizedHashMapEnabled) { ctx.INPUT_ROW = vectorizedRowBuffer - val vectorizedRowEvals = - updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val vectorizedRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } val updateVectorizedRow = vectorizedRowEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(vectorizedRowBuffer, dt, i, ev, updateExpr(i).nullable, @@ -659,6 +670,8 @@ case class TungstenAggregate( } Option( s""" + |// common sub-expressions + |$effectiveCodes |// evaluate aggregate function |${evaluateVariables(vectorizedRowEvals)} |// update vectorized row @@ -701,13 +714,19 @@ case class TungstenAggregate( val updateRowInUnsafeRowMap: String = { ctx.INPUT_ROW = unsafeRowBuffer - val unsafeRowBufferEvals = - updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val unsafeRowBufferEvals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } val updateUnsafeRowBuffer = unsafeRowBufferEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(unsafeRowBuffer, dt, i, ev, updateExpr(i).nullable) } s""" + |// common sub-expressions + |$effectiveCodes |// evaluate aggregate function |${evaluateVariables(unsafeRowBufferEvals)} |// update unsafe row buffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 535e64cb3444..edca816cb1d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -31,31 +31,9 @@ object TypedAggregateExpression { def apply[BUF : Encoder, OUT : Encoder]( aggregator: Aggregator[_, BUF, OUT]): TypedAggregateExpression = { val bufferEncoder = encoderFor[BUF] - // We will insert the deserializer and function call expression at the bottom of each serializer - // expression while executing `TypedAggregateExpression`, which means multiply serializer - // expressions will all evaluate the same sub-expression at bottom. To avoid the re-evaluating, - // here we always use one single serializer expression to serialize the buffer object into a - // single-field row, no matter whether the encoder is flat or not. We also need to update the - // deserializer to read in all fields from that single-field row. - // TODO: remove this trick after we have better integration of subexpression elimination and - // whole stage codegen. - val bufferSerializer = if (bufferEncoder.flat) { - bufferEncoder.namedExpressions.head - } else { - Alias(CreateStruct(bufferEncoder.serializer), "buffer")() - } - - val bufferDeserializer = if (bufferEncoder.flat) { - bufferEncoder.deserializer transformUp { - case b: BoundReference => bufferSerializer.toAttribute - } - } else { - bufferEncoder.deserializer transformUp { - case UnresolvedAttribute(nameParts) => - assert(nameParts.length == 1) - UnresolvedExtractValue(bufferSerializer.toAttribute, Literal(nameParts.head)) - case BoundReference(ordinal, dt, _) => GetStructField(bufferSerializer.toAttribute, ordinal) - } + val bufferSerializer = bufferEncoder.namedExpressions + val bufferDeserializer = bufferEncoder.deserializer.transform { + case b: BoundReference => bufferSerializer(b.ordinal).toAttribute } val outputEncoder = encoderFor[OUT] @@ -82,7 +60,7 @@ object TypedAggregateExpression { case class TypedAggregateExpression( aggregator: Aggregator[Any, Any, Any], inputDeserializer: Option[Expression], - bufferSerializer: NamedExpression, + bufferSerializer: Seq[NamedExpression], bufferDeserializer: Expression, outputSerializer: Seq[Expression], outputExternalType: DataType, @@ -106,11 +84,11 @@ case class TypedAggregateExpression( private def bufferExternalType = bufferDeserializer.dataType override lazy val aggBufferAttributes: Seq[AttributeReference] = - bufferSerializer.toAttribute.asInstanceOf[AttributeReference] :: Nil + bufferSerializer.map(_.toAttribute.asInstanceOf[AttributeReference]) override lazy val initialValues: Seq[Expression] = { val zero = Literal.fromObject(aggregator.zero, bufferExternalType) - ReferenceToExpressions(bufferSerializer, zero :: Nil) :: Nil + bufferSerializer.map(ReferenceToExpressions(_, zero :: Nil)) } override lazy val updateExpressions: Seq[Expression] = { @@ -120,7 +98,7 @@ case class TypedAggregateExpression( bufferExternalType, bufferDeserializer :: inputDeserializer.get :: Nil) - ReferenceToExpressions(bufferSerializer, reduced :: Nil) :: Nil + bufferSerializer.map(ReferenceToExpressions(_, reduced :: Nil)) } override lazy val mergeExpressions: Seq[Expression] = { @@ -136,7 +114,7 @@ case class TypedAggregateExpression( bufferExternalType, leftBuffer :: rightBuffer :: Nil) - ReferenceToExpressions(bufferSerializer, merged :: Nil) :: Nil + bufferSerializer.map(ReferenceToExpressions(_, merged :: Nil)) } override lazy val evaluateExpression: Expression = { From 54d90bd3eb18253b3f3d67ae8a52471314a061ad Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Wed, 4 May 2016 10:59:36 -0700 Subject: [PATCH 063/313] [MINOR] Add python3 compatibility in python examples ## What changes were proposed in this pull request? Add python3 compatibility in python examples ## How was this patch tested? manual tests Author: Zheng RuiFeng Closes #12868 from zhengruifeng/fix_gmm_py. (cherry picked from commit 4530250f5a51a77f9d0b91f036f8e44e0f943a32) Signed-off-by: Davies Liu --- examples/src/main/python/ml/als_example.py | 4 ++++ examples/src/main/python/mllib/gaussian_mixture_model.py | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index 922173308c6a..0c9ac583b2da 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -17,6 +17,10 @@ from __future__ import print_function +import sys +if sys.version >= '3': + long = int + from pyspark import SparkContext from pyspark.sql import SQLContext diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index 69e836fc1d06..6b46e27ddaaa 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -20,6 +20,10 @@ """ from __future__ import print_function +import sys +if sys.version >= '3': + long = int + import random import argparse import numpy as np From c5961543206cbccc7a9b3103110f1a84b4ccddce Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 May 2016 11:00:01 -0700 Subject: [PATCH 064/313] [SPARK-15115][SQL] Reorganize whole stage codegen benchmark suites ## What changes were proposed in this pull request? We currently have a single suite that is very large, making it difficult to maintain and play with specific primitives. This patch reorganizes the file by creating multiple benchmark suites in a single package. Most of the changes are straightforward move of code. On top of the code moving, I did: 1. Use SparkSession instead of SQLContext. 2. Turned most benchmark scenarios into a their own test cases, rather than having multiple scenarios in a single test case, which takes forever to run. ## How was this patch tested? This is a test only change. Author: Reynold Xin Closes #12891 from rxin/SPARK-15115. (cherry picked from commit 6274a520fa743b7d079fde4a3033da5c3a2532a1) Signed-off-by: Reynold Xin --- .../sort/RecordPointerAndKeyPrefix.java | 2 +- .../unsafe/sort/UnsafeSortDataFormat.java | 3 +- .../unsafe/sort/RadixSortSuite.scala | 78 ---- .../AggregateBenchmark.scala} | 392 +++--------------- .../execution/benchmark/BenchmarkBase.scala | 54 +++ .../execution/benchmark/JoinBenchmark.scala | 229 ++++++++++ .../execution/benchmark/MiscBenchmark.scala | 135 ++++++ .../execution/benchmark/SortBenchmark.scala | 132 ++++++ 8 files changed, 603 insertions(+), 422 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/execution/{BenchmarkWholeStageCodegen.scala => benchmark/AggregateBenchmark.scala} (52%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java index de92b8db4713..e9571aa8bb05 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java @@ -17,7 +17,7 @@ package org.apache.spark.util.collection.unsafe.sort; -final class RecordPointerAndKeyPrefix { +public final class RecordPointerAndKeyPrefix { /** * A pointer to a record; see {@link org.apache.spark.memory.TaskMemoryManager} for a * description of how these addresses are encoded. diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java index 12fb62fb77f0..d19b71fbc1bc 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java @@ -29,7 +29,8 @@ * Within each long[] buffer, position {@code 2 * i} holds a pointer pointer to the record at * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. */ -final class UnsafeSortDataFormat extends SortDataFormat { +public final class UnsafeSortDataFormat + extends SortDataFormat { public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat(); diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index b03df1a94d84..def0752b46f6 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.unsafe.array.LongArray import org.apache.spark.unsafe.memory.MemoryBlock -import org.apache.spark.util.Benchmark import org.apache.spark.util.collection.Sorter import org.apache.spark.util.random.XORShiftRandom @@ -184,81 +183,4 @@ class RadixSortSuite extends SparkFunSuite with Logging { assert(res1.view == res2.view) } } - - ignore("microbenchmarks") { - val size = 25000000 - val rand = new XORShiftRandom(123) - val benchmark = new Benchmark("radix sort " + size, size) - benchmark.addTimerCase("reference TimSort key prefix array") { timer => - val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) - timer.stopTiming() - } - benchmark.addTimerCase("reference Arrays.sort") { timer => - val ref = Array.tabulate[Long](size) { i => rand.nextLong } - timer.startTiming() - Arrays.sort(ref) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort one byte") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong & 0xff - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort two bytes") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong & 0xffff - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort eight bytes") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort key prefix array") { timer => - val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) - timer.startTiming() - RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.run() - - /** - Running benchmark: radix sort 25000000 - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 3.13.0-44-generic - Intel(R) Core(TM) i7-4600U CPU @ 2.10GHz - - radix sort 25000000: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - reference TimSort key prefix array 15546 / 15859 1.6 621.9 1.0X - reference Arrays.sort 2416 / 2446 10.3 96.6 6.4X - radix sort one byte 133 / 137 188.4 5.3 117.2X - radix sort two bytes 255 / 258 98.2 10.2 61.1X - radix sort eight bytes 991 / 997 25.2 39.6 15.7X - radix sort key prefix array 1540 / 1563 16.2 61.6 10.1X - */ - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala similarity index 52% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 7ca4b75f480b..b31338e82768 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -15,59 +15,35 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.execution.benchmark import java.util.HashMap -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.SparkConf import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} -import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{IntegerType, LongType, StructType} +import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.util.Benchmark /** - * Benchmark to measure whole stage codegen performance. + * Benchmark to measure performance for aggregate primitives. * To run this: - * build/sbt "sql/test-only *BenchmarkWholeStageCodegen" + * build/sbt "sql/test-only *benchmark.AggregateBenchmark" * * Benchmarks in this file are skipped in normal builds. */ -class BenchmarkWholeStageCodegen extends SparkFunSuite { - lazy val conf = new SparkConf().setMaster("local[1]").setAppName("benchmark") - .set("spark.sql.shuffle.partitions", "1") - .set("spark.sql.autoBroadcastJoinThreshold", "1") - lazy val sc = SparkContext.getOrCreate(conf) - lazy val sqlContext = SQLContext.getOrCreate(sc) - - /** Runs function `f` with whole stage codegen on and off. */ - def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { - val benchmark = new Benchmark(name, cardinality) - - benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") - f - } - - benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - f - } - - benchmark.run() - } +class AggregateBenchmark extends BenchmarkBase { ignore("aggregate without grouping") { val N = 500L << 22 val benchmark = new Benchmark("agg without grouping", N) runBenchmark("agg w/o group", N) { - sqlContext.range(N).selectExpr("sum(id)").collect() + sparkSession.range(N).selectExpr("sum(id)").collect() } /* agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative @@ -77,79 +53,19 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { */ } - ignore("filter & aggregate without group") { - val N = 500L << 22 - runBenchmark("range/filter/sum", N) { - sqlContext.range(N).filter("(id & 1) = 1").groupBy().sum().collect() - } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X - range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X - */ - } - - ignore("range/limit/sum") { - val N = 500L << 20 - runBenchmark("range/limit/sum", N) { - sqlContext.range(N).limit(1000000).groupBy().sum().collect() - } - /* - Westmere E56xx/L56xx/X56xx (Nehalem-C) - range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - range/limit/sum codegen=false 609 / 672 861.6 1.2 1.0X - range/limit/sum codegen=true 561 / 621 935.3 1.1 1.1X - */ - } - - ignore("sample") { - val N = 500 << 18 - runBenchmark("sample with replacement", N) { - sqlContext.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() - } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X - sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X - */ - - runBenchmark("sample without replacement", N) { - sqlContext.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() - } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X - sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X - */ - } - ignore("stat functions") { val N = 100L << 20 runBenchmark("stddev", N) { - sqlContext.range(N).groupBy().agg("id" -> "stddev").collect() + sparkSession.range(N).groupBy().agg("id" -> "stddev").collect() } runBenchmark("kurtosis", N) { - sqlContext.range(N).groupBy().agg("id" -> "kurtosis").collect() + sparkSession.range(N).groupBy().agg("id" -> "kurtosis").collect() } - - /** - Using ImperativeAggregate (as implemented in Spark 1.6): + /* + Using ImperativeAggregate (as implemented in Spark 1.6): Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz stddev: Avg Time(ms) Avg Rate(M/s) Relative Rate @@ -172,29 +88,31 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { ------------------------------------------------------------------------------------------- kurtosis codegen=false 14847 / 15084 7.0 142.9 1.0X kurtosis codegen=true 1652 / 2124 63.0 15.9 9.0X - */ + */ } ignore("aggregate with linear keys") { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) - def f(): Unit = sqlContext.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + def f(): Unit = { + sparkSession.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + } benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "3") f() } @@ -216,24 +134,24 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) - sqlContext.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") - def f(): Unit = sqlContext.sql("select k, k, sum(id) from test group by k, k").collect() + def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 3) f() } @@ -255,23 +173,23 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 20 val benchmark = new Benchmark("Aggregate w string key", N) - def f(): Unit = sqlContext.range(N).selectExpr("id", "cast(id & 1023 as string) as k") + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 1023 as string) as k") .groupBy("k").count().collect() benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "3") f() } @@ -292,23 +210,23 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 20 val benchmark = new Benchmark("Aggregate w decimal key", N) - def f(): Unit = sqlContext.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") .groupBy("k").count().collect() benchmark.addCase(s"codegen = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "3") f() } @@ -329,7 +247,7 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 20 val benchmark = new Benchmark("Aggregate w multiple keys", N) - def f(): Unit = sqlContext.range(N) + def f(): Unit = sparkSession.range(N) .selectExpr( "id", "(id & 1023) as k1", @@ -343,19 +261,19 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { .collect() benchmark.addCase(s"codegen = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "10") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "10") f() } @@ -372,187 +290,22 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { */ } - ignore("broadcast hash join") { - val N = 20 << 20 - val M = 1 << 16 - val dim = broadcast(sqlContext.range(M).selectExpr("id as k", "cast(id as string) as v")) - - runBenchmark("Join w long", N) { - sqlContext.range(N).join(dim, (col("id") % M) === col("k")).count() - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w long codegen=false 3002 / 3262 7.0 143.2 1.0X - Join w long codegen=true 321 / 371 65.3 15.3 9.3X - */ - - runBenchmark("Join w long duplicated", N) { - val dim = broadcast(sqlContext.range(M).selectExpr("cast(id/10 as long) as k")) - sqlContext.range(N).join(dim, (col("id") % M) === col("k")).count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w long duplicated codegen=false 3446 / 3478 6.1 164.3 1.0X - Join w long duplicated codegen=true 322 / 351 65.2 15.3 10.7X - */ - - val dim2 = broadcast(sqlContext.range(M) - .selectExpr("cast(id as int) as k1", "cast(id as int) as k2", "cast(id as string) as v")) - - runBenchmark("Join w 2 ints", N) { - sqlContext.range(N).join(dim2, - (col("id") % M).cast(IntegerType) === col("k1") - && (col("id") % M).cast(IntegerType) === col("k2")).count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w 2 ints codegen=false 4426 / 4501 4.7 211.1 1.0X - Join w 2 ints codegen=true 791 / 818 26.5 37.7 5.6X - */ - - val dim3 = broadcast(sqlContext.range(M) - .selectExpr("id as k1", "id as k2", "cast(id as string) as v")) - - runBenchmark("Join w 2 longs", N) { - sqlContext.range(N).join(dim3, - (col("id") % M) === col("k1") && (col("id") % M) === col("k2")) - .count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w 2 longs codegen=false 5905 / 6123 3.6 281.6 1.0X - Join w 2 longs codegen=true 2230 / 2529 9.4 106.3 2.6X - */ - - val dim4 = broadcast(sqlContext.range(M) - .selectExpr("cast(id/10 as long) as k1", "cast(id/10 as long) as k2")) - - runBenchmark("Join w 2 longs duplicated", N) { - sqlContext.range(N).join(dim4, - (col("id") bitwiseAND M) === col("k1") && (col("id") bitwiseAND M) === col("k2")) - .count() - } - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w 2 longs duplicated codegen=false 6420 / 6587 3.3 306.1 1.0X - Join w 2 longs duplicated codegen=true 2080 / 2139 10.1 99.2 3.1X - */ - - runBenchmark("outer join w long", N) { - sqlContext.range(N).join(dim, (col("id") % M) === col("k"), "left").count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - outer join w long codegen=false 3055 / 3189 6.9 145.7 1.0X - outer join w long codegen=true 261 / 276 80.5 12.4 11.7X - */ - - runBenchmark("semi join w long", N) { - sqlContext.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi").count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - semi join w long codegen=false 1912 / 1990 11.0 91.2 1.0X - semi join w long codegen=true 237 / 244 88.3 11.3 8.1X - */ - } - - ignore("sort merge join") { - val N = 2 << 20 - runBenchmark("merge join", N) { - val df1 = sqlContext.range(N).selectExpr(s"id * 2 as k1") - val df2 = sqlContext.range(N).selectExpr(s"id * 3 as k2") - df1.join(df2, col("k1") === col("k2")).count() - } - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - merge join codegen=false 1588 / 1880 1.3 757.1 1.0X - merge join codegen=true 1477 / 1531 1.4 704.2 1.1X - */ - - runBenchmark("sort merge join", N) { - val df1 = sqlContext.range(N) - .selectExpr(s"(id * 15485863) % ${N*10} as k1") - val df2 = sqlContext.range(N) - .selectExpr(s"(id * 15485867) % ${N*10} as k2") - df1.join(df2, col("k1") === col("k2")).count() - } - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - sort merge join codegen=false 3626 / 3667 0.6 1728.9 1.0X - sort merge join codegen=true 3405 / 3438 0.6 1623.8 1.1X - */ - } - - ignore("shuffle hash join") { - val N = 4 << 20 - sqlContext.setConf("spark.sql.shuffle.partitions", "2") - sqlContext.setConf("spark.sql.autoBroadcastJoinThreshold", "10000000") - sqlContext.setConf("spark.sql.join.preferSortMergeJoin", "false") - runBenchmark("shuffle hash join", N) { - val df1 = sqlContext.range(N).selectExpr(s"id as k1") - val df2 = sqlContext.range(N / 5).selectExpr(s"id * 3 as k2") - df1.join(df2, col("k1") === col("k2")).count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - shuffle hash join codegen=false 1101 / 1391 3.8 262.6 1.0X - shuffle hash join codegen=true 528 / 578 7.9 125.8 2.1X - */ - } ignore("cube") { val N = 5 << 20 runBenchmark("cube", N) { - sqlContext.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") + sparkSession.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") .cube("k1", "k2").sum("id").collect() } /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------- cube codegen=false 3188 / 3392 1.6 608.2 1.0X cube codegen=true 1239 / 1394 4.2 236.3 2.6X - */ + */ } ignore("hash and BytesToBytesMap") { @@ -802,7 +555,7 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { } } - /** + /* Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------- @@ -822,49 +575,4 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { benchmark.run() } - ignore("collect") { - val N = 1 << 20 - - val benchmark = new Benchmark("collect", N) - benchmark.addCase("collect 1 million") { iter => - sqlContext.range(N).collect() - } - benchmark.addCase("collect 2 millions") { iter => - sqlContext.range(N * 2).collect() - } - benchmark.addCase("collect 4 millions") { iter => - sqlContext.range(N * 4).collect() - } - benchmark.run() - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - collect 1 million 439 / 654 2.4 418.7 1.0X - collect 2 millions 961 / 1907 1.1 916.4 0.5X - collect 4 millions 3193 / 3895 0.3 3044.7 0.1X - */ - } - - ignore("collect limit") { - val N = 1 << 20 - - val benchmark = new Benchmark("collect limit", N) - benchmark.addCase("collect limit 1 million") { iter => - sqlContext.range(N * 4).limit(N).collect() - } - benchmark.addCase("collect limit 2 millions") { iter => - sqlContext.range(N * 4).limit(N * 2).collect() - } - benchmark.run() - - /** - model name : Westmere E56xx/L56xx/X56xx (Nehalem-C) - collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - collect limit 1 million 833 / 1284 1.3 794.4 1.0X - collect limit 2 millions 3348 / 4005 0.3 3193.3 0.2X - */ - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala new file mode 100644 index 000000000000..c99a5aec1cd6 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala @@ -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. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.Benchmark + +/** + * Common base trait for micro benchmarks that are supposed to run standalone (i.e. not together + * with other test suites). + */ +private[benchmark] trait BenchmarkBase extends SparkFunSuite { + + lazy val sparkSession = SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.autoBroadcastJoinThreshold", 1) + .getOrCreate() + + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality) + + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f + } + + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + f + } + + benchmark.run() + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala new file mode 100644 index 000000000000..46db41a8abad --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.IntegerType + +/** + * Benchmark to measure performance for aggregate primitives. + * To run this: + * build/sbt "sql/test-only *benchmark.JoinBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class JoinBenchmark extends BenchmarkBase { + + ignore("broadcast hash join, long key") { + val N = 20 << 20 + val M = 1 << 16 + + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("Join w long", N) { + sparkSession.range(N).join(dim, (col("id") % M) === col("k")).count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + Join w long codegen=false 3002 / 3262 7.0 143.2 1.0X + Join w long codegen=true 321 / 371 65.3 15.3 9.3X + */ + } + + ignore("broadcast hash join, long key with duplicates") { + val N = 20 << 20 + val M = 1 << 16 + + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("Join w long duplicated", N) { + val dim = broadcast(sparkSession.range(M).selectExpr("cast(id/10 as long) as k")) + sparkSession.range(N).join(dim, (col("id") % M) === col("k")).count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w long duplicated codegen=false 3446 / 3478 6.1 164.3 1.0X + *Join w long duplicated codegen=true 322 / 351 65.2 15.3 10.7X + */ + } + + ignore("broadcast hash join, two int key") { + val N = 20 << 20 + val M = 1 << 16 + val dim2 = broadcast(sparkSession.range(M) + .selectExpr("cast(id as int) as k1", "cast(id as int) as k2", "cast(id as string) as v")) + + runBenchmark("Join w 2 ints", N) { + sparkSession.range(N).join(dim2, + (col("id") % M).cast(IntegerType) === col("k1") + && (col("id") % M).cast(IntegerType) === col("k2")).count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 ints codegen=false 4426 / 4501 4.7 211.1 1.0X + *Join w 2 ints codegen=true 791 / 818 26.5 37.7 5.6X + */ + } + + ignore("broadcast hash join, two long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim3 = broadcast(sparkSession.range(M) + .selectExpr("id as k1", "id as k2", "cast(id as string) as v")) + + runBenchmark("Join w 2 longs", N) { + sparkSession.range(N).join(dim3, + (col("id") % M) === col("k1") && (col("id") % M) === col("k2")) + .count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 longs codegen=false 5905 / 6123 3.6 281.6 1.0X + *Join w 2 longs codegen=true 2230 / 2529 9.4 106.3 2.6X + */ + } + + ignore("broadcast hash join, two long key with duplicates") { + val N = 20 << 20 + val M = 1 << 16 + val dim4 = broadcast(sparkSession.range(M) + .selectExpr("cast(id/10 as long) as k1", "cast(id/10 as long) as k2")) + + runBenchmark("Join w 2 longs duplicated", N) { + sparkSession.range(N).join(dim4, + (col("id") bitwiseAND M) === col("k1") && (col("id") bitwiseAND M) === col("k2")) + .count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 longs duplicated codegen=false 6420 / 6587 3.3 306.1 1.0X + *Join w 2 longs duplicated codegen=true 2080 / 2139 10.1 99.2 3.1X + */ + } + + ignore("broadcast hash join, outer join long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("outer join w long", N) { + sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "left").count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *outer join w long codegen=false 3055 / 3189 6.9 145.7 1.0X + *outer join w long codegen=true 261 / 276 80.5 12.4 11.7X + */ + } + + ignore("broadcast hash join, semi join long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("semi join w long", N) { + sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi").count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *semi join w long codegen=false 1912 / 1990 11.0 91.2 1.0X + *semi join w long codegen=true 237 / 244 88.3 11.3 8.1X + */ + } + + ignore("sort merge join") { + val N = 2 << 20 + runBenchmark("merge join", N) { + val df1 = sparkSession.range(N).selectExpr(s"id * 2 as k1") + val df2 = sparkSession.range(N).selectExpr(s"id * 3 as k2") + df1.join(df2, col("k1") === col("k2")).count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *merge join codegen=false 1588 / 1880 1.3 757.1 1.0X + *merge join codegen=true 1477 / 1531 1.4 704.2 1.1X + */ + } + + ignore("sort merge join with duplicates") { + val N = 2 << 20 + runBenchmark("sort merge join", N) { + val df1 = sparkSession.range(N) + .selectExpr(s"(id * 15485863) % ${N*10} as k1") + val df2 = sparkSession.range(N) + .selectExpr(s"(id * 15485867) % ${N*10} as k2") + df1.join(df2, col("k1") === col("k2")).count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *sort merge join codegen=false 3626 / 3667 0.6 1728.9 1.0X + *sort merge join codegen=true 3405 / 3438 0.6 1623.8 1.1X + */ + } + + ignore("shuffle hash join") { + val N = 4 << 20 + sparkSession.conf.set("spark.sql.shuffle.partitions", "2") + sparkSession.conf.set("spark.sql.autoBroadcastJoinThreshold", "10000000") + sparkSession.conf.set("spark.sql.join.preferSortMergeJoin", "false") + runBenchmark("shuffle hash join", N) { + val df1 = sparkSession.range(N).selectExpr(s"id as k1") + val df2 = sparkSession.range(N / 5).selectExpr(s"id * 3 as k2") + df1.join(df2, col("k1") === col("k2")).count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *shuffle hash join codegen=false 1101 / 1391 3.8 262.6 1.0X + *shuffle hash join codegen=true 528 / 578 7.9 125.8 2.1X + */ + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala new file mode 100644 index 000000000000..470c78120b19 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure whole stage codegen performance. + * To run this: + * build/sbt "sql/test-only *benchmark.MiscBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class MiscBenchmark extends BenchmarkBase { + + ignore("filter & aggregate without group") { + val N = 500L << 22 + runBenchmark("range/filter/sum", N) { + sparkSession.range(N).filter("(id & 1) = 1").groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X + range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X + */ + } + + ignore("range/limit/sum") { + val N = 500L << 20 + runBenchmark("range/limit/sum", N) { + sparkSession.range(N).limit(1000000).groupBy().sum().collect() + } + /* + Westmere E56xx/L56xx/X56xx (Nehalem-C) + range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + range/limit/sum codegen=false 609 / 672 861.6 1.2 1.0X + range/limit/sum codegen=true 561 / 621 935.3 1.1 1.1X + */ + } + + ignore("sample") { + val N = 500 << 18 + runBenchmark("sample with replacement", N) { + sparkSession.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X + sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X + */ + + runBenchmark("sample without replacement", N) { + sparkSession.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X + sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X + */ + } + + ignore("collect") { + val N = 1 << 20 + + val benchmark = new Benchmark("collect", N) + benchmark.addCase("collect 1 million") { iter => + sparkSession.range(N).collect() + } + benchmark.addCase("collect 2 millions") { iter => + sparkSession.range(N * 2).collect() + } + benchmark.addCase("collect 4 millions") { iter => + sparkSession.range(N * 4).collect() + } + benchmark.run() + + /** + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + collect 1 million 439 / 654 2.4 418.7 1.0X + collect 2 millions 961 / 1907 1.1 916.4 0.5X + collect 4 millions 3193 / 3895 0.3 3044.7 0.1X + */ + } + + ignore("collect limit") { + val N = 1 << 20 + + val benchmark = new Benchmark("collect limit", N) + benchmark.addCase("collect limit 1 million") { iter => + sparkSession.range(N * 4).limit(N).collect() + } + benchmark.addCase("collect limit 2 millions") { iter => + sparkSession.range(N * 4).limit(N * 2).collect() + } + benchmark.run() + + /** + model name : Westmere E56xx/L56xx/X56xx (Nehalem-C) + collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + collect limit 1 million 833 / 1284 1.3 794.4 1.0X + collect limit 2 millions 3348 / 4005 0.3 3193.3 0.2X + */ + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala new file mode 100644 index 000000000000..0e1868dd6656 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import java.util.{Arrays, Comparator} + +import org.apache.spark.unsafe.array.LongArray +import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.util.Benchmark +import org.apache.spark.util.collection.Sorter +import org.apache.spark.util.collection.unsafe.sort._ +import org.apache.spark.util.random.XORShiftRandom + +/** + * Benchmark to measure performance for aggregate primitives. + * To run this: + * build/sbt "sql/test-only *benchmark.SortBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class SortBenchmark extends BenchmarkBase { + + private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { + new Sorter(UnsafeSortDataFormat.INSTANCE).sort( + buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { + override def compare( + r1: RecordPointerAndKeyPrefix, + r2: RecordPointerAndKeyPrefix): Int = { + refCmp.compare(r1.keyPrefix, r2.keyPrefix) + } + }) + } + + private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { + val ref = Array.tabulate[Long](size * 2) { i => rand } + val extended = ref ++ Array.fill[Long](size * 2)(0) + (new LongArray(MemoryBlock.fromLongArray(ref)), + new LongArray(MemoryBlock.fromLongArray(extended))) + } + + ignore("sort") { + val size = 25000000 + val rand = new XORShiftRandom(123) + val benchmark = new Benchmark("radix sort " + size, size) + benchmark.addTimerCase("reference TimSort key prefix array") { timer => + val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) + timer.stopTiming() + } + benchmark.addTimerCase("reference Arrays.sort") { timer => + val ref = Array.tabulate[Long](size) { i => rand.nextLong } + timer.startTiming() + Arrays.sort(ref) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort one byte") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong & 0xff + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort two bytes") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong & 0xffff + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort eight bytes") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort key prefix array") { timer => + val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) + timer.startTiming() + RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.run() + + /* + Running benchmark: radix sort 25000000 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 3.13.0-44-generic + Intel(R) Core(TM) i7-4600U CPU @ 2.10GHz + + radix sort 25000000: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + reference TimSort key prefix array 15546 / 15859 1.6 621.9 1.0X + reference Arrays.sort 2416 / 2446 10.3 96.6 6.4X + radix sort one byte 133 / 137 188.4 5.3 117.2X + radix sort two bytes 255 / 258 98.2 10.2 61.1X + radix sort eight bytes 991 / 997 25.2 39.6 15.7X + radix sort key prefix array 1540 / 1563 16.2 61.6 10.1X + */ + } +} From e868a15a7a3ce8895092131f45110c27b734bfb7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 4 May 2016 11:02:48 -0700 Subject: [PATCH 065/313] [SPARK-15103][SQL] Refactored FileCatalog class to allow StreamFileCatalog to infer partitioning ## What changes were proposed in this pull request? File Stream Sink writes the list of written files in a metadata log. StreamFileCatalog reads the list of the files for processing. However StreamFileCatalog does not infer partitioning like HDFSFileCatalog. This PR enables that by refactoring HDFSFileCatalog to create an abstract class PartitioningAwareFileCatalog, that has all the functionality to infer partitions from a list of leaf files. - HDFSFileCatalog has been renamed to ListingFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from recursive directory scanning. - StreamFileCatalog has been renamed to MetadataLogFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from the metadata log. - The above two classes has been moved into their own files as they are not interfaces that should be in fileSourceInterfaces.scala. ## How was this patch tested? - FileStreamSinkSuite was update to see if partitioning gets inferred, and on reading whether the partitions get pruned correctly based on the query. - Other unit tests are unchanged and pass as expected. Author: Tathagata Das Closes #12879 from tdas/SPARK-15103. (cherry picked from commit 0fd3a4748416233f034ec137d95f0a4c8712d396) Signed-off-by: Tathagata Das --- .../execution/datasources/DataSource.scala | 8 +- .../datasources/ListingFileCatalog.scala | 127 +++++++++++ .../PartitioningAwareFileCatalog.scala | 155 +++++++++++++ .../datasources/fileSourceInterfaces.scala | 215 +----------------- .../streaming/MetadataLogFileCatalog.scala | 59 +++++ .../streaming/StreamFileCatalog.scala | 58 ----- .../sql/streaming/FileStreamSinkSuite.scala | 64 +++++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 9 +- 8 files changed, 410 insertions(+), 285 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6114142cefea..618ea3d669bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -136,7 +136,7 @@ case class DataSource( val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) SparkHadoopUtil.get.globPathIfNecessary(qualified) }.toArray - val fileCatalog: FileCatalog = new HDFSFileCatalog(sparkSession, options, globbedPaths, None) + val fileCatalog = new ListingFileCatalog(sparkSession, globbedPaths, options, None) format.inferSchema( sparkSession, caseInsensitiveOptions, @@ -258,7 +258,7 @@ case class DataSource( case (format: FileFormat, _) if hasMetadata(caseInsensitiveOptions.get("path").toSeq ++ paths) => val basePath = new Path((caseInsensitiveOptions.get("path").toSeq ++ paths).head) - val fileCatalog = new StreamFileCatalog(sparkSession, basePath) + val fileCatalog = new MetadataLogFileCatalog(sparkSession, basePath) val dataSchema = userSpecifiedSchema.orElse { format.inferSchema( sparkSession, @@ -310,8 +310,8 @@ case class DataSource( }) } - val fileCatalog: FileCatalog = - new HDFSFileCatalog(sparkSession, options, globbedPaths, partitionSchema) + val fileCatalog = + new ListingFileCatalog(sparkSession, globbedPaths, options, partitionSchema) val dataSchema = userSpecifiedSchema.map { schema => val equality = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala new file mode 100644 index 000000000000..bdf43e02f4a0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import scala.collection.mutable +import scala.util.Try + +import org.apache.hadoop.fs.{FileStatus, LocatedFileStatus, Path} +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType + + +/** + * A [[FileCatalog]] that generates the list of files to process by recursively listing all the + * files present in `paths`. + * + * @param parameters as set of options to control discovery + * @param paths a list of paths to scan + * @param partitionSchema an optional partition schema that will be use to provide types for the + * discovered partitions + */ +class ListingFileCatalog( + sparkSession: SparkSession, + override val paths: Seq[Path], + parameters: Map[String, String], + partitionSchema: Option[StructType]) + extends PartitioningAwareFileCatalog(sparkSession, parameters, partitionSchema) { + + @volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _ + @volatile private var cachedLeafDirToChildrenFiles: Map[Path, Array[FileStatus]] = _ + @volatile private var cachedPartitionSpec: PartitionSpec = _ + + refresh() + + override def partitionSpec(): PartitionSpec = { + if (cachedPartitionSpec == null) { + cachedPartitionSpec = inferPartitioning() + } + cachedPartitionSpec + } + + override protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = { + cachedLeafFiles + } + + override protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = { + cachedLeafDirToChildrenFiles + } + + override def refresh(): Unit = { + val files = listLeafFiles(paths) + cachedLeafFiles = + new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f) + cachedLeafDirToChildrenFiles = files.toArray.groupBy(_.getPath.getParent) + cachedPartitionSpec = null + } + + protected def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext) + } else { + val statuses: Seq[FileStatus] = paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + logInfo(s"Listing $path on driver") + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + + val statuses = { + val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) + if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats + } + + statuses.map { + case f: LocatedFileStatus => f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should a + // a big deal since we always use to `listLeafFilesInParallel` when the number of paths + // exceeds threshold. + case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + } + }.filterNot { status => + val name = status.getPath.getName + HadoopFsRelation.shouldFilterOut(name) + } + + val (dirs, files) = statuses.partition(_.isDirectory) + + // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) + if (dirs.isEmpty) { + mutable.LinkedHashSet(files: _*) + } else { + mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) + } + } + } + + override def equals(other: Any): Boolean = other match { + case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet + case _ => false + } + + override def hashCode(): Int = paths.toSet.hashCode() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala new file mode 100644 index 000000000000..9d997d628579 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{expressions, InternalRow} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.{StringType, StructType} + + +/** + * An abstract class that represents [[FileCatalog]]s that are aware of partitioned tables. + * It provides the necessary methods to parse partition data based on a set of files. + * + * @param parameters as set of options to control partition discovery + * @param partitionSchema an optional partition schema that will be use to provide types for the + * discovered partitions +*/ +abstract class PartitioningAwareFileCatalog( + sparkSession: SparkSession, + parameters: Map[String, String], + partitionSchema: Option[StructType]) + extends FileCatalog with Logging { + + protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) + + protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] + + protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] + + override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + if (partitionSpec().partitionColumns.isEmpty) { + Partition(InternalRow.empty, allFiles().filterNot(_.getPath.getName startsWith "_")) :: Nil + } else { + prunePartitions(filters, partitionSpec()).map { + case PartitionDirectory(values, path) => + Partition( + values, + leafDirToChildrenFiles(path).filterNot(_.getPath.getName startsWith "_")) + } + } + } + + override def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq + + protected def inferPartitioning(): PartitionSpec = { + // We use leaf dirs containing data files to discover the schema. + val leafDirs = leafDirToChildrenFiles.keys.toSeq + partitionSchema match { + case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => + val spec = PartitioningUtils.parsePartitions( + leafDirs, + PartitioningUtils.DEFAULT_PARTITION_NAME, + typeInference = false, + basePaths = basePaths) + + // Without auto inference, all of value in the `row` should be null or in StringType, + // we need to cast into the data type that user specified. + def castPartitionValuesToUserSchema(row: InternalRow) = { + InternalRow((0 until row.numFields).map { i => + Cast( + Literal.create(row.getUTF8String(i), StringType), + userProvidedSchema.fields(i).dataType).eval() + }: _*) + } + + PartitionSpec(userProvidedSchema, spec.partitions.map { part => + part.copy(values = castPartitionValuesToUserSchema(part.values)) + }) + case _ => + PartitioningUtils.parsePartitions( + leafDirs, + PartitioningUtils.DEFAULT_PARTITION_NAME, + typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled(), + basePaths = basePaths) + } + } + + private def prunePartitions( + predicates: Seq[Expression], + partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { + val PartitionSpec(partitionColumns, partitions) = partitionSpec + val partitionColumnNames = partitionColumns.map(_.name).toSet + val partitionPruningPredicates = predicates.filter { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + + if (partitionPruningPredicates.nonEmpty) { + val predicate = partitionPruningPredicates.reduce(expressions.And) + + val boundPredicate = InterpretedPredicate.create(predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }) + + val selected = partitions.filter { + case PartitionDirectory(values, _) => boundPredicate(values) + } + logInfo { + val total = partitions.length + val selectedSize = selected.length + val percentPruned = (1 - selectedSize.toDouble / total.toDouble) * 100 + s"Selected $selectedSize partitions out of $total, pruned $percentPruned% partitions." + } + + selected + } else { + partitions + } + } + + /** + * Contains a set of paths that are considered as the base dirs of the input datasets. + * The partitioning discovery logic will make sure it will stop when it reaches any + * base path. By default, the paths of the dataset provided by users will be base paths. + * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path + * will be `/path/something=true/`, and the returned DataFrame will not contain a column of + * `something`. If users want to override the basePath. They can set `basePath` in the options + * to pass the new base path to the data source. + * For the above example, if the user-provided base path is `/path/`, the returned + * DataFrame will have the column of `something`. + */ + private def basePaths: Set[Path] = { + val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) + userDefinedBasePath.getOrElse { + // If the user does not provide basePath, we will just use paths. + paths.toSet + }.map { hdfsPath => + // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). + val fs = hdfsPath.getFileSystem(hadoopConf) + hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 24e2bf6d136a..c87e67296109 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -304,232 +304,31 @@ case class Partition(values: InternalRow, files: Seq[FileStatus]) * as the partitioning characteristics of those files. */ trait FileCatalog { + + /** Returns the list of input paths from which the catalog will get files. */ def paths: Seq[Path] + /** Returns the specification of the partitions inferred from the data. */ def partitionSpec(): PartitionSpec /** * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with not partition values. + * unpartitioned, this will return a single partition with no partition values. * - * @param filters the filters used to prune which partitions are returned. These filters must + * @param filters The filters used to prune which partitions are returned. These filters must * only refer to partition columns and this method will only return files * where these predicates are guaranteed to evaluate to `true`. Thus, these * filters will not need to be evaluated again on the returned data. */ def listFiles(filters: Seq[Expression]): Seq[Partition] + /** Returns all the valid files. */ def allFiles(): Seq[FileStatus] - def getStatus(path: Path): Array[FileStatus] - + /** Refresh the file listing */ def refresh(): Unit } -/** - * A file catalog that caches metadata gathered by scanning all the files present in `paths` - * recursively. - * - * @param parameters as set of options to control discovery - * @param paths a list of paths to scan - * @param partitionSchema an optional partition schema that will be use to provide types for the - * discovered partitions - */ -class HDFSFileCatalog( - sparkSession: SparkSession, - parameters: Map[String, String], - override val paths: Seq[Path], - partitionSchema: Option[StructType]) - extends FileCatalog with Logging { - - private val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) - - var leafFiles = mutable.LinkedHashMap.empty[Path, FileStatus] - var leafDirToChildrenFiles = mutable.Map.empty[Path, Array[FileStatus]] - var cachedPartitionSpec: PartitionSpec = _ - - def partitionSpec(): PartitionSpec = { - if (cachedPartitionSpec == null) { - cachedPartitionSpec = inferPartitioning(partitionSchema) - } - - cachedPartitionSpec - } - - refresh() - - override def listFiles(filters: Seq[Expression]): Seq[Partition] = { - if (partitionSpec().partitionColumns.isEmpty) { - Partition(InternalRow.empty, allFiles().filterNot(_.getPath.getName startsWith "_")) :: Nil - } else { - prunePartitions(filters, partitionSpec()).map { - case PartitionDirectory(values, path) => - Partition( - values, - getStatus(path).filterNot(_.getPath.getName startsWith "_")) - } - } - } - - protected def prunePartitions( - predicates: Seq[Expression], - partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { - val PartitionSpec(partitionColumns, partitions) = partitionSpec - val partitionColumnNames = partitionColumns.map(_.name).toSet - val partitionPruningPredicates = predicates.filter { - _.references.map(_.name).toSet.subsetOf(partitionColumnNames) - } - - if (partitionPruningPredicates.nonEmpty) { - val predicate = partitionPruningPredicates.reduce(expressions.And) - - val boundPredicate = InterpretedPredicate.create(predicate.transform { - case a: AttributeReference => - val index = partitionColumns.indexWhere(a.name == _.name) - BoundReference(index, partitionColumns(index).dataType, nullable = true) - }) - - val selected = partitions.filter { - case PartitionDirectory(values, _) => boundPredicate(values) - } - logInfo { - val total = partitions.length - val selectedSize = selected.length - val percentPruned = (1 - selectedSize.toDouble / total.toDouble) * 100 - s"Selected $selectedSize partitions out of $total, pruned $percentPruned% partitions." - } - - selected - } else { - partitions - } - } - - def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq - - def getStatus(path: Path): Array[FileStatus] = leafDirToChildrenFiles(path) - - private def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext) - } else { - val statuses: Seq[FileStatus] = paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - logInfo(s"Listing $path on driver") - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - - val statuses = { - val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) - if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats - } - - statuses.map { - case f: LocatedFileStatus => f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should a - // a big deal since we always use to `listLeafFilesInParallel` when the number of paths - // exceeds threshold. - case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) - } - }.filterNot { status => - val name = status.getPath.getName - HadoopFsRelation.shouldFilterOut(name) - } - - val (dirs, files) = statuses.partition(_.isDirectory) - - // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) - if (dirs.isEmpty) { - mutable.LinkedHashSet(files: _*) - } else { - mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) - } - } - } - - def inferPartitioning(schema: Option[StructType]): PartitionSpec = { - // We use leaf dirs containing data files to discover the schema. - val leafDirs = leafDirToChildrenFiles.keys.toSeq - schema match { - case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => - val spec = PartitioningUtils.parsePartitions( - leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = false, - basePaths = basePaths) - - // Without auto inference, all of value in the `row` should be null or in StringType, - // we need to cast into the data type that user specified. - def castPartitionValuesToUserSchema(row: InternalRow) = { - InternalRow((0 until row.numFields).map { i => - Cast( - Literal.create(row.getUTF8String(i), StringType), - userProvidedSchema.fields(i).dataType).eval() - }: _*) - } - - PartitionSpec(userProvidedSchema, spec.partitions.map { part => - part.copy(values = castPartitionValuesToUserSchema(part.values)) - }) - case _ => - PartitioningUtils.parsePartitions( - leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled(), - basePaths = basePaths) - } - } - - /** - * Contains a set of paths that are considered as the base dirs of the input datasets. - * The partitioning discovery logic will make sure it will stop when it reaches any - * base path. By default, the paths of the dataset provided by users will be base paths. - * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path - * will be `/path/something=true/`, and the returned DataFrame will not contain a column of - * `something`. If users want to override the basePath. They can set `basePath` in the options - * to pass the new base path to the data source. - * For the above example, if the user-provided base path is `/path/`, the returned - * DataFrame will have the column of `something`. - */ - private def basePaths: Set[Path] = { - val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) - userDefinedBasePath.getOrElse { - // If the user does not provide basePath, we will just use paths. - paths.toSet - }.map { hdfsPath => - // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). - val fs = hdfsPath.getFileSystem(hadoopConf) - hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - } - } - - def refresh(): Unit = { - val files = listLeafFiles(paths) - - leafFiles.clear() - leafDirToChildrenFiles.clear() - - leafFiles ++= files.map(f => f.getPath -> f) - leafDirToChildrenFiles ++= files.toArray.groupBy(_.getPath.getParent) - - cachedPartitionSpec = null - } - - override def equals(other: Any): Boolean = other match { - case hdfs: HDFSFileCatalog => paths.toSet == hdfs.paths.toSet - case _ => false - } - - override def hashCode(): Int = paths.toSet.hashCode() -} /** * Helper methods for gathering metadata from HDFS. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala new file mode 100644 index 000000000000..20ade12e3796 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources._ + + +/** + * A [[FileCatalog]] that generates the list of files to processing by reading them from the + * metadata log files generated by the [[FileStreamSink]]. + */ +class MetadataLogFileCatalog(sparkSession: SparkSession, path: Path) + extends PartitioningAwareFileCatalog(sparkSession, Map.empty, None) { + + private val metadataDirectory = new Path(path, FileStreamSink.metadataDir) + logInfo(s"Reading streaming file log from $metadataDirectory") + private val metadataLog = new FileStreamSinkLog(sparkSession, metadataDirectory.toUri.toString) + private val allFilesFromLog = metadataLog.allFiles().map(_.toFileStatus).filterNot(_.isDirectory) + private var cachedPartitionSpec: PartitionSpec = _ + + override protected val leafFiles: mutable.LinkedHashMap[Path, FileStatus] = { + new mutable.LinkedHashMap ++= allFilesFromLog.map(f => f.getPath -> f) + } + + override protected val leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = { + allFilesFromLog.toArray.groupBy(_.getPath.getParent) + } + + override def paths: Seq[Path] = path :: Nil + + override def refresh(): Unit = { } + + override def partitionSpec(): PartitionSpec = { + if (cachedPartitionSpec == null) { + cachedPartitionSpec = inferPartitioning() + } + cachedPartitionSpec + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala deleted file mode 100644 index 4f699719c276..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming - -import org.apache.hadoop.fs.{FileStatus, Path} - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.datasources.{FileCatalog, Partition, PartitionSpec} -import org.apache.spark.sql.types.StructType - -class StreamFileCatalog(sparkSession: SparkSession, path: Path) extends FileCatalog with Logging { - val metadataDirectory = new Path(path, FileStreamSink.metadataDir) - logInfo(s"Reading streaming file log from $metadataDirectory") - val metadataLog = new FileStreamSinkLog(sparkSession, metadataDirectory.toUri.toString) - val fs = path.getFileSystem(sparkSession.sessionState.newHadoopConf()) - - override def paths: Seq[Path] = path :: Nil - - override def partitionSpec(): PartitionSpec = PartitionSpec(StructType(Nil), Nil) - - /** - * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with not partition values. - * - * @param filters the filters used to prune which partitions are returned. These filters must - * only refer to partition columns and this method will only return files - * where these predicates are guaranteed to evaluate to `true`. Thus, these - * filters will not need to be evaluated again on the returned data. - */ - override def listFiles(filters: Seq[Expression]): Seq[Partition] = - Partition(InternalRow.empty, allFiles()) :: Nil - - override def getStatus(path: Path): Array[FileStatus] = fs.listStatus(path) - - override def refresh(): Unit = {} - - override def allFiles(): Seq[FileStatus] = { - metadataLog.allFiles().map(_.toFileStatus) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 609ca976a016..e937fc3e876e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -22,12 +22,14 @@ import java.io.File import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.{DirectoryFileFilter, RegexFileFilter} -import org.apache.spark.sql.{ContinuousQuery, Row, StreamTest} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream} +import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream, MetadataLogFileCatalog} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils class FileStreamSinkSuite extends StreamTest with SharedSQLContext { @@ -147,7 +149,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { } } - test("FileStreamSink - partitioned writing and batch reading [IGNORES PARTITION COLUMN]") { + test("FileStreamSink - partitioned writing and batch reading") { val inputData = MemoryStream[Int] val ds = inputData.toDS() @@ -157,7 +159,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { var query: ContinuousQuery = null try { - query = + query = ds.map(i => (i, i * 1000)) .toDF("id", "value") .write @@ -171,12 +173,58 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { query.processAllAvailable() } - // TODO (tdas): Test partition column can be read or not val outputDf = sqlContext.read.parquet(outputDir) + val expectedSchema = new StructType() + .add(StructField("value", IntegerType)) + .add(StructField("id", IntegerType)) + assert(outputDf.schema === expectedSchema) + + // Verify that MetadataLogFileCatalog is being used and the correct partitioning schema has + // been inferred + val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { + case LogicalRelation(baseRelation, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => + baseRelation.asInstanceOf[HadoopFsRelation] + } + assert(hadoopdFsRelations.size === 1) + assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileCatalog]) + assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id")) + assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value")) + + // Verify the data is correctly read checkDataset( - outputDf.as[Int], - 1000, 2000, 3000) + outputDf.as[(Int, Int)], + (1000, 1), (2000, 2), (3000, 3)) + + /** Check some condition on the partitions of the FileScanRDD generated by a DF */ + def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = { + val getFileScanRDD = df.queryExecution.executedPlan.collect { + case scan: DataSourceScanExec if scan.rdd.isInstanceOf[FileScanRDD] => + scan.rdd.asInstanceOf[FileScanRDD] + }.headOption.getOrElse { + fail(s"No FileScan in query\n${df.queryExecution}") + } + func(getFileScanRDD.filePartitions) + } + // Read without pruning + checkFileScanPartitions(outputDf) { partitions => + // There should be as many distinct partition values as there are distinct ids + assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3) + } + + // Read with pruning, should read only files in partition dir id=1 + checkFileScanPartitions(outputDf.filter("id = 1")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(filesToBeRead.map(_.filePath).forall(_.contains("/id=1/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 1) + } + + // Read with pruning, should read only files in partition dir id=1 and id=2 + checkFileScanPartitions(outputDf.filter("id in (1,2)")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(!filesToBeRead.map(_.filePath).exists(_.contains("/id=3/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 2) + } } finally { if (query != null) { query.stop() 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 0b24d358548b..7a799b6c87e9 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 @@ -536,17 +536,12 @@ private[hive] class MetaStoreFileCatalog( sparkSession: SparkSession, paths: Seq[Path], partitionSpecFromHive: PartitionSpec) - extends HDFSFileCatalog( + extends ListingFileCatalog( sparkSession, - Map.empty, paths, + Map.empty, Some(partitionSpecFromHive.partitionColumns)) { - override def getStatus(path: Path): Array[FileStatus] = { - val fs = path.getFileSystem(sparkSession.sessionState.newHadoopConf()) - fs.listStatus(path) - } - override def partitionSpec(): PartitionSpec = partitionSpecFromHive } From 45862f6c935c36969a62a3fbb863cce55c4a6426 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 May 2016 14:26:05 -0700 Subject: [PATCH 066/313] [SPARK-15126][SQL] RuntimeConfig.set should return Unit ## What changes were proposed in this pull request? Currently we return RuntimeConfig itself to facilitate chaining. However, it makes the output in interactive environments (e.g. notebooks, scala repl) weird because it'd show the response of calling set as a RuntimeConfig itself. ## How was this patch tested? Updated unit tests. Author: Reynold Xin Closes #12902 from rxin/SPARK-15126. (cherry picked from commit 6ae9fc00ed6ef530a9c42c8407fc66fd873239cc) Signed-off-by: Andrew Or --- python/pyspark/sql/conf.py | 1 - python/pyspark/sql/session.py | 3 --- .../org/apache/spark/sql/RuntimeConfig.scala | 7 +++---- .../sql/{internal => }/RuntimeConfigSuite.scala | 16 ++++++++-------- 4 files changed, 11 insertions(+), 16 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/{internal => }/RuntimeConfigSuite.scala (85%) diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index 7428c919915f..609d882a95a3 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -23,7 +23,6 @@ class RuntimeConfig(object): """User-facing configuration API, accessible through `SparkSession.conf`. Options set here are automatically propagated to the Hadoop configuration during I/O. - This a thin wrapper around its Scala implementation org.apache.spark.sql.RuntimeConfig. """ def __init__(self, jconf): diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index fb3e318163e8..04842f6185c7 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -71,9 +71,6 @@ class SparkSession(object): .config("spark.some.config.option", "some-value") \ .getOrCreate() - :param sparkContext: The :class:`SparkContext` backing this SparkSession. - :param jsparkSession: An optional JVM Scala SparkSession. If set, we do not instantiate a new - SparkSession in the JVM, instead we make all calls to this object. """ class Builder(object): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 4fd6e42640d2..7e07e0cb84a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -35,9 +35,8 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * * @since 2.0.0 */ - def set(key: String, value: String): RuntimeConfig = { + def set(key: String, value: String): Unit = { sqlConf.setConfString(key, value) - this } /** @@ -45,7 +44,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * * @since 2.0.0 */ - def set(key: String, value: Boolean): RuntimeConfig = { + def set(key: String, value: Boolean): Unit = { set(key, value.toString) } @@ -54,7 +53,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * * @since 2.0.0 */ - def set(key: String, value: Long): RuntimeConfig = { + def set(key: String, value: Long): Unit = { set(key, value.toString) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala similarity index 85% rename from sql/core/src/test/scala/org/apache/spark/sql/internal/RuntimeConfigSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index a629b73ac046..cfe2e9f2dbc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.internal +package org.apache.spark.sql import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.RuntimeConfig class RuntimeConfigSuite extends SparkFunSuite { @@ -26,10 +25,9 @@ class RuntimeConfigSuite extends SparkFunSuite { test("set and get") { val conf = newConf() - conf - .set("k1", "v1") - .set("k2", 2) - .set("k3", value = false) + conf.set("k1", "v1") + conf.set("k2", 2) + conf.set("k3", value = false) assert(conf.get("k1") == "v1") assert(conf.get("k2") == "2") @@ -41,13 +39,15 @@ class RuntimeConfigSuite extends SparkFunSuite { } test("getOption") { - val conf = newConf().set("k1", "v1") + val conf = newConf() + conf.set("k1", "v1") assert(conf.getOption("k1") == Some("v1")) assert(conf.getOption("notset") == None) } test("unset") { - val conf = newConf().set("k1", "v1") + val conf = newConf() + conf.set("k1", "v1") assert(conf.get("k1") == "v1") conf.unset("k1") intercept[NoSuchElementException] { From eeb18f6d70bc75f6d1292938292ad066d85ced8a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 4 May 2016 14:28:26 -0700 Subject: [PATCH 067/313] [SPARK-15121] Improve logging of external shuffle handler ## What changes were proposed in this pull request? Add more informative logging in the external shuffle service to aid in debugging who is connecting to the YARN Nodemanager when the external shuffle service runs under it. ## How was this patch tested? Ran and saw logs coming out in log file. Author: Thomas Graves Closes #12900 from tgravescs/SPARK-15121. (cherry picked from commit 0c00391f77359efdbf9dbd26d4c8186be8839922) Signed-off-by: Andrew Or --- .../spark/network/shuffle/ExternalShuffleBlockHandler.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index f8d03b3b9433..fb1226c09e24 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -35,6 +35,7 @@ import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; import org.apache.spark.network.shuffle.protocol.*; +import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -86,7 +87,8 @@ protected void handleMessage( blocks.add(blockManager.getBlockData(msg.appId, msg.execId, blockId)); } long streamId = streamManager.registerStream(client.getClientId(), blocks.iterator()); - logger.trace("Registered streamId {} with {} buffers", streamId, msg.blockIds.length); + logger.trace("Registered streamId {} with {} buffers for client {} from host {}", streamId, + msg.blockIds.length, client.getClientId(), NettyUtils.getRemoteAddress(client.getChannel())); callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); } else if (msgObj instanceof RegisterExecutor) { From c0715f33b456f8379117a6ecae3ff2cda6f59a7c Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Wed, 4 May 2016 14:29:54 -0700 Subject: [PATCH 068/313] [SPARK-12299][CORE] Remove history serving functionality from Master Remove history server functionality from standalone Master. Previously, the Master process rebuilt a SparkUI once the application was completed which sometimes caused problems, such as OOM, when the application event log is large (see SPARK-6270). Keeping this functionality out of the Master will help to simplify the process and increase stability. Testing for this change included running core unit tests and manually running an application on a standalone cluster to verify that it completed successfully and that the Master UI functioned correctly. Also added 2 unit tests to verify killing an application and driver from MasterWebUI makes the correct request to the Master. Author: Bryan Cutler Closes #10991 from BryanCutler/remove-history-master-SPARK-12299. (cherry picked from commit cf2e9da612397233ae7bca0e9ce57309f16226b5) Signed-off-by: Andrew Or --- .../spark/deploy/master/ApplicationInfo.scala | 9 -- .../apache/spark/deploy/master/Master.scala | 109 +--------------- .../spark/deploy/master/MasterMessages.scala | 2 - .../deploy/master/ui/ApplicationPage.scala | 6 +- .../master/ui/HistoryNotFoundPage.scala | 73 ----------- .../spark/deploy/master/ui/MasterPage.scala | 8 +- .../spark/deploy/master/ui/MasterWebUI.scala | 40 +----- .../spark/status/api/v1/ApiRootResource.scala | 2 +- .../api/v1/ApplicationListResource.scala | 30 ----- .../deploy/master/ui/MasterWebUISuite.scala | 118 +++++++++++------- docs/monitoring.md | 5 - 11 files changed, 86 insertions(+), 316 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 4ffb5283e99a..53564d0e9515 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -41,7 +41,6 @@ private[spark] class ApplicationInfo( @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ - @transient @volatile var appUIUrlAtHistoryServer: Option[String] = None // A cap on the number of executors this application can have at any given time. // By default, this is infinite. Only after the first allocation request is issued by the @@ -66,7 +65,6 @@ private[spark] class ApplicationInfo( nextExecutorId = 0 removedExecutors = new ArrayBuffer[ExecutorDesc] executorLimit = desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE) - appUIUrlAtHistoryServer = None } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -136,11 +134,4 @@ private[spark] class ApplicationInfo( System.currentTimeMillis() - startTime } } - - /** - * Returns the original application UI url unless there is its address at history server - * is defined - */ - def curAppUIUrl: String = appUIUrlAtHistoryServer.getOrElse(desc.appUiUrl) - } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index edc9be2a8a8c..faed4f4dc957 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -17,25 +17,17 @@ package org.apache.spark.deploy.master -import java.io.FileNotFoundException -import java.net.URLEncoder import java.text.SimpleDateFormat import java.util.Date -import java.util.concurrent.{ConcurrentHashMap, ScheduledFuture, TimeUnit} +import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration -import scala.language.postfixOps import scala.util.Random -import org.apache.hadoop.fs.Path - import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ -import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI @@ -43,9 +35,7 @@ import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ -import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.serializer.{JavaSerializer, Serializer} -import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ThreadUtils, Utils} private[deploy] class Master( @@ -59,10 +49,6 @@ private[deploy] class Master( private val forwardMessageThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("master-forward-message-thread") - private val rebuildUIThread = - ThreadUtils.newDaemonSingleThreadExecutor("master-rebuild-ui-thread") - private val rebuildUIContext = ExecutionContext.fromExecutor(rebuildUIThread) - private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs @@ -85,8 +71,6 @@ private[deploy] class Master( private val addressToApp = new HashMap[RpcAddress, ApplicationInfo] private val completedApps = new ArrayBuffer[ApplicationInfo] private var nextAppNumber = 0 - // Using ConcurrentHashMap so that master-rebuild-ui-thread can add a UI after asyncRebuildUI - private val appIdToUI = new ConcurrentHashMap[String, SparkUI] private val drivers = new HashSet[DriverInfo] private val completedDrivers = new ArrayBuffer[DriverInfo] @@ -199,7 +183,6 @@ private[deploy] class Master( checkForWorkerTimeOutTask.cancel(true) } forwardMessageThread.shutdownNow() - rebuildUIThread.shutdownNow() webUi.stop() restServer.foreach(_.stop()) masterMetricsSystem.stop() @@ -391,9 +374,6 @@ private[deploy] class Master( case CheckForWorkerTimeOut => timeOutDeadWorkers() - case AttachCompletedRebuildUI(appId) => - // An asyncRebuildSparkUI has completed, so need to attach to master webUi - Option(appIdToUI.get(appId)).foreach { ui => webUi.attachSparkUI(ui) } } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -844,7 +824,6 @@ private[deploy] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach { a => - Option(appIdToUI.remove(a.id)).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) } completedApps.trimStart(toRemove) @@ -852,9 +831,6 @@ private[deploy] class Master( completedApps += app // Remember it in our history waitingApps -= app - // If application events are logged, use them to rebuild the UI - asyncRebuildSparkUI(app) - for (exec <- app.executors.values) { killExecutor(exec) } @@ -953,89 +929,6 @@ private[deploy] class Master( exec.state = ExecutorState.KILLED } - /** - * Rebuild a new SparkUI from the given application's event logs. - * Return the UI if successful, else None - */ - private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { - val futureUI = asyncRebuildSparkUI(app) - ThreadUtils.awaitResult(futureUI, Duration.Inf) - } - - /** Rebuild a new SparkUI asynchronously to not block RPC event loop */ - private[master] def asyncRebuildSparkUI(app: ApplicationInfo): Future[Option[SparkUI]] = { - val appName = app.desc.name - val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogDir = app.desc.eventLogDir - .getOrElse { - // Event logging is disabled for this application - app.appUIUrlAtHistoryServer = Some(notFoundBasePath) - return Future.successful(None) - } - val futureUI = Future { - val eventLogFilePrefix = EventLoggingListener.getLogPath( - eventLogDir, app.id, appAttemptId = None, compressionCodecName = app.desc.eventLogCodec) - val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) - val inProgressExists = fs.exists(new Path(eventLogFilePrefix + - EventLoggingListener.IN_PROGRESS)) - - val eventLogFile = if (inProgressExists) { - // Event logging is enabled for this application, but the application is still in progress - logWarning(s"Application $appName is still in progress, it may be terminated abnormally.") - eventLogFilePrefix + EventLoggingListener.IN_PROGRESS - } else { - eventLogFilePrefix - } - - val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) - val replayBus = new ReplayListenerBus() - val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime) - try { - replayBus.replay(logInput, eventLogFile, inProgressExists) - } finally { - logInput.close() - } - - Some(ui) - }(rebuildUIContext) - - futureUI.onSuccess { case Some(ui) => - appIdToUI.put(app.id, ui) - // `self` can be null if we are already in the process of shutting down - // This happens frequently in tests where `local-cluster` is used - if (self != null) { - self.send(AttachCompletedRebuildUI(app.id)) - } - // Application UI is successfully rebuilt, so link the Master UI to it - // NOTE - app.appUIUrlAtHistoryServer is volatile - app.appUIUrlAtHistoryServer = Some(ui.basePath) - }(ThreadUtils.sameThread) - - futureUI.onFailure { - case fnf: FileNotFoundException => - // Event logging is enabled for this application, but no event logs are found - val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir.get}." - logWarning(msg) - msg += " Did you specify the correct logging directory?" - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = Some(notFoundBasePath + s"?msg=$msg&title=$title") - - case e: Exception => - // Relay exception message to application UI page - val title = s"Application history load error (${app.id})" - val exception = URLEncoder.encode(Utils.exceptionString(e), "UTF-8") - var msg = s"Exception in replaying log for application $appName!" - logError(msg, e) - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = - Some(notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title") - }(ThreadUtils.sameThread) - - futureUI - } - /** Generate a new app ID given a app's submission date */ private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala index a055d097674c..a952cee36eb4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -39,6 +39,4 @@ private[master] object MasterMessages { case object BoundPortsRequest case class BoundPortsResponse(rpcEndpointPort: Int, webUIPort: Int, restPort: Option[Int]) - - case class AttachCompletedRebuildUI(appId: String) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 96274958d142..8875fc223250 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -75,7 +75,11 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • -
  • Application Detail UI
  • + { + if (!app.isFinished) { +
  • Application Detail UI
  • + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala deleted file mode 100644 index e021f1eef794..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.master.ui - -import java.net.URLDecoder -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.spark.ui.{UIUtils, WebUIPage} - -private[ui] class HistoryNotFoundPage(parent: MasterWebUI) - extends WebUIPage("history/not-found") { - - /** - * Render a page that conveys failure in loading application history. - * - * This accepts 3 HTTP parameters: - * msg = message to display to the user - * title = title of the page - * exception = detailed description of the exception in loading application history (if any) - * - * Parameters "msg" and "exception" are assumed to be UTF-8 encoded. - */ - def render(request: HttpServletRequest): Seq[Node] = { - val titleParam = request.getParameter("title") - val msgParam = request.getParameter("msg") - val exceptionParam = request.getParameter("exception") - - // If no parameters are specified, assume the user did not enable event logging - val defaultTitle = "Event logging is not enabled" - val defaultContent = -
    -
    - No event logs were found for this application! To - enable event logging, - set spark.eventLog.enabled to true and - spark.eventLog.dir to the directory to which your - event logs are written. -
    -
    - - val title = Option(titleParam).getOrElse(defaultTitle) - val content = Option(msgParam) - .map { msg => URLDecoder.decode(msg, "UTF-8") } - .map { msg => -
    -
    {msg}
    -
    ++ - Option(exceptionParam) - .map { e => URLDecoder.decode(e, "UTF-8") } - .map { e =>
    {e}
    } - .getOrElse(Seq.empty) - }.getOrElse(defaultContent) - - UIUtils.basicSparkPage(content, title) - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 363f4b84f885..75de3ede78b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -206,7 +206,13 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {killLink} - {app.desc.name} + { + if (app.isFinished) { + app.desc.name + } else { + {app.desc.name} + } + } {app.coresGranted} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index ae16ce90c84b..a0727ad83fb6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -19,8 +19,6 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging -import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, - UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -30,60 +28,26 @@ import org.apache.spark.ui.JettyUtils._ private[master] class MasterWebUI( val master: Master, - requestedPort: Int, - customMasterPage: Option[MasterPage] = None) + requestedPort: Int) extends WebUI(master.securityMgr, master.securityMgr.getSSLOptions("standalone"), - requestedPort, master.conf, name = "MasterUI") with Logging with UIRoot { + requestedPort, master.conf, name = "MasterUI") with Logging { val masterEndpointRef = master.self val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) - val masterPage = customMasterPage.getOrElse(new MasterPage(this)) - initialize() /** Initialize all components of the server. */ def initialize() { val masterPage = new MasterPage(this) attachPage(new ApplicationPage(this)) - attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(ApiRootResource.getServletHandler(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST"))) } - - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - ui.getHandlers.foreach(attachHandler) - } - - /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - ui.getHandlers.foreach(detachHandler) - } - - def getApplicationInfoList: Iterator[ApplicationInfo] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++ - completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) } - } - - def getSparkUI(appId: String): Option[SparkUI] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - (activeApps ++ completedApps).find { _.id == appId }.flatMap { - master.rebuildSparkUI - } - } } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index ba9cd711f18e..2cd51a9ed541 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -205,7 +205,7 @@ private[spark] object ApiRootResource { /** * This trait is shared by the all the root containers for application UI information -- - * the HistoryServer, the Master UI, and the application UI. This provides the common + * the HistoryServer and the application UI. This provides the common * interface needed for them all to expose application info as json. */ private[spark] trait UIRoot { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 0f3018368246..02fd2985fa20 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -21,7 +21,6 @@ import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam} import javax.ws.rs.core.MediaType import org.apache.spark.deploy.history.ApplicationHistoryInfo -import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo} @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class ApplicationListResource(uiRoot: UIRoot) { @@ -84,33 +83,4 @@ private[spark] object ApplicationsListResource { } ) } - - def convertApplicationInfo( - internal: InternalApplicationInfo, - completed: Boolean): ApplicationInfo = { - // standalone application info always has just one attempt - new ApplicationInfo( - id = internal.id, - name = internal.desc.name, - coresGranted = Some(internal.coresGranted), - maxCores = internal.desc.maxCores, - coresPerExecutor = internal.desc.coresPerExecutor, - memoryPerExecutorMB = Some(internal.desc.memoryPerExecutorMB), - attempts = Seq(new ApplicationAttemptInfo( - attemptId = None, - startTime = new Date(internal.startTime), - endTime = new Date(internal.endTime), - duration = - if (internal.endTime > 0) { - internal.endTime - internal.startTime - } else { - 0 - }, - lastUpdated = new Date(internal.endTime), - sparkUser = internal.desc.user, - completed = completed - )) - ) - } - } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index 0c9382a92bca..69a460fbc7db 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -17,74 +17,96 @@ package org.apache.spark.deploy.master.ui +import java.io.DataOutputStream +import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets import java.util.Date -import scala.io.Source -import scala.language.postfixOps +import scala.collection.mutable.HashMap -import org.json4s.jackson.JsonMethods._ -import org.json4s.JsonAST.{JInt, JNothing, JString} -import org.mockito.Mockito.{mock, when} -import org.scalatest.BeforeAndAfter +import org.mockito.Mockito.{mock, times, verify, when} +import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.DeployMessages.MasterStateResponse +import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, RequestKillDriver} import org.apache.spark.deploy.DeployTestUtils._ import org.apache.spark.deploy.master._ -import org.apache.spark.rpc.RpcEnv +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} -class MasterWebUISuite extends SparkFunSuite with BeforeAndAfter { +class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { - val masterPage = mock(classOf[MasterPage]) - val master = { - val conf = new SparkConf - val securityMgr = new SecurityManager(conf) - val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, securityMgr) - val master = new Master(rpcEnv, rpcEnv.address, 0, securityMgr, conf) - master - } - val masterWebUI = new MasterWebUI(master, 0, customMasterPage = Some(masterPage)) + val conf = new SparkConf + val securityMgr = new SecurityManager(conf) + val rpcEnv = mock(classOf[RpcEnv]) + val master = mock(classOf[Master]) + val masterEndpointRef = mock(classOf[RpcEndpointRef]) + when(master.securityMgr).thenReturn(securityMgr) + when(master.conf).thenReturn(conf) + when(master.rpcEnv).thenReturn(rpcEnv) + when(master.self).thenReturn(masterEndpointRef) + val masterWebUI = new MasterWebUI(master, 0) - before { + override def beforeAll() { + super.beforeAll() masterWebUI.bind() } - after { + override def afterAll() { masterWebUI.stop() + super.afterAll() } - test("list applications") { - val worker = createWorkerInfo() + test("kill application") { val appDesc = createAppDesc() // use new start date so it isn't filtered by UI val activeApp = new ApplicationInfo( - new Date().getTime, "id", appDesc, new Date(), null, Int.MaxValue) - activeApp.addExecutor(worker, 2) - - val workers = Array[WorkerInfo](worker) - val activeApps = Array(activeApp) - val completedApps = Array[ApplicationInfo]() - val activeDrivers = Array[DriverInfo]() - val completedDrivers = Array[DriverInfo]() - val stateResponse = new MasterStateResponse( - "host", 8080, None, workers, activeApps, completedApps, - activeDrivers, completedDrivers, RecoveryState.ALIVE) - - when(masterPage.getMasterState).thenReturn(stateResponse) - - val resultJson = Source.fromURL( - s"http://localhost:${masterWebUI.boundPort}/api/v1/applications") - .mkString - val parsedJson = parse(resultJson) - val firstApp = parsedJson(0) - - assert(firstApp \ "id" === JString(activeApp.id)) - assert(firstApp \ "name" === JString(activeApp.desc.name)) - assert(firstApp \ "coresGranted" === JInt(2)) - assert(firstApp \ "maxCores" === JInt(4)) - assert(firstApp \ "memoryPerExecutorMB" === JInt(1234)) - assert(firstApp \ "coresPerExecutor" === JNothing) + new Date().getTime, "app-0", appDesc, new Date(), null, Int.MaxValue) + + when(master.idToApp).thenReturn(HashMap[String, ApplicationInfo]((activeApp.id, activeApp))) + + val url = s"http://localhost:${masterWebUI.boundPort}/app/kill/" + val body = convPostDataToString(Map(("id", activeApp.id), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify the master was called to remove the active app + verify(master, times(1)).removeApplication(activeApp, ApplicationState.KILLED) + } + + test("kill driver") { + val activeDriverId = "driver-0" + val url = s"http://localhost:${masterWebUI.boundPort}/driver/kill/" + val body = convPostDataToString(Map(("id", activeDriverId), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify that master was asked to kill driver with the correct id + verify(masterEndpointRef, times(1)).ask[KillDriverResponse](RequestKillDriver(activeDriverId)) } + private def convPostDataToString(data: Map[String, String]): String = { + (for ((name, value) <- data) yield s"$name=$value").mkString("&") + } + + /** + * Send an HTTP request to the given URL using the method and the body specified. + * Return the connection object. + */ + private def sendHttpRequest( + url: String, + method: String, + body: String = ""): HttpURLConnection = { + val conn = new URL(url).openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod(method) + if (body.nonEmpty) { + conn.setDoOutput(true) + conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded") + conn.setRequestProperty("Content-Length", Integer.toString(body.length)) + val out = new DataOutputStream(conn.getOutputStream) + out.write(body.getBytes(StandardCharsets.UTF_8)) + out.close() + } + conn + } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 697962ae3a4c..78a3470abf40 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -27,11 +27,6 @@ in the UI to persisted storage. ## Viewing After the Fact -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over -the course of its lifetime, then the Standalone master's web UI will automatically re-render the -application's UI after the application has finished. - If Spark is run on Mesos or YARN, it is still possible to construct the UI of an application through Spark's history server, provided that the application's event logs exist. You can start the history server by executing: From 23789e358589505121113d504adee97e56c00929 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 4 May 2016 14:31:36 -0700 Subject: [PATCH 069/313] [SPARK-15031][EXAMPLE] Use SparkSession in Scala/Python/Java example. ## What changes were proposed in this pull request? This PR aims to update Scala/Python/Java examples by replacing `SQLContext` with newly added `SparkSession`. - Use **SparkSession Builder Pattern** in 154(Scala 55, Java 52, Python 47) files. - Add `getConf` in Python SparkContext class: `python/pyspark/context.py` - Replace **SQLContext Singleton Pattern** with **SparkSession Singleton Pattern**: - `SqlNetworkWordCount.scala` - `JavaSqlNetworkWordCount.java` - `sql_network_wordcount.py` Now, `SQLContexts` are used only in R examples and the following two Python examples. The python examples are untouched in this PR since it already fails some unknown issue. - `simple_params_example.py` - `aft_survival_regression.py` ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #12809 from dongjoon-hyun/SPARK-15031. (cherry picked from commit cdce4e62a5674e2034e5d395578b1a60e3d8c435) Signed-off-by: Andrew Or --- .../ml/JavaAFTSurvivalRegressionExample.java | 12 ++--- .../spark/examples/ml/JavaALSExample.java | 15 +++--- .../examples/ml/JavaBinarizerExample.java | 15 +++--- .../ml/JavaBisectingKMeansExample.java | 18 +++---- .../examples/ml/JavaBucketizerExample.java | 18 +++---- .../examples/ml/JavaChiSqSelectorExample.java | 15 +++--- .../ml/JavaCountVectorizerExample.java | 19 +++---- .../spark/examples/ml/JavaDCTExample.java | 15 +++--- ...JavaDecisionTreeClassificationExample.java | 13 ++--- .../ml/JavaDecisionTreeRegressionExample.java | 13 ++--- .../examples/ml/JavaDeveloperApiExample.java | 15 ++---- .../ml/JavaElementwiseProductExample.java | 15 +++--- .../JavaEstimatorTransformerParamExample.java | 16 +++--- ...aGradientBoostedTreeClassifierExample.java | 11 ++--- ...vaGradientBoostedTreeRegressorExample.java | 14 ++---- .../examples/ml/JavaIndexToStringExample.java | 18 +++---- .../spark/examples/ml/JavaKMeansExample.java | 14 ++---- .../spark/examples/ml/JavaLDAExample.java | 14 ++---- ...LinearRegressionWithElasticNetExample.java | 13 ++--- .../JavaLogisticRegressionSummaryExample.java | 13 ++--- ...gisticRegressionWithElasticNetExample.java | 13 ++--- .../examples/ml/JavaMaxAbsScalerExample.java | 12 ++--- .../examples/ml/JavaMinMaxScalerExample.java | 12 ++--- ...delSelectionViaCrossValidationExample.java | 16 +++--- ...lectionViaTrainValidationSplitExample.java | 14 ++---- ...MultilayerPerceptronClassifierExample.java | 13 ++--- .../spark/examples/ml/JavaNGramExample.java | 18 +++---- .../examples/ml/JavaNaiveBayesExample.java | 14 ++---- .../examples/ml/JavaNormalizerExample.java | 13 ++--- .../examples/ml/JavaOneHotEncoderExample.java | 18 +++---- .../examples/ml/JavaOneVsRestExample.java | 14 ++---- .../spark/examples/ml/JavaPCAExample.java | 18 +++---- .../examples/ml/JavaPipelineExample.java | 16 ++---- .../ml/JavaPolynomialExpansionExample.java | 17 +++---- .../ml/JavaQuantileDiscretizerExample.java | 29 +++++------ .../examples/ml/JavaRFormulaExample.java | 18 +++---- .../ml/JavaRandomForestClassifierExample.java | 14 ++---- .../ml/JavaRandomForestRegressorExample.java | 14 ++---- .../ml/JavaSQLTransformerExample.java | 19 +++---- .../examples/ml/JavaSimpleParamsExample.java | 14 ++---- .../JavaSimpleTextClassificationPipeline.java | 15 +++--- .../ml/JavaStandardScalerExample.java | 13 ++--- .../ml/JavaStopWordsRemoverExample.java | 18 +++---- .../examples/ml/JavaStringIndexerExample.java | 18 +++---- .../spark/examples/ml/JavaTfIdfExample.java | 18 +++---- .../examples/ml/JavaTokenizerExample.java | 18 +++---- .../ml/JavaVectorAssemblerExample.java | 14 ++---- .../examples/ml/JavaVectorIndexerExample.java | 12 ++--- .../examples/ml/JavaVectorSlicerExample.java | 19 +++---- .../examples/ml/JavaWord2VecExample.java | 19 +++---- .../spark/examples/sql/JavaSparkSQL.java | 33 ++++++------- .../streaming/JavaSqlNetworkWordCount.java | 19 ++++--- examples/src/main/python/ml/als_example.py | 14 +++--- .../src/main/python/ml/binarizer_example.py | 10 ++-- .../python/ml/bisecting_k_means_example.py | 16 +++--- .../src/main/python/ml/bucketizer_example.py | 10 ++-- .../main/python/ml/chisq_selector_example.py | 10 ++-- .../python/ml/count_vectorizer_example.py | 10 ++-- .../src/main/python/ml/cross_validator.py | 49 +++++++++---------- .../src/main/python/ml/dataframe_example.py | 14 +++--- examples/src/main/python/ml/dct_example.py | 10 ++-- .../decision_tree_classification_example.py | 9 ++-- .../ml/decision_tree_regression_example.py | 9 ++-- .../python/ml/elementwise_product_example.py | 10 ++-- .../ml/estimator_transformer_param_example.py | 13 +++-- ...radient_boosted_tree_classifier_example.py | 9 ++-- ...gradient_boosted_tree_regressor_example.py | 9 ++-- .../main/python/ml/index_to_string_example.py | 10 ++-- examples/src/main/python/ml/kmeans_example.py | 16 +++--- .../ml/linear_regression_with_elastic_net.py | 10 ++-- .../logistic_regression_with_elastic_net.py | 10 ++-- .../main/python/ml/max_abs_scaler_example.py | 10 ++-- .../main/python/ml/min_max_scaler_example.py | 10 ++-- .../multilayer_perceptron_classification.py | 12 ++--- examples/src/main/python/ml/n_gram_example.py | 10 ++-- .../src/main/python/ml/naive_bayes_example.py | 11 ++--- .../src/main/python/ml/normalizer_example.py | 10 ++-- .../main/python/ml/onehot_encoder_example.py | 10 ++-- examples/src/main/python/ml/pca_example.py | 10 ++-- .../src/main/python/ml/pipeline_example.py | 13 +++-- .../python/ml/polynomial_expansion_example.py | 10 ++-- .../ml/random_forest_classifier_example.py | 9 ++-- .../ml/random_forest_regressor_example.py | 9 ++-- .../src/main/python/ml/rformula_example.py | 10 ++-- .../ml/simple_text_classification_pipeline.py | 32 ++++++------ .../src/main/python/ml/sql_transformer.py | 10 ++-- .../main/python/ml/standard_scaler_example.py | 10 ++-- .../python/ml/stopwords_remover_example.py | 10 ++-- .../main/python/ml/string_indexer_example.py | 10 ++-- examples/src/main/python/ml/tf_idf_example.py | 10 ++-- .../src/main/python/ml/tokenizer_example.py | 10 ++-- .../main/python/ml/train_validation_split.py | 10 ++-- .../python/ml/vector_assembler_example.py | 10 ++-- .../main/python/ml/vector_indexer_example.py | 10 ++-- .../main/python/ml/vector_slicer_example.py | 10 ++-- .../src/main/python/ml/word2vec_example.py | 10 ++-- .../binary_classification_metrics_example.py | 6 ++- examples/src/main/python/sql.py | 2 +- .../python/streaming/sql_network_wordcount.py | 19 +++---- .../ml/AFTSurvivalRegressionExample.scala | 11 ++--- .../apache/spark/examples/ml/ALSExample.scala | 14 +++--- .../spark/examples/ml/BinarizerExample.scala | 12 ++--- .../spark/examples/ml/BucketizerExample.scala | 11 ++--- .../examples/ml/ChiSqSelectorExample.scala | 14 ++---- .../examples/ml/CountVectorizerExample.scala | 11 ++--- .../apache/spark/examples/ml/DCTExample.scala | 12 ++--- .../spark/examples/ml/DataFrameExample.scala | 14 ++---- .../DecisionTreeClassificationExample.scala | 11 ++--- .../examples/ml/DecisionTreeExample.scala | 18 +++---- .../ml/DecisionTreeRegressionExample.scala | 11 ++--- .../examples/ml/DeveloperApiExample.scala | 17 +++---- .../ml/ElementwiseProductExample.scala | 12 ++--- .../ml/EstimatorTransformerParamExample.scala | 13 ++--- ...GradientBoostedTreeClassifierExample.scala | 11 ++--- .../GradientBoostedTreeRegressorExample.scala | 11 ++--- .../examples/ml/IndexToStringExample.scala | 13 ++--- .../spark/examples/ml/KMeansExample.scala | 11 ++--- .../apache/spark/examples/ml/LDAExample.scala | 13 ++--- ...inearRegressionWithElasticNetExample.scala | 11 ++--- .../ml/LogisticRegressionSummaryExample.scala | 13 ++--- ...isticRegressionWithElasticNetExample.scala | 12 ++--- .../examples/ml/MaxAbsScalerExample.scala | 14 ++---- .../examples/ml/MinMaxScalerExample.scala | 12 ++--- ...elSelectionViaCrossValidationExample.scala | 14 +++--- ...ectionViaTrainValidationSplitExample.scala | 12 ++--- ...ultilayerPerceptronClassifierExample.scala | 11 ++--- .../spark/examples/ml/NGramExample.scala | 12 ++--- .../spark/examples/ml/NaiveBayesExample.scala | 13 +++-- .../spark/examples/ml/NormalizerExample.scala | 12 ++--- .../examples/ml/OneHotEncoderExample.scala | 12 ++--- .../spark/examples/ml/OneVsRestExample.scala | 13 ++--- .../apache/spark/examples/ml/PCAExample.scala | 12 ++--- .../spark/examples/ml/PipelineExample.scala | 13 ++--- .../ml/PolynomialExpansionExample.scala | 12 ++--- .../ml/QuantileDiscretizerExample.scala | 16 +++--- .../spark/examples/ml/RFormulaExample.scala | 12 ++--- .../ml/RandomForestClassifierExample.scala | 11 ++--- .../ml/RandomForestRegressorExample.scala | 11 ++--- .../examples/ml/SQLTransformerExample.scala | 11 ++--- .../examples/ml/SimpleParamsExample.scala | 19 +++---- .../ml/SimpleTextClassificationPipeline.scala | 15 +++--- .../examples/ml/StandardScalerExample.scala | 12 ++--- .../examples/ml/StopWordsRemoverExample.scala | 12 ++--- .../examples/ml/StringIndexerExample.scala | 12 ++--- .../spark/examples/ml/TfIdfExample.scala | 11 ++--- .../spark/examples/ml/TokenizerExample.scala | 12 ++--- .../examples/ml/VectorAssemblerExample.scala | 12 ++--- .../examples/ml/VectorIndexerExample.scala | 12 ++--- .../examples/ml/VectorSlicerExample.scala | 17 +++---- .../spark/examples/ml/Word2VecExample.scala | 11 ++--- .../spark/examples/mllib/LDAExample.scala | 6 +-- .../mllib/RankingMetricsExample.scala | 11 ++--- .../mllib/RegressionMetricsExample.scala | 18 ++++--- .../streaming/SqlNetworkWordCount.scala | 21 ++++---- python/pyspark/context.py | 5 ++ 155 files changed, 852 insertions(+), 1232 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java index 22b93a3a85c5..ecb7084e0312 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java @@ -21,23 +21,19 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.regression.AFTSurvivalRegression; import org.apache.spark.ml.regression.AFTSurvivalRegressionModel; import org.apache.spark.mllib.linalg.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaAFTSurvivalRegressionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaAFTSurvivalRegressionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaAFTSurvivalRegressionExample").getOrCreate(); // $example on$ List data = Arrays.asList( @@ -52,7 +48,7 @@ public static void main(String[] args) { new StructField("censor", DataTypes.DoubleType, false, Metadata.empty()), new StructField("features", new VectorUDT(), false, Metadata.empty()) }); - Dataset training = jsql.createDataFrame(data, schema); + Dataset training = spark.createDataFrame(data, schema); double[] quantileProbabilities = new double[]{0.3, 0.6}; AFTSurvivalRegression aft = new AFTSurvivalRegression() .setQuantileProbabilities(quantileProbabilities) @@ -66,6 +62,6 @@ public static void main(String[] args) { model.transform(training).show(false); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 088037d427f5..9a9a10489b50 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -17,11 +17,9 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.io.Serializable; @@ -83,18 +81,17 @@ public static Rating parseRating(String str) { // $example off$ public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaALSExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaALSExample").getOrCreate(); // $example on$ - JavaRDD ratingsRDD = jsc.textFile("data/mllib/als/sample_movielens_ratings.txt") + JavaRDD ratingsRDD = spark + .read().text("data/mllib/als/sample_movielens_ratings.txt").javaRDD() .map(new Function() { public Rating call(String str) { return Rating.parseRating(str); } }); - Dataset ratings = sqlContext.createDataFrame(ratingsRDD, Rating.class); + Dataset ratings = spark.createDataFrame(ratingsRDD, Rating.class); Dataset[] splits = ratings.randomSplit(new double[]{0.8, 0.2}); Dataset training = splits[0]; Dataset test = splits[1]; @@ -121,6 +118,6 @@ public Rating call(String str) { Double rmse = evaluator.evaluate(predictions); System.out.println("Root-mean-square error = " + rmse); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java index 0a6e9c2a1f93..88e4298a6191 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -20,10 +20,11 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Binarizer; @@ -37,21 +38,19 @@ public class JavaBinarizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBinarizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaBinarizerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, 0.1), RowFactory.create(1, 0.8), RowFactory.create(2, 0.2) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset continuousDataFrame = jsql.createDataFrame(jrdd, schema); + Dataset continuousDataFrame = spark.createDataFrame(data, schema); Binarizer binarizer = new Binarizer() .setInputCol("feature") .setOutputCol("binarized_feature") @@ -63,6 +62,6 @@ public static void main(String[] args) { System.out.println(binarized_value); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 1d1a518bbca1..51aa35084e84 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -18,12 +18,10 @@ package org.apache.spark.examples.ml; import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; @@ -44,25 +42,23 @@ public class JavaBisectingKMeansExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBisectingKMeansExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaBisectingKMeansExample").getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(0.1, 0.1, 0.1)), RowFactory.create(Vectors.dense(0.3, 0.3, 0.25)), RowFactory.create(Vectors.dense(0.1, 0.1, -0.1)), RowFactory.create(Vectors.dense(20.3, 20.1, 19.9)), RowFactory.create(Vectors.dense(20.2, 20.1, 19.7)), RowFactory.create(Vectors.dense(18.9, 20.0, 19.7)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); BisectingKMeans bkm = new BisectingKMeans().setK(2); BisectingKMeansModel model = bkm.fit(dataset); @@ -76,6 +72,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java index 68ffa702ea5e..0c24f52cf58a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Bucketizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,23 +35,21 @@ public class JavaBucketizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBucketizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaBucketizerExample").getOrCreate(); // $example on$ double[] splits = {Double.NEGATIVE_INFINITY, -0.5, 0.0, 0.5, Double.POSITIVE_INFINITY}; - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(-0.5), RowFactory.create(-0.3), RowFactory.create(0.0), RowFactory.create(0.2) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset dataFrame = jsql.createDataFrame(data, schema); + Dataset dataFrame = spark.createDataFrame(data, schema); Bucketizer bucketizer = new Bucketizer() .setInputCol("features") @@ -64,7 +60,7 @@ public static void main(String[] args) { Dataset bucketedData = bucketizer.transform(dataFrame); bucketedData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java index b1bf1cfeb215..684cf9a7145d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java @@ -21,10 +21,11 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.ml.feature.ChiSqSelector; import org.apache.spark.mllib.linalg.VectorUDT; @@ -39,23 +40,21 @@ public class JavaChiSqSelectorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaChiSqSelectorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaChiSqSelectorExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(7, Vectors.dense(0.0, 0.0, 18.0, 1.0), 1.0), RowFactory.create(8, Vectors.dense(0.0, 1.0, 12.0, 0.0), 0.0), RowFactory.create(9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("features", new VectorUDT(), false, Metadata.empty()), new StructField("clicked", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); ChiSqSelector selector = new ChiSqSelector() .setNumTopFeatures(1) @@ -66,6 +65,6 @@ public static void main(String[] args) { Dataset result = selector.fit(df).transform(df); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java index ec3ac202bea4..0631f9d6d5d7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java @@ -19,36 +19,31 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.CountVectorizer; import org.apache.spark.ml.feature.CountVectorizerModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaCountVectorizerExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaCountVectorizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaCountVectorizerExample").getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("a", "b", "c")), RowFactory.create(Arrays.asList("a", "b", "b", "c", "a")) - )); + ); StructType schema = new StructType(new StructField [] { new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); // fit a CountVectorizerModel from the corpus CountVectorizerModel cvModel = new CountVectorizer() @@ -66,6 +61,6 @@ public static void main(String[] args) { cvModel.transform(df).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java index 4b15fde9c35f..ec57a2445138 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java @@ -20,10 +20,11 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.DCT; @@ -38,20 +39,18 @@ public class JavaDCTExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDCTExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaDCTExample").getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(0.0, 1.0, -2.0, 3.0)), RowFactory.create(Vectors.dense(-1.0, 2.0, 4.0, -7.0)), RowFactory.create(Vectors.dense(14.0, -2.0, -5.0, 1.0)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); DCT dct = new DCT() .setInputCol("features") .setOutputCol("featuresDCT") @@ -59,7 +58,7 @@ public static void main(String[] args) { Dataset dctDf = dct.transform(df); dctDf.select("featuresDCT").show(3); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java index 8214952f8069..733bc4181c70 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java @@ -17,8 +17,6 @@ // scalastyle:off println package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -28,18 +26,17 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaDecisionTreeClassificationExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDecisionTreeClassificationExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaDecisionTreeClassificationExample").getOrCreate(); // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - Dataset data = sqlContext + Dataset data = spark .read() .format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); @@ -100,6 +97,6 @@ public static void main(String[] args) { System.out.println("Learned classification tree model:\n" + treeModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java index a4f3e97bf318..bd6dc3edd363 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java @@ -17,8 +17,6 @@ // scalastyle:off println package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -29,17 +27,16 @@ import org.apache.spark.ml.regression.DecisionTreeRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaDecisionTreeRegressionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDecisionTreeRegressionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaDecisionTreeRegressionExample").getOrCreate(); // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - Dataset data = sqlContext.read().format("libsvm") + Dataset data = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. @@ -85,6 +82,6 @@ public static void main(String[] args) { System.out.println("Learned regression tree model:\n" + treeModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 0ba94786d4e5..90023ac06b79 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -21,9 +21,7 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.Classifier; import org.apache.spark.ml.classification.ClassificationModel; import org.apache.spark.ml.param.IntParam; @@ -35,7 +33,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** @@ -51,9 +49,7 @@ public class JavaDeveloperApiExample { public static void main(String[] args) throws Exception { - SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaDeveloperApiExample").getOrCreate(); // Prepare training data. List localTraining = Lists.newArrayList( @@ -61,8 +57,7 @@ public static void main(String[] args) throws Exception { new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - Dataset training = jsql.createDataFrame( - jsc.parallelize(localTraining), LabeledPoint.class); + Dataset training = spark.createDataFrame(localTraining, LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); @@ -80,7 +75,7 @@ public static void main(String[] args) throws Exception { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); + Dataset test = spark.createDataFrame(localTest, LabeledPoint.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). Dataset results = model.transform(test); @@ -93,7 +88,7 @@ public static void main(String[] args) throws Exception { " even though all coefficients are 0!"); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java index 37de9cf3596a..a062a6fcd0c1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -20,7 +20,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.ArrayList; @@ -41,16 +41,15 @@ public class JavaElementwiseProductExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaElementwiseProductExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaElementwiseProductExample").getOrCreate(); // $example on$ // Create some vector data; also works for sparse vectors - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create("a", Vectors.dense(1.0, 2.0, 3.0)), RowFactory.create("b", Vectors.dense(4.0, 5.0, 6.0)) - )); + ); List fields = new ArrayList<>(2); fields.add(DataTypes.createStructField("id", DataTypes.StringType, false)); @@ -58,7 +57,7 @@ public static void main(String[] args) { StructType schema = DataTypes.createStructType(fields); - Dataset dataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset dataFrame = spark.createDataFrame(data, schema); Vector transformingVector = Vectors.dense(0.0, 1.0, 2.0); @@ -70,6 +69,6 @@ public static void main(String[] args) { // Batch transform the vectors to create new column: transformer.transform(dataFrame).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java index 604b193dd489..5ba8e6cf4490 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java @@ -21,8 +21,6 @@ import java.util.Arrays; // $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; // $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.LogisticRegressionModel; @@ -32,23 +30,21 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for Estimator, Transformer, and Param. */ public class JavaEstimatorTransformerParamExample { public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("JavaEstimatorTransformerParamExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession + .builder().appName("JavaEstimatorTransformerParamExample").getOrCreate(); // $example on$ // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans into // DataFrames, where it uses the bean metadata to infer the schema. - Dataset training = sqlContext.createDataFrame( + Dataset training = spark.createDataFrame( Arrays.asList( new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), @@ -89,7 +85,7 @@ public static void main(String[] args) { System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); // Prepare test documents. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)) @@ -107,6 +103,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java index 553070dace88..a7c89b9d1997 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -29,18 +29,17 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaGradientBoostedTreeClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaGradientBoostedTreeClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaGradientBoostedTreeClassifierExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = sqlContext.read().format("libsvm") + Dataset data = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. @@ -99,6 +98,6 @@ public static void main(String[] args) { System.out.println("Learned classification GBT model:\n" + gbtModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java index 83fd89e3bd59..6d3f21fdafa0 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -30,19 +28,17 @@ import org.apache.spark.ml.regression.GBTRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaGradientBoostedTreeRegressorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaGradientBoostedTreeRegressorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaGradientBoostedTreeRegressorExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -87,6 +83,6 @@ public static void main(String[] args) { System.out.println("Learned regression GBT model:\n" + gbtModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java index 9b8c22f3bdfd..ccd74f2920ce 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.ml.feature.IndexToString; import org.apache.spark.ml.feature.StringIndexer; @@ -39,24 +37,22 @@ public class JavaIndexToStringExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaIndexToStringExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaIndexToStringExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("category", DataTypes.StringType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexerModel indexer = new StringIndexer() .setInputCol("category") @@ -70,6 +66,6 @@ public static void main(String[] args) { Dataset converted = converter.transform(indexed); converted.select("id", "originalCategory").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index c5022f4c0b8f..e6d82a0513a3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -19,12 +19,10 @@ import java.util.regex.Pattern; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.expressions.GenericRow; // $example on$ import org.apache.spark.ml.clustering.KMeansModel; @@ -72,16 +70,14 @@ public static void main(String[] args) { int k = Integer.parseInt(args[1]); // Parses the arguments - SparkConf conf = new SparkConf().setAppName("JavaKMeansExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaKMeansExample").getOrCreate(); // $example on$ // Loads data - JavaRDD points = jsc.textFile(inputFile).map(new ParsePoint()); + JavaRDD points = spark.read().text(inputFile).javaRDD().map(new ParsePoint()); StructField[] fields = {new StructField("features", new VectorUDT(), false, Metadata.empty())}; StructType schema = new StructType(fields); - Dataset dataset = sqlContext.createDataFrame(points, schema); + Dataset dataset = spark.createDataFrame(points, schema); // Trains a k-means model KMeans kmeans = new KMeans() @@ -96,6 +92,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java index 351bc401180c..b8baca5920e8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java @@ -19,9 +19,7 @@ // $example on$ import java.util.regex.Pattern; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.ml.clustering.LDA; import org.apache.spark.ml.clustering.LDAModel; @@ -30,7 +28,7 @@ import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.expressions.GenericRow; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -67,15 +65,13 @@ public static void main(String[] args) { String inputFile = "data/mllib/sample_lda_data.txt"; // Parses the arguments - SparkConf conf = new SparkConf().setAppName("JavaLDAExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaLDAExample").getOrCreate(); // Loads data - JavaRDD points = jsc.textFile(inputFile).map(new ParseVector()); + JavaRDD points = spark.read().text(inputFile).javaRDD().map(new ParseVector()); StructField[] fields = {new StructField("features", new VectorUDT(), false, Metadata.empty())}; StructType schema = new StructType(fields); - Dataset dataset = sqlContext.createDataFrame(points, schema); + Dataset dataset = spark.createDataFrame(points, schema); // Trains a LDA model LDA lda = new LDA() @@ -91,7 +87,7 @@ public static void main(String[] args) { topics.show(false); model.transform(dataset).show(false); - jsc.stop(); + spark.stop(); } // $example off$ } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java index 08fce89359fc..b6ea1fed25f8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.regression.LinearRegression; import org.apache.spark.ml.regression.LinearRegressionModel; @@ -26,18 +24,17 @@ import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaLinearRegressionWithElasticNetExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaLinearRegressionWithElasticNetExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaLinearRegressionWithElasticNetExample").getOrCreate(); // $example on$ // Load training data - Dataset training = sqlContext.read().format("libsvm") + Dataset training = spark.read().format("libsvm") .load("data/mllib/sample_linear_regression_data.txt"); LinearRegression lr = new LinearRegression() @@ -61,6 +58,6 @@ public static void main(String[] args) { System.out.println("r2: " + trainingSummary.r2()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java index 73b028fb4440..fd040aead410 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary; import org.apache.spark.ml.classification.LogisticRegression; @@ -26,18 +24,17 @@ import org.apache.spark.ml.classification.LogisticRegressionTrainingSummary; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.functions; // $example off$ public class JavaLogisticRegressionSummaryExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaLogisticRegressionSummaryExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaLogisticRegressionSummaryExample").getOrCreate(); // Load training data - Dataset training = sqlContext.read().format("libsvm") + Dataset training = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); LogisticRegression lr = new LogisticRegression() @@ -80,6 +77,6 @@ public static void main(String[] args) { lrModel.setThreshold(bestThreshold); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java index 691166852206..f00c7a05cd12 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java @@ -17,25 +17,22 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.LogisticRegressionModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaLogisticRegressionWithElasticNetExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaLogisticRegressionWithElasticNetExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaLogisticRegressionWithElasticNetExample").getOrCreate(); // $example on$ // Load training data - Dataset training = sqlContext.read().format("libsvm") + Dataset training = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); LogisticRegression lr = new LogisticRegression() @@ -51,6 +48,6 @@ public static void main(String[] args) { + lrModel.coefficients() + " Intercept: " + lrModel.intercept()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java index a2a072b253f3..80cdd364b937 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java @@ -17,25 +17,21 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.feature.MaxAbsScaler; import org.apache.spark.ml.feature.MaxAbsScalerModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; public class JavaMaxAbsScalerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaMaxAbsScalerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaMaxAbsScalerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); MaxAbsScaler scaler = new MaxAbsScaler() .setInputCol("features") .setOutputCol("scaledFeatures"); @@ -47,7 +43,7 @@ public static void main(String[] args) { Dataset scaledData = scalerModel.transform(dataFrame); scaledData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java index 4aee18eeabfc..022940fd1e67 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.feature.MinMaxScaler; @@ -30,12 +28,10 @@ public class JavaMinMaxScalerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JaveMinMaxScalerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaMinMaxScalerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); MinMaxScaler scaler = new MinMaxScaler() .setInputCol("features") .setOutputCol("scaledFeatures"); @@ -47,6 +43,6 @@ public static void main(String[] args) { Dataset scaledData = scalerModel.transform(dataFrame); scaledData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java index c4122d1247a9..a4ec4f58154f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java @@ -21,8 +21,6 @@ import java.util.Arrays; // $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineStage; @@ -37,21 +35,19 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for Model Selection via Cross Validation. */ public class JavaModelSelectionViaCrossValidationExample { public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("JavaModelSelectionViaCrossValidationExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession + .builder().appName("JavaModelSelectionViaCrossValidationExample").getOrCreate(); // $example on$ // Prepare training documents, which are labeled. - Dataset training = sqlContext.createDataFrame(Arrays.asList( + Dataset training = spark.createDataFrame(Arrays.asList( new JavaLabeledDocument(0L, "a b c d e spark", 1.0), new JavaLabeledDocument(1L, "b d", 0.0), new JavaLabeledDocument(2L,"spark f g h", 1.0), @@ -102,7 +98,7 @@ public static void main(String[] args) { CrossValidatorModel cvModel = cv.fit(training); // Prepare test documents, which are unlabeled. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new JavaDocument(4L, "spark i j k"), new JavaDocument(5L, "l m n"), new JavaDocument(6L, "mapreduce spark"), @@ -117,6 +113,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java index 4994f8f9fa85..63a0ad1cb883 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator; import org.apache.spark.ml.param.ParamMap; @@ -29,7 +27,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example demonstrating model selection using TrainValidationSplit. @@ -44,13 +42,11 @@ */ public class JavaModelSelectionViaTrainValidationSplitExample { public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("JavaModelSelectionViaTrainValidationSplitExample"); - SparkContext sc = new SparkContext(conf); - SQLContext jsql = new SQLContext(sc); + SparkSession spark = SparkSession + .builder().appName("JavaModelSelectionViaTrainValidationSplitExample").getOrCreate(); // $example on$ - Dataset data = jsql.read().format("libsvm") + Dataset data = spark.read().format("libsvm") .load("data/mllib/sample_linear_regression_data.txt"); // Prepare training and test data. @@ -87,6 +83,6 @@ public static void main(String[] args) { .show(); // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java index 0ca528d8cd07..d547a2a64be5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java @@ -18,11 +18,9 @@ package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel; import org.apache.spark.ml.classification.MultilayerPerceptronClassifier; import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; @@ -34,14 +32,13 @@ public class JavaMultilayerPerceptronClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaMultilayerPerceptronClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaMultilayerPerceptronClassifierExample").getOrCreate(); // $example on$ // Load training data String path = "data/mllib/sample_multiclass_classification_data.txt"; - Dataset dataFrame = jsql.read().format("libsvm").load(path); + Dataset dataFrame = spark.read().format("libsvm").load(path); // Split the data into train and test Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L); Dataset train = splits[0]; @@ -66,6 +63,6 @@ public static void main(String[] args) { System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels)); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java index 608bd8028565..325b7b587480 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java @@ -17,15 +17,13 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.NGram; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -37,16 +35,14 @@ public class JavaNGramExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaNGramExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaNGramExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0.0, Arrays.asList("Hi", "I", "heard", "about", "Spark")), RowFactory.create(1.0, Arrays.asList("I", "wish", "Java", "could", "use", "case", "classes")), RowFactory.create(2.0, Arrays.asList("Logistic", "regression", "models", "are", "neat")) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), @@ -54,7 +50,7 @@ public static void main(String[] args) { "words", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) }); - Dataset wordDataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset wordDataFrame = spark.createDataFrame(data, schema); NGram ngramTransformer = new NGram().setInputCol("words").setOutputCol("ngrams"); @@ -66,6 +62,6 @@ public static void main(String[] args) { System.out.println(); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java index 41d7ad75b9d4..1f24a23609eb 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java @@ -17,16 +17,13 @@ package org.apache.spark.examples.ml; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.NaiveBayes; import org.apache.spark.ml.classification.NaiveBayesModel; import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ /** @@ -35,13 +32,12 @@ public class JavaNaiveBayesExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaNaiveBayesExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaNaiveBayesExample").getOrCreate(); // $example on$ // Load training data - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Split the data into train and test Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L); Dataset train = splits[0]; @@ -59,6 +55,6 @@ public static void main(String[] args) { System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels)); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java index 31cd75213668..4b3a718ea92c 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.feature.Normalizer; @@ -29,12 +27,11 @@ public class JavaNormalizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaNormalizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaNormalizerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Normalize each Vector using $L^1$ norm. Normalizer normalizer = new Normalizer() @@ -50,6 +47,6 @@ public static void main(String[] args) { normalizer.transform(dataFrame, normalizer.p().w(Double.POSITIVE_INFINITY)); lInfNormData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java index 882438ca28eb..d6e4d21ead16 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.OneHotEncoder; import org.apache.spark.ml.feature.StringIndexer; import org.apache.spark.ml.feature.StringIndexerModel; @@ -39,26 +37,24 @@ public class JavaOneHotEncoderExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaOneHotEncoderExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaOneHotEncoderExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.DoubleType, false, Metadata.empty()), new StructField("category", DataTypes.StringType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexerModel indexer = new StringIndexer() .setInputCol("category") @@ -72,7 +68,7 @@ public static void main(String[] args) { Dataset encoded = encoder.transform(indexed); encoded.select("id", "categoryVec").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java index 1f13b48bf82a..9cc983bd115d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java @@ -19,8 +19,6 @@ import org.apache.commons.cli.*; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.OneVsRest; @@ -31,7 +29,7 @@ import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructField; // $example off$ @@ -60,9 +58,7 @@ private static class Params { public static void main(String[] args) { // parse the arguments Params params = parse(args); - SparkConf conf = new SparkConf().setAppName("JavaOneVsRestExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaOneVsRestExample").getOrCreate(); // $example on$ // configure the base classifier @@ -82,7 +78,7 @@ public static void main(String[] args) { OneVsRest ovr = new OneVsRest().setClassifier(classifier); String input = params.input; - Dataset inputData = jsql.read().format("libsvm").load(input); + Dataset inputData = spark.read().format("libsvm").load(input); Dataset train; Dataset test; @@ -92,7 +88,7 @@ public static void main(String[] args) { train = inputData; // compute the number of features in the training set. int numFeatures = inputData.first().getAs(1).size(); - test = jsql.read().format("libsvm").option("numFeatures", + test = spark.read().format("libsvm").option("numFeatures", String.valueOf(numFeatures)).load(testInput); } else { double f = params.fracTest; @@ -131,7 +127,7 @@ public static void main(String[] args) { System.out.println(results); // $example off$ - jsc.stop(); + spark.stop(); } private static Params parse(String[] args) { diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java index a792fd7d47cc..6b1dcb68ba30 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.PCA; import org.apache.spark.ml.feature.PCAModel; import org.apache.spark.mllib.linalg.VectorUDT; @@ -39,22 +37,20 @@ public class JavaPCAExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPCAExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaPCAExample").getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.sparse(5, new int[]{1, 3}, new double[]{1.0, 7.0})), RowFactory.create(Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0)), RowFactory.create(Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); PCAModel pca = new PCA() .setInputCol("features") @@ -65,7 +61,7 @@ public static void main(String[] args) { Dataset result = pca.transform(df).select("pcaFeatures"); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java index 305420f208b7..556a4573263f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java @@ -19,11 +19,7 @@ // $example on$ import java.util.Arrays; -// $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; -// $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -33,20 +29,18 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for simple text document 'Pipeline'. */ public class JavaPipelineExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPipelineExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession.builder().appName("JavaPipelineExample").getOrCreate(); // $example on$ // Prepare training documents, which are labeled. - Dataset training = sqlContext.createDataFrame(Arrays.asList( + Dataset training = spark.createDataFrame(Arrays.asList( new JavaLabeledDocument(0L, "a b c d e spark", 1.0), new JavaLabeledDocument(1L, "b d", 0.0), new JavaLabeledDocument(2L, "spark f g h", 1.0), @@ -71,7 +65,7 @@ public static void main(String[] args) { PipelineModel model = pipeline.fit(training); // Prepare test documents, which are unlabeled. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new JavaDocument(4L, "spark i j k"), new JavaDocument(5L, "l m n"), new JavaDocument(6L, "mapreduce spark"), @@ -86,6 +80,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java index 48fc3c8acb0c..e328454c7059 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -17,15 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.PolynomialExpansion; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -39,9 +36,7 @@ public class JavaPolynomialExpansionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPolynomialExpansionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaPolynomialExpansionExample").getOrCreate(); // $example on$ PolynomialExpansion polyExpansion = new PolynomialExpansion() @@ -49,17 +44,17 @@ public static void main(String[] args) { .setOutputCol("polyFeatures") .setDegree(3); - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(-2.0, 2.3)), RowFactory.create(Vectors.dense(0.0, 0.0)), RowFactory.create(Vectors.dense(0.6, -1.1)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); Dataset polyDF = polyExpansion.transform(df); List rows = polyDF.select("polyFeatures").takeAsList(3); @@ -67,6 +62,6 @@ public static void main(String[] args) { System.out.println(r.get(0)); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java index 7b226fede996..94e3fafcab18 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java @@ -17,13 +17,11 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.QuantileDiscretizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -36,19 +34,16 @@ public class JavaQuantileDiscretizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaQuantileDiscretizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaQuantileDiscretizerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize( - Arrays.asList( - RowFactory.create(0, 18.0), - RowFactory.create(1, 19.0), - RowFactory.create(2, 8.0), - RowFactory.create(3, 5.0), - RowFactory.create(4, 2.2) - ) + List data = Arrays.asList( + RowFactory.create(0, 18.0), + RowFactory.create(1, 19.0), + RowFactory.create(2, 8.0), + RowFactory.create(3, 5.0), + RowFactory.create(4, 2.2) ); StructType schema = new StructType(new StructField[]{ @@ -56,7 +51,7 @@ public static void main(String[] args) { new StructField("hour", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); QuantileDiscretizer discretizer = new QuantileDiscretizer() .setInputCol("hour") @@ -66,6 +61,6 @@ public static void main(String[] args) { Dataset result = discretizer.fit(df).transform(df); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java index 8c453bf80d64..8282ce01d36f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.RFormula; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,9 +35,7 @@ public class JavaRFormulaExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRFormulaExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaRFormulaExample").getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ @@ -49,13 +45,13 @@ public static void main(String[] args) { createStructField("clicked", DoubleType, false) }); - JavaRDD rdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(7, "US", 18, 1.0), RowFactory.create(8, "CA", 12, 0.0), RowFactory.create(9, "NZ", 15, 0.0) - )); + ); - Dataset dataset = sqlContext.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(data, schema); RFormula formula = new RFormula() .setFormula("clicked ~ country + hour") .setFeaturesCol("features") @@ -63,7 +59,7 @@ public static void main(String[] args) { Dataset output = formula.fit(dataset).transform(dataset); output.select("features", "label").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java index 05c2bc9622e1..21e783a96897 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -29,19 +27,17 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaRandomForestClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRandomForestClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaRandomForestClassifierExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -98,6 +94,6 @@ public static void main(String[] args) { System.out.println("Learned classification forest model:\n" + rfModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java index d366967083a1..ece184a8784f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -30,19 +28,17 @@ import org.apache.spark.ml.regression.RandomForestRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaRandomForestRegressorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRandomForestRegressorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaRandomForestRegressorExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -87,6 +83,6 @@ public static void main(String[] args) { System.out.println("Learned regression forest model:\n" + rfModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java index 7e3ca99d7cb9..492718bbdba8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java @@ -19,36 +19,31 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.SQLTransformer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaSQLTransformerExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaSQLTransformerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaSQLTransformerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, 1.0, 3.0), RowFactory.create(2, 2.0, 5.0) - )); + ); StructType schema = new StructType(new StructField [] { new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("v1", DataTypes.DoubleType, false, Metadata.empty()), new StructField("v2", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); SQLTransformer sqlTrans = new SQLTransformer().setStatement( "SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); @@ -56,6 +51,6 @@ public static void main(String[] args) { sqlTrans.transform(df).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index cb911ef5ef58..f906843640ed 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -21,8 +21,6 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionModel; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.classification.LogisticRegression; @@ -30,7 +28,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -42,9 +40,7 @@ public class JavaSimpleParamsExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaSimpleParamsExample").getOrCreate(); // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans @@ -55,7 +51,7 @@ public static void main(String[] args) { new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); Dataset training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); + spark.createDataFrame(localTraining, LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -96,7 +92,7 @@ public static void main(String[] args) { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); + Dataset test = spark.createDataFrame(localTest, LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegressionModel.transform will only use the 'features' column. @@ -109,6 +105,6 @@ public static void main(String[] args) { + ", prediction=" + r.get(3)); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index a18a60f44816..9516ce1f4fb1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -21,8 +21,6 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -31,7 +29,7 @@ import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * A simple text classification pipeline that recognizes "spark" from input text. It uses the Java @@ -44,9 +42,8 @@ public class JavaSimpleTextClassificationPipeline { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaSimpleTextClassificationPipeline").getOrCreate(); // Prepare training documents, which are labeled. List localTraining = Lists.newArrayList( @@ -55,7 +52,7 @@ public static void main(String[] args) { new LabeledDocument(2L, "spark f g h", 1.0), new LabeledDocument(3L, "hadoop mapreduce", 0.0)); Dataset training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); + spark.createDataFrame(localTraining, LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -80,7 +77,7 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "spark hadoop spark"), new Document(7L, "apache hadoop")); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); + Dataset test = spark.createDataFrame(localTest, Document.class); // Make predictions on test documents. Dataset predictions = model.transform(test); @@ -89,6 +86,6 @@ public static void main(String[] args) { + ", prediction=" + r.get(3)); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java index e2dd759c0a40..10f82f223386 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.feature.StandardScaler; @@ -30,12 +28,11 @@ public class JavaStandardScalerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStandardScalerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaStandardScalerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); StandardScaler scaler = new StandardScaler() .setInputCol("features") @@ -50,6 +47,6 @@ public static void main(String[] args) { Dataset scaledData = scalerModel.transform(dataFrame); scaledData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java index 0ff3782cb3e9..23ed071c9f6e 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.StopWordsRemover; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -38,28 +36,26 @@ public class JavaStopWordsRemoverExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStopWordsRemoverExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaStopWordsRemoverExample").getOrCreate(); // $example on$ StopWordsRemover remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered"); - JavaRDD rdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField( "raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) }); - Dataset dataset = jsql.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(data, schema); remover.transform(dataset).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java index ceacbb4fb3f3..d4c2cf96a7cf 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.StringIndexer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,30 +35,28 @@ public class JavaStringIndexerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStringIndexerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaStringIndexerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ createStructField("id", IntegerType, false), createStructField("category", StringType, false) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexer indexer = new StringIndexer() .setInputCol("category") .setOutputCol("categoryIndex"); Dataset indexed = indexer.fit(df).transform(df); indexed.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index 107c835f2e01..a8169917772c 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -19,10 +19,8 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.IDF; import org.apache.spark.ml.feature.IDFModel; @@ -31,7 +29,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -40,21 +38,19 @@ public class JavaTfIdfExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTfIdfExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaTfIdfExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "Hi I heard about Spark"), RowFactory.create(0, "I wish Java could use case classes"), RowFactory.create(1, "Logistic regression models are neat") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - Dataset sentenceData = sqlContext.createDataFrame(jrdd, schema); + Dataset sentenceData = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); Dataset wordsData = tokenizer.transform(sentenceData); int numFeatures = 20; @@ -76,6 +72,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java index 9225fe2262f5..a65735a5e514 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.RegexTokenizer; import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.Dataset; @@ -38,23 +36,21 @@ public class JavaTokenizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTokenizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaTokenizerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "Hi I heard about Spark"), RowFactory.create(1, "I wish Java could use case classes"), RowFactory.create(2, "Logistic,regression,models,are,neat") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.IntegerType, false, Metadata.empty()), new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - Dataset sentenceDataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset sentenceDataFrame = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); @@ -70,6 +66,6 @@ public static void main(String[] args) { .setOutputCol("words") .setPattern("\\W"); // alternatively .setPattern("\\w+").setGaps(false); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java index 953ad455b1dc..9569bc2412de 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java @@ -17,14 +17,11 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.VectorAssembler; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -38,9 +35,7 @@ public class JavaVectorAssemblerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorAssemblerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaVectorAssemblerExample").getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ @@ -51,8 +46,7 @@ public static void main(String[] args) { createStructField("clicked", DoubleType, false) }); Row row = RowFactory.create(0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0); - JavaRDD rdd = jsc.parallelize(Arrays.asList(row)); - Dataset dataset = sqlContext.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(Arrays.asList(row), schema); VectorAssembler assembler = new VectorAssembler() .setInputCols(new String[]{"hour", "mobile", "userFeatures"}) @@ -61,7 +55,7 @@ public static void main(String[] args) { Dataset output = assembler.transform(dataset); System.out.println(output.select("features", "clicked").first()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java index b3b5953ee7bb..217d5a06d10a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Map; @@ -32,12 +30,10 @@ public class JavaVectorIndexerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorIndexerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaVectorIndexerExample").getOrCreate(); // $example on$ - Dataset data = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); VectorIndexer indexer = new VectorIndexer() .setInputCol("features") @@ -57,6 +53,6 @@ public static void main(String[] args) { Dataset indexedData = indexerModel.transform(data); indexedData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java index 2ae57c3577ef..4f1ea824a3a9 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java @@ -17,14 +17,13 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ +import java.util.List; + import com.google.common.collect.Lists; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; @@ -38,9 +37,7 @@ public class JavaVectorSlicerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorSlicerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaVectorSlicerExample").getOrCreate(); // $example on$ Attribute[] attrs = new Attribute[]{ @@ -50,13 +47,13 @@ public static void main(String[] args) { }; AttributeGroup group = new AttributeGroup("userFeatures", attrs); - JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + List data = Lists.newArrayList( RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) - )); + ); Dataset dataset = - jsql.createDataFrame(jrdd, (new StructType()).add(group.toStructField())); + spark.createDataFrame(data, (new StructType()).add(group.toStructField())); VectorSlicer vectorSlicer = new VectorSlicer() .setInputCol("userFeatures").setOutputCol("features"); @@ -68,7 +65,7 @@ public static void main(String[] args) { System.out.println(output.select("userFeatures", "features").first()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java index c5bb1eaaa344..d9b1a79b5254 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java @@ -19,37 +19,32 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.Word2Vec; import org.apache.spark.ml.feature.Word2VecModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaWord2VecExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaWord2VecExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaWord2VecExample").getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("Hi I heard about Spark".split(" "))), RowFactory.create(Arrays.asList("I wish Java could use case classes".split(" "))), RowFactory.create(Arrays.asList("Logistic regression models are neat".split(" "))) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - Dataset documentDF = sqlContext.createDataFrame(jrdd, schema); + Dataset documentDF = spark.createDataFrame(data, schema); // Learn a mapping from words to Vectors. Word2Vec word2Vec = new Word2Vec() @@ -64,6 +59,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index 354a5306ed45..ec2142e75695 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -21,14 +21,12 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; public class JavaSparkSQL { public static class Person implements Serializable { @@ -53,13 +51,12 @@ public void setAge(int age) { } public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName("JavaSparkSQL"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(ctx); + SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate(); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. - JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( + String file = "examples/src/main/resources/people.txt"; + JavaRDD people = spark.read().text(file).javaRDD().map( new Function() { @Override public Person call(String line) { @@ -74,12 +71,11 @@ public Person call(String line) { }); // Apply a schema to an RDD of Java Beans and register it as a table. - Dataset schemaPeople = sqlContext.createDataFrame(people, Person.class); + Dataset schemaPeople = spark.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. - Dataset teenagers = - sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + Dataset teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -100,12 +96,12 @@ public String call(Row row) { // Read in the parquet file created above. // Parquet files are self-describing so the schema is preserved. // The result of loading a parquet file is also a DataFrame. - Dataset parquetFile = sqlContext.read().parquet("people.parquet"); + Dataset parquetFile = spark.read().parquet("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); Dataset teenagers2 = - sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -121,7 +117,7 @@ public String call(Row row) { // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; // Create a DataFrame from the file(s) pointed by path - Dataset peopleFromJsonFile = sqlContext.read().json(path); + Dataset peopleFromJsonFile = spark.read().json(path); // Because the schema of a JSON dataset is automatically inferred, to write queries, // it is better to take a look at what is the schema. @@ -135,8 +131,7 @@ public String call(Row row) { peopleFromJsonFile.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlContext. - Dataset teenagers3 = - sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + Dataset teenagers3 = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrame and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -152,8 +147,8 @@ public String call(Row row) { // a RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); - JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - Dataset peopleFromJsonRDD = sqlContext.read().json(anotherPeopleRDD.rdd()); + JavaRDD anotherPeopleRDD = spark.createDataFrame(jsonData, String.class).toJSON().javaRDD(); + Dataset peopleFromJsonRDD = spark.read().json(anotherPeopleRDD); // Take a look at the schema of this new DataFrame. peopleFromJsonRDD.printSchema(); @@ -166,7 +161,7 @@ public String call(Row row) { peopleFromJsonRDD.registerTempTable("people2"); - Dataset peopleWithCity = sqlContext.sql("SELECT name, address.city FROM people2"); + Dataset peopleWithCity = spark.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -177,6 +172,6 @@ public String call(Row row) { System.out.println(name); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 7aa8862761d2..44f1e800fe44 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -22,14 +22,13 @@ import java.util.regex.Pattern; import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.VoidFunction2; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.api.java.StorageLevels; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.Time; @@ -82,7 +81,7 @@ public Iterator call(String x) { words.foreachRDD(new VoidFunction2, Time>() { @Override public void call(JavaRDD rdd, Time time) { - SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); + SparkSession spark = JavaSparkSessionSingleton.getInstance(rdd.context().getConf()); // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame JavaRDD rowRDD = rdd.map(new Function() { @@ -93,14 +92,14 @@ public JavaRecord call(String word) { return record; } }); - Dataset wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRecord.class); + Dataset wordsDataFrame = spark.createDataFrame(rowRDD, JavaRecord.class); // Register as table wordsDataFrame.registerTempTable("words"); // Do word count on table using SQL and print it Dataset wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word"); + spark.sql("select word, count(*) as total from words group by word"); System.out.println("========= " + time + "========="); wordCountsDataFrame.show(); } @@ -111,12 +110,12 @@ public JavaRecord call(String word) { } } -/** Lazily instantiated singleton instance of SQLContext */ -class JavaSQLContextSingleton { - private static transient SQLContext instance = null; - public static SQLContext getInstance(SparkContext sparkContext) { +/** Lazily instantiated singleton instance of SparkSession */ +class JavaSparkSessionSingleton { + private static transient SparkSession instance = null; + public static SparkSession getInstance(SparkConf sparkConf) { if (instance == null) { - instance = new SQLContext(sparkContext); + instance = SparkSession.builder().config(sparkConf).getOrCreate(); } return instance; } diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index 0c9ac583b2da..e36444f18506 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -21,8 +21,7 @@ if sys.version >= '3': long = int -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.evaluation import RegressionEvaluator @@ -31,15 +30,14 @@ # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="ALSExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("ALSExample").getOrCreate() # $example on$ - lines = sc.textFile("data/mllib/als/sample_movielens_ratings.txt") - parts = lines.map(lambda l: l.split("::")) + lines = spark.read.text("data/mllib/als/sample_movielens_ratings.txt").rdd + parts = lines.map(lambda row: row.value.split("::")) ratingsRDD = parts.map(lambda p: Row(userId=int(p[0]), movieId=int(p[1]), rating=float(p[2]), timestamp=long(p[3]))) - ratings = sqlContext.createDataFrame(ratingsRDD) + ratings = spark.createDataFrame(ratingsRDD) (training, test) = ratings.randomSplit([0.8, 0.2]) # Build the recommendation model using ALS on the training data @@ -56,4 +54,4 @@ rmse = evaluator.evaluate(predictions) print("Root-mean-square error = " + str(rmse)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/binarizer_example.py b/examples/src/main/python/ml/binarizer_example.py index 317cfa638a5a..072187e64564 100644 --- a/examples/src/main/python/ml/binarizer_example.py +++ b/examples/src/main/python/ml/binarizer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Binarizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BinarizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() # $example on$ - continuousDataFrame = sqlContext.createDataFrame([ + continuousDataFrame = spark.createDataFrame([ (0, 0.1), (1, 0.8), (2, 0.2) @@ -40,4 +38,4 @@ print(binarized_feature) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index e6f6bfd7e84e..836a89cde0d6 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -17,28 +17,26 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.clustering import BisectingKMeans, BisectingKMeansModel from pyspark.mllib.linalg import VectorUDT, _convert_to_vector, Vectors from pyspark.mllib.linalg import Vectors from pyspark.sql.types import Row # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession """ A simple example demonstrating a bisecting k-means clustering. """ if __name__ == "__main__": - - sc = SparkContext(appName="PythonBisectingKMeansExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PythonBisectingKMeansExample").getOrCreate() # $example on$ - data = sc.textFile("data/mllib/kmeans_data.txt") - parsed = data.map(lambda l: Row(features=Vectors.dense([float(x) for x in l.split(' ')]))) - training = sqlContext.createDataFrame(parsed) + data = spark.read.text("data/mllib/kmeans_data.txt").rdd + parsed = data\ + .map(lambda row: Row(features=Vectors.dense([float(x) for x in row.value.split(' ')]))) + training = spark.createDataFrame(parsed) kmeans = BisectingKMeans().setK(2).setSeed(1).setFeaturesCol("features") @@ -54,4 +52,4 @@ print(center) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/bucketizer_example.py b/examples/src/main/python/ml/bucketizer_example.py index 4304255f350d..288ec62bdfbb 100644 --- a/examples/src/main/python/ml/bucketizer_example.py +++ b/examples/src/main/python/ml/bucketizer_example.py @@ -17,21 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Bucketizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BucketizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() # $example on$ splits = [-float("inf"), -0.5, 0.0, 0.5, float("inf")] data = [(-0.5,), (-0.3,), (0.0,), (0.2,)] - dataFrame = sqlContext.createDataFrame(data, ["features"]) + dataFrame = spark.createDataFrame(data, ["features"]) bucketizer = Bucketizer(splits=splits, inputCol="features", outputCol="bucketedFeatures") @@ -40,4 +38,4 @@ bucketedData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py index 997a50473536..8f58fc28deb9 100644 --- a/examples/src/main/python/ml/chisq_selector_example.py +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import ChiSqSelector from pyspark.mllib.linalg import Vectors # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="ChiSqSelectorExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (7, Vectors.dense([0.0, 0.0, 18.0, 1.0]), 1.0,), (8, Vectors.dense([0.0, 1.0, 12.0, 0.0]), 0.0,), (9, Vectors.dense([1.0, 0.0, 15.0, 0.1]), 0.0,)], ["id", "features", "clicked"]) @@ -41,4 +39,4 @@ result.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/count_vectorizer_example.py b/examples/src/main/python/ml/count_vectorizer_example.py index e839f645f70b..9dbf9959d17e 100644 --- a/examples/src/main/python/ml/count_vectorizer_example.py +++ b/examples/src/main/python/ml/count_vectorizer_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import CountVectorizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="CountVectorizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("CountVectorizerExample").getOrCreate() # $example on$ # Input data: Each row is a bag of words with a ID. - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, "a b c".split(" ")), (1, "a b b c a".split(" ")) ], ["id", "words"]) @@ -41,4 +39,4 @@ result.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/cross_validator.py b/examples/src/main/python/ml/cross_validator.py index 5f0ef20218c4..a61d0f63d277 100644 --- a/examples/src/main/python/ml/cross_validator.py +++ b/examples/src/main/python/ml/cross_validator.py @@ -17,15 +17,14 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.evaluation import BinaryClassificationEvaluator from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.ml.tuning import CrossValidator, ParamGridBuilder -from pyspark.sql import Row, SQLContext # $example off$ +from pyspark.sql import Row, SparkSession """ A simple example demonstrating model selection using CrossValidator. @@ -36,25 +35,23 @@ """ if __name__ == "__main__": - sc = SparkContext(appName="CrossValidatorExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("CrossValidatorExample").getOrCreate() # $example on$ # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0, "a b c d e spark", 1.0), - (1, "b d", 0.0), - (2, "spark f g h", 1.0), - (3, "hadoop mapreduce", 0.0), - (4, "b spark who", 1.0), - (5, "g d a y", 0.0), - (6, "spark fly", 1.0), - (7, "was mapreduce", 0.0), - (8, "e spark program", 1.0), - (9, "a e c l", 0.0), - (10, "spark compile", 1.0), - (11, "hadoop software", 0.0) - ]) \ - .map(lambda x: LabeledDocument(*x)).toDF() + training = spark.createDataFrame([ + (0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0), + (4, "b spark who", 1.0), + (5, "g d a y", 0.0), + (6, "spark fly", 1.0), + (7, "was mapreduce", 0.0), + (8, "e spark program", 1.0), + (9, "a e c l", 0.0), + (10, "spark compile", 1.0), + (11, "hadoop software", 0.0) + ], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -82,12 +79,12 @@ cvModel = crossval.fit(training) # Prepare test documents, which are unlabeled. - Document = Row("id", "text") - test = sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() + test = spark.createDataFrame([ + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop") + ], ["id", "text"]) # Make predictions on test documents. cvModel uses the best model found (lrModel). prediction = cvModel.transform(test) @@ -96,4 +93,4 @@ print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py index d2644ca33565..b3e671038eb7 100644 --- a/examples/src/main/python/ml/dataframe_example.py +++ b/examples/src/main/python/ml/dataframe_example.py @@ -26,16 +26,14 @@ import tempfile import shutil -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.mllib.stat import Statistics if __name__ == "__main__": if len(sys.argv) > 2: print("Usage: dataframe_example.py ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="DataFrameExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("DataFrameExample").getOrCreate() if len(sys.argv) == 2: input = sys.argv[1] else: @@ -43,7 +41,7 @@ # Load input data print("Loading LIBSVM file with UDT from " + input + ".") - df = sqlContext.read.format("libsvm").load(input).cache() + df = spark.read.format("libsvm").load(input).cache() print("Schema from LIBSVM:") df.printSchema() print("Loaded training data as a DataFrame with " + @@ -54,7 +52,7 @@ labelSummary.show() # Convert features column to an RDD of vectors. - features = df.select("features").map(lambda r: r.features) + features = df.select("features").rdd.map(lambda r: r.features) summary = Statistics.colStats(features) print("Selected features column with average values:\n" + str(summary.mean())) @@ -67,9 +65,9 @@ # Load the records back. print("Loading Parquet file with UDT from " + tempdir) - newDF = sqlContext.read.parquet(tempdir) + newDF = spark.read.parquet(tempdir) print("Schema from Parquet:") newDF.printSchema() shutil.rmtree(tempdir) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py index 264d47f404cb..1bf8fc6d140c 100644 --- a/examples/src/main/python/ml/dct_example.py +++ b/examples/src/main/python/ml/dct_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import DCT from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="DCTExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("DCTExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (Vectors.dense([0.0, 1.0, -2.0, 3.0]),), (Vectors.dense([-1.0, 2.0, 4.0, -7.0]),), (Vectors.dense([14.0, -2.0, -5.0, 1.0]),)], ["features"]) @@ -42,4 +40,4 @@ print(dcts) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index 86bdc65392bb..d2318e24369d 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -21,20 +21,19 @@ from __future__ import print_function # $example on$ -from pyspark import SparkContext, SQLContext from pyspark.ml import Pipeline from pyspark.ml.classification import DecisionTreeClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="decision_tree_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,3 +71,5 @@ # summary only print(treeModel) # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index 8e20d5d8572a..9e8cb382a9bf 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import DecisionTreeRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="decision_tree_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # We specify maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,3 +68,5 @@ # summary only print(treeModel) # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index c85cb0d89543..6fa641b772d9 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -17,23 +17,21 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import ElementwiseProduct from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="ElementwiseProductExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() # $example on$ data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] - df = sqlContext.createDataFrame(data, ["vector"]) + df = spark.createDataFrame(data, ["vector"]) transformer = ElementwiseProduct(scalingVec=Vectors.dense([0.0, 1.0, 2.0]), inputCol="vector", outputCol="transformedVector") transformer.transform(df).show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py index 9a8993dac4f6..4993b5a9846c 100644 --- a/examples/src/main/python/ml/estimator_transformer_param_example.py +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -18,20 +18,19 @@ """ Estimator Transformer Param Example. """ -from pyspark import SparkContext, SQLContext + # $example on$ from pyspark.mllib.linalg import Vectors from pyspark.ml.classification import LogisticRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="EstimatorTransformerParamExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() # $example on$ # Prepare training data from a list of (label, features) tuples. - training = sqlContext.createDataFrame([ + training = spark.createDataFrame([ (1.0, Vectors.dense([0.0, 1.1, 0.1])), (0.0, Vectors.dense([2.0, 1.0, -1.0])), (0.0, Vectors.dense([2.0, 1.3, 1.0])), @@ -69,7 +68,7 @@ print model2.extractParamMap() # Prepare test data - test = sqlContext.createDataFrame([ + test = spark.createDataFrame([ (1.0, Vectors.dense([-1.0, 1.5, 1.3])), (0.0, Vectors.dense([3.0, 2.0, -0.1])), (1.0, Vectors.dense([0.0, 2.2, -1.5]))], ["label", "features"]) @@ -84,4 +83,4 @@ print row # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index f7e842f4b303..b09ad41da313 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import GBTClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="gradient_boosted_tree_classifier_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("gradient_boosted_tree_classifier_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,4 +71,4 @@ print(gbtModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py index f8b4de651c76..caa7cfc4e11f 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import GBTRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="gradient_boosted_tree_regressor_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("gradient_boosted_tree_regressor_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,4 +68,4 @@ print(gbtModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/index_to_string_example.py b/examples/src/main/python/ml/index_to_string_example.py index fb0ba2950bbd..dd04b2c4b0bf 100644 --- a/examples/src/main/python/ml/index_to_string_example.py +++ b/examples/src/main/python/ml/index_to_string_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import IndexToString, StringIndexer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="IndexToStringExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame( + df = spark.createDataFrame( [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], ["id", "category"]) @@ -42,4 +40,4 @@ converted.select("id", "originalCategory").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index fa57a4d3ada1..7d9d80e6452b 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -20,10 +20,9 @@ import sys import numpy as np -from pyspark import SparkContext from pyspark.ml.clustering import KMeans, KMeansModel from pyspark.mllib.linalg import VectorUDT, _convert_to_vector -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.sql.types import Row, StructField, StructType """ @@ -35,8 +34,8 @@ """ -def parseVector(line): - array = np.array([float(x) for x in line.split(' ')]) +def parseVector(row): + array = np.array([float(x) for x in row.value.split(' ')]) return _convert_to_vector(array) @@ -50,14 +49,13 @@ def parseVector(line): path = sys.argv[1] k = sys.argv[2] - sc = SparkContext(appName="PythonKMeansExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PythonKMeansExample").getOrCreate() - lines = sc.textFile(path) + lines = spark.read.text(path).rdd data = lines.map(parseVector) row_rdd = data.map(lambda x: Row(x)) schema = StructType([StructField(FEATURES_COL, VectorUDT(), False)]) - df = sqlContext.createDataFrame(row_rdd, schema) + df = spark.createDataFrame(row_rdd, schema) kmeans = KMeans().setK(2).setSeed(1).setFeaturesCol(FEATURES_COL) model = kmeans.fit(df) @@ -67,4 +65,4 @@ def parseVector(line): for center in centers: print(center) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/linear_regression_with_elastic_net.py b/examples/src/main/python/ml/linear_regression_with_elastic_net.py index a4cd40cf2672..99b7f7fe99de 100644 --- a/examples/src/main/python/ml/linear_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/linear_regression_with_elastic_net.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.regression import LinearRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="LinearRegressionWithElasticNet") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("LinearRegressionWithElasticNet").getOrCreate() # $example on$ # Load training data - training = sqlContext.read.format("libsvm")\ + training = spark.read.format("libsvm")\ .load("data/mllib/sample_linear_regression_data.txt") lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) @@ -42,4 +40,4 @@ print("Intercept: " + str(lrModel.intercept)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py index b0b1d27e13bb..0d7112e72354 100644 --- a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import LogisticRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="LogisticRegressionWithElasticNet") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("LogisticRegressionWithElasticNet").getOrCreate() # $example on$ # Load training data - training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") lr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) @@ -41,4 +39,4 @@ print("Intercept: " + str(lrModel.intercept)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/max_abs_scaler_example.py b/examples/src/main/python/ml/max_abs_scaler_example.py index d9b69eef1cd8..1cb95a98f08d 100644 --- a/examples/src/main/python/ml/max_abs_scaler_example.py +++ b/examples/src/main/python/ml/max_abs_scaler_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import MaxAbsScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="MaxAbsScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = MaxAbsScaler(inputCol="features", outputCol="scaledFeatures") @@ -40,4 +38,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/min_max_scaler_example.py b/examples/src/main/python/ml/min_max_scaler_example.py index 2f8e4ade468b..8d91a59e2b0b 100644 --- a/examples/src/main/python/ml/min_max_scaler_example.py +++ b/examples/src/main/python/ml/min_max_scaler_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import MinMaxScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="MinMaxScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures") @@ -40,4 +38,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/multilayer_perceptron_classification.py b/examples/src/main/python/ml/multilayer_perceptron_classification.py index f84588f547ff..8bededc14db4 100644 --- a/examples/src/main/python/ml/multilayer_perceptron_classification.py +++ b/examples/src/main/python/ml/multilayer_perceptron_classification.py @@ -17,21 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import MultilayerPerceptronClassifier from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="multilayer_perceptron_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder.appName("multilayer_perceptron_classification_example").getOrCreate() # $example on$ # Load training data - data = sqlContext.read.format("libsvm")\ + data = spark.read.format("libsvm")\ .load("data/mllib/sample_multiclass_classification_data.txt") # Split the data into train and test splits = data.randomSplit([0.6, 0.4], 1234) @@ -52,4 +50,4 @@ print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/n_gram_example.py b/examples/src/main/python/ml/n_gram_example.py index f2d85f53e721..b7fecf0d685c 100644 --- a/examples/src/main/python/ml/n_gram_example.py +++ b/examples/src/main/python/ml/n_gram_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import NGram # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="NGramExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("NGramExample").getOrCreate() # $example on$ - wordDataFrame = sqlContext.createDataFrame([ + wordDataFrame = spark.createDataFrame([ (0, ["Hi", "I", "heard", "about", "Spark"]), (1, ["I", "wish", "Java", "could", "use", "case", "classes"]), (2, ["Logistic", "regression", "models", "are", "neat"]) @@ -39,4 +37,4 @@ print(ngrams_label) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py index db8fbea9bf9b..e37035542c7e 100644 --- a/examples/src/main/python/ml/naive_bayes_example.py +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -17,21 +17,18 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import NaiveBayes from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="naive_bayes_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("naive_bayes_example").getOrCreate() # $example on$ # Load training data - data = sqlContext.read.format("libsvm") \ + data = spark.read.format("libsvm") \ .load("data/mllib/sample_libsvm_data.txt") # Split the data into train and test splits = data.randomSplit([0.6, 0.4], 1234) @@ -50,4 +47,4 @@ print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/normalizer_example.py b/examples/src/main/python/ml/normalizer_example.py index d490221474c2..ae2553761928 100644 --- a/examples/src/main/python/ml/normalizer_example.py +++ b/examples/src/main/python/ml/normalizer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Normalizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="NormalizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Normalize each Vector using $L^1$ norm. normalizer = Normalizer(inputCol="features", outputCol="normFeatures", p=1.0) @@ -40,4 +38,4 @@ lInfNormData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/onehot_encoder_example.py b/examples/src/main/python/ml/onehot_encoder_example.py index 0f94c26638d3..9acc363dc9e2 100644 --- a/examples/src/main/python/ml/onehot_encoder_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import OneHotEncoder, StringIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="OneHotEncoderExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, "a"), (1, "b"), (2, "c"), @@ -45,4 +43,4 @@ encoded.select("id", "categoryVec").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py index a17181f1b8a5..adab151734aa 100644 --- a/examples/src/main/python/ml/pca_example.py +++ b/examples/src/main/python/ml/pca_example.py @@ -17,26 +17,24 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import PCA from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="PCAExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PCAExample").getOrCreate() # $example on$ data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] - df = sqlContext.createDataFrame(data, ["features"]) + df = spark.createDataFrame(data, ["features"]) pca = PCA(k=3, inputCol="features", outputCol="pcaFeatures") model = pca.fit(df) result = model.transform(df).select("pcaFeatures") result.show(truncate=False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/pipeline_example.py b/examples/src/main/python/ml/pipeline_example.py index 3288568f0c28..ed9765d96165 100644 --- a/examples/src/main/python/ml/pipeline_example.py +++ b/examples/src/main/python/ml/pipeline_example.py @@ -18,21 +18,20 @@ """ Pipeline Example. """ -from pyspark import SparkContext, SQLContext + # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.feature import HashingTF, Tokenizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="PipelineExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PipelineExample").getOrCreate() # $example on$ # Prepare training documents from a list of (id, text, label) tuples. - training = sqlContext.createDataFrame([ + training = spark.createDataFrame([ (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -48,7 +47,7 @@ model = pipeline.fit(training) # Prepare test documents, which are unlabeled (id, text) tuples. - test = sqlContext.createDataFrame([ + test = spark.createDataFrame([ (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -61,4 +60,4 @@ print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 89f5cbe8f2f4..328b559320f8 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import PolynomialExpansion from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="PolynomialExpansionExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() # $example on$ - df = sqlContext\ + df = spark\ .createDataFrame([(Vectors.dense([-2.0, 2.3]),), (Vectors.dense([0.0, 0.0]),), (Vectors.dense([0.6, -1.1]),)], @@ -40,4 +38,4 @@ print(expanded) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index c3570438c51d..b0a93e050c54 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import RandomForestClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="random_forest_classifier_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("random_forest_classifier_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,4 +71,4 @@ print(rfModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py index b77014f37923..4bb84f0de8ff 100644 --- a/examples/src/main/python/ml/random_forest_regressor_example.py +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import RandomForestRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="random_forest_regressor_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("random_forest_regressor_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,4 +68,4 @@ print(rfModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/rformula_example.py b/examples/src/main/python/ml/rformula_example.py index b544a1470076..45cc116ac2f6 100644 --- a/examples/src/main/python/ml/rformula_example.py +++ b/examples/src/main/python/ml/rformula_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import RFormula # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="RFormulaExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() # $example on$ - dataset = sqlContext.createDataFrame( + dataset = spark.createDataFrame( [(7, "US", 18, 1.0), (8, "CA", 12, 0.0), (9, "NZ", 15, 0.0)], @@ -41,4 +39,4 @@ output.select("features", "label").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index b4f06bf88874..3600c12211d0 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -17,11 +17,10 @@ from __future__ import print_function -from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.feature import HashingTF, Tokenizer -from pyspark.sql import Row, SQLContext +from pyspark.sql import Row, SparkSession """ @@ -34,16 +33,15 @@ if __name__ == "__main__": - sc = SparkContext(appName="SimpleTextClassificationPipeline") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0, "a b c d e spark", 1.0), - (1, "b d", 0.0), - (2, "spark f g h", 1.0), - (3, "hadoop mapreduce", 0.0)]) \ - .map(lambda x: LabeledDocument(*x)).toDF() + training = spark.createDataFrame([ + (0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0) + ], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -55,12 +53,12 @@ model = pipeline.fit(training) # Prepare test documents, which are unlabeled. - Document = Row("id", "text") - test = sc.parallelize([(4, "spark i j k"), - (5, "l m n"), - (6, "spark hadoop spark"), - (7, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() + test = spark.createDataFrame([ + (4, "spark i j k"), + (5, "l m n"), + (6, "spark hadoop spark"), + (7, "apache hadoop") + ], ["id", "text"]) # Make predictions on test documents and print columns of interest. prediction = model.transform(test) @@ -68,4 +66,4 @@ for row in selected.collect(): print(row) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/sql_transformer.py b/examples/src/main/python/ml/sql_transformer.py index 9575d728d815..26045db4be6b 100644 --- a/examples/src/main/python/ml/sql_transformer.py +++ b/examples/src/main/python/ml/sql_transformer.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import SQLTransformer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="SQLTransformerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, 1.0, 3.0), (2, 2.0, 5.0) ], ["id", "v1", "v2"]) @@ -37,4 +35,4 @@ sqlTrans.transform(df).show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/standard_scaler_example.py b/examples/src/main/python/ml/standard_scaler_example.py index ae7aa85005bc..c50804f6bf4e 100644 --- a/examples/src/main/python/ml/standard_scaler_example.py +++ b/examples/src/main/python/ml/standard_scaler_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StandardScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StandardScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = StandardScaler(inputCol="features", outputCol="scaledFeatures", withStd=True, withMean=False) @@ -40,4 +38,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/stopwords_remover_example.py b/examples/src/main/python/ml/stopwords_remover_example.py index 01f94af8ca75..57362673df78 100644 --- a/examples/src/main/python/ml/stopwords_remover_example.py +++ b/examples/src/main/python/ml/stopwords_remover_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StopWordsRemover # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StopWordsRemoverExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() # $example on$ - sentenceData = sqlContext.createDataFrame([ + sentenceData = spark.createDataFrame([ (0, ["I", "saw", "the", "red", "baloon"]), (1, ["Mary", "had", "a", "little", "lamb"]) ], ["label", "raw"]) @@ -37,4 +35,4 @@ remover.transform(sentenceData).show(truncate=False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/string_indexer_example.py b/examples/src/main/python/ml/string_indexer_example.py index 58a8cb5d56b7..aacd4f999bb1 100644 --- a/examples/src/main/python/ml/string_indexer_example.py +++ b/examples/src/main/python/ml/string_indexer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StringIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StringIndexerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame( + df = spark.createDataFrame( [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], ["id", "category"]) indexer = StringIndexer(inputCol="category", outputCol="categoryIndex") @@ -36,4 +34,4 @@ indexed.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index 141324d45853..25df8166efc2 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import HashingTF, IDF, Tokenizer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="TfIdfExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() # $example on$ - sentenceData = sqlContext.createDataFrame([ + sentenceData = spark.createDataFrame([ (0, "Hi I heard about Spark"), (0, "I wish Java could use case classes"), (1, "Logistic regression models are neat") @@ -46,4 +44,4 @@ print(features_label) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/tokenizer_example.py b/examples/src/main/python/ml/tokenizer_example.py index ce9b225be535..5be4b4cfe3a5 100644 --- a/examples/src/main/python/ml/tokenizer_example.py +++ b/examples/src/main/python/ml/tokenizer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Tokenizer, RegexTokenizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="TokenizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() # $example on$ - sentenceDataFrame = sqlContext.createDataFrame([ + sentenceDataFrame = spark.createDataFrame([ (0, "Hi I heard about Spark"), (1, "I wish Java could use case classes"), (2, "Logistic,regression,models,are,neat") @@ -41,4 +39,4 @@ # alternatively, pattern="\\w+", gaps(False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/train_validation_split.py b/examples/src/main/python/ml/train_validation_split.py index 161a200c61b6..2e43a0f8aec5 100644 --- a/examples/src/main/python/ml/train_validation_split.py +++ b/examples/src/main/python/ml/train_validation_split.py @@ -15,13 +15,12 @@ # limitations under the License. # -from pyspark import SparkContext # $example on$ from pyspark.ml.evaluation import RegressionEvaluator from pyspark.ml.regression import LinearRegression from pyspark.ml.tuning import ParamGridBuilder, TrainValidationSplit -from pyspark.sql import SQLContext # $example off$ +from pyspark.sql import SparkSession """ This example demonstrates applying TrainValidationSplit to split data @@ -32,11 +31,10 @@ """ if __name__ == "__main__": - sc = SparkContext(appName="TrainValidationSplit") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("TrainValidationSplit").getOrCreate() # $example on$ # Prepare training and test data. - data = sqlContext.read.format("libsvm")\ + data = spark.read.format("libsvm")\ .load("data/mllib/sample_linear_regression_data.txt") train, test = data.randomSplit([0.7, 0.3]) lr = LinearRegression(maxIter=10, regParam=0.1) @@ -65,4 +63,4 @@ for row in prediction.take(5): print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py index 04f64839f188..019a9ea6f750 100644 --- a/examples/src/main/python/ml/vector_assembler_example.py +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.mllib.linalg import Vectors from pyspark.ml.feature import VectorAssembler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorAssemblerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() # $example on$ - dataset = sqlContext.createDataFrame( + dataset = spark.createDataFrame( [(0, 18, 1.0, Vectors.dense([0.0, 10.0, 0.5]), 1.0)], ["id", "hour", "mobile", "userFeatures", "clicked"]) assembler = VectorAssembler( @@ -39,4 +37,4 @@ print(output.select("features", "clicked").first()) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_indexer_example.py b/examples/src/main/python/ml/vector_indexer_example.py index 146f41c1dd90..3cf5b8ebf17e 100644 --- a/examples/src/main/python/ml/vector_indexer_example.py +++ b/examples/src/main/python/ml/vector_indexer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import VectorIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorIndexerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() # $example on$ - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") indexer = VectorIndexer(inputCol="features", outputCol="indexed", maxCategories=10) indexerModel = indexer.fit(data) @@ -37,4 +35,4 @@ indexedData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index 31a753073c13..0531bcdb0667 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -17,20 +17,18 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import VectorSlicer from pyspark.mllib.linalg import Vectors from pyspark.sql.types import Row # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorSlicerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ Row(userFeatures=Vectors.sparse(3, {0: -2.0, 1: 2.3}),), Row(userFeatures=Vectors.dense([-2.0, 2.3, 0.0]),)]) @@ -41,4 +39,4 @@ output.select("userFeatures", "features").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/word2vec_example.py b/examples/src/main/python/ml/word2vec_example.py index 53c77feb1014..6766a7b6aa45 100644 --- a/examples/src/main/python/ml/word2vec_example.py +++ b/examples/src/main/python/ml/word2vec_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Word2Vec # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="Word2VecExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("Word2VecExample").getOrCreate() # $example on$ # Input data: Each row is a bag of words from a sentence or document. - documentDF = sqlContext.createDataFrame([ + documentDF = spark.createDataFrame([ ("Hi I heard about Spark".split(" "), ), ("I wish Java could use case classes".split(" "), ), ("Logistic regression models are neat".split(" "), ) @@ -42,4 +40,4 @@ print(feature) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py index 4e7ea289b253..8f0fc9d45da2 100644 --- a/examples/src/main/python/mllib/binary_classification_metrics_example.py +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -18,7 +18,7 @@ Binary Classification Metrics Example. """ from __future__ import print_function -from pyspark import SparkContext, SQLContext +from pyspark import SparkContext # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS from pyspark.mllib.evaluation import BinaryClassificationMetrics @@ -27,7 +27,7 @@ if __name__ == "__main__": sc = SparkContext(appName="BinaryClassificationMetricsExample") - sqlContext = SQLContext(sc) + # $example on$ # Several of the methods available in scala are currently missing from pyspark # Load training data in LIBSVM format @@ -52,3 +52,5 @@ # Area under ROC curve print("Area under ROC = %s" % metrics.areaUnderROC) # $example off$ + + sc.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index ea6a22dbfe82..59a46cb283c4 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -63,7 +63,7 @@ # |-- age: long (nullable = true) # |-- name: string (nullable = true) - # Register this DataFrame as a table. + # Register this DataFrame as a temporary table. people.registerTempTable("people") # SQL statements can be run by using the sql methods provided by sqlContext diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index 1ba5e9fb7899..588cbfee14ba 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -33,13 +33,14 @@ from pyspark import SparkContext from pyspark.streaming import StreamingContext -from pyspark.sql import SQLContext, Row +from pyspark.sql import Row, SparkSession -def getSqlContextInstance(sparkContext): - if ('sqlContextSingletonInstance' not in globals()): - globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) - return globals()['sqlContextSingletonInstance'] +def getSparkSessionInstance(sparkConf): + if ('sparkSessionSingletonInstance' not in globals()): + globals()['sparkSessionSingletonInstance'] =\ + SparkSession.builder.config(conf=sparkConf).getOrCreate() + return globals()['sparkSessionSingletonInstance'] if __name__ == "__main__": @@ -60,19 +61,19 @@ def process(time, rdd): print("========= %s =========" % str(time)) try: - # Get the singleton instance of SQLContext - sqlContext = getSqlContextInstance(rdd.context) + # Get the singleton instance of SparkSession + spark = getSparkSessionInstance(rdd.context.getConf()) # Convert RDD[String] to RDD[Row] to DataFrame rowRdd = rdd.map(lambda w: Row(word=w)) - wordsDataFrame = sqlContext.createDataFrame(rowRdd) + wordsDataFrame = spark.createDataFrame(rowRdd) # Register as table wordsDataFrame.registerTempTable("words") # Do word count on table using SQL and print it wordCountsDataFrame = \ - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() except: pass diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala index 21f58ddf3cfb..3795af83094a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.regression.AFTSurvivalRegression import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * An example for AFTSurvivalRegression. @@ -31,12 +30,10 @@ import org.apache.spark.sql.SQLContext object AFTSurvivalRegressionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("AFTSurvivalRegressionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("AFTSurvivalRegressionExample").getOrCreate() // $example on$ - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (1.218, 1.0, Vectors.dense(1.560, -0.605)), (2.949, 0.0, Vectors.dense(0.346, 2.158)), (3.627, 0.0, Vectors.dense(1.380, 0.231)), @@ -56,7 +53,7 @@ object AFTSurvivalRegressionExample { model.transform(training).show(false) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index a79e15c767e1..41750ca77914 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.recommendation.ALS // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession // $example on$ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType @@ -43,13 +42,11 @@ object ALSExample { // $example off$ def main(args: Array[String]) { - val conf = new SparkConf().setAppName("ALSExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("ALSExample").getOrCreate() + import spark.implicits._ // $example on$ - val ratings = sc.textFile("data/mllib/als/sample_movielens_ratings.txt") + val ratings = spark.read.text("data/mllib/als/sample_movielens_ratings.txt") .map(Rating.parseRating) .toDF() val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2)) @@ -75,7 +72,8 @@ object ALSExample { val rmse = evaluator.evaluate(predictions) println(s"Root-mean-square error = $rmse") // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala index 2ed8101c133c..93c153f92323 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Binarizer // $example off$ -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} object BinarizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("BinarizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() // $example on$ val data = Array((0, 0.1), (1, 0.8), (2, 0.2)) - val dataFrame: DataFrame = sqlContext.createDataFrame(data).toDF("label", "feature") + val dataFrame: DataFrame = spark.createDataFrame(data).toDF("label", "feature") val binarizer: Binarizer = new Binarizer() .setInputCol("feature") @@ -42,7 +39,8 @@ object BinarizerExample { val binarizedFeatures = binarizedDataFrame.select("binarized_feature") binarizedFeatures.collect().foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala index 6f6236a2b058..779ad33dbda0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala @@ -18,23 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Bucketizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object BucketizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("BucketizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() // $example on$ val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) val data = Array(-0.5, -0.3, 0.0, 0.2) - val dataFrame = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val dataFrame = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val bucketizer = new Bucketizer() .setInputCol("features") @@ -45,7 +42,7 @@ object BucketizerExample { val bucketedData = bucketizer.transform(dataFrame) bucketedData.show() // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 2be61537e613..84ca1f0b564d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -18,20 +18,16 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.ChiSqSelector import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object ChiSqSelectorExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("ChiSqSelectorExample") - val sc = new SparkContext(conf) - - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() + import spark.implicits._ // $example on$ val data = Seq( @@ -40,7 +36,7 @@ object ChiSqSelectorExample { (9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) ) - val df = sc.parallelize(data).toDF("id", "features", "clicked") + val df = spark.createDataset(data).toDF("id", "features", "clicked") val selector = new ChiSqSelector() .setNumTopFeatures(1) @@ -51,7 +47,7 @@ object ChiSqSelectorExample { val result = selector.fit(df).transform(df) result.show() // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index 7d07fc7dd113..9ab43a48bff8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object CountVectorizerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("CounterVectorizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("CounterVectorizerExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, Array("a", "b", "c")), (1, Array("a", "b", "b", "c", "a")) )).toDF("id", "words") @@ -51,6 +48,8 @@ object CountVectorizerExample { cvModel.transform(df).select("features").show() // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala index dc26b55a768a..b415333c710c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala @@ -18,18 +18,15 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.DCT import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DCTExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DCTExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("DCTExample").getOrCreate() // $example on$ val data = Seq( @@ -37,7 +34,7 @@ object DCTExample { Vectors.dense(-1.0, 2.0, 4.0, -7.0), Vectors.dense(14.0, -2.0, -5.0, 1.0)) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val dct = new DCT() .setInputCol("features") @@ -47,7 +44,8 @@ object DCTExample { val dctDf = dct.transform(df) dctDf.select("featuresDCT").show(3) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 7e608a281203..2f892f8d72c1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -23,11 +23,10 @@ import java.io.File import com.google.common.io.Files import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} /** * An example of how to use [[org.apache.spark.sql.DataFrame]] for ML. Run with @@ -62,14 +61,11 @@ object DataFrameExample { } def run(params: Params) { - - val conf = new SparkConf().setAppName(s"DataFrameExample with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"DataFrameExample with $params").getOrCreate() // Load input data println(s"Loading LIBSVM file with UDT from ${params.input}.") - val df: DataFrame = sqlContext.read.format("libsvm").load(params.input).cache() + val df: DataFrame = spark.read.format("libsvm").load(params.input).cache() println("Schema from LIBSVM:") df.printSchema() println(s"Loaded training data as a DataFrame with ${df.count()} records.") @@ -94,11 +90,11 @@ object DataFrameExample { // Load the records back. println(s"Loading Parquet file with UDT from $outputDir.") - val newDF = sqlContext.read.parquet(outputDir) + val newDF = spark.read.parquet(outputDir) println(s"Schema from Parquet:") newDF.printSchema() - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala index 224d8da5f0ec..a0a2e1fb33dc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.DecisionTreeClassificationModel @@ -26,16 +25,14 @@ import org.apache.spark.ml.classification.DecisionTreeClassifier import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DecisionTreeClassificationExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DecisionTreeClassificationExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("DecisionTreeClassificationExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -88,6 +85,8 @@ object DecisionTreeClassificationExample { val treeModel = model.stages(2).asInstanceOf[DecisionTreeClassificationModel] println("Learned classification tree model:\n" + treeModel.toDebugString) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index d2560cc00ba0..cea1d801aa73 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -33,7 +33,7 @@ import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.evaluation.{MulticlassMetrics, RegressionMetrics} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for decision trees. Run with @@ -134,18 +134,18 @@ object DecisionTreeExample { /** Load a dataset from the given path, using the given format */ private[ml] def loadData( - sqlContext: SQLContext, + spark: SparkSession, path: String, format: String, expectedNumFeatures: Option[Int] = None): DataFrame = { - import sqlContext.implicits._ + import spark.implicits._ format match { - case "dense" => MLUtils.loadLabeledPoints(sqlContext.sparkContext, path).toDF() + case "dense" => MLUtils.loadLabeledPoints(spark.sparkContext, path).toDF() case "libsvm" => expectedNumFeatures match { - case Some(numFeatures) => sqlContext.read.option("numFeatures", numFeatures.toString) + case Some(numFeatures) => spark.read.option("numFeatures", numFeatures.toString) .format("libsvm").load(path) - case None => sqlContext.read.format("libsvm").load(path) + case None => spark.read.format("libsvm").load(path) } case _ => throw new IllegalArgumentException(s"Bad data format: $format") } @@ -167,17 +167,17 @@ object DecisionTreeExample { testInput: String, algo: String, fracTest: Double): (DataFrame, DataFrame) = { - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.getOrCreate() // Load training data - val origExamples: DataFrame = loadData(sqlContext, input, dataFormat) + val origExamples: DataFrame = loadData(spark, input, dataFormat) // Load or create test set val dataframes: Array[DataFrame] = if (testInput != "") { // Load testInput. val numFeatures = origExamples.first().getAs[Vector](1).size val origTestExamples: DataFrame = - loadData(sqlContext, testInput, dataFormat, Some(numFeatures)) + loadData(spark, testInput, dataFormat, Some(numFeatures)) Array(origExamples, origTestExamples) } else { // Split input into training, test. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala index ad32e5635a3e..26b52d0489e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator @@ -26,17 +25,15 @@ import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.regression.DecisionTreeRegressor // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DecisionTreeRegressionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DecisionTreeRegressionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("DecisionTreeRegressionExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Here, we treat features with > 4 distinct values as continuous. @@ -78,6 +75,8 @@ object DecisionTreeRegressionExample { val treeModel = model.stages(1).asInstanceOf[DecisionTreeRegressionModel] println("Learned regression tree model:\n" + treeModel.toDebugString) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 8d127f9b3542..2aa1ab1ec855 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -18,13 +18,12 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.{ClassificationModel, Classifier, ClassifierParams} import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util.Identifiable import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql.{Dataset, Row, SparkSession} /** * A simple example demonstrating how to write your own learning algorithm using Estimator, @@ -38,13 +37,11 @@ import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} object DeveloperApiExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("DeveloperApiExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("DeveloperApiExample").getOrCreate() + import spark.implicits._ // Prepare training data. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -62,13 +59,13 @@ object DeveloperApiExample { val model = lr.fit(training.toDF()) // Prepare test data. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) // Make predictions on test data. - val sumPredictions: Double = model.transform(test.toDF()) + val sumPredictions: Double = model.transform(test) .select("features", "label", "prediction") .collect() .map { case Row(features: Vector, label: Double, prediction: Double) => @@ -77,7 +74,7 @@ object DeveloperApiExample { assert(sumPredictions == 0.0, "MyLogisticRegression predicted something other than 0, even though all coefficients are 0!") - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala index 629d322c4357..f289c28df9b5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -18,22 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.ElementwiseProduct import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object ElementwiseProductExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ElementwiseProductExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() // $example on$ // Create some vector data; also works for sparse vectors - val dataFrame = sqlContext.createDataFrame(Seq( + val dataFrame = spark.createDataFrame(Seq( ("a", Vectors.dense(1.0, 2.0, 3.0)), ("b", Vectors.dense(4.0, 5.0, 6.0)))).toDF("id", "vector") @@ -46,7 +43,8 @@ object ElementwiseProductExample { // Batch transform the vectors to create new column: transformer.transform(dataFrame).show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala index 65e3c365abb3..91076ccbc14d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -18,25 +18,22 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.param.ParamMap import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object EstimatorTransformerParamExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("EstimatorTransformerParamExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() // $example on$ // Prepare training data from a list of (label, features) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (1.0, Vectors.dense(0.0, 1.1, 0.1)), (0.0, Vectors.dense(2.0, 1.0, -1.0)), (0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -76,7 +73,7 @@ object EstimatorTransformerParamExample { println("Model 2 was fit using parameters: " + model2.parent.extractParamMap) // Prepare test data. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (1.0, Vectors.dense(-1.0, 1.5, 1.3)), (0.0, Vectors.dense(3.0, 2.0, -0.1)), (1.0, Vectors.dense(0.0, 2.2, -1.5)) @@ -94,7 +91,7 @@ object EstimatorTransformerParamExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala index cd62a803820c..412c54db7d5c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier} import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object GradientBoostedTreeClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("GradientBoostedTreeClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("GradientBoostedTreeClassifierExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -91,7 +88,7 @@ object GradientBoostedTreeClassifierExample { println("Learned classification GBT model:\n" + gbtModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala index b8cf9629bbda..fd43553cc69d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object GradientBoostedTreeRegressorExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("GradientBoostedTreeRegressorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("GradientBoostedTreeRegressorExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -79,7 +76,7 @@ object GradientBoostedTreeRegressorExample { println("Learned regression GBT model:\n" + gbtModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala index 4cea09ba1265..d873618726c6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala @@ -18,21 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{IndexToString, StringIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object IndexToStringExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("IndexToStringExample") - val sc = new SparkContext(conf) - - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a"), (1, "b"), (2, "c"), @@ -54,7 +50,8 @@ object IndexToStringExample { val converted = converter.transform(indexed) converted.select("id", "originalCategory").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala index 7af011571f76..d2573fad3596 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala @@ -19,11 +19,10 @@ package org.apache.spark.examples.ml // scalastyle:off println -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.clustering.KMeans import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} // $example off$ /** @@ -37,13 +36,11 @@ object KMeansExample { def main(args: Array[String]): Unit = { // Creates a Spark context and a SQL context - val conf = new SparkConf().setAppName(s"${this.getClass.getSimpleName}") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() // $example on$ // Crates a DataFrame - val dataset: DataFrame = sqlContext.createDataFrame(Seq( + val dataset: DataFrame = spark.createDataFrame(Seq( (1, Vectors.dense(0.0, 0.0, 0.0)), (2, Vectors.dense(0.1, 0.1, 0.1)), (3, Vectors.dense(0.2, 0.2, 0.2)), @@ -64,7 +61,7 @@ object KMeansExample { model.clusterCenters.foreach(println) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala index f9ddac77090e..c23adee1a3ea 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala @@ -18,11 +18,10 @@ package org.apache.spark.examples.ml // scalastyle:off println -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.clustering.LDA import org.apache.spark.mllib.linalg.{Vectors, VectorUDT} -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.types.{StructField, StructType} // $example off$ @@ -41,16 +40,14 @@ object LDAExample { val input = "data/mllib/sample_lda_data.txt" // Creates a Spark context and a SQL context - val conf = new SparkConf().setAppName(s"${this.getClass.getSimpleName}") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() // $example on$ // Loads data - val rowRDD = sc.textFile(input).filter(_.nonEmpty) + val rowRDD = spark.read.text(input).rdd.filter(_.nonEmpty) .map(_.split(" ").map(_.toDouble)).map(Vectors.dense).map(Row(_)) val schema = StructType(Array(StructField(FEATURES_COL, new VectorUDT, false))) - val dataset = sqlContext.createDataFrame(rowRDD, schema) + val dataset = spark.createDataFrame(rowRDD, schema) // Trains a LDA model val lda = new LDA() @@ -71,7 +68,7 @@ object LDAExample { transformed.show(false) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala index f68aef708201..cb6e2492f513 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala @@ -18,22 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.regression.LinearRegression // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object LinearRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LinearRegressionWithElasticNetExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("LinearRegressionWithElasticNetExample").getOrCreate() // $example on$ // Load training data - val training = sqlContext.read.format("libsvm") + val training = spark.read.format("libsvm") .load("data/mllib/sample_linear_regression_data.txt") val lr = new LinearRegression() @@ -56,7 +53,7 @@ object LinearRegressionWithElasticNetExample { println(s"r2: ${trainingSummary.r2}") // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala index 89c5edf1ace9..50670d7b3841 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala @@ -18,23 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.{BinaryLogisticRegressionSummary, LogisticRegression} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions.max object LogisticRegressionSummaryExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LogisticRegressionSummaryExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("LogisticRegressionSummaryExample").getOrCreate() + import spark.implicits._ // Load training data - val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val lr = new LogisticRegression() .setMaxIter(10) @@ -71,7 +68,7 @@ object LogisticRegressionSummaryExample { lrModel.setThreshold(bestThreshold) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala index 6e27571f1dc1..fcba813d5be4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala @@ -18,22 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.LogisticRegression // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object LogisticRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LogisticRegressionWithElasticNetExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder.appName("LogisticRegressionWithElasticNetExample").getOrCreate() // $example on$ // Load training data - val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val lr = new LogisticRegression() .setMaxIter(10) @@ -47,7 +45,7 @@ object LogisticRegressionWithElasticNetExample { println(s"Coefficients: ${lrModel.coefficients} Intercept: ${lrModel.intercept}") // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala index aafb5efd698e..896d8fadbe67 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala @@ -15,23 +15,19 @@ * limitations under the License. */ -// scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.MaxAbsScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object MaxAbsScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MaxAbsScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new MaxAbsScaler() .setInputCol("features") .setOutputCol("scaledFeatures") @@ -43,7 +39,7 @@ object MaxAbsScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala index 9a03f69f5af0..bcdca0fa0440 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.MinMaxScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object MinMaxScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MinMaxScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new MinMaxScaler() .setInputCol("features") @@ -44,7 +41,8 @@ object MinMaxScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala index d1441b5497a8..5fb3536060c9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression @@ -28,7 +27,7 @@ import org.apache.spark.ml.tuning.{CrossValidator, ParamGridBuilder} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * A simple example demonstrating model selection using CrossValidator. @@ -42,13 +41,12 @@ import org.apache.spark.sql.SQLContext object ModelSelectionViaCrossValidationExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ModelSelectionViaCrossValidationExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder.appName("ModelSelectionViaCrossValidationExample").getOrCreate() // $example on$ // Prepare training data from a list of (id, text, label) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -98,7 +96,7 @@ object ModelSelectionViaCrossValidationExample { val cvModel = cv.fit(training) // Prepare test documents, which are unlabeled (id, text) tuples. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -114,7 +112,7 @@ object ModelSelectionViaCrossValidationExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala index fcad17a81758..6bc082982c89 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala @@ -17,13 +17,12 @@ package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * A simple example demonstrating model selection using TrainValidationSplit. @@ -36,13 +35,12 @@ import org.apache.spark.sql.SQLContext object ModelSelectionViaTrainValidationSplitExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ModelSelectionViaTrainValidationSplitExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder.appName("ModelSelectionViaTrainValidationSplitExample").getOrCreate() // $example on$ // Prepare training and test data. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) val lr = new LinearRegression() @@ -75,6 +73,6 @@ object ModelSelectionViaTrainValidationSplitExample { .show() // $example off$ - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala index d7d1e82f6f84..a11fe1b4b262 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.MultilayerPerceptronClassifier import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * An example for Multilayer Perceptron Classification. @@ -31,13 +30,11 @@ import org.apache.spark.sql.SQLContext object MultilayerPerceptronClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MultilayerPerceptronClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("MultilayerPerceptronClassifierExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm") + val data = spark.read.format("libsvm") .load("data/mllib/sample_multiclass_classification_data.txt") // Split the data into train and test val splits = data.randomSplit(Array(0.6, 0.4), seed = 1234L) @@ -63,7 +60,7 @@ object MultilayerPerceptronClassifierExample { println("Precision:" + evaluator.evaluate(predictionAndLabels)) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala index 77b913aaa3fa..1b71a398905e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.NGram // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NGramExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NGramExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("NGramExample").getOrCreate() // $example on$ - val wordDataFrame = sqlContext.createDataFrame(Seq( + val wordDataFrame = spark.createDataFrame(Seq( (0, Array("Hi", "I", "heard", "about", "Spark")), (1, Array("I", "wish", "Java", "could", "use", "case", "classes")), (2, Array("Logistic", "regression", "models", "are", "neat")) @@ -41,7 +38,8 @@ object NGramExample { val ngramDataFrame = ngram.transform(wordDataFrame) ngramDataFrame.take(3).map(_.getAs[Stream[String]]("ngrams").toList).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala index 5ea1270c9781..8d54555cd3c6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ -import org.apache.spark.ml.classification.{NaiveBayes} +import org.apache.spark.ml.classification.NaiveBayes import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NaiveBayesExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NaiveBayesExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("NaiveBayesExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Split the data into training and test sets (30% held out for testing) val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) @@ -53,6 +50,8 @@ object NaiveBayesExample { val precision = evaluator.evaluate(predictions) println("Precision:" + precision) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala index 6b33c16c7403..4622d69ef98c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Normalizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NormalizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NormalizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Normalize each Vector using $L^1$ norm. val normalizer = new Normalizer() @@ -46,7 +43,8 @@ object NormalizerExample { val lInfNormData = normalizer.transform(dataFrame, normalizer.p -> Double.PositiveInfinity) lInfNormData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala index cb9fe65a85e8..338436100c5c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object OneHotEncoderExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("OneHotEncoderExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a"), (1, "b"), (2, "c"), @@ -52,7 +49,8 @@ object OneHotEncoderExample { val encoded = encoder.transform(indexed) encoded.select("id", "categoryVec").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index 0b5d31c0ff90..e2351c682d75 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit.{NANOSECONDS => NANO} import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.classification.{LogisticRegression, OneVsRest} @@ -31,7 +30,7 @@ import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.DataFrame // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * An example runner for Multiclass to Binary Reduction with One Vs Rest. @@ -110,18 +109,16 @@ object OneVsRestExample { } private def run(params: Params) { - val conf = new SparkConf().setAppName(s"OneVsRestExample with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"OneVsRestExample with $params").getOrCreate() // $example on$ - val inputData = sqlContext.read.format("libsvm").load(params.input) + val inputData = spark.read.format("libsvm").load(params.input) // compute the train/test split: if testInput is not provided use part of input. val data = params.testInput match { case Some(t) => // compute the number of features in the training set. val numFeatures = inputData.first().getAs[Vector](1).size - val testData = sqlContext.read.option("numFeatures", numFeatures.toString) + val testData = spark.read.option("numFeatures", numFeatures.toString) .format("libsvm").load(t) Array[DataFrame](inputData, testData) case None => @@ -175,7 +172,7 @@ object OneVsRestExample { println(fprs.map {case (label, fpr) => label + "\t" + fpr}.mkString("\n")) // $example off$ - sc.stop() + spark.stop() } private def time[R](block: => R): (Long, R) = { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala index 535652ec6c79..14394d562468 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala @@ -18,18 +18,15 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.PCA import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PCAExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PCAExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("PCAExample").getOrCreate() // $example on$ val data = Array( @@ -37,7 +34,7 @@ object PCAExample { Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) ) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val pca = new PCA() .setInputCol("features") .setOutputCol("pcaFeatures") @@ -47,7 +44,8 @@ object PCAExample { val result = pcaDF.select("pcaFeatures") result.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala index 6c29063626ba..61b34aebd9c3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.{Pipeline, PipelineModel} import org.apache.spark.ml.classification.LogisticRegression @@ -26,18 +25,16 @@ import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PipelineExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PipelineExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("PipelineExample").getOrCreate() // $example on$ // Prepare training documents from a list of (id, text, label) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -71,7 +68,7 @@ object PipelineExample { val sameModel = PipelineModel.load("/tmp/spark-logistic-regression-model") // Prepare test documents, which are unlabeled (id, text) tuples. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -87,7 +84,7 @@ object PipelineExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala index 3014008ea0ce..4d8c672a55c3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -18,18 +18,15 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.PolynomialExpansion import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PolynomialExpansionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PolynomialExpansionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() // $example on$ val data = Array( @@ -37,7 +34,7 @@ object PolynomialExpansionExample { Vectors.dense(0.0, 0.0), Vectors.dense(0.6, -1.1) ) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") .setOutputCol("polyFeatures") @@ -45,7 +42,8 @@ object PolynomialExpansionExample { val polyDF = polynomialExpansion.transform(df) polyDF.select("polyFeatures").take(3).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala index e64e673a485e..0839c609f1d9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -15,25 +15,21 @@ * limitations under the License. */ -// scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.QuantileDiscretizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object QuantileDiscretizerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("QuantileDiscretizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("QuantileDiscretizerExample").getOrCreate() + import spark.implicits._ // $example on$ val data = Array((0, 18.0), (1, 19.0), (2, 8.0), (3, 5.0), (4, 2.2)) - val df = sc.parallelize(data).toDF("id", "hour") + val df = spark.createDataFrame(data).toDF("id", "hour") val discretizer = new QuantileDiscretizer() .setInputCol("hour") @@ -43,7 +39,7 @@ object QuantileDiscretizerExample { val result = discretizer.fit(df).transform(df) result.show() // $example off$ - sc.stop() + + spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala index bec831d51c58..699b621db90a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.RFormula // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RFormulaExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RFormulaExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() // $example on$ - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( (7, "US", 18, 1.0), (8, "CA", 12, 0.0), (9, "NZ", 15, 0.0) @@ -43,7 +40,8 @@ object RFormulaExample { val output = formula.fit(dataset).transform(dataset) output.select("features", "label").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala index 6c9b52cf259e..4192a9c73704 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RandomForestClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RandomForestClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RandomForestClassifierExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -91,7 +88,7 @@ object RandomForestClassifierExample { println("Learned classification forest model:\n" + rfModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala index 4d2db017f346..5632f0419a80 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RandomForestRegressorExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RandomForestRegressorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RandomForestRegressorExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -78,7 +75,7 @@ object RandomForestRegressorExample { println("Learned regression forest model:\n" + rfModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala index 202925acadff..f03b29ba327c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.SQLTransformer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object SQLTransformerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SQLTransformerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame( + val df = spark.createDataFrame( Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") val sqlTrans = new SQLTransformer().setStatement( @@ -39,6 +36,8 @@ object SQLTransformerExample { sqlTrans.transform(df).show() // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index f4d1fe57856a..dff771950715 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.param.ParamMap import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -35,15 +34,13 @@ import org.apache.spark.sql.{Row, SQLContext} object SimpleParamsExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleParamsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("SimpleParamsExample").getOrCreate() + import spark.implicits._ // Prepare training data. // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes // into DataFrames, where it uses the case class metadata to infer the schema. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -59,7 +56,7 @@ object SimpleParamsExample { .setRegParam(0.01) // Learn a LogisticRegression model. This uses the parameters stored in lr. - val model1 = lr.fit(training.toDF()) + val model1 = lr.fit(training) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this @@ -82,7 +79,7 @@ object SimpleParamsExample { println("Model 2 was fit using parameters: " + model2.parent.extractParamMap()) // Prepare test data. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) @@ -91,14 +88,14 @@ object SimpleParamsExample { // LogisticRegressionModel.transform will only use the 'features' column. // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. - model2.transform(test.toDF()) + model2.transform(test) .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => println(s"($features, $label) -> prob=$prob, prediction=$prediction") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 960280137cbf..05199007f015 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -20,12 +20,11 @@ package org.apache.spark.examples.ml import scala.beans.BeanInfo -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} @BeanInfo case class LabeledDocument(id: Long, text: String, label: Double) @@ -43,13 +42,11 @@ case class Document(id: Long, text: String) object SimpleTextClassificationPipeline { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() + import spark.implicits._ // Prepare training documents, which are labeled. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledDocument(0L, "a b c d e spark", 1.0), LabeledDocument(1L, "b d", 0.0), LabeledDocument(2L, "spark f g h", 1.0), @@ -73,7 +70,7 @@ object SimpleTextClassificationPipeline { val model = pipeline.fit(training.toDF()) // Prepare test documents, which are unlabeled. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( Document(4L, "spark i j k"), Document(5L, "l m n"), Document(6L, "spark hadoop spark"), @@ -87,7 +84,7 @@ object SimpleTextClassificationPipeline { println(s"($id, $text) --> prob=$prob, prediction=$prediction") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala index e3439677e78d..55f777c6e228 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StandardScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StandardScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StandardScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new StandardScaler() .setInputCol("features") @@ -46,7 +43,8 @@ object StandardScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala index 8199be12c155..85e79c8cb38a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala @@ -18,31 +18,29 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StopWordsRemover // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StopWordsRemoverExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StopWordsRemoverExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() // $example on$ val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = spark.createDataFrame(Seq( (0, Seq("I", "saw", "the", "red", "baloon")), (1, Seq("Mary", "had", "a", "little", "lamb")) )).toDF("id", "raw") remover.transform(dataSet).show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala index 3f0e870c8dc6..e01a768da988 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StringIndexer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StringIndexerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StringIndexerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame( + val df = spark.createDataFrame( Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) ).toDF("id", "category") @@ -42,7 +39,8 @@ object StringIndexerExample { val indexed = indexer.fit(df).transform(df) indexed.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index 396f073e6b32..910ef62a2670 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{HashingTF, IDF, Tokenizer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object TfIdfExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("TfIdfExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() // $example on$ - val sentenceData = sqlContext.createDataFrame(Seq( + val sentenceData = spark.createDataFrame(Seq( (0, "Hi I heard about Spark"), (0, "I wish Java could use case classes"), (1, "Logistic regression models are neat") @@ -50,6 +47,8 @@ object TfIdfExample { val rescaledData = idfModel.transform(featurizedData) rescaledData.select("features", "label").take(3).foreach(println) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala index c667728d6326..4f0c47b3c84c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{RegexTokenizer, Tokenizer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object TokenizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("TokenizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() // $example on$ - val sentenceDataFrame = sqlContext.createDataFrame(Seq( + val sentenceDataFrame = spark.createDataFrame(Seq( (0, "Hi I heard about Spark"), (1, "I wish Java could use case classes"), (2, "Logistic,regression,models,are,neat") @@ -48,7 +45,8 @@ object TokenizerExample { val regexTokenized = regexTokenizer.transform(sentenceDataFrame) regexTokenized.select("words", "label").take(3).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala index 768a8c069047..56b7263b192e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.VectorAssembler import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorAssemblerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorAssemblerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() // $example on$ - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( Seq((0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0)) ).toDF("id", "hour", "mobile", "userFeatures", "clicked") @@ -43,7 +40,8 @@ object VectorAssemblerExample { val output = assembler.transform(dataset) println(output.select("features", "clicked").first()) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala index 3bef37ba360b..214ad91634ee 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.VectorIndexer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorIndexerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorIndexerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() // $example on$ - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val indexer = new VectorIndexer() .setInputCol("features") @@ -48,7 +45,8 @@ object VectorIndexerExample { val indexedData = indexerModel.transform(data) indexedData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala index 01377d80e7e5..716bf023a808 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -18,31 +18,29 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ +import java.util.Arrays + import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} import org.apache.spark.ml.feature.VectorSlicer import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.Row import org.apache.spark.sql.types.StructType // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorSlicerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorSlicerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() // $example on$ - val data = Array(Row(Vectors.dense(-2.0, 2.3, 0.0))) + val data = Arrays.asList(Row(Vectors.dense(-2.0, 2.3, 0.0))) val defaultAttr = NumericAttribute.defaultAttr val attrs = Array("f1", "f2", "f3").map(defaultAttr.withName) val attrGroup = new AttributeGroup("userFeatures", attrs.asInstanceOf[Array[Attribute]]) - val dataRDD = sc.parallelize(data) - val dataset = sqlContext.createDataFrame(dataRDD, StructType(Array(attrGroup.toStructField()))) + val dataset = spark.createDataFrame(data, StructType(Array(attrGroup.toStructField()))) val slicer = new VectorSlicer().setInputCol("userFeatures").setOutputCol("features") @@ -52,7 +50,8 @@ object VectorSlicerExample { val output = slicer.transform(dataset) println(output.select("userFeatures", "features").first()) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala index e77aa59ba32b..292b6d9f7776 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Word2Vec // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object Word2VecExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("Word2Vec example") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("Word2Vec example").getOrCreate() // $example on$ // Input data: Each row is a bag of words from a sentence or document. - val documentDF = sqlContext.createDataFrame(Seq( + val documentDF = spark.createDataFrame(Seq( "Hi I heard about Spark".split(" "), "I wish Java could use case classes".split(" "), "Logistic regression models are neat".split(" ") @@ -48,6 +45,8 @@ object Word2VecExample { val result = model.transform(documentDF) result.select("result").take(3).foreach(println) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index e89d555884dd..c2bf1548b568 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -27,7 +27,7 @@ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel, Regex import org.apache.spark.mllib.clustering.{DistributedLDAModel, EMLDAOptimizer, LDA, OnlineLDAOptimizer} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * An example Latent Dirichlet Allocation (LDA) app. Run with @@ -189,8 +189,8 @@ object LDAExample { vocabSize: Int, stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.getOrCreate() + import spark.implicits._ // Get dataset of document texts // One document per line in each text file. If the input consists of many small files, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala index fdb01b86dd78..cd4f0bb0de60 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -18,22 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.mllib -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.evaluation.{RankingMetrics, RegressionMetrics} import org.apache.spark.mllib.recommendation.{ALS, Rating} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RankingMetricsExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("RankingMetricsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("RankingMetricsExample").getOrCreate() + import spark.implicits._ // $example on$ // Read in the ratings data - val ratings = sc.textFile("data/mllib/sample_movielens_data.txt").map { line => + val ratings = spark.read.text("data/mllib/sample_movielens_data.txt").rdd.map { line => val fields = line.split("::") Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble - 2.5) }.cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala index add634c957b4..22c47a694d34 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -18,22 +18,22 @@ package org.apache.spark.examples.mllib -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RegressionMetricsExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RegressionMetricsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RegressionMetricsExample").getOrCreate() // $example on$ // Load the data - val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_linear_regression_data.txt").cache() + val data = spark + .read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + .rdd.map(row => LabeledPoint(row.getDouble(0), row.get(1).asInstanceOf[Vector])) + .cache() // Build the model val numIterations = 100 @@ -61,6 +61,8 @@ object RegressionMetricsExample { // Explained variance println(s"Explained variance = ${metrics.explainedVariance}") // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 918e124065e4..2f0fe704f709 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -19,9 +19,8 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext, Time} @@ -60,9 +59,9 @@ object SqlNetworkWordCount { // Convert RDDs of the words DStream to DataFrame and run SQL query words.foreachRDD { (rdd: RDD[String], time: Time) => - // Get the singleton instance of SQLContext - val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) - import sqlContext.implicits._ + // Get the singleton instance of SparkSession + val spark = SparkSessionSingleton.getInstance(rdd.sparkContext.getConf) + import spark.implicits._ // Convert RDD[String] to RDD[case class] to DataFrame val wordsDataFrame = rdd.map(w => Record(w)).toDF() @@ -72,7 +71,7 @@ object SqlNetworkWordCount { // Do word count on table using SQL and print it val wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") println(s"========= $time =========") wordCountsDataFrame.show() } @@ -87,14 +86,14 @@ object SqlNetworkWordCount { case class Record(word: String) -/** Lazily instantiated singleton instance of SQLContext */ -object SQLContextSingleton { +/** Lazily instantiated singleton instance of SparkSession */ +object SparkSessionSingleton { - @transient private var instance: SQLContext = _ + @transient private var instance: SparkSession = _ - def getInstance(sparkContext: SparkContext): SQLContext = { + def getInstance(sparkConf: SparkConf): SparkSession = { if (instance == null) { - instance = new SQLContext(sparkContext) + instance = SparkSession.builder.config(sparkConf).getOrCreate() } instance } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index cb15b4b91f91..aec0215b4094 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -952,6 +952,11 @@ def dump_profiles(self, path): """ self.profiler_collector.dump_profiles(path) + def getConf(self): + conf = SparkConf() + conf.setAll(self._conf.getAll()) + return conf + def _test(): import atexit From 1e7d9bfb5a41f5c2479ab3b4d4081f00bf00bd31 Mon Sep 17 00:00:00 2001 From: Sebastien Rainville Date: Wed, 4 May 2016 14:32:36 -0700 Subject: [PATCH 070/313] [SPARK-13001][CORE][MESOS] Prevent getting offers when reached max cores Similar to https://github.com/apache/spark/pull/8639 This change rejects offers for 120s when reached `spark.cores.max` in coarse-grained mode to mitigate offer starvation. This prevents Mesos to send us offers again and again, starving other frameworks. This is especially problematic when running many small frameworks on the same Mesos cluster, e.g. many small Sparks streaming jobs, and cause the bigger spark jobs to stop receiving offers. By rejecting the offers for a long period of time, they become available to those other frameworks. Author: Sebastien Rainville Closes #10924 from sebastienrainville/master. (cherry picked from commit eb019af9a9cadb127eab1b6d30312169ed90f808) Signed-off-by: Andrew Or --- .../mesos/CoarseMesosSchedulerBackend.scala | 53 +++++++++++++------ .../cluster/mesos/MesosSchedulerUtils.scala | 4 ++ .../CoarseMesosSchedulerBackendSuite.scala | 13 +++++ 3 files changed, 53 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 50b452c72f8a..2c5be1f52889 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -109,10 +109,14 @@ private[spark] class CoarseMesosSchedulerBackend( private val slaveOfferConstraints = parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) - // reject offers with mismatched constraints in seconds + // Reject offers with mismatched constraints in seconds private val rejectOfferDurationForUnmetConstraints = getRejectOfferDurationForUnmetConstraints(sc) + // Reject offers when we reached the maximum number of cores for this framework + private val rejectOfferDurationForReachedMaxCores = + getRejectOfferDurationForReachedMaxCores(sc) + // A client for talking to the external shuffle service private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { if (shuffleServiceEnabled) { @@ -279,18 +283,32 @@ private[spark] class CoarseMesosSchedulerBackend( } private def declineUnmatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { - for (offer <- offers) { - val id = offer.getId.getValue - val offerAttributes = toAttributeMap(offer.getAttributesList) - val mem = getResource(offer.getResourcesList, "mem") - val cpus = getResource(offer.getResourcesList, "cpus") - val filters = Filters.newBuilder() - .setRefuseSeconds(rejectOfferDurationForUnmetConstraints).build() - - logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus" - + s" for $rejectOfferDurationForUnmetConstraints seconds") + offers.foreach { offer => + declineOffer(d, offer, Some("unmet constraints"), + Some(rejectOfferDurationForUnmetConstraints)) + } + } - d.declineOffer(offer.getId, filters) + private def declineOffer( + d: SchedulerDriver, + offer: Offer, + reason: Option[String] = None, + refuseSeconds: Option[Long] = None): Unit = { + + val id = offer.getId.getValue + val offerAttributes = toAttributeMap(offer.getAttributesList) + val mem = getResource(offer.getResourcesList, "mem") + val cpus = getResource(offer.getResourcesList, "cpus") + + logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem" + + s" cpu: $cpus for $refuseSeconds seconds" + + reason.map(r => s" (reason: $r)").getOrElse("")) + + refuseSeconds match { + case Some(seconds) => + val filters = Filters.newBuilder().setRefuseSeconds(seconds).build() + d.declineOffer(offer.getId, filters) + case _ => d.declineOffer(offer.getId) } } @@ -326,11 +344,12 @@ private[spark] class CoarseMesosSchedulerBackend( d.launchTasks( Collections.singleton(offer.getId), offerTasks.asJava) - } else { // decline - logDebug(s"Declining offer: $id with attributes: $offerAttributes " + - s"mem: $offerMem cpu: $offerCpus") - - d.declineOffer(offer.getId) + } else if (totalCoresAcquired >= maxCores) { + // Reject an offer for a configurable amount of time to avoid starving other frameworks + declineOffer(d, offer, Some("reached spark.cores.max"), + Some(rejectOfferDurationForReachedMaxCores)) + } else { + declineOffer(d, offer) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 1e322ac67941..7355ba317d9a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -352,4 +352,8 @@ private[mesos] trait MesosSchedulerUtils extends Logging { sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForUnmetConstraints", "120s") } + protected def getRejectOfferDurationForReachedMaxCores(sc: SparkContext): Long = { + sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForReachedMaxCores", "120s") + } + } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala index b18f0eb162b1..15d59e705291 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala @@ -147,6 +147,19 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite verifyDeclinedOffer(driver, createOfferId("o1"), true) } + test("mesos declines offers with a filter when reached spark.cores.max") { + val maxCores = 3 + setBackend(Map("spark.cores.max" -> maxCores.toString)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List( + (executorMemory, maxCores + 1), + (executorMemory, maxCores + 1))) + + verifyTaskLaunched("o1") + verifyDeclinedOffer(driver, createOfferId("o2"), true) + } + test("mesos assigns tasks round-robin on offers") { val executorCores = 4 val maxCores = executorCores * 2 From 701c667296fdc8a374852ec38366c91a02350e17 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 4 May 2016 14:40:54 -0700 Subject: [PATCH 071/313] [SPARK-15116] In REPL we should create SparkSession first and get SparkContext from it ## What changes were proposed in this pull request? see https://github.com/apache/spark/pull/12873#discussion_r61993910. The problem is, if we create `SparkContext` first and then call `SparkSession.builder.enableHiveSupport().getOrCreate()`, we will reuse the existing `SparkContext` and the hive flag won't be set. ## How was this patch tested? verified it locally. Author: Wenchen Fan Closes #12890 from cloud-fan/repl. --- .../org/apache/spark/repl/SparkILoop.scala | 20 ++++++-------- .../apache/spark/repl/SparkILoopInit.scala | 11 +++----- .../scala/org/apache/spark/repl/Main.scala | 27 +++++++++---------- .../org/apache/spark/repl/SparkILoop.scala | 11 +++----- 4 files changed, 26 insertions(+), 43 deletions(-) diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index c4f64505a216..b1e95d8fdb60 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1003,7 +1003,7 @@ class SparkILoop( // NOTE: Must be public for visibility @DeveloperApi - def createSparkContext(): SparkContext = { + def createSparkSession(): SparkSession = { val execUri = System.getenv("SPARK_EXECUTOR_URI") val jars = SparkILoop.getAddedJars val conf = new SparkConf() @@ -1019,22 +1019,18 @@ class SparkILoop( if (execUri != null) { conf.set("spark.executor.uri", execUri) } - sparkContext = new SparkContext(conf) - logInfo("Created spark context..") - Signaling.cancelOnInterrupt(sparkContext) - sparkContext - } - @DeveloperApi - // TODO: don't duplicate this code - def createSparkSession(): SparkSession = { - if (SparkSession.hiveClassesArePresent) { + val builder = SparkSession.builder.config(conf) + val sparkSession = if (SparkSession.hiveClassesArePresent) { logInfo("Creating Spark session with Hive support") - SparkSession.builder.enableHiveSupport().getOrCreate() + builder.enableHiveSupport().getOrCreate() } else { logInfo("Creating Spark session") - SparkSession.builder.getOrCreate() + builder.getOrCreate() } + sparkContext = sparkSession.sparkContext + Signaling.cancelOnInterrupt(sparkContext) + sparkSession } private def getMaster(): String = { diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index f1febb9497c7..29f63de8a0fa 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -123,19 +123,14 @@ private[repl] trait SparkILoopInit { def initializeSpark() { intp.beQuietDuring { command(""" + @transient val spark = org.apache.spark.repl.Main.interp.createSparkSession() @transient val sc = { - val _sc = org.apache.spark.repl.Main.interp.createSparkContext() + val _sc = spark.sparkContext _sc.uiWebUrl.foreach(webUrl => println(s"Spark context Web UI available at ${webUrl}")) println("Spark context available as 'sc' " + s"(master = ${_sc.master}, app id = ${_sc.applicationId}).") - _sc - } - """) - command(""" - @transient val spark = { - val _session = org.apache.spark.repl.Main.interp.createSparkSession() println("Spark session available as 'spark'.") - _session + _sc } """) command("import org.apache.spark.SparkContext._") diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index a17175980925..005edda2bee7 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -71,35 +71,32 @@ object Main extends Logging { } } - def createSparkContext(): SparkContext = { + def createSparkSession(): SparkSession = { val execUri = System.getenv("SPARK_EXECUTOR_URI") conf.setIfMissing("spark.app.name", "Spark shell") - // SparkContext will detect this configuration and register it with the RpcEnv's - // file server, setting spark.repl.class.uri to the actual URI for executors to - // use. This is sort of ugly but since executors are started as part of SparkContext - // initialization in certain cases, there's an initialization order issue that prevents - // this from being set after SparkContext is instantiated. - .set("spark.repl.class.outputDir", outputDir.getAbsolutePath()) + // SparkContext will detect this configuration and register it with the RpcEnv's + // file server, setting spark.repl.class.uri to the actual URI for executors to + // use. This is sort of ugly but since executors are started as part of SparkContext + // initialization in certain cases, there's an initialization order issue that prevents + // this from being set after SparkContext is instantiated. + conf.set("spark.repl.class.outputDir", outputDir.getAbsolutePath()) if (execUri != null) { conf.set("spark.executor.uri", execUri) } if (System.getenv("SPARK_HOME") != null) { conf.setSparkHome(System.getenv("SPARK_HOME")) } - sparkContext = new SparkContext(conf) - logInfo("Created spark context..") - Signaling.cancelOnInterrupt(sparkContext) - sparkContext - } - def createSparkSession(): SparkSession = { + val builder = SparkSession.builder.config(conf) if (SparkSession.hiveClassesArePresent) { - sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate() + sparkSession = builder.enableHiveSupport().getOrCreate() logInfo("Created Spark session with Hive support") } else { - sparkSession = SparkSession.builder.getOrCreate() + sparkSession = builder.getOrCreate() logInfo("Created Spark session") } + sparkContext = sparkSession.sparkContext + Signaling.cancelOnInterrupt(sparkContext) sparkSession } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala index d74b7965316f..bbdb992d8a5e 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -36,19 +36,14 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) def initializeSpark() { intp.beQuietDuring { processLine(""" + @transient val spark = org.apache.spark.repl.Main.createSparkSession() @transient val sc = { - val _sc = org.apache.spark.repl.Main.createSparkContext() + val _sc = spark.sparkContext _sc.uiWebUrl.foreach(webUrl => println(s"Spark context Web UI available at ${webUrl}")) println("Spark context available as 'sc' " + s"(master = ${_sc.master}, app id = ${_sc.applicationId}).") - _sc - } - """) - processLine(""" - @transient val spark = { - val _session = org.apache.spark.repl.Main.createSparkSession() println("Spark session available as 'spark'.") - _session + _sc } """) processLine("import org.apache.spark.SparkContext._") From aca46ecf8ebc1e477cf1ca8aabf45861bf12e225 Mon Sep 17 00:00:00 2001 From: sethah Date: Wed, 4 May 2016 16:46:13 -0700 Subject: [PATCH 072/313] [MINOR][SQL] Fix typo in DataFrameReader csv documentation ## What changes were proposed in this pull request? Typo fix ## How was this patch tested? No tests My apologies for the tiny PR, but I stumbled across this today and wanted to get it corrected for 2.0. Author: sethah Closes #12912 from sethah/csv_typo. (cherry picked from commit b28137764716f56fa1a923c4278624a56364a505) Signed-off-by: Andrew Or --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 2d4a68f3c3a9..5bf696c1c392 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -407,7 +407,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `header` (default `false`): uses the first line as names of columns.
  • *
  • `ignoreLeadingWhiteSpace` (default `false`): defines whether or not leading whitespaces * from values being read should be skipped.
  • - *
  • `ignoreTrailingWhiteSpace` (default `fDataFraalse`): defines whether or not trailing + *
  • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing * whitespaces from values being read should be skipped.
  • *
  • `nullValue` (default empty string): sets the string representation of a null value.
  • *
  • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
  • From fa3c5507fb7b612f78750abfd60dfdde0ad86da3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 4 May 2016 17:39:30 -0700 Subject: [PATCH 073/313] [SPARK-14896][SQL] Deprecate HiveContext in python ## What changes were proposed in this pull request? See title. ## How was this patch tested? PySpark tests. Author: Andrew Or Closes #12917 from andrewor14/deprecate-hive-context-python. (cherry picked from commit fa79d346e1a79ceda6ccd20e74eb850e769556ea) Signed-off-by: Andrew Or --- python/pyspark/sql/column.py | 2 -- python/pyspark/sql/context.py | 9 ++++++++- python/pyspark/sql/streaming.py | 2 +- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 43e9baece2de..90fb76f9b5a5 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -418,8 +418,6 @@ def over(self, window): >>> window = Window.partitionBy("name").orderBy("age").rowsBetween(-1, 1) >>> from pyspark.sql.functions import rank, min >>> # df.select(rank().over(window), min('age').over(window)) - - .. note:: Window functions is only supported with HiveContext in 1.4 """ from pyspark.sql.window import WindowSpec if not isinstance(window, WindowSpec): diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 2096236d7f36..78ab2e81bfce 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -17,6 +17,7 @@ from __future__ import print_function import sys +import warnings if sys.version >= '3': basestring = unicode = str @@ -434,7 +435,6 @@ def streams(self): return ContinuousQueryManager(self._ssql_ctx.streams()) -# TODO(andrew): deprecate this class HiveContext(SQLContext): """A variant of Spark SQL that integrates with data stored in Hive. @@ -444,8 +444,15 @@ class HiveContext(SQLContext): :param sparkContext: The SparkContext to wrap. :param jhiveContext: An optional JVM Scala HiveContext. If set, we do not instantiate a new :class:`HiveContext` in the JVM, instead we make all calls to this object. + + .. note:: Deprecated in 2.0.0. Use SparkSession.builder.enableHiveSupport().getOrCreate(). """ + warnings.warn( + "HiveContext is deprecated in Spark 2.0.0. Please use " + + "SparkSession.builder.enableHiveSupport().getOrCreate() instead.", + DeprecationWarning) + def __init__(self, sparkContext, jhiveContext=None): if jhiveContext is None: sparkSession = SparkSession.withHiveSupport(sparkContext) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index bf03fdca9139..8238b8e7cde6 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -221,7 +221,7 @@ def _test(): globs['os'] = os globs['sc'] = sc globs['sqlContext'] = SQLContext(sc) - globs['hiveContext'] = HiveContext(sc) + globs['hiveContext'] = HiveContext._createForTesting(sc) globs['df'] = \ globs['sqlContext'].read.format('text').stream('python/test_support/sql/streaming') From d90359d63b49e1fd1079858c6aad9b444ba2c253 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 4 May 2016 18:27:25 -0700 Subject: [PATCH 074/313] [SPARK-6339][SQL] Supports CREATE TEMPORARY VIEW tableIdentifier AS query ## What changes were proposed in this pull request? This PR support new SQL syntax CREATE TEMPORARY VIEW. Like: ``` CREATE TEMPORARY VIEW viewName AS SELECT * from xx CREATE OR REPLACE TEMPORARY VIEW viewName AS SELECT * from xx CREATE TEMPORARY VIEW viewName (c1 COMMENT 'blabla', c2 COMMENT 'blabla') AS SELECT * FROM xx ``` ## How was this patch tested? Unit tests. Author: Sean Zhong Closes #12872 from clockfly/spark-6399. (cherry picked from commit 8fb1463d6a832f187f323d97635e5bec1e93c6f3) Signed-off-by: Yin Huai --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 13 ++- .../spark/sql/execution/command/views.scala | 86 ++++++++++---- .../spark/sql/hive/HiveDDLCommandSuite.scala | 2 +- .../sql/hive/execution/SQLViewSuite.scala | 110 ++++++++++++++++-- 5 files changed, 175 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 273ad9289169..ee27d69ab3f9 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -84,7 +84,7 @@ statement | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? tableIdentifier #dropTable - | CREATE (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier + | CREATE (OR REPLACE)? TEMPORARY? VIEW (IF NOT EXISTS)? tableIdentifier identifierCommentList? (COMMENT STRING)? (PARTITIONED ON identifierList)? (TBLPROPERTIES tablePropertyList)? AS query #createView diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 60388df59646..146e036bb484 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -935,7 +935,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * For example: * {{{ - * CREATE VIEW [IF NOT EXISTS] [db_name.]view_name + * CREATE [TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name * [(column_name [COMMENT column_comment], ...) ] * [COMMENT view_comment] * [TBLPROPERTIES (property_name = property_value, ...)] @@ -958,7 +958,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.query, Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty), ctx.EXISTS != null, - ctx.REPLACE != null + ctx.REPLACE != null, + ctx.TEMPORARY != null ) } } @@ -975,7 +976,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.query, Map.empty, allowExist = false, - replace = true) + replace = true, + isTemporary = false) } /** @@ -989,7 +991,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { query: QueryContext, properties: Map[String, String], allowExist: Boolean, - replace: Boolean): LogicalPlan = { + replace: Boolean, + isTemporary: Boolean): LogicalPlan = { val sql = Option(source(query)) val tableDesc = CatalogTable( identifier = visitTableIdentifier(name), @@ -1000,7 +1003,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { viewOriginalText = sql, viewText = sql, comment = comment) - CreateViewCommand(tableDesc, plan(query), allowExist, replace, command(ctx)) + CreateViewCommand(tableDesc, plan(query), allowExist, replace, isTemporary, command(ctx)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 0f656ef53e39..70ce5c842901 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, Row, SparkSession} -import org.apache.spark.sql.catalyst.SQLBuilder +import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} @@ -37,6 +37,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} * already exists, throws analysis exception. * @param replace if true, and if the view already exists, updates it; if false, and if the view * already exists, throws analysis exception. + * @param isTemporary if true, the view is created as a temporary view. Temporary views are dropped + * at the end of current Spark session. Existing permanent relations with the same + * name are not visible to the current session while the temporary view exists, + * unless they are specified with full qualified table name with database prefix. * @param sql the original sql */ case class CreateViewCommand( @@ -44,6 +48,7 @@ case class CreateViewCommand( child: LogicalPlan, allowExisting: Boolean, replace: Boolean, + isTemporary: Boolean, sql: String) extends RunnableCommand { @@ -55,12 +60,23 @@ case class CreateViewCommand( require(tableDesc.tableType == CatalogTableType.VIEW) require(tableDesc.viewText.isDefined) - private val tableIdentifier = tableDesc.identifier - if (allowExisting && replace) { throw new AnalysisException("CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed.") } + // Disallows 'CREATE TEMPORARY VIEW IF NOT EXISTS' to be consistent with 'CREATE TEMPORARY TABLE' + if (allowExisting && isTemporary) { + throw new AnalysisException( + "It is not allowed to define a TEMPORARY view with IF NOT EXISTS.") + } + + // Temporary view names should NOT contain database prefix like "database.table" + if (isTemporary && tableDesc.identifier.database.isDefined) { + val database = tableDesc.identifier.database.get + throw new AnalysisException( + s"It is not allowed to add database prefix ${database} for the TEMPORARY view name.") + } + override def run(sparkSession: SparkSession): Seq[Row] = { // If the plan cannot be analyzed, throw an exception and don't proceed. val qe = sparkSession.executePlan(child) @@ -70,29 +86,59 @@ case class CreateViewCommand( require(tableDesc.schema == Nil || tableDesc.schema.length == analyzedPlan.output.length) val sessionState = sparkSession.sessionState - if (sessionState.catalog.tableExists(tableIdentifier)) { - if (allowExisting) { - // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view - // already exists. - } else if (replace) { - // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` - sessionState.catalog.alterTable(prepareTable(sparkSession, analyzedPlan)) + if (isTemporary) { + createTemporaryView(tableDesc.identifier, sparkSession, analyzedPlan) + } else { + // Adds default database for permanent table if it doesn't exist, so that tableExists() + // only check permanent tables. + val database = tableDesc.identifier.database.getOrElse( + sessionState.catalog.getCurrentDatabase) + val tableIdentifier = tableDesc.identifier.copy(database = Option(database)) + + if (sessionState.catalog.tableExists(tableIdentifier)) { + if (allowExisting) { + // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view + // already exists. + } else if (replace) { + // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` + sessionState.catalog.alterTable(prepareTable(sparkSession, analyzedPlan)) + } else { + // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already + // exists. + throw new AnalysisException( + s"View $tableIdentifier already exists. If you want to update the view definition, " + + "please use ALTER VIEW AS or CREATE OR REPLACE VIEW AS") + } } else { - // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already - // exists. - throw new AnalysisException(s"View $tableIdentifier already exists. " + - "If you want to update the view definition, please use ALTER VIEW AS or " + - "CREATE OR REPLACE VIEW AS") + // Create the view if it doesn't exist. + sessionState.catalog.createTable( + prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) } - } else { - // Create the view if it doesn't exist. - sessionState.catalog.createTable( - prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) } - Seq.empty[Row] } + private def createTemporaryView( + table: TableIdentifier, sparkSession: SparkSession, analyzedPlan: LogicalPlan): Unit = { + + val sessionState = sparkSession.sessionState + val catalog = sessionState.catalog + + // Projects column names to alias names + val logicalPlan = { + if (tableDesc.schema.isEmpty) { + analyzedPlan + } else { + val projectList = analyzedPlan.output.zip(tableDesc.schema).map { + case (attr, col) => Alias(attr, col.name)() + } + sparkSession.executePlan(Project(projectList, analyzedPlan)).analyzed + } + } + + catalog.createTempTable(table.table, logicalPlan, replace) + } + /** * Returns a [[CatalogTable]] that can be used to save in the catalog. This comment canonicalize * SQL based on the analyzed plan, and also creates the proper schema for the view. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index c4ebc604dc13..3d74235dc52b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -39,7 +39,7 @@ class HiveDDLCommandSuite extends PlanTest { parser.parsePlan(sql).collect { case CreateTable(desc, allowExisting) => (desc, allowExisting) case CreateTableAsSelectLogicalPlan(desc, _, allowExisting) => (desc, allowExisting) - case CreateViewCommand(desc, _, allowExisting, _, _) => (desc, allowExisting) + case CreateViewCommand(desc, _, allowExisting, _, _, _) => (desc, allowExisting) }.head } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 51848470502a..72f9fba13d4b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -37,11 +37,21 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sqlContext.sql(s"DROP TABLE IF EXISTS jt") } - test("nested views") { - withView("jtv1", "jtv2") { - sql("CREATE VIEW jtv1 AS SELECT * FROM jt WHERE id > 3").collect() - sql("CREATE VIEW jtv2 AS SELECT * FROM jtv1 WHERE id < 6").collect() + test("nested views (interleaved with temporary views)") { + withView("jtv1", "jtv2", "jtv3", "temp_jtv1", "temp_jtv2", "temp_jtv3") { + sql("CREATE VIEW jtv1 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE VIEW jtv2 AS SELECT * FROM jtv1 WHERE id < 6") checkAnswer(sql("select count(*) FROM jtv2"), Row(2)) + + // Checks temporary views + sql("CREATE TEMPORARY VIEW temp_jtv1 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE TEMPORARY VIEW temp_jtv2 AS SELECT * FROM temp_jtv1 WHERE id < 6") + checkAnswer(sql("select count(*) FROM temp_jtv2"), Row(2)) + + // Checks interleaved temporary view and normal view + sql("CREATE TEMPORARY VIEW temp_jtv3 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE VIEW jtv3 AS SELECT * FROM temp_jtv3 WHERE id < 6") + checkAnswer(sql("select count(*) FROM jtv3"), Row(2)) } } @@ -57,6 +67,33 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("error handling: fail if the temp view name contains the database prefix") { + // Fully qualified table name like "database.table" is not allowed for temporary view + val e = intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW default.myabcdview AS SELECT * FROM jt") + } + assert(e.message.contains("It is not allowed to add database prefix")) + } + + test("error handling: disallow IF NOT EXISTS for CREATE TEMPORARY VIEW") { + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW IF NOT EXISTS myabcdview AS SELECT * FROM jt") + } + assert(e.message.contains("It is not allowed to define a TEMPORARY view with IF NOT EXISTS")) + } + + test("error handling: fail if the temp view sql itself is invalid") { + // A table that does not exist for temporary view + intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM table_not_exist1345") + } + + // A column that does not exist, for temporary view + intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT random1234 FROM jt") + } + } + test("correctly parse CREATE VIEW statement") { withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { sql( @@ -69,18 +106,70 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("correctly parse CREATE TEMPORARY VIEW statement") { + withView("testView") { + sql( + """CREATE TEMPORARY VIEW + |testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') + |TBLPROPERTIES ('a' = 'b') + |AS SELECT * FROM jt + |""".stripMargin) + checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) + } + } + + test("should NOT allow CREATE TEMPORARY VIEW when TEMPORARY VIEW with same name exists") { + withView("testView") { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + } + + assert(e.message.contains("Temporary table") && e.message.contains("already exists")) + } + } + + test("should allow CREATE TEMPORARY VIEW when a permanent VIEW with same name exists") { + withView("testView", "default.testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + } + } + + test("should allow CREATE permanent VIEW when a TEMPORARY VIEW with same name exists") { + withView("testView", "default.testView") { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + sql("CREATE VIEW testView AS SELECT id FROM jt") + } + } + test("correctly handle CREATE VIEW IF NOT EXISTS") { withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { withTable("jt2") { - sql("CREATE VIEW testView AS SELECT id FROM jt") + withView("testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") - val df = (1 until 10).map(i => i -> i).toDF("i", "j") - df.write.format("json").saveAsTable("jt2") - sql("CREATE VIEW IF NOT EXISTS testView AS SELECT * FROM jt2") + val df = (1 until 10).map(i => i -> i).toDF("i", "j") + df.write.format("json").saveAsTable("jt2") + sql("CREATE VIEW IF NOT EXISTS testView AS SELECT * FROM jt2") - // make sure our view doesn't change. + // make sure our view doesn't change. + checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) + } + } + } + } + + test(s"correctly handle CREATE OR REPLACE TEMPORARY VIEW") { + withTable("jt2") { + withView("testView") { + sql("CREATE OR REPLACE TEMPORARY VIEW testView AS SELECT id FROM jt") checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) - sql("DROP VIEW testView") + + sql("CREATE OR REPLACE TEMPORARY VIEW testView AS SELECT id AS i, id AS j FROM jt") + // make sure the view has been changed. + checkAnswer(sql("SELECT * FROM testView ORDER BY i"), (1 to 9).map(i => Row(i, i))) } } } @@ -215,5 +304,4 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } } - } From 689b0fc819a16d6013bd4bfec2063354841a916e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 4 May 2016 18:47:27 -0700 Subject: [PATCH 075/313] [SPARK-14993][SQL] Fix Partition Discovery Inconsistency when Input is a Path to Parquet File #### What changes were proposed in this pull request? When we load a dataset, if we set the path to ```/path/a=1```, we will not take `a` as the partitioning column. However, if we set the path to ```/path/a=1/file.parquet```, we take `a` as the partitioning column and it shows up in the schema. This PR is to fix the behavior inconsistency issue. The base path contains a set of paths that are considered as the base dirs of the input datasets. The partitioning discovery logic will make sure it will stop when it reaches any base path. By default, the paths of the dataset provided by users will be base paths. Below are three typical cases, **Case 1**```sqlContext.read.parquet("/path/something=true/")```: the base path will be `/path/something=true/`, and the returned DataFrame will not contain a column of `something`. **Case 2**```sqlContext.read.parquet("/path/something=true/a.parquet")```: the base path will be still `/path/something=true/`, and the returned DataFrame will also not contain a column of `something`. **Case 3**```sqlContext.read.parquet("/path/")```: the base path will be `/path/`, and the returned DataFrame will have the column of `something`. Users also can override the basePath by setting `basePath` in the options to pass the new base path to the data source. For example, ```sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")```, and the returned DataFrame will have the column of `something`. The related PRs: - https://github.com/apache/spark/pull/9651 - https://github.com/apache/spark/pull/10211 #### How was this patch tested? Added a couple of test cases Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #12828 from gatorsmile/readPartitionedTable. (cherry picked from commit ef55e46c9225ddceebeaf19398519cbe651c1728) Signed-off-by: Yin Huai --- .../PartitioningAwareFileCatalog.scala | 42 ++++++++----- .../ParquetPartitionDiscoverySuite.scala | 60 +++++++++++++++++++ 2 files changed, 88 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index 9d997d628579..2c44b399cb95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -133,23 +133,37 @@ abstract class PartitioningAwareFileCatalog( /** * Contains a set of paths that are considered as the base dirs of the input datasets. * The partitioning discovery logic will make sure it will stop when it reaches any - * base path. By default, the paths of the dataset provided by users will be base paths. - * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path - * will be `/path/something=true/`, and the returned DataFrame will not contain a column of - * `something`. If users want to override the basePath. They can set `basePath` in the options - * to pass the new base path to the data source. - * For the above example, if the user-provided base path is `/path/`, the returned + * base path. + * + * By default, the paths of the dataset provided by users will be base paths. + * Below are three typical examples, + * Case 1) `sqlContext.read.parquet("/path/something=true/")`: the base path will be + * `/path/something=true/`, and the returned DataFrame will not contain a column of `something`. + * Case 2) `sqlContext.read.parquet("/path/something=true/a.parquet")`: the base path will be + * still `/path/something=true/`, and the returned DataFrame will also not contain a column of + * `something`. + * Case 3) `sqlContext.read.parquet("/path/")`: the base path will be `/path/`, and the returned * DataFrame will have the column of `something`. + * + * Users also can override the basePath by setting `basePath` in the options to pass the new base + * path to the data source. + * For example, `sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")`, + * and the returned DataFrame will have the column of `something`. */ private def basePaths: Set[Path] = { - val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) - userDefinedBasePath.getOrElse { - // If the user does not provide basePath, we will just use paths. - paths.toSet - }.map { hdfsPath => - // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). - val fs = hdfsPath.getFileSystem(hadoopConf) - hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + parameters.get("basePath").map(new Path(_)) match { + case Some(userDefinedBasePath) => + val fs = userDefinedBasePath.getFileSystem(hadoopConf) + if (!fs.isDirectory(userDefinedBasePath)) { + throw new IllegalArgumentException("Option 'basePath' must be a directory") + } + Set(fs.makeQualified(userDefinedBasePath)) + + case None => + paths.map { path => + // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). + val qualifiedPath = path.getFileSystem(hadoopConf).makeQualified(path) + if (leafFiles.contains(qualifiedPath)) qualifiedPath.getParent else qualifiedPath }.toSet } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 5bffb307ec80..cb2c2522b20c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -191,6 +191,29 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha checkThrows[AssertionError]("file://path/a=", "Empty partition column value") } + test("parse partition with base paths") { + // when the basePaths is the same as the path to a leaf directory + val partitionSpec1: Option[PartitionValues] = parsePartition( + path = new Path("file://path/a=10"), + defaultPartitionName = defaultPartitionName, + typeInference = true, + basePaths = Set(new Path("file://path/a=10")))._1 + + assert(partitionSpec1.isEmpty) + + // when the basePaths is the path to a base directory of leaf directories + val partitionSpec2: Option[PartitionValues] = parsePartition( + path = new Path("file://path/a=10"), + defaultPartitionName = defaultPartitionName, + typeInference = true, + basePaths = Set(new Path("file://path")))._1 + + assert(partitionSpec2 == + Option(PartitionValues( + ArrayBuffer("a"), + ArrayBuffer(Literal.create(10, IntegerType))))) + } + test("parse partitions") { def check( paths: Seq[String], @@ -413,6 +436,43 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } + test("read partitioned table using different path options") { + withTempDir { base => + val pi = 1 + val ps = "foo" + val path = makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps) + makeParquetFile( + (1 to 10).map(i => ParquetData(i, i.toString)), path) + + // when the input is the base path containing partitioning directories + val baseDf = sqlContext.read.parquet(base.getCanonicalPath) + assert(baseDf.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) + + // when the input is a path to the leaf directory containing a parquet file + val partDf = sqlContext.read.parquet(path.getCanonicalPath) + assert(partDf.schema.map(_.name) === Seq("intField", "stringField")) + + path.listFiles().foreach { f => + if (f.getName.toLowerCase().endsWith(".parquet")) { + // when the input is a path to a parquet file + val df = sqlContext.read.parquet(f.getCanonicalPath) + assert(df.schema.map(_.name) === Seq("intField", "stringField")) + } + } + + path.listFiles().foreach { f => + if (f.getName.toLowerCase().endsWith(".parquet")) { + // when the input is a path to a parquet file but `basePath` is overridden to + // the base path containing partitioning directories + val df = sqlContext + .read.option("basePath", base.getCanonicalPath) + .parquet(f.getCanonicalPath) + assert(df.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) + } + } + } + } + test("read partitioned table - partition key included in Parquet file") { withTempDir { base => for { From e12ec46c6b9b89dff5b8bfe610227d84b16388aa Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 4 May 2016 21:19:53 -0700 Subject: [PATCH 076/313] [SPARK-15131][SQL] Shutdown StateStore management thread when SparkContext has been shutdown ## What changes were proposed in this pull request? Make sure that whenever the StateStoreCoordinator cannot be contacted, assume that the SparkContext and RpcEnv on the driver has been shutdown, and therefore stop the StateStore management thread, and unload all loaded stores. ## How was this patch tested? Updated unit tests. Author: Tathagata Das Closes #12905 from tdas/SPARK-15131. (cherry picked from commit bde27b89a243247bd3069b24cb4bb1eee94edfd7) Signed-off-by: Shixiong Zhu --- .../streaming/state/StateStore.scala | 46 ++++++++++--------- .../streaming/state/StateStoreSuite.scala | 15 +++++- .../streaming/StreamingAggregationSuite.scala | 10 +++- 3 files changed, 48 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 952150632519..9948292470ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -113,7 +113,7 @@ case class KeyRemoved(key: UnsafeRow) extends StoreUpdate * the store is the active instance. Accordingly, it either keeps it loaded and performs * maintenance, or unloads the store. */ -private[state] object StateStore extends Logging { +private[sql] object StateStore extends Logging { val MAINTENANCE_INTERVAL_CONFIG = "spark.streaming.stateStore.maintenanceInterval" val MAINTENANCE_INTERVAL_DEFAULT_SECS = 60 @@ -155,6 +155,10 @@ private[state] object StateStore extends Logging { loadedProviders.contains(storeId) } + def isMaintenanceRunning: Boolean = loadedProviders.synchronized { + maintenanceTask != null + } + /** Unload and stop all state store providers */ def stop(): Unit = loadedProviders.synchronized { loadedProviders.clear() @@ -187,44 +191,44 @@ private[state] object StateStore extends Logging { */ private def doMaintenance(): Unit = { logDebug("Doing maintenance") - loadedProviders.synchronized { loadedProviders.toSeq }.foreach { case (id, provider) => - try { - if (verifyIfStoreInstanceActive(id)) { - provider.doMaintenance() - } else { - unload(id) - logInfo(s"Unloaded $provider") + if (SparkEnv.get == null) { + stop() + } else { + loadedProviders.synchronized { loadedProviders.toSeq }.foreach { case (id, provider) => + try { + if (verifyIfStoreInstanceActive(id)) { + provider.doMaintenance() + } else { + unload(id) + logInfo(s"Unloaded $provider") + } + } catch { + case NonFatal(e) => + logWarning(s"Error managing $provider, stopping management thread") + stop() } - } catch { - case NonFatal(e) => - logWarning(s"Error managing $provider") } } } private def reportActiveStoreInstance(storeId: StateStoreId): Unit = { - try { + if (SparkEnv.get != null) { val host = SparkEnv.get.blockManager.blockManagerId.host val executorId = SparkEnv.get.blockManager.blockManagerId.executorId coordinatorRef.foreach(_.reportActiveInstance(storeId, host, executorId)) logDebug(s"Reported that the loaded instance $storeId is active") - } catch { - case NonFatal(e) => - logWarning(s"Error reporting active instance of $storeId") } } private def verifyIfStoreInstanceActive(storeId: StateStoreId): Boolean = { - try { + if (SparkEnv.get != null) { val executorId = SparkEnv.get.blockManager.blockManagerId.executorId val verified = coordinatorRef.map(_.verifyIfInstanceActive(storeId, executorId)).getOrElse(false) - logDebug(s"Verified whether the loaded instance $storeId is active: $verified" ) + logDebug(s"Verified whether the loaded instance $storeId is active: $verified") verified - } catch { - case NonFatal(e) => - logWarning(s"Error verifying active instance of $storeId") - false + } else { + false } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index dd23925716b0..f8f8bc7d6ff8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -47,8 +47,14 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth private val keySchema = StructType(Seq(StructField("key", StringType, true))) private val valueSchema = StructType(Seq(StructField("value", IntegerType, true))) + before { + StateStore.stop() + require(!StateStore.isMaintenanceRunning) + } + after { StateStore.stop() + require(!StateStore.isMaintenanceRunning) } test("get, put, remove, commit, and all data iterator") { @@ -352,7 +358,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth } } - ignore("maintenance") { + test("maintenance") { val conf = new SparkConf() .setMaster("local") .setAppName("test") @@ -366,20 +372,26 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth val provider = new HDFSBackedStateStoreProvider( storeId, keySchema, valueSchema, storeConf, hadoopConf) + quietly { withSpark(new SparkContext(conf)) { sc => withCoordinatorRef(sc) { coordinatorRef => + require(!StateStore.isMaintenanceRunning, "StateStore is unexpectedly running") + for (i <- 1 to 20) { val store = StateStore.get( storeId, keySchema, valueSchema, i - 1, storeConf, hadoopConf) put(store, "a", i) store.commit() } + eventually(timeout(10 seconds)) { assert(coordinatorRef.getLocation(storeId).nonEmpty, "active instance was not reported") } // Background maintenance should clean up and generate snapshots + assert(StateStore.isMaintenanceRunning, "Maintenance task is not running") + eventually(timeout(10 seconds)) { // Earliest delta file should get cleaned up assert(!fileExists(provider, 1, isSnapshot = false), "earliest file not deleted") @@ -418,6 +430,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth require(SparkEnv.get === null) eventually(timeout(10 seconds)) { assert(!StateStore.isLoaded(storeId)) + assert(!StateStore.isMaintenanceRunning) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index bdf40f5cd45d..8da7742ffe0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.streaming +import org.scalatest.BeforeAndAfterAll + import org.apache.spark.SparkException import org.apache.spark.sql.StreamTest import org.apache.spark.sql.catalyst.analysis.Update import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scala.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext @@ -29,7 +32,12 @@ object FailureSinglton { var firstTime = true } -class StreamingAggregationSuite extends StreamTest with SharedSQLContext { +class StreamingAggregationSuite extends StreamTest with SharedSQLContext with BeforeAndAfterAll { + + override def afterAll(): Unit = { + super.afterAll() + StateStore.stop() + } import testImplicits._ From 2023faf6c5433cdca4cd654bf16b165a57e2b5dd Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 4 May 2016 21:30:13 -0700 Subject: [PATCH 077/313] [MINOR] remove dead code --- python/pyspark/sql/dataframe.py | 9 --------- 1 file changed, 9 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index bbe15f5f900d..5378c32a1a96 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -363,15 +363,6 @@ def coalesce(self, numPartitions): """ return DataFrame(self._jdf.coalesce(numPartitions), self.sql_ctx) - @since(1.3) - def repartition(self, numPartitions): - """Returns a new :class:`DataFrame` that has exactly ``numPartitions`` partitions. - - >>> df.repartition(10).rdd.getNumPartitions() - 10 - """ - return DataFrame(self._jdf.repartition(numPartitions), self.sql_ctx) - @since(1.3) def repartition(self, numPartitions, *cols): """ From 0914296cb847c8abaca2fc523b8dc22cdfafb776 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 4 May 2016 22:18:55 -0700 Subject: [PATCH 078/313] [SPARK-15132][MINOR][SQL] Debug log for generated code should be printed with proper indentation ## What changes were proposed in this pull request? Similar to #11990, GenerateOrdering and GenerateColumnAccessor should print debug log for generated code with proper indentation. ## How was this patch tested? Manually checked. Author: Kousuke Saruta Closes #12908 from sarutak/SPARK-15132. (cherry picked from commit 1a9b341581478f39421c7313e5e3981e8ab46abb) Signed-off-by: Reynold Xin --- .../sql/catalyst/expressions/codegen/GenerateOrdering.scala | 2 +- .../spark/sql/execution/columnar/GenerateColumnAccessor.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 5635c91830f4..dc4825cdd876 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -136,7 +136,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR } }""" - logDebug(s"Generated Ordering: ${CodeFormatter.format(code)}") + logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index e2e33e32463f..bd5cb800dde3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -224,7 +224,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera } }""" - logDebug(s"Generated ColumnarIterator: ${CodeFormatter.format(code)}") + logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] } From e28d21d3f5c71598bc45d920db706772c7deb19e Mon Sep 17 00:00:00 2001 From: Abhinav Gupta Date: Wed, 4 May 2016 22:22:01 -0700 Subject: [PATCH 079/313] [SPARK-15045] [CORE] Remove dead code in TaskMemoryManager.cleanUpAllAllocatedMemory for pageTable ## What changes were proposed in this pull request? Removed the DeadCode as suggested. Author: Abhinav Gupta Closes #12829 from abhi951990/master. (cherry picked from commit 1a5c6fcef131c60db4bc8852ca96445b9cb3bfac) Signed-off-by: Davies Liu --- .../org/apache/spark/memory/TaskMemoryManager.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 2796114fc545..89521c1d6c30 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -379,7 +379,6 @@ public long getOffsetInPage(long pagePlusOffsetAddress) { */ public long cleanUpAllAllocatedMemory() { synchronized (this) { - Arrays.fill(pageTable, null); for (MemoryConsumer c: consumers) { if (c != null && c.getUsed() > 0) { // In case of failed task, it's normal to see leaked memory @@ -387,14 +386,16 @@ public long cleanUpAllAllocatedMemory() { } } consumers.clear(); - } - for (MemoryBlock page : pageTable) { - if (page != null) { - memoryManager.tungstenMemoryAllocator().free(page); + for (MemoryBlock page : pageTable) { + if (page != null) { + logger.warn("leak a page: " + page + " in task " + taskAttemptId); + memoryManager.tungstenMemoryAllocator().free(page); + } } + Arrays.fill(pageTable, null); } - Arrays.fill(pageTable, null); + // release the memory that is not used by any consumer. memoryManager.releaseExecutionMemory(acquiredButNotUsed, taskAttemptId, tungstenMemoryMode); From 433bc34b1b51739f27a04dff6119bce0e15a48de Mon Sep 17 00:00:00 2001 From: Lining Sun Date: Thu, 5 May 2016 10:47:39 +0100 Subject: [PATCH 080/313] [SPARK-15123] upgrade org.json4s to 3.2.11 version ## What changes were proposed in this pull request? We had the issue when using snowplow in our Spark applications. Snowplow requires json4s version 3.2.11 while Spark still use a few years old version 3.2.10. The change is to upgrade json4s jar to 3.2.11. ## How was this patch tested? We built Spark jar and successfully ran our applications in local and cluster modes. Author: Lining Sun Closes #12901 from liningalex/master. (cherry picked from commit 592fc455639462fcf00ec02860d7c33470b73273) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 6 +++--- dev/deps/spark-deps-hadoop-2.3 | 6 +++--- dev/deps/spark-deps-hadoop-2.4 | 6 +++--- dev/deps/spark-deps-hadoop-2.6 | 6 +++--- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- pom.xml | 2 +- 6 files changed, 16 insertions(+), 16 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index a4ef9a9af293..8d3063b8169f 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -110,9 +110,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 19c8fad984b3..48c8ebddc321 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -101,9 +101,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index c2365f9cc562..df55fca1e740 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -102,9 +102,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 0d8afd19c561..2125b917e319 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -108,9 +108,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index f4274a9441a0..eece3c7b2d23 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -108,9 +108,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsp-api-2.1.jar jsr305-1.3.9.jar jta-1.1.jar diff --git a/pom.xml b/pom.xml index 1dce91b7dfe4..f33b2abae53f 100644 --- a/pom.xml +++ b/pom.xml @@ -619,7 +619,7 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.2.10 + 3.2.11 com.sun.jersey From 0c4e42beaa16d99da093302bd768f868e9583399 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 5 May 2016 10:51:03 +0100 Subject: [PATCH 081/313] [SPARK-12154] Upgrade to Jersey 2 ## What changes were proposed in this pull request? Replace com.sun.jersey with org.glassfish.jersey. Changes to the Spark Web UI code were required to compile. The changes were relatively standard Jersey migration things. ## How was this patch tested? I did a manual test for the standalone web APIs. Although I didn't test the functionality of the security filter itself, the code that changed non-trivially is how we actually register the filter. I attached a debugger to the Spark master and verified that the SecurityFilter code is indeed invoked upon hitting /api/v1/applications. Author: mcheah Closes #12715 from mccheah/feature/upgrade-jersey. (cherry picked from commit b7fdc23ccc5967de5799d8cf6f14289e71f29a1e) Signed-off-by: Sean Owen --- NOTICE | 11 +- core/pom.xml | 18 ++- .../spark/status/api/v1/ApiRootResource.scala | 11 +- .../spark/status/api/v1/SecurityFilter.scala | 16 +-- dev/deps/spark-deps-hadoop-2.2 | 42 +++--- dev/deps/spark-deps-hadoop-2.3 | 26 ++-- dev/deps/spark-deps-hadoop-2.4 | 27 ++-- dev/deps/spark-deps-hadoop-2.6 | 25 ++-- dev/deps/spark-deps-hadoop-2.7 | 25 ++-- external/kafka-assembly/pom.xml | 10 -- external/kinesis-asl-assembly/pom.xml | 13 +- pom.xml | 124 +++++++++++++++--- sql/hive-thriftserver/pom.xml | 12 -- yarn/pom.xml | 15 +++ 14 files changed, 244 insertions(+), 131 deletions(-) diff --git a/NOTICE b/NOTICE index 2a6fe237dcbe..f4b1260a2217 100644 --- a/NOTICE +++ b/NOTICE @@ -12,7 +12,9 @@ Common Development and Distribution License 1.0 The following components are provided under the Common Development and Distribution License 1.0. See project link for details. (CDDL 1.0) Glassfish Jasper (org.mortbay.jetty:jsp-2.1:6.1.14 - http://jetty.mortbay.org/project/modules/jsp-2.1) + (CDDL 1.0) JAX-RS (https://jax-rs-spec.java.net/) (CDDL 1.0) Servlet Specification 2.5 API (org.mortbay.jetty:servlet-api-2.5:6.1.14 - http://jetty.mortbay.org/project/modules/servlet-api-2.5) + (CDDL 1.0) (GPL2 w/ CPE) javax.annotation API (https://glassfish.java.net/nonav/public/CDDL+GPL.html) (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined) (Common Development and Distribution License (CDDL) v1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) @@ -22,15 +24,10 @@ Common Development and Distribution License 1.1 The following components are provided under the Common Development and Distribution License 1.1. See project link for details. + (CDDL 1.1) (GPL2 w/ CPE) org.glassfish.hk2 (https://hk2.java.net) (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/) (CDDL 1.1) (GPL2 w/ CPE) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.8 - https://jersey.dev.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.8 - https://jersey.dev.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.8 - https://jersey.dev.java.net/jersey-server/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/) + (CDDL 1.1) (GPL2 w/ CPE) Jersey 2 (https://jersey.java.net) ======================================================================== Common Public License 1.0 diff --git a/core/pom.xml b/core/pom.xml index 7349ad35b959..07b589637602 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -194,12 +194,24 @@ json4s-jackson_${scala.binary.version} - com.sun.jersey + org.glassfish.jersey.core + jersey-client + + + org.glassfish.jersey.core + jersey-common + + + org.glassfish.jersey.core jersey-server - com.sun.jersey - jersey-core + org.glassfish.jersey.containers + jersey-container-servlet + + + org.glassfish.jersey.containers + jersey-container-servlet-core org.apache.mesos diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 2cd51a9ed541..681f295006e3 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -21,10 +21,10 @@ import javax.servlet.ServletContext import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} -import com.sun.jersey.api.core.ResourceConfig -import com.sun.jersey.spi.container.servlet.ServletContainer import org.eclipse.jetty.server.handler.ContextHandler import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.glassfish.jersey.server.ServerProperties +import org.glassfish.jersey.servlet.ServletContainer import org.apache.spark.SecurityManager import org.apache.spark.ui.SparkUI @@ -191,12 +191,7 @@ private[spark] object ApiRootResource { val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) jerseyContext.setContextPath("/api") val holder: ServletHolder = new ServletHolder(classOf[ServletContainer]) - holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", - "com.sun.jersey.api.core.PackagesResourceConfig") - holder.setInitParameter("com.sun.jersey.config.property.packages", - "org.apache.spark.status.api.v1") - holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, - classOf[SecurityFilter].getCanonicalName) + holder.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.spark.status.api.v1") UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) jerseyContext.addServlet(holder, "/*") jerseyContext diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala index 95fbd96ade5a..b4a991eda35f 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala @@ -16,18 +16,16 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.WebApplicationException +import javax.ws.rs.container.{ContainerRequestContext, ContainerRequestFilter} import javax.ws.rs.core.Response +import javax.ws.rs.ext.Provider -import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter} - +@Provider private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext { - def filter(req: ContainerRequest): ContainerRequest = { - val user = Option(req.getUserPrincipal).map { _.getName }.orNull - if (uiRoot.securityManager.checkUIViewPermissions(user)) { - req - } else { - throw new WebApplicationException( + override def filter(req: ContainerRequestContext): Unit = { + val user = Option(req.getSecurityContext.getUserPrincipal).map { _.getName }.orNull + if (!uiRoot.securityManager.checkUIViewPermissions(user)) { + req.abortWith( Response .status(Response.Status.FORBIDDEN) .entity(raw"""user "$user"is not authorized""") diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 8d3063b8169f..669c88369356 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -1,14 +1,13 @@ JavaEWAH-0.3.2.jar RoaringBitmap-0.5.11.jar ST4-4.0.4.jar -activation-1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -49,12 +48,6 @@ datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar derby-10.10.1.1.jar eigenbase-properties-1.1.5.jar -gmbal-api-only-3.0.0-b023.jar -grizzly-framework-2.1.2.jar -grizzly-http-2.1.2.jar -grizzly-http-server-2.1.2.jar -grizzly-http-servlet-2.1.2.jar -grizzly-rcm-2.1.2.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar @@ -73,6 +66,9 @@ hadoop-yarn-client-2.2.0.jar hadoop-yarn-common-2.2.0.jar hadoop-yarn-server-common-2.2.0.jar hadoop-yarn-server-web-proxy-2.2.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar @@ -80,30 +76,26 @@ jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar -javax.servlet-3.1.jar -javax.servlet-api-3.0.1.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-grizzly2-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar -jersey-test-framework-core-1.9.jar -jersey-test-framework-grizzly2-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.7.1.jar -jettison-1.1.jar jetty-util-6.1.26.jar jline-2.12.jar joda-time-2.9.3.jar @@ -123,7 +115,6 @@ libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar lz4-1.3.0.jar -management-api-3.0.0-b012.jar mesos-0.21.1-shaded-protobuf.jar metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar @@ -135,6 +126,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -162,12 +154,12 @@ snappy-0.2.jar snappy-java-1.1.2.4.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar -stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 48c8ebddc321..c6e835d6a586 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -6,9 +6,9 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -68,6 +68,9 @@ hadoop-yarn-client-2.3.0.jar hadoop-yarn-common-2.3.0.jar hadoop-yarn-server-common-2.3.0.jar hadoop-yarn-server-web-proxy-2.3.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar @@ -75,25 +78,28 @@ jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -127,6 +133,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -160,6 +167,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index df55fca1e740..79c09b2d0e86 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -6,9 +6,9 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -68,6 +68,9 @@ hadoop-yarn-client-2.4.0.jar hadoop-yarn-common-2.4.0.jar hadoop-yarn-server-common-2.4.0.jar hadoop-yarn-server-web-proxy-2.4.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar @@ -75,26 +78,28 @@ jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -128,6 +133,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -161,6 +167,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 2125b917e319..be9b52cbef26 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -6,13 +6,13 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -73,6 +73,9 @@ hadoop-yarn-client-2.6.0.jar hadoop-yarn-common-2.6.0.jar hadoop-yarn-server-common-2.6.0.jar hadoop-yarn-server-web-proxy-2.6.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar httpclient-4.3.2.jar httpcore-4.3.2.jar @@ -87,20 +90,24 @@ jackson-module-scala_2.11-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -134,6 +141,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -167,6 +175,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index eece3c7b2d23..08baa3a13728 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -6,13 +6,13 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -73,6 +73,9 @@ hadoop-yarn-client-2.7.0.jar hadoop-yarn-common-2.7.0.jar hadoop-yarn-server-common-2.7.0.jar hadoop-yarn-server-web-proxy-2.7.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar httpclient-4.3.2.jar httpcore-4.3.2.jar @@ -87,20 +90,24 @@ jackson-module-scala_2.11-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -135,6 +142,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -168,6 +176,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index 62818f5e8f43..e1b5a7e713d7 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -65,16 +65,6 @@ protobuf-java provided - - com.sun.jersey - jersey-server - provided - - - com.sun.jersey - jersey-core - provided - net.jpountz.lz4 lz4 diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index d1c38c7ca5d6..e057b78abdc8 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -66,13 +66,18 @@ provided - com.sun.jersey - jersey-server + org.glassfish.jersey.core + jersey-client provided - com.sun.jersey - jersey-core + org.glassfish.jersey.core + jersey-common + provided + + + org.glassfish.jersey.core + jersey-server provided diff --git a/pom.xml b/pom.xml index f33b2abae53f..852136a998ab 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 3.3.2 3.2.10 2.7.8 - 1.9 + 2.22.2 2.9.3 3.5.2 1.3.9 @@ -360,7 +360,6 @@ provided - org.apache.commons commons-lang3 @@ -588,16 +587,44 @@ - com.sun.jersey + com.fasterxml.jackson.module + jackson-module-jaxb-annotations + ${fasterxml.jackson.version} + + + org.glassfish.jersey.core jersey-server ${jersey.version} - ${hadoop.deps.scope} - com.sun.jersey - jersey-core + org.glassfish.jersey.core + jersey-common + ${jersey.version} + + + org.glassfish.jersey.core + jersey-client + ${jersey.version} + + + org.glassfish.jersey.containers + jersey-container-servlet + ${jersey.version} + + + org.glassfish.jersey.containers + jersey-container-servlet-core ${jersey.version} - ${hadoop.deps.scope} + + + org.glassfish.jersey + jersey-client + ${jersey.version} + + + javax.ws.rs + javax.ws.rs-api + 2.0.1 org.scalanlp @@ -621,17 +648,6 @@ json4s-jackson_${scala.binary.version} 3.2.11 - - com.sun.jersey - jersey-json - ${jersey.version} - - - stax - stax-api - - - org.scala-lang scala-compiler @@ -814,6 +830,18 @@ junit junit + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -926,6 +954,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -954,6 +994,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -983,6 +1035,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1011,6 +1075,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1039,6 +1115,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 987103b1a953..a8b7aaff62a1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -64,18 +64,6 @@ ${hive.group} hive-beeline - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-json - - - com.sun.jersey - jersey-server - org.seleniumhq.selenium diff --git a/yarn/pom.xml b/yarn/pom.xml index 328bb6678db9..db7f3e51d3be 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -30,6 +30,7 @@ Spark Project YARN yarn + 1.9 @@ -147,20 +148,34 @@ test + + com.sun.jersey jersey-core test + ${jersey-1.version} com.sun.jersey jersey-json test + ${jersey-1.version} com.sun.jersey jersey-server test + ${jersey-1.version} + + + com.sun.jersey.contribs + jersey-guice + test + ${jersey-1.version} C:\TEMP\ Win XP --> C:\TEMP Solaris --> /var/tmp/ Linux --> /var/tmp ``` Second, a couple of test cases are added to verify if the commands work properly. #### How was this patch tested? Added a test case for it and correct the previous test cases. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #12081 from gatorsmile/mkdir. (cherry picked from commit 8cba57a75cf9e29b54d97366a039a97a2f305d5d) Signed-off-by: Andrew Or --- .../sql/catalyst/catalog/SessionCatalog.scala | 4 + .../spark/sql/execution/command/ddl.scala | 5 +- .../sql/execution/command/DDLSuite.scala | 249 +++++++++++------- .../sql/hive/execution/HiveDDLSuite.scala | 150 ++++++++++- 4 files changed, 311 insertions(+), 97 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index ff6303471e14..eff420eb4c5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -146,6 +146,10 @@ class SessionCatalog( currentDb = db } + /** + * Get the path for creating a non-default database when database location is not provided + * by users. + */ def getDefaultDBPath(db: String): String = { val database = if (conf.caseSensitiveAnalysis) db else db.toLowerCase new Path(new Path(conf.warehousePath), database + ".db").toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index aa06c014fb0a..085bdaff4e03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -40,7 +40,10 @@ import org.apache.spark.sql.types._ * unless 'ifNotExists' is true. * The syntax of using this command in SQL is: * {{{ - * CREATE DATABASE|SCHEMA [IF NOT EXISTS] database_name + * CREATE (DATABASE|SCHEMA) [IF NOT EXISTS] database_name + * [COMMENT database_comment] + * [LOCATION database_directory] + * [WITH DBPROPERTIES (property_name=property_value, ...)]; * }}} */ case class CreateDatabase( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 0ae099ecc2bd..6085098a709e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -95,49 +95,81 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) } + private def appendTrailingSlash(path: String): String = { + if (!path.endsWith(File.separator)) path + File.separator else path + } + test("the qualified path of a database is stored in the catalog") { val catalog = sqlContext.sessionState.catalog - val path = System.getProperty("java.io.tmpdir") - // The generated temp path is not qualified. - assert(!path.startsWith("file:/")) - sql(s"CREATE DATABASE db1 LOCATION '$path'") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri - assert("file" === pathInCatalog.getScheme) - assert(path === pathInCatalog.getPath) - - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir"))) { - sql(s"CREATE DATABASE db2") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + withTempDir { tmpDir => + val path = tmpDir.toString + // The generated temp path is not qualified. + assert(!path.startsWith("file:/")) + sql(s"CREATE DATABASE db1 LOCATION '$path'") + val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri assert("file" === pathInCatalog.getScheme) - assert(s"${sqlContext.conf.warehousePath}/db2.db" === pathInCatalog.getPath) - } + val expectedPath = if (path.endsWith(File.separator)) path.dropRight(1) else path + assert(expectedPath === pathInCatalog.getPath) + + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + sql(s"CREATE DATABASE db2") + val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + assert("file" === pathInCatalog.getScheme) + val expectedPath = appendTrailingSlash(sqlContext.conf.warehousePath) + "db2.db" + assert(expectedPath === pathInCatalog.getPath) + } - sql("DROP DATABASE db1") - sql("DROP DATABASE db2") + sql("DROP DATABASE db1") + sql("DROP DATABASE db2") + } } test("Create/Drop Database") { - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir") + File.separator)) { - val catalog = sqlContext.sessionState.catalog + withTempDir { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val databaseNames = Seq("db1", "`database`") + sql(s"CREATE DATABASE $dbName") + val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + val expectedLocation = + "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + assert(db1 == CatalogDatabase( + dbNameWithoutBackTicks, + "", + expectedLocation, + Map.empty)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.databaseExists(dbNameWithoutBackTicks)) + } finally { + catalog.reset() + } + } + } + } + } + test("Create/Drop Database - location") { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + withTempDir { tmpDir => + val path = tmpDir.toString + val dbPath = "file:" + path databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - - sql(s"CREATE DATABASE $dbName") + sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + System.getProperty("java.io.tmpdir") + - File.separator + s"$dbNameWithoutBackTicks.db" assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", - expectedLocation, + if (dbPath.endsWith(File.separator)) dbPath.dropRight(1) else dbPath, Map.empty)) sql(s"DROP DATABASE $dbName CASCADE") assert(!catalog.databaseExists(dbNameWithoutBackTicks)) @@ -149,77 +181,78 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("Create Database - database already exists") { - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir") + File.separator)) { - val catalog = sqlContext.sessionState.catalog - val databaseNames = Seq("db1", "`database`") - - databaseNames.foreach { dbName => - try { - val dbNameWithoutBackTicks = cleanIdentifier(dbName) - sql(s"CREATE DATABASE $dbName") - val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + System.getProperty("java.io.tmpdir") + - File.separator + s"$dbNameWithoutBackTicks.db" - assert(db1 == CatalogDatabase( - dbNameWithoutBackTicks, - "", - expectedLocation, - Map.empty)) - - val message = intercept[AnalysisException] { + withTempDir { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") - }.getMessage - assert(message.contains(s"Database '$dbNameWithoutBackTicks' already exists.")) - } finally { - catalog.reset() + val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + val expectedLocation = + "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + assert(db1 == CatalogDatabase( + dbNameWithoutBackTicks, + "", + expectedLocation, + Map.empty)) + + val message = intercept[AnalysisException] { + sql(s"CREATE DATABASE $dbName") + }.getMessage + assert(message.contains(s"Database '$dbNameWithoutBackTicks' already exists.")) + } finally { + catalog.reset() + } } } } } test("Alter/Describe Database") { - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir") + File.separator)) { - val catalog = sqlContext.sessionState.catalog - val databaseNames = Seq("db1", "`database`") + withTempDir { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") - databaseNames.foreach { dbName => - try { - val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = - "file:" + System.getProperty("java.io.tmpdir") + - File.separator + s"$dbNameWithoutBackTicks.db" - - sql(s"CREATE DATABASE $dbName") - - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), - Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: - Row("Location", location) :: - Row("Properties", "") :: Nil) - - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") - - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), - Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: - Row("Location", location) :: - Row("Properties", "((a,a), (b,b), (c,c))") :: Nil) - - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") - - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), - Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: - Row("Location", location) :: - Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) - } finally { - catalog.reset() + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) + val location = "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + + sql(s"CREATE DATABASE $dbName") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", location) :: + Row("Properties", "") :: Nil) + + sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", location) :: + Row("Properties", "((a,a), (b,b), (c,c))") :: Nil) + + sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", location) :: + Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) + } finally { + catalog.reset() + } } } } @@ -251,7 +284,43 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } - // TODO: test drop database in restrict mode + test("drop non-empty database in restrict mode") { + val catalog = sqlContext.sessionState.catalog + val dbName = "db1" + sql(s"CREATE DATABASE $dbName") + + // create a table in database + val tableIdent1 = TableIdentifier("tab1", Some(dbName)) + createTable(catalog, tableIdent1) + + // drop a non-empty database in Restrict mode + val message = intercept[AnalysisException] { + sql(s"DROP DATABASE $dbName RESTRICT") + }.getMessage + assert(message.contains(s"Database '$dbName' is not empty. One or more tables exist")) + + catalog.dropTable(tableIdent1, ignoreIfNotExists = false) + + assert(catalog.listDatabases().contains(dbName)) + sql(s"DROP DATABASE $dbName RESTRICT") + assert(!catalog.listDatabases().contains(dbName)) + } + + test("drop non-empty database in cascade mode") { + val catalog = sqlContext.sessionState.catalog + val dbName = "db1" + sql(s"CREATE DATABASE $dbName") + + // create a table in database + val tableIdent1 = TableIdentifier("tab1", Some(dbName)) + createTable(catalog, tableIdent1) + + // drop a non-empty database in CASCADE mode + assert(catalog.listTables(dbName).contains(tableIdent1)) + assert(catalog.listDatabases().contains(dbName)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.listDatabases().contains(dbName)) + } test("create table in default db") { val catalog = sqlContext.sessionState.catalog diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 373d1a1e0ebc..d55ddb251d00 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -20,21 +20,37 @@ package org.apache.spark.sql.hive.execution import java.io.File import org.apache.hadoop.fs.Path +import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} -import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTableType} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils -class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { +class HiveDDLSuite + extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { import hiveContext.implicits._ + override def afterEach(): Unit = { + try { + // drop all databases, tables and functions after each test + sqlContext.sessionState.catalog.reset() + } finally { + super.afterEach() + } + } // check if the directory for recording the data of the table exists. - private def tableDirectoryExists(tableIdentifier: TableIdentifier): Boolean = { + private def tableDirectoryExists( + tableIdentifier: TableIdentifier, + dbPath: Option[String] = None): Boolean = { val expectedTablePath = - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdentifier) + if (dbPath.isEmpty) { + hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdentifier) + } else { + new Path(new Path(dbPath.get), tableIdentifier.table).toString + } val filesystemPath = new Path(expectedTablePath) val fs = filesystemPath.getFileSystem(hiveContext.sessionState.newHadoopConf()) fs.exists(filesystemPath) @@ -56,7 +72,7 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("drop managed tables") { + test("drop managed tables in default database") { withTempDir { tmpDir => val tabName = "tab1" withTable(tabName) { @@ -83,7 +99,7 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("drop external data source table") { + test("drop external data source table in default database") { withTempDir { tmpDir => val tabName = "tab1" withTable(tabName) { @@ -365,4 +381,126 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { .exists(_.getString(0) == "# Detailed Table Information")) } } + + private def createDatabaseWithLocation(tmpDir: File, dirExists: Boolean): Unit = { + val catalog = sqlContext.sessionState.catalog + val dbName = "db1" + val tabName = "tab1" + val fs = new Path(tmpDir.toString).getFileSystem(hiveContext.sessionState.newHadoopConf()) + withTable(tabName) { + if (dirExists) { + assert(tmpDir.listFiles.isEmpty) + } else { + assert(!fs.exists(new Path(tmpDir.toString))) + } + sql(s"CREATE DATABASE $dbName Location '$tmpDir'") + val db1 = catalog.getDatabaseMetadata(dbName) + val dbPath = "file:" + tmpDir + assert(db1 == CatalogDatabase( + dbName, + "", + if (dbPath.endsWith(File.separator)) dbPath.dropRight(1) else dbPath, + Map.empty)) + sql("USE db1") + + sql(s"CREATE TABLE $tabName as SELECT 1") + assert(tableDirectoryExists(TableIdentifier(tabName), Option(tmpDir.toString))) + + assert(tmpDir.listFiles.nonEmpty) + sql(s"DROP TABLE $tabName") + + assert(tmpDir.listFiles.isEmpty) + sql(s"DROP DATABASE $dbName") + assert(!fs.exists(new Path(tmpDir.toString))) + } + } + + test("create/drop database - location without pre-created directory") { + withTempPath { tmpDir => + createDatabaseWithLocation(tmpDir, dirExists = false) + } + } + + test("create/drop database - location with pre-created directory") { + withTempDir { tmpDir => + createDatabaseWithLocation(tmpDir, dirExists = true) + } + } + + private def appendTrailingSlash(path: String): String = { + if (!path.endsWith(File.separator)) path + File.separator else path + } + + private def dropDatabase(cascade: Boolean, tableExists: Boolean): Unit = { + withTempPath { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val dbName = "db1" + val fs = new Path(path).getFileSystem(hiveContext.sessionState.newHadoopConf()) + val dbPath = new Path(path) + // the database directory does not exist + assert(!fs.exists(dbPath)) + + sql(s"CREATE DATABASE $dbName") + val catalog = sqlContext.sessionState.catalog + val expectedDBLocation = "file:" + appendTrailingSlash(dbPath.toString) + s"$dbName.db" + val db1 = catalog.getDatabaseMetadata(dbName) + assert(db1 == CatalogDatabase( + dbName, + "", + expectedDBLocation, + Map.empty)) + // the database directory was created + assert(fs.exists(dbPath) && fs.isDirectory(dbPath)) + sql(s"USE $dbName") + + val tabName = "tab1" + assert(!tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + sql(s"CREATE TABLE $tabName as SELECT 1") + assert(tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + + if (!tableExists) { + sql(s"DROP TABLE $tabName") + assert(!tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + } + + val sqlDropDatabase = s"DROP DATABASE $dbName ${if (cascade) "CASCADE" else "RESTRICT"}" + if (tableExists && !cascade) { + val message = intercept[AnalysisException] { + sql(sqlDropDatabase) + }.getMessage + assert(message.contains(s"Database $dbName is not empty. One or more tables exist.")) + // the database directory was not removed + assert(fs.exists(new Path(expectedDBLocation))) + } else { + sql(sqlDropDatabase) + // the database directory was removed and the inclusive table directories are also removed + assert(!fs.exists(new Path(expectedDBLocation))) + } + } + } + } + + test("drop database containing tables - CASCADE") { + dropDatabase(cascade = true, tableExists = true) + } + + test("drop an empty database - CASCADE") { + dropDatabase(cascade = true, tableExists = false) + } + + test("drop database containing tables - RESTRICT") { + dropDatabase(cascade = false, tableExists = true) + } + + test("drop an empty database - RESTRICT") { + dropDatabase(cascade = false, tableExists = false) + } + + test("drop default database") { + val message = intercept[AnalysisException] { + sql("DROP DATABASE default") + }.getMessage + assert(message.contains("Can not drop default database")) + } } From 59fa480b6d014369d58db9c5b77d82ddd17ee2a2 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 5 May 2016 14:35:15 -0700 Subject: [PATCH 090/313] [SPARK-15072][SQL][REPL][EXAMPLES] Remove SparkSession.withHiveSupport ## What changes were proposed in this pull request? Removing the `withHiveSupport` method of `SparkSession`, instead use `enableHiveSupport` ## How was this patch tested? ran tests locally Author: Sandeep Singh Closes #12851 from techaddict/SPARK-15072. (cherry picked from commit ed6f3f8a5f3a6bf7c53e13c2798de398c9a526a6) Signed-off-by: Andrew Or --- .../spark/examples/sql/hive/HiveFromSpark.scala | 14 +++++++++----- .../scala/org/apache/spark/sql/SparkSession.scala | 13 ------------- .../spark/sql/hive/thriftserver/SparkSQLEnv.scala | 10 ++++++---- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 7 +++++-- 4 files changed, 20 insertions(+), 24 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index ff33091621c1..a15cf5ded0e7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -36,15 +36,19 @@ object HiveFromSpark { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") - val sc = new SparkContext(sparkConf) // A hive context adds support for finding tables in the MetaStore and writing queries // using HiveQL. Users who do not have an existing Hive deployment can still create a // HiveContext. When not configured by the hive-site.xml, the context automatically // creates metastore_db and warehouse in the current directory. - val sparkSession = SparkSession.withHiveSupport(sc) - import sparkSession.implicits._ - import sparkSession.sql + val spark = SparkSession.builder + .config(sparkConf) + .enableHiveSupport() + .getOrCreate() + val sc = spark.sparkContext + + import spark.implicits._ + import spark.sql sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") @@ -74,7 +78,7 @@ object HiveFromSpark { println("Result of SELECT *:") sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index aa7c335c53d2..9ed3756628c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -816,17 +816,4 @@ object SparkSession { } } - /** - * Create a new [[SparkSession]] with a catalog backed by Hive. - */ - def withHiveSupport(sc: SparkContext): SparkSession = { - if (hiveClassesArePresent) { - sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive") - new SparkSession(sc) - } else { - throw new IllegalArgumentException( - "Unable to instantiate SparkSession with Hive support because Hive classes are not found.") - } - } - } 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 665a44e51a0c..8de223f444f7 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 @@ -54,13 +54,15 @@ private[hive] object SparkSQLEnv extends Logging { "spark.kryo.referenceTracking", maybeKryoReferenceTracking.getOrElse("false")) - sparkContext = new SparkContext(sparkConf) - sqlContext = SparkSession.withHiveSupport(sparkContext).wrapped - val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] + val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() + sparkContext = sparkSession.sparkContext + sqlContext = sparkSession.wrapped + + val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState] sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) sessionState.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) sessionState.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) - sqlContext.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) + sparkSession.conf.set("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 77a6a94a6719..a32001179949 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -290,8 +290,11 @@ object SetWarehouseLocationTest extends Logging { conf.set("spark.sql.warehouse.dir", warehouseLocation.toString) conf.set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString) - val sc = new SparkContext(conf) - val sparkSession = SparkSession.withHiveSupport(sc) + val sparkSession = SparkSession.builder + .config(conf) + .enableHiveSupport() + .getOrCreate() + val catalog = sparkSession.sessionState.catalog sparkSession.sql("drop table if exists testLocation") From e78b31b72d85ffcc596176a2c91b050f3a3ff3ee Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 5 May 2016 14:36:47 -0700 Subject: [PATCH 091/313] [SPARK-15135][SQL] Make sure SparkSession thread safe ## What changes were proposed in this pull request? Went through SparkSession and its members and fixed non-thread-safe classes used by SparkSession ## How was this patch tested? Existing unit tests Author: Shixiong Zhu Closes #12915 from zsxwing/spark-session-thread-safe. (cherry picked from commit bb9991dec5dd631b22a05e2e1b83b9082a845e8f) Signed-off-by: Andrew Or --- .../catalyst/analysis/FunctionRegistry.scala | 10 +- .../catalyst/catalog/InMemoryCatalog.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 102 ++++++++++-------- .../spark/sql/ExperimentalMethods.scala | 4 +- .../org/apache/spark/sql/SparkSession.scala | 7 +- .../apache/spark/sql/hive/test/TestHive.scala | 4 +- 6 files changed, 73 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 1bada2ce67ea..ac05dd3d0ef9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -28,7 +28,11 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.util.StringKeyHashMap -/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ +/** + * A catalog for looking up user defined functions, used by an [[Analyzer]]. + * + * Note: The implementation should be thread-safe to allow concurrent access. + */ trait FunctionRegistry { final def registerFunction(name: String, builder: FunctionBuilder): Unit = { @@ -62,7 +66,7 @@ trait FunctionRegistry { class SimpleFunctionRegistry extends FunctionRegistry { - private[sql] val functionBuilders = + protected val functionBuilders = StringKeyHashMap[(ExpressionInfo, FunctionBuilder)](caseSensitive = false) override def registerFunction( @@ -97,7 +101,7 @@ class SimpleFunctionRegistry extends FunctionRegistry { functionBuilders.remove(name).isDefined } - override def clear(): Unit = { + override def clear(): Unit = synchronized { functionBuilders.clear() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 1d2ca2863fae..c65f4611295e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -340,7 +340,7 @@ class InMemoryCatalog extends ExternalCatalog { catalog(db).functions(funcName) } - override def functionExists(db: String, funcName: String): Boolean = { + override def functionExists(db: String, funcName: String): Boolean = synchronized { requireDbExists(db) catalog(db).functions.contains(funcName) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index eff420eb4c5a..712770784bf9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.catalog +import javax.annotation.concurrent.GuardedBy + import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -37,7 +39,7 @@ import org.apache.spark.sql.catalyst.util.StringUtils * proxy to the underlying metastore (e.g. Hive Metastore) and it also manages temporary * tables and functions of the Spark Session that it belongs to. * - * This class is not thread-safe. + * This class must be thread-safe. */ class SessionCatalog( externalCatalog: ExternalCatalog, @@ -66,12 +68,14 @@ class SessionCatalog( } /** List of temporary tables, mapping from table name to their logical plan. */ + @GuardedBy("this") protected val tempTables = new mutable.HashMap[String, LogicalPlan] // Note: we track current database here because certain operations do not explicitly // specify the database (e.g. DROP TABLE my_table). In these cases we must first // check whether the temporary table or function exists, then, if not, operate on // the corresponding item in the current database. + @GuardedBy("this") protected var currentDb = { val defaultName = "default" val defaultDbDefinition = @@ -137,13 +141,13 @@ class SessionCatalog( externalCatalog.listDatabases(pattern) } - def getCurrentDatabase: String = currentDb + def getCurrentDatabase: String = synchronized { currentDb } def setCurrentDatabase(db: String): Unit = { if (!databaseExists(db)) { throw new AnalysisException(s"Database '$db' does not exist.") } - currentDb = db + synchronized { currentDb = db } } /** @@ -173,7 +177,7 @@ class SessionCatalog( * If no such database is specified, create it in the current database. */ def createTable(tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = { - val db = tableDefinition.identifier.database.getOrElse(currentDb) + val db = tableDefinition.identifier.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableDefinition.identifier.table) val newTableDefinition = tableDefinition.copy(identifier = TableIdentifier(table, Some(db))) externalCatalog.createTable(db, newTableDefinition, ignoreIfExists) @@ -189,7 +193,7 @@ class SessionCatalog( * this becomes a no-op. */ def alterTable(tableDefinition: CatalogTable): Unit = { - val db = tableDefinition.identifier.database.getOrElse(currentDb) + val db = tableDefinition.identifier.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableDefinition.identifier.table) val newTableDefinition = tableDefinition.copy(identifier = TableIdentifier(table, Some(db))) externalCatalog.alterTable(db, newTableDefinition) @@ -201,7 +205,7 @@ class SessionCatalog( * If the specified table is not found in the database then an [[AnalysisException]] is thrown. */ def getTableMetadata(name: TableIdentifier): CatalogTable = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.getTable(db, table) } @@ -212,7 +216,7 @@ class SessionCatalog( * If the specified table is not found in the database then return None if it doesn't exist. */ def getTableMetadataOption(name: TableIdentifier): Option[CatalogTable] = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.getTableOption(db, table) } @@ -227,7 +231,7 @@ class SessionCatalog( loadPath: String, isOverwrite: Boolean, holdDDLTime: Boolean): Unit = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.loadTable(db, table, loadPath, isOverwrite, holdDDLTime) } @@ -245,14 +249,14 @@ class SessionCatalog( holdDDLTime: Boolean, inheritTableSpecs: Boolean, isSkewedStoreAsSubdir: Boolean): Unit = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.loadPartition(db, table, loadPath, partition, isOverwrite, holdDDLTime, inheritTableSpecs, isSkewedStoreAsSubdir) } def defaultTablePath(tableIdent: TableIdentifier): String = { - val dbName = tableIdent.database.getOrElse(currentDb) + val dbName = tableIdent.database.getOrElse(getCurrentDatabase) val dbLocation = getDatabaseMetadata(dbName).locationUri new Path(new Path(dbLocation), formatTableName(tableIdent.table)).toString @@ -268,7 +272,7 @@ class SessionCatalog( def createTempTable( name: String, tableDefinition: LogicalPlan, - overrideIfExists: Boolean): Unit = { + overrideIfExists: Boolean): Unit = synchronized { val table = formatTableName(name) if (tempTables.contains(table) && !overrideIfExists) { throw new AnalysisException(s"Temporary table '$name' already exists.") @@ -285,7 +289,7 @@ class SessionCatalog( * * This assumes the database specified in `oldName` matches the one specified in `newName`. */ - def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = { + def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = synchronized { val db = oldName.database.getOrElse(currentDb) val newDb = newName.database.getOrElse(currentDb) if (db != newDb) { @@ -310,7 +314,7 @@ class SessionCatalog( * If no database is specified, this will first attempt to drop a temporary table with * the same name, then, if that does not exist, drop the table from the current database. */ - def dropTable(name: TableIdentifier, ignoreIfNotExists: Boolean): Unit = { + def dropTable(name: TableIdentifier, ignoreIfNotExists: Boolean): Unit = synchronized { val db = name.database.getOrElse(currentDb) val table = formatTableName(name.table) if (name.database.isDefined || !tempTables.contains(table)) { @@ -334,19 +338,21 @@ class SessionCatalog( * the same name, then, if that does not exist, return the table from the current database. */ def lookupRelation(name: TableIdentifier, alias: Option[String] = None): LogicalPlan = { - val db = name.database.getOrElse(currentDb) - val table = formatTableName(name.table) - val relation = - if (name.database.isDefined || !tempTables.contains(table)) { - val metadata = externalCatalog.getTable(db, table) - SimpleCatalogRelation(db, metadata, alias) - } else { - tempTables(table) - } - val qualifiedTable = SubqueryAlias(table, relation) - // If an alias was specified by the lookup, wrap the plan in a subquery so that - // attributes are properly qualified with this alias. - alias.map(a => SubqueryAlias(a, qualifiedTable)).getOrElse(qualifiedTable) + synchronized { + val db = name.database.getOrElse(currentDb) + val table = formatTableName(name.table) + val relation = + if (name.database.isDefined || !tempTables.contains(table)) { + val metadata = externalCatalog.getTable(db, table) + SimpleCatalogRelation(db, metadata, alias) + } else { + tempTables(table) + } + val qualifiedTable = SubqueryAlias(table, relation) + // If an alias was specified by the lookup, wrap the plan in a subquery so that + // attributes are properly qualified with this alias. + alias.map(a => SubqueryAlias(a, qualifiedTable)).getOrElse(qualifiedTable) + } } /** @@ -357,7 +363,7 @@ class SessionCatalog( * table with the same name, we will return false if the specified database does not * contain the table. */ - def tableExists(name: TableIdentifier): Boolean = { + def tableExists(name: TableIdentifier): Boolean = synchronized { val db = name.database.getOrElse(currentDb) val table = formatTableName(name.table) if (name.database.isDefined || !tempTables.contains(table)) { @@ -373,7 +379,7 @@ class SessionCatalog( * Note: The temporary table cache is checked only when database is not * explicitly specified. */ - def isTemporaryTable(name: TableIdentifier): Boolean = { + def isTemporaryTable(name: TableIdentifier): Boolean = synchronized { name.database.isEmpty && tempTables.contains(formatTableName(name.table)) } @@ -388,9 +394,11 @@ class SessionCatalog( def listTables(db: String, pattern: String): Seq[TableIdentifier] = { val dbTables = externalCatalog.listTables(db, pattern).map { t => TableIdentifier(t, Some(db)) } - val _tempTables = StringUtils.filterPattern(tempTables.keys.toSeq, pattern) - .map { t => TableIdentifier(t) } - dbTables ++ _tempTables + synchronized { + val _tempTables = StringUtils.filterPattern(tempTables.keys.toSeq, pattern) + .map { t => TableIdentifier(t) } + dbTables ++ _tempTables + } } // TODO: It's strange that we have both refresh and invalidate here. @@ -409,7 +417,7 @@ class SessionCatalog( * Drop all existing temporary tables. * For testing only. */ - def clearTempTables(): Unit = { + def clearTempTables(): Unit = synchronized { tempTables.clear() } @@ -417,7 +425,7 @@ class SessionCatalog( * Return a temporary table exactly as it was stored. * For testing only. */ - private[catalog] def getTempTable(name: String): Option[LogicalPlan] = { + private[catalog] def getTempTable(name: String): Option[LogicalPlan] = synchronized { tempTables.get(name) } @@ -441,7 +449,7 @@ class SessionCatalog( tableName: TableIdentifier, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.createPartitions(db, table, parts, ignoreIfExists) } @@ -454,7 +462,7 @@ class SessionCatalog( tableName: TableIdentifier, parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.dropPartitions(db, table, parts, ignoreIfNotExists) } @@ -469,7 +477,7 @@ class SessionCatalog( tableName: TableIdentifier, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.renamePartitions(db, table, specs, newSpecs) } @@ -484,7 +492,7 @@ class SessionCatalog( * this becomes a no-op. */ def alterPartitions(tableName: TableIdentifier, parts: Seq[CatalogTablePartition]): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.alterPartitions(db, table, parts) } @@ -494,7 +502,7 @@ class SessionCatalog( * If no database is specified, assume the table is in the current database. */ def getPartition(tableName: TableIdentifier, spec: TablePartitionSpec): CatalogTablePartition = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.getPartition(db, table, spec) } @@ -509,7 +517,7 @@ class SessionCatalog( def listPartitions( tableName: TableIdentifier, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.listPartitions(db, table, partialSpec) } @@ -532,7 +540,7 @@ class SessionCatalog( * If no such database is specified, create it in the current database. */ def createFunction(funcDefinition: CatalogFunction, ignoreIfExists: Boolean): Unit = { - val db = funcDefinition.identifier.database.getOrElse(currentDb) + val db = funcDefinition.identifier.database.getOrElse(getCurrentDatabase) val identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)) val newFuncDefinition = funcDefinition.copy(identifier = identifier) if (!functionExists(identifier)) { @@ -547,7 +555,7 @@ class SessionCatalog( * If no database is specified, assume the function is in the current database. */ def dropFunction(name: FunctionIdentifier, ignoreIfNotExists: Boolean): Unit = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val identifier = name.copy(database = Some(db)) if (functionExists(identifier)) { // TODO: registry should just take in FunctionIdentifier for type safety @@ -571,7 +579,7 @@ class SessionCatalog( * If no database is specified, this will return the function in the current database. */ def getFunctionMetadata(name: FunctionIdentifier): CatalogFunction = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) externalCatalog.getFunction(db, name.funcName) } @@ -579,7 +587,7 @@ class SessionCatalog( * Check if the specified function exists. */ def functionExists(name: FunctionIdentifier): Boolean = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) functionRegistry.functionExists(name.unquotedString) || externalCatalog.functionExists(db, name.funcName) } @@ -644,7 +652,7 @@ class SessionCatalog( /** * Look up the [[ExpressionInfo]] associated with the specified function, assuming it exists. */ - private[spark] def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = { + private[spark] def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = synchronized { // TODO: just make function registry take in FunctionIdentifier instead of duplicating this val qualifiedName = name.copy(database = name.database.orElse(Some(currentDb))) functionRegistry.lookupFunction(name.funcName) @@ -673,7 +681,9 @@ class SessionCatalog( * based on the function class and put the builder into the FunctionRegistry. * The name of this function in the FunctionRegistry will be `databaseName.functionName`. */ - def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { + def lookupFunction( + name: FunctionIdentifier, + children: Seq[Expression]): Expression = synchronized { // Note: the implementation of this function is a little bit convoluted. // We probably shouldn't use a single FunctionRegistry to register all three kinds of functions // (built-in, temp, and external). @@ -741,7 +751,7 @@ class SessionCatalog( * * This is mainly used for tests. */ - private[sql] def reset(): Unit = { + private[sql] def reset(): Unit = synchronized { val default = "default" listDatabases().filter(_ != default).foreach { db => dropDatabase(db, ignoreIfNotExists = false, cascade = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index c5df02848537..a49da6dc2b8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -42,9 +42,9 @@ class ExperimentalMethods private[sql]() { * @since 1.3.0 */ @Experimental - var extraStrategies: Seq[Strategy] = Nil + @volatile var extraStrategies: Seq[Strategy] = Nil @Experimental - var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil + @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 9ed3756628c8..2a893c6478d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -115,14 +115,17 @@ class SparkSession private( @transient private var _wrapped: SQLContext = _ - protected[sql] def wrapped: SQLContext = { + @transient + private val _wrappedLock = new Object + + protected[sql] def wrapped: SQLContext = _wrappedLock.synchronized { if (_wrapped == null) { _wrapped = new SQLContext(self, isRootContext = false) } _wrapped } - protected[sql] def setWrappedContext(sqlContext: SQLContext): Unit = { + protected[sql] def setWrappedContext(sqlContext: SQLContext): Unit = _wrappedLock.synchronized { _wrapped = sqlContext } 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 42746ece3c37..6d418c1dcf46 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 @@ -485,11 +485,11 @@ private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { private val removedFunctions = collection.mutable.ArrayBuffer.empty[(String, (ExpressionInfo, FunctionBuilder))] - def unregisterFunction(name: String): Unit = { + def unregisterFunction(name: String): Unit = synchronized { functionBuilders.remove(name).foreach(f => removedFunctions += name -> f) } - def restore(): Unit = { + def restore(): Unit = synchronized { removedFunctions.foreach { case (name, (info, builder)) => registerFunction(name, info, builder) } From 8b4ab590cb18b926c71c4cb4ec5b184b1b566770 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 5 May 2016 14:37:50 -0700 Subject: [PATCH 092/313] [SPARK-15134][EXAMPLE] Indent SparkSession builder patterns and update binary_classification_metrics_example.py ## What changes were proposed in this pull request? This issue addresses the comments in SPARK-15031 and also fix java-linter errors. - Use multiline format in SparkSession builder patterns. - Update `binary_classification_metrics_example.py` to use `SparkSession`. - Fix Java Linter errors (in SPARK-13745, SPARK-15031, and so far) ## How was this patch tested? After passing the Jenkins tests and run `dev/lint-java` manually. Author: Dongjoon Hyun Closes #12911 from dongjoon-hyun/SPARK-15134. (cherry picked from commit 2c170dd3d731bd848d62265431795e1c141d75d7) Signed-off-by: Andrew Or --- .../shuffle/ExternalShuffleBlockHandler.java | 7 +++++-- .../ml/JavaAFTSurvivalRegressionExample.java | 5 ++++- .../apache/spark/examples/ml/JavaALSExample.java | 5 ++++- .../spark/examples/ml/JavaBinarizerExample.java | 8 ++++---- .../examples/ml/JavaBisectingKMeansExample.java | 5 ++++- .../spark/examples/ml/JavaBucketizerExample.java | 5 ++++- .../examples/ml/JavaChiSqSelectorExample.java | 8 ++++---- .../examples/ml/JavaCountVectorizerExample.java | 5 ++++- .../apache/spark/examples/ml/JavaDCTExample.java | 8 ++++---- .../examples/ml/JavaDeveloperApiExample.java | 5 ++++- .../ml/JavaElementwiseProductExample.java | 7 +++---- .../JavaGradientBoostedTreeClassifierExample.java | 10 ++++++---- .../examples/ml/JavaIndexToStringExample.java | 5 ++++- .../spark/examples/ml/JavaKMeansExample.java | 5 ++++- .../apache/spark/examples/ml/JavaLDAExample.java | 5 ++++- .../examples/ml/JavaMaxAbsScalerExample.java | 14 ++++++++++---- .../examples/ml/JavaMinMaxScalerExample.java | 10 ++++++++-- .../spark/examples/ml/JavaNGramExample.java | 5 ++++- .../spark/examples/ml/JavaNaiveBayesExample.java | 5 ++++- .../spark/examples/ml/JavaNormalizerExample.java | 5 ++++- .../examples/ml/JavaOneHotEncoderExample.java | 5 ++++- .../spark/examples/ml/JavaOneVsRestExample.java | 5 ++++- .../apache/spark/examples/ml/JavaPCAExample.java | 5 ++++- .../spark/examples/ml/JavaPipelineExample.java | 5 ++++- .../ml/JavaPolynomialExpansionExample.java | 5 ++++- .../spark/examples/ml/JavaRFormulaExample.java | 5 ++++- .../examples/ml/JavaSQLTransformerExample.java | 5 ++++- .../examples/ml/JavaSimpleParamsExample.java | 5 ++++- .../examples/ml/JavaStandardScalerExample.java | 5 ++++- .../examples/ml/JavaStopWordsRemoverExample.java | 5 ++++- .../examples/ml/JavaStringIndexerExample.java | 5 ++++- .../spark/examples/ml/JavaTfIdfExample.java | 5 ++++- .../spark/examples/ml/JavaTokenizerExample.java | 5 ++++- .../examples/ml/JavaVectorAssemblerExample.java | 5 ++++- .../examples/ml/JavaVectorIndexerExample.java | 5 ++++- .../examples/ml/JavaVectorSlicerExample.java | 5 ++++- .../spark/examples/ml/JavaWord2VecExample.java | 5 ++++- .../apache/spark/examples/sql/JavaSparkSQL.java | 8 ++++++-- .../streaming/JavaSqlNetworkWordCount.java | 5 ++++- examples/src/main/python/ml/als_example.py | 5 ++++- examples/src/main/python/ml/binarizer_example.py | 5 ++++- .../main/python/ml/bisecting_k_means_example.py | 5 ++++- examples/src/main/python/ml/bucketizer_example.py | 5 ++++- .../src/main/python/ml/chisq_selector_example.py | 5 ++++- .../main/python/ml/count_vectorizer_example.py | 5 ++++- examples/src/main/python/ml/cross_validator.py | 5 ++++- examples/src/main/python/ml/dataframe_example.py | 5 ++++- examples/src/main/python/ml/dct_example.py | 5 ++++- .../ml/decision_tree_classification_example.py | 5 ++++- .../python/ml/decision_tree_regression_example.py | 5 ++++- .../main/python/ml/elementwise_product_example.py | 5 ++++- .../ml/estimator_transformer_param_example.py | 5 ++++- .../gradient_boosted_tree_classifier_example.py | 5 ++++- .../ml/gradient_boosted_tree_regressor_example.py | 5 ++++- .../src/main/python/ml/index_to_string_example.py | 5 ++++- examples/src/main/python/ml/kmeans_example.py | 5 ++++- .../ml/linear_regression_with_elastic_net.py | 5 ++++- .../ml/logistic_regression_with_elastic_net.py | 5 ++++- .../src/main/python/ml/max_abs_scaler_example.py | 5 ++++- .../src/main/python/ml/min_max_scaler_example.py | 5 ++++- examples/src/main/python/ml/n_gram_example.py | 5 ++++- .../src/main/python/ml/naive_bayes_example.py | 5 ++++- examples/src/main/python/ml/normalizer_example.py | 5 ++++- .../src/main/python/ml/onehot_encoder_example.py | 5 ++++- examples/src/main/python/ml/pca_example.py | 5 ++++- examples/src/main/python/ml/pipeline_example.py | 5 ++++- .../python/ml/polynomial_expansion_example.py | 5 ++++- .../python/ml/random_forest_classifier_example.py | 5 ++++- .../python/ml/random_forest_regressor_example.py | 5 ++++- examples/src/main/python/ml/rformula_example.py | 5 ++++- .../ml/simple_text_classification_pipeline.py | 5 ++++- examples/src/main/python/ml/sql_transformer.py | 5 ++++- .../src/main/python/ml/standard_scaler_example.py | 5 ++++- .../main/python/ml/stopwords_remover_example.py | 5 ++++- .../src/main/python/ml/string_indexer_example.py | 5 ++++- examples/src/main/python/ml/tf_idf_example.py | 5 ++++- examples/src/main/python/ml/tokenizer_example.py | 5 ++++- .../src/main/python/ml/train_validation_split.py | 5 ++++- .../main/python/ml/vector_assembler_example.py | 5 ++++- .../src/main/python/ml/vector_indexer_example.py | 5 ++++- .../src/main/python/ml/vector_slicer_example.py | 5 ++++- examples/src/main/python/ml/word2vec_example.py | 5 ++++- .../binary_classification_metrics_example.py | 15 ++++++++++----- examples/src/main/python/sql.py | 5 ++++- .../python/streaming/sql_network_wordcount.py | 6 ++++-- .../ml/AFTSurvivalRegressionExample.scala | 5 ++++- .../org/apache/spark/examples/ml/ALSExample.scala | 5 ++++- .../spark/examples/ml/BinarizerExample.scala | 5 ++++- .../spark/examples/ml/BucketizerExample.scala | 5 ++++- .../spark/examples/ml/ChiSqSelectorExample.scala | 5 ++++- .../examples/ml/CountVectorizerExample.scala | 5 ++++- .../org/apache/spark/examples/ml/DCTExample.scala | 5 ++++- .../spark/examples/ml/DataFrameExample.scala | 5 ++++- .../ml/DecisionTreeClassificationExample.scala | 5 ++++- .../spark/examples/ml/DecisionTreeExample.scala | 4 +++- .../ml/DecisionTreeRegressionExample.scala | 5 ++++- .../spark/examples/ml/DeveloperApiExample.scala | 5 ++++- .../examples/ml/ElementwiseProductExample.scala | 5 ++++- .../ml/EstimatorTransformerParamExample.scala | 5 ++++- .../ml/GradientBoostedTreeClassifierExample.scala | 5 ++++- .../ml/GradientBoostedTreeRegressorExample.scala | 5 ++++- .../spark/examples/ml/IndexToStringExample.scala | 5 ++++- .../apache/spark/examples/ml/KMeansExample.scala | 5 ++++- .../org/apache/spark/examples/ml/LDAExample.scala | 5 ++++- .../LinearRegressionWithElasticNetExample.scala | 5 ++++- .../ml/LogisticRegressionSummaryExample.scala | 5 ++++- .../spark/examples/ml/MaxAbsScalerExample.scala | 5 ++++- .../spark/examples/ml/MinMaxScalerExample.scala | 5 ++++- .../MultilayerPerceptronClassifierExample.scala | 5 ++++- .../apache/spark/examples/ml/NGramExample.scala | 5 ++++- .../spark/examples/ml/NaiveBayesExample.scala | 5 ++++- .../spark/examples/ml/NormalizerExample.scala | 5 ++++- .../spark/examples/ml/OneHotEncoderExample.scala | 5 ++++- .../spark/examples/ml/OneVsRestExample.scala | 5 ++++- .../org/apache/spark/examples/ml/PCAExample.scala | 5 ++++- .../spark/examples/ml/PipelineExample.scala | 5 ++++- .../examples/ml/PolynomialExpansionExample.scala | 5 ++++- .../examples/ml/QuantileDiscretizerExample.scala | 5 ++++- .../spark/examples/ml/RFormulaExample.scala | 5 ++++- .../ml/RandomForestClassifierExample.scala | 5 ++++- .../ml/RandomForestRegressorExample.scala | 5 ++++- .../spark/examples/ml/SQLTransformerExample.scala | 5 ++++- .../spark/examples/ml/SimpleParamsExample.scala | 5 ++++- .../ml/SimpleTextClassificationPipeline.scala | 5 ++++- .../spark/examples/ml/StandardScalerExample.scala | 5 ++++- .../examples/ml/StopWordsRemoverExample.scala | 5 ++++- .../spark/examples/ml/StringIndexerExample.scala | 5 ++++- .../apache/spark/examples/ml/TfIdfExample.scala | 5 ++++- .../spark/examples/ml/TokenizerExample.scala | 5 ++++- .../examples/ml/VectorAssemblerExample.scala | 5 ++++- .../spark/examples/ml/VectorIndexerExample.scala | 5 ++++- .../spark/examples/ml/VectorSlicerExample.scala | 5 ++++- .../spark/examples/ml/Word2VecExample.scala | 5 ++++- .../apache/spark/examples/mllib/LDAExample.scala | 4 +++- .../examples/mllib/RankingMetricsExample.scala | 5 ++++- .../examples/mllib/RegressionMetricsExample.scala | 5 ++++- .../apache/spark/examples/sql/RDDRelation.scala | 5 ++++- .../examples/streaming/SqlNetworkWordCount.scala | 5 ++++- .../parquet/VectorizedPlainValuesReader.java | 5 +++-- .../execution/vectorized/OffHeapColumnVector.java | 15 ++++++++------- .../execution/vectorized/OnHeapColumnVector.java | 7 ++++--- .../hive/service/cli/session/SessionManager.java | 2 -- 142 files changed, 585 insertions(+), 178 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index fb1226c09e24..22fd592a321d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -87,8 +87,11 @@ protected void handleMessage( blocks.add(blockManager.getBlockData(msg.appId, msg.execId, blockId)); } long streamId = streamManager.registerStream(client.getClientId(), blocks.iterator()); - logger.trace("Registered streamId {} with {} buffers for client {} from host {}", streamId, - msg.blockIds.length, client.getClientId(), NettyUtils.getRemoteAddress(client.getChannel())); + logger.trace("Registered streamId {} with {} buffers for client {} from host {}", + streamId, + msg.blockIds.length, + client.getClientId(), + NettyUtils.getRemoteAddress(client.getChannel())); callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); } else if (msgObj instanceof RegisterExecutor) { diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java index ecb7084e0312..2c2aa6df47c7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java @@ -33,7 +33,10 @@ public class JavaAFTSurvivalRegressionExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaAFTSurvivalRegressionExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaAFTSurvivalRegressionExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 9a9a10489b50..4b13ba6f9cea 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -81,7 +81,10 @@ public static Rating parseRating(String str) { // $example off$ public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaALSExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaALSExample") + .getOrCreate(); // $example on$ JavaRDD ratingsRDD = spark diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java index 88e4298a6191..5f964aca9209 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -26,7 +24,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Binarizer; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -38,7 +35,10 @@ public class JavaBinarizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaBinarizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaBinarizerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 51aa35084e84..810ad905c56a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -42,7 +42,10 @@ public class JavaBisectingKMeansExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaBisectingKMeansExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaBisectingKMeansExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java index 0c24f52cf58a..691df3887a9b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java @@ -35,7 +35,10 @@ public class JavaBucketizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaBucketizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaBucketizerExample") + .getOrCreate(); // $example on$ double[] splits = {Double.NEGATIVE_INFINITY, -0.5, 0.0, 0.5, Double.POSITIVE_INFINITY}; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java index 684cf9a7145d..f8f2fb14be1f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java @@ -17,9 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -40,7 +37,10 @@ public class JavaChiSqSelectorExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaChiSqSelectorExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaChiSqSelectorExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java index 0631f9d6d5d7..0a6b13601425 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java @@ -32,7 +32,10 @@ public class JavaCountVectorizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaCountVectorizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaCountVectorizerExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java index ec57a2445138..eee92c77a8c5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -26,7 +24,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.DCT; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -39,7 +36,10 @@ public class JavaDCTExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaDCTExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaDCTExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 90023ac06b79..49bad0afc0b7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -49,7 +49,10 @@ public class JavaDeveloperApiExample { public static void main(String[] args) throws Exception { - SparkSession spark = SparkSession.builder().appName("JavaDeveloperApiExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaDeveloperApiExample") + .getOrCreate(); // Prepare training data. List localTraining = Lists.newArrayList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java index a062a6fcd0c1..9126242f9eb7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -27,7 +25,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.ElementwiseProduct; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.VectorUDT; @@ -42,7 +39,9 @@ public class JavaElementwiseProductExample { public static void main(String[] args) { SparkSession spark = SparkSession - .builder().appName("JavaElementwiseProductExample").getOrCreate(); + .builder() + .appName("JavaElementwiseProductExample") + .getOrCreate(); // $example on$ // Create some vector data; also works for sparse vectors diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java index a7c89b9d1997..baacd796a058 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -35,11 +33,15 @@ public class JavaGradientBoostedTreeClassifierExample { public static void main(String[] args) { SparkSession spark = SparkSession - .builder().appName("JavaGradientBoostedTreeClassifierExample").getOrCreate(); + .builder() + .appName("JavaGradientBoostedTreeClassifierExample") + .getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = spark.read().format("libsvm") + Dataset data = spark + .read() + .format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java index ccd74f2920ce..0064beb8c8f3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java @@ -37,7 +37,10 @@ public class JavaIndexToStringExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaIndexToStringExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaIndexToStringExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index e6d82a0513a3..65e29ade299d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -70,7 +70,10 @@ public static void main(String[] args) { int k = Integer.parseInt(args[1]); // Parses the arguments - SparkSession spark = SparkSession.builder().appName("JavaKMeansExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaKMeansExample") + .getOrCreate(); // $example on$ // Loads data diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java index b8baca5920e8..1c52f37867a8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java @@ -65,7 +65,10 @@ public static void main(String[] args) { String inputFile = "data/mllib/sample_lda_data.txt"; // Parses the arguments - SparkSession spark = SparkSession.builder().appName("JavaLDAExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaLDAExample") + .getOrCreate(); // Loads data JavaRDD points = spark.read().text(inputFile).javaRDD().map(new ParseVector()); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java index 80cdd364b937..9a27b0e9e23b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java @@ -28,13 +28,19 @@ public class JavaMaxAbsScalerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaMaxAbsScalerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaMaxAbsScalerExample") + .getOrCreate(); // $example on$ - Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); MaxAbsScaler scaler = new MaxAbsScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures"); + .setInputCol("features") + .setOutputCol("scaledFeatures"); // Compute summary statistics and generate MaxAbsScalerModel MaxAbsScalerModel scalerModel = scaler.fit(dataFrame); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java index 022940fd1e67..37fa1c5434ea 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java @@ -28,10 +28,16 @@ public class JavaMinMaxScalerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaMinMaxScalerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaMinMaxScalerExample") + .getOrCreate(); // $example on$ - Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); MinMaxScaler scaler = new MinMaxScaler() .setInputCol("features") .setOutputCol("scaledFeatures"); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java index 325b7b587480..899815f57c84 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java @@ -35,7 +35,10 @@ public class JavaNGramExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaNGramExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaNGramExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java index 1f24a23609eb..50a46a5774fc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java @@ -32,7 +32,10 @@ public class JavaNaiveBayesExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaNaiveBayesExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaNaiveBayesExample") + .getOrCreate(); // $example on$ // Load training data diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java index 4b3a718ea92c..abc38f85ea77 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java @@ -27,7 +27,10 @@ public class JavaNormalizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaNormalizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaNormalizerExample") + .getOrCreate(); // $example on$ Dataset dataFrame = diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java index d6e4d21ead16..5d29e5454921 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -37,7 +37,10 @@ public class JavaOneHotEncoderExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaOneHotEncoderExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaOneHotEncoderExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java index 9cc983bd115d..e0cb752224f7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java @@ -58,7 +58,10 @@ private static class Params { public static void main(String[] args) { // parse the arguments Params params = parse(args); - SparkSession spark = SparkSession.builder().appName("JavaOneVsRestExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaOneVsRestExample") + .getOrCreate(); // $example on$ // configure the base classifier diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java index 6b1dcb68ba30..ffa979ee013a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java @@ -37,7 +37,10 @@ public class JavaPCAExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaPCAExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaPCAExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java index 556a4573263f..9a43189c9146 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java @@ -36,7 +36,10 @@ */ public class JavaPipelineExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaPipelineExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaPipelineExample") + .getOrCreate(); // $example on$ // Prepare training documents, which are labeled. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java index e328454c7059..7afcd0e50cd9 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -36,7 +36,10 @@ public class JavaPolynomialExpansionExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaPolynomialExpansionExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaPolynomialExpansionExample") + .getOrCreate(); // $example on$ PolynomialExpansion polyExpansion = new PolynomialExpansion() diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java index 8282ce01d36f..428067e0f7ef 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java @@ -35,7 +35,10 @@ public class JavaRFormulaExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaRFormulaExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaRFormulaExample") + .getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java index 492718bbdba8..2a3d62de41ab 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java @@ -31,7 +31,10 @@ public class JavaSQLTransformerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaSQLTransformerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaSQLTransformerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index f906843640ed..0787079ba4e5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -40,7 +40,10 @@ public class JavaSimpleParamsExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaSimpleParamsExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaSimpleParamsExample") + .getOrCreate(); // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java index 10f82f223386..08ea285a0d53 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java @@ -28,7 +28,10 @@ public class JavaStandardScalerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaStandardScalerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaStandardScalerExample") + .getOrCreate(); // $example on$ Dataset dataFrame = diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java index 23ed071c9f6e..def599442912 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java @@ -36,7 +36,10 @@ public class JavaStopWordsRemoverExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaStopWordsRemoverExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaStopWordsRemoverExample") + .getOrCreate(); // $example on$ StopWordsRemover remover = new StopWordsRemover() diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java index d4c2cf96a7cf..7533c1835e32 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java @@ -35,7 +35,10 @@ public class JavaStringIndexerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaStringIndexerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaStringIndexerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index a8169917772c..6e0753959efd 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -38,7 +38,10 @@ public class JavaTfIdfExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaTfIdfExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaTfIdfExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java index a65735a5e514..1cc16bb60d17 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -36,7 +36,10 @@ public class JavaTokenizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaTokenizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaTokenizerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java index 9569bc2412de..41f1d8750ac4 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java @@ -35,7 +35,10 @@ public class JavaVectorAssemblerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaVectorAssemblerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorAssemblerExample") + .getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java index 217d5a06d10a..dd9d757dd683 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java @@ -30,7 +30,10 @@ public class JavaVectorIndexerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaVectorIndexerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorIndexerExample") + .getOrCreate(); // $example on$ Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java index 4f1ea824a3a9..24959c0e10f2 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java @@ -37,7 +37,10 @@ public class JavaVectorSlicerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaVectorSlicerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorSlicerExample") + .getOrCreate(); // $example on$ Attribute[] attrs = new Attribute[]{ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java index d9b1a79b5254..9be6e6353adc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java @@ -32,7 +32,10 @@ public class JavaWord2VecExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaWord2VecExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaWord2VecExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index ec2142e75695..755b4f538104 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -51,7 +51,10 @@ public void setAge(int age) { } public static void main(String[] args) throws Exception { - SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaSparkSQL") + .getOrCreate(); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. @@ -147,7 +150,8 @@ public String call(Row row) { // a RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); - JavaRDD anotherPeopleRDD = spark.createDataFrame(jsonData, String.class).toJSON().javaRDD(); + JavaRDD anotherPeopleRDD = spark + .createDataFrame(jsonData, String.class).toJSON().javaRDD(); Dataset peopleFromJsonRDD = spark.read().json(anotherPeopleRDD); // Take a look at the schema of this new DataFrame. diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 44f1e800fe44..57953ef74f79 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -115,7 +115,10 @@ class JavaSparkSessionSingleton { private static transient SparkSession instance = null; public static SparkSession getInstance(SparkConf sparkConf) { if (instance == null) { - instance = SparkSession.builder().config(sparkConf).getOrCreate(); + instance = SparkSession + .builder() + .config(sparkConf) + .getOrCreate(); } return instance; } diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index e36444f18506..ff0829b0dd45 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -30,7 +30,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("ALSExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("ALSExample")\ + .getOrCreate() # $example on$ lines = spark.read.text("data/mllib/als/sample_movielens_ratings.txt").rdd diff --git a/examples/src/main/python/ml/binarizer_example.py b/examples/src/main/python/ml/binarizer_example.py index 072187e64564..4224a27dbef0 100644 --- a/examples/src/main/python/ml/binarizer_example.py +++ b/examples/src/main/python/ml/binarizer_example.py @@ -23,7 +23,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("BinarizerExample")\ + .getOrCreate() # $example on$ continuousDataFrame = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index 836a89cde0d6..540a4bc3e4bf 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -30,7 +30,10 @@ """ if __name__ == "__main__": - spark = SparkSession.builder.appName("PythonBisectingKMeansExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PythonBisectingKMeansExample")\ + .getOrCreate() # $example on$ data = spark.read.text("data/mllib/kmeans_data.txt").rdd diff --git a/examples/src/main/python/ml/bucketizer_example.py b/examples/src/main/python/ml/bucketizer_example.py index 288ec62bdfbb..8177e560ddef 100644 --- a/examples/src/main/python/ml/bucketizer_example.py +++ b/examples/src/main/python/ml/bucketizer_example.py @@ -23,7 +23,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("BucketizerExample")\ + .getOrCreate() # $example on$ splits = [-float("inf"), -0.5, 0.0, 0.5, float("inf")] diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py index 8f58fc28deb9..8bafb942e0d2 100644 --- a/examples/src/main/python/ml/chisq_selector_example.py +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -24,7 +24,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("ChiSqSelectorExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/count_vectorizer_example.py b/examples/src/main/python/ml/count_vectorizer_example.py index 9dbf9959d17e..38cfac82fbe2 100644 --- a/examples/src/main/python/ml/count_vectorizer_example.py +++ b/examples/src/main/python/ml/count_vectorizer_example.py @@ -23,7 +23,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("CountVectorizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("CountVectorizerExample")\ + .getOrCreate() # $example on$ # Input data: Each row is a bag of words with a ID. diff --git a/examples/src/main/python/ml/cross_validator.py b/examples/src/main/python/ml/cross_validator.py index a61d0f63d277..a41df6cf946f 100644 --- a/examples/src/main/python/ml/cross_validator.py +++ b/examples/src/main/python/ml/cross_validator.py @@ -35,7 +35,10 @@ """ if __name__ == "__main__": - spark = SparkSession.builder.appName("CrossValidatorExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("CrossValidatorExample")\ + .getOrCreate() # $example on$ # Prepare training documents, which are labeled. training = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py index b3e671038eb7..a7d8b9056dff 100644 --- a/examples/src/main/python/ml/dataframe_example.py +++ b/examples/src/main/python/ml/dataframe_example.py @@ -33,7 +33,10 @@ if len(sys.argv) > 2: print("Usage: dataframe_example.py ", file=sys.stderr) exit(-1) - spark = SparkSession.builder.appName("DataFrameExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("DataFrameExample")\ + .getOrCreate() if len(sys.argv) == 2: input = sys.argv[1] else: diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py index 1bf8fc6d140c..e36fcdeaeed2 100644 --- a/examples/src/main/python/ml/dct_example.py +++ b/examples/src/main/python/ml/dct_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("DCTExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("DCTExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index d2318e24369d..9b40b701ecaa 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("decision_tree_classification_example")\ + .getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index 9e8cb382a9bf..b734d4974a4f 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("decision_tree_classification_example")\ + .getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index 6fa641b772d9..41727edcdb09 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("ElementwiseProductExample")\ + .getOrCreate() # $example on$ data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py index 4993b5a9846c..0fcae0e3fc22 100644 --- a/examples/src/main/python/ml/estimator_transformer_param_example.py +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -26,7 +26,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("EstimatorTransformerParamExample")\ + .getOrCreate() # $example on$ # Prepare training data from a list of (label, features) tuples. diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index b09ad41da313..50026d7b7e3e 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("gradient_boosted_tree_classifier_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("gradient_boosted_tree_classifier_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py index caa7cfc4e11f..5dd2272748d7 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("gradient_boosted_tree_regressor_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("gradient_boosted_tree_regressor_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/index_to_string_example.py b/examples/src/main/python/ml/index_to_string_example.py index dd04b2c4b0bf..523caac00c18 100644 --- a/examples/src/main/python/ml/index_to_string_example.py +++ b/examples/src/main/python/ml/index_to_string_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("IndexToStringExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame( diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index 7d9d80e6452b..73823969554f 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -49,7 +49,10 @@ def parseVector(row): path = sys.argv[1] k = sys.argv[2] - spark = SparkSession.builder.appName("PythonKMeansExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PythonKMeansExample")\ + .getOrCreate() lines = spark.read.text(path).rdd data = lines.map(parseVector) diff --git a/examples/src/main/python/ml/linear_regression_with_elastic_net.py b/examples/src/main/python/ml/linear_regression_with_elastic_net.py index 99b7f7fe99de..620ab5b87e59 100644 --- a/examples/src/main/python/ml/linear_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/linear_regression_with_elastic_net.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("LinearRegressionWithElasticNet").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("LinearRegressionWithElasticNet")\ + .getOrCreate() # $example on$ # Load training data diff --git a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py index 0d7112e72354..33d0689f75cd 100644 --- a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("LogisticRegressionWithElasticNet").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("LogisticRegressionWithElasticNet")\ + .getOrCreate() # $example on$ # Load training data diff --git a/examples/src/main/python/ml/max_abs_scaler_example.py b/examples/src/main/python/ml/max_abs_scaler_example.py index 1cb95a98f08d..ab91198b083d 100644 --- a/examples/src/main/python/ml/max_abs_scaler_example.py +++ b/examples/src/main/python/ml/max_abs_scaler_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("MaxAbsScalerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/min_max_scaler_example.py b/examples/src/main/python/ml/min_max_scaler_example.py index 8d91a59e2b0b..e3e7bc205b1e 100644 --- a/examples/src/main/python/ml/min_max_scaler_example.py +++ b/examples/src/main/python/ml/min_max_scaler_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("MinMaxScalerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/n_gram_example.py b/examples/src/main/python/ml/n_gram_example.py index b7fecf0d685c..9ac07f2c8ee2 100644 --- a/examples/src/main/python/ml/n_gram_example.py +++ b/examples/src/main/python/ml/n_gram_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("NGramExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("NGramExample")\ + .getOrCreate() # $example on$ wordDataFrame = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py index e37035542c7e..89255a2bae64 100644 --- a/examples/src/main/python/ml/naive_bayes_example.py +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("naive_bayes_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("naive_bayes_example")\ + .getOrCreate() # $example on$ # Load training data diff --git a/examples/src/main/python/ml/normalizer_example.py b/examples/src/main/python/ml/normalizer_example.py index ae2553761928..19012f51f402 100644 --- a/examples/src/main/python/ml/normalizer_example.py +++ b/examples/src/main/python/ml/normalizer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("NormalizerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/onehot_encoder_example.py b/examples/src/main/python/ml/onehot_encoder_example.py index 9acc363dc9e2..b9fceef68e70 100644 --- a/examples/src/main/python/ml/onehot_encoder_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("OneHotEncoderExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py index adab151734aa..f1b3cdec7bd7 100644 --- a/examples/src/main/python/ml/pca_example.py +++ b/examples/src/main/python/ml/pca_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("PCAExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PCAExample")\ + .getOrCreate() # $example on$ data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), diff --git a/examples/src/main/python/ml/pipeline_example.py b/examples/src/main/python/ml/pipeline_example.py index ed9765d96165..bd10cfd7a252 100644 --- a/examples/src/main/python/ml/pipeline_example.py +++ b/examples/src/main/python/ml/pipeline_example.py @@ -27,7 +27,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("PipelineExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PipelineExample")\ + .getOrCreate() # $example on$ # Prepare training documents from a list of (id, text, label) tuples. diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 328b559320f8..08882bcb256f 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PolynomialExpansionExample")\ + .getOrCreate() # $example on$ df = spark\ diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index b0a93e050c54..c618eaf60c2e 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("random_forest_classifier_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("random_forest_classifier_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py index 4bb84f0de8ff..3a793737dba8 100644 --- a/examples/src/main/python/ml/random_forest_regressor_example.py +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("random_forest_regressor_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("random_forest_regressor_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/rformula_example.py b/examples/src/main/python/ml/rformula_example.py index 45cc116ac2f6..d5df3ce4f591 100644 --- a/examples/src/main/python/ml/rformula_example.py +++ b/examples/src/main/python/ml/rformula_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("RFormulaExample")\ + .getOrCreate() # $example on$ dataset = spark.createDataFrame( diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index 3600c12211d0..886f43c0b08e 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -33,7 +33,10 @@ if __name__ == "__main__": - spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("SimpleTextClassificationPipeline")\ + .getOrCreate() # Prepare training documents, which are labeled. training = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/sql_transformer.py b/examples/src/main/python/ml/sql_transformer.py index 26045db4be6b..0bf8f35720c9 100644 --- a/examples/src/main/python/ml/sql_transformer.py +++ b/examples/src/main/python/ml/sql_transformer.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("SQLTransformerExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/standard_scaler_example.py b/examples/src/main/python/ml/standard_scaler_example.py index c50804f6bf4e..c0027480e69b 100644 --- a/examples/src/main/python/ml/standard_scaler_example.py +++ b/examples/src/main/python/ml/standard_scaler_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("StandardScalerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/stopwords_remover_example.py b/examples/src/main/python/ml/stopwords_remover_example.py index 57362673df78..395fdeffc537 100644 --- a/examples/src/main/python/ml/stopwords_remover_example.py +++ b/examples/src/main/python/ml/stopwords_remover_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("StopWordsRemoverExample")\ + .getOrCreate() # $example on$ sentenceData = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/string_indexer_example.py b/examples/src/main/python/ml/string_indexer_example.py index aacd4f999bb1..a328e040f563 100644 --- a/examples/src/main/python/ml/string_indexer_example.py +++ b/examples/src/main/python/ml/string_indexer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("StringIndexerExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame( diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index 25df8166efc2..fb4ad992fb80 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("TfIdfExample")\ + .getOrCreate() # $example on$ sentenceData = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/tokenizer_example.py b/examples/src/main/python/ml/tokenizer_example.py index 5be4b4cfe3a5..e61ec920d228 100644 --- a/examples/src/main/python/ml/tokenizer_example.py +++ b/examples/src/main/python/ml/tokenizer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("TokenizerExample")\ + .getOrCreate() # $example on$ sentenceDataFrame = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/train_validation_split.py b/examples/src/main/python/ml/train_validation_split.py index 2e43a0f8aec5..5f5c52aca8c4 100644 --- a/examples/src/main/python/ml/train_validation_split.py +++ b/examples/src/main/python/ml/train_validation_split.py @@ -31,7 +31,10 @@ """ if __name__ == "__main__": - spark = SparkSession.builder.appName("TrainValidationSplit").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("TrainValidationSplit")\ + .getOrCreate() # $example on$ # Prepare training and test data. data = spark.read.format("libsvm")\ diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py index 019a9ea6f750..b955ff00a819 100644 --- a/examples/src/main/python/ml/vector_assembler_example.py +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("VectorAssemblerExample")\ + .getOrCreate() # $example on$ dataset = spark.createDataFrame( diff --git a/examples/src/main/python/ml/vector_indexer_example.py b/examples/src/main/python/ml/vector_indexer_example.py index 3cf5b8ebf17e..9b00e0f84136 100644 --- a/examples/src/main/python/ml/vector_indexer_example.py +++ b/examples/src/main/python/ml/vector_indexer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("VectorIndexerExample")\ + .getOrCreate() # $example on$ data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index 0531bcdb0667..b833a894eb84 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -25,7 +25,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("VectorSlicerExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/word2vec_example.py b/examples/src/main/python/ml/word2vec_example.py index 6766a7b6aa45..66500bee152f 100644 --- a/examples/src/main/python/ml/word2vec_example.py +++ b/examples/src/main/python/ml/word2vec_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("Word2VecExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("Word2VecExample")\ + .getOrCreate() # $example on$ # Input data: Each row is a bag of words from a sentence or document. diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py index 8f0fc9d45da2..daf000e38dcd 100644 --- a/examples/src/main/python/mllib/binary_classification_metrics_example.py +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -18,20 +18,25 @@ Binary Classification Metrics Example. """ from __future__ import print_function -from pyspark import SparkContext +from pyspark.sql import SparkSession # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS from pyspark.mllib.evaluation import BinaryClassificationMetrics -from pyspark.mllib.util import MLUtils +from pyspark.mllib.regression import LabeledPoint # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BinaryClassificationMetricsExample") + spark = SparkSession\ + .builder\ + .appName("BinaryClassificationMetricsExample")\ + .getOrCreate() # $example on$ # Several of the methods available in scala are currently missing from pyspark # Load training data in LIBSVM format - data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_binary_classification_data.txt") + data = spark\ + .read.format("libsvm").load("data/mllib/sample_binary_classification_data.txt")\ + .rdd.map(lambda row: LabeledPoint(row[0], row[1])) # Split data into training (60%) and test (40%) training, test = data.randomSplit([0.6, 0.4], seed=11L) @@ -53,4 +58,4 @@ print("Area under ROC = %s" % metrics.areaUnderROC) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 59a46cb283c4..5594223465f8 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -25,7 +25,10 @@ if __name__ == "__main__": - spark = SparkSession.builder.appName("PythonSQL").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PythonSQL")\ + .getOrCreate() # A list of Rows. Infer schema from the first row, create a DataFrame and print the schema rows = [Row(name="John", age=19), Row(name="Smith", age=23), Row(name="Sarah", age=18)] diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index 588cbfee14ba..f8801d4ea63e 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -38,8 +38,10 @@ def getSparkSessionInstance(sparkConf): if ('sparkSessionSingletonInstance' not in globals()): - globals()['sparkSessionSingletonInstance'] =\ - SparkSession.builder.config(conf=sparkConf).getOrCreate() + globals()['sparkSessionSingletonInstance'] = SparkSession\ + .builder\ + .config(conf=sparkConf)\ + .getOrCreate() return globals()['sparkSessionSingletonInstance'] diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala index 3795af83094a..2b224d50a0a3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -30,7 +30,10 @@ import org.apache.spark.sql.SparkSession object AFTSurvivalRegressionExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("AFTSurvivalRegressionExample").getOrCreate() + val spark = SparkSession + .builder + .appName("AFTSurvivalRegressionExample") + .getOrCreate() // $example on$ val training = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index 41750ca77914..7c1cfe293717 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -42,7 +42,10 @@ object ALSExample { // $example off$ def main(args: Array[String]) { - val spark = SparkSession.builder.appName("ALSExample").getOrCreate() + val spark = SparkSession + .builder + .appName("ALSExample") + .getOrCreate() import spark.implicits._ // $example on$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala index 93c153f92323..82bc14789b46 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.{DataFrame, SparkSession} object BinarizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("BinarizerExample") + .getOrCreate() // $example on$ val data = Array((0, 0.1), (1, 0.8), (2, 0.2)) val dataFrame: DataFrame = spark.createDataFrame(data).toDF("label", "feature") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala index 779ad33dbda0..38cce34bb509 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object BucketizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("BucketizerExample") + .getOrCreate() // $example on$ val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 84ca1f0b564d..80f50cd3556a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object ChiSqSelectorExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() + val spark = SparkSession + .builder + .appName("ChiSqSelectorExample") + .getOrCreate() import spark.implicits._ // $example on$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index 9ab43a48bff8..51aa5179fa4a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object CountVectorizerExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("CounterVectorizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("CounterVectorizerExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala index b415333c710c..5a888b15eb1c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object DCTExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("DCTExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DCTExample") + .getOrCreate() // $example on$ val data = Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 2f892f8d72c1..6cb81cde6f79 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -61,7 +61,10 @@ object DataFrameExample { } def run(params: Params) { - val spark = SparkSession.builder.appName(s"DataFrameExample with $params").getOrCreate() + val spark = SparkSession + .builder + .appName(s"DataFrameExample with $params") + .getOrCreate() // Load input data println(s"Loading LIBSVM file with UDT from ${params.input}.") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala index a0a2e1fb33dc..7f6c8de9679f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.SparkSession object DecisionTreeClassificationExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("DecisionTreeClassificationExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DecisionTreeClassificationExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index cea1d801aa73..eadb02ab0d7f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -167,7 +167,9 @@ object DecisionTreeExample { testInput: String, algo: String, fracTest: Double): (DataFrame, DataFrame) = { - val spark = SparkSession.builder.getOrCreate() + val spark = SparkSession + .builder + .getOrCreate() // Load training data val origExamples: DataFrame = loadData(spark, input, dataFormat) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala index 26b52d0489e1..799070ef47da 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.SparkSession object DecisionTreeRegressionExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("DecisionTreeRegressionExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DecisionTreeRegressionExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 2aa1ab1ec855..a522d2127edc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -37,7 +37,10 @@ import org.apache.spark.sql.{Dataset, Row, SparkSession} object DeveloperApiExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("DeveloperApiExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DeveloperApiExample") + .getOrCreate() import spark.implicits._ // Prepare training data. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala index f289c28df9b5..b99b76e58cef 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object ElementwiseProductExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() + val spark = SparkSession + .builder + .appName("ElementwiseProductExample") + .getOrCreate() // $example on$ // Create some vector data; also works for sparse vectors diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala index 91076ccbc14d..972241e76922 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.SparkSession object EstimatorTransformerParamExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() + val spark = SparkSession + .builder + .appName("EstimatorTransformerParamExample") + .getOrCreate() // $example on$ // Prepare training data from a list of (label, features) tuples. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala index 412c54db7d5c..b6a8baba2d95 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object GradientBoostedTreeClassifierExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("GradientBoostedTreeClassifierExample").getOrCreate() + val spark = SparkSession + .builder + .appName("GradientBoostedTreeClassifierExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala index fd43553cc69d..62285b83cbb9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object GradientBoostedTreeRegressorExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("GradientBoostedTreeRegressorExample").getOrCreate() + val spark = SparkSession + .builder + .appName("GradientBoostedTreeRegressorExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala index d873618726c6..950733831c3d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object IndexToStringExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() + val spark = SparkSession + .builder + .appName("IndexToStringExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala index d2573fad3596..2abd588c6f0e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala @@ -36,7 +36,10 @@ object KMeansExample { def main(args: Array[String]): Unit = { // Creates a Spark context and a SQL context - val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() // $example on$ // Crates a DataFrame diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala index c23adee1a3ea..c2920f6a5d4a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala @@ -40,7 +40,10 @@ object LDAExample { val input = "data/mllib/sample_lda_data.txt" // Creates a Spark context and a SQL context - val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() // $example on$ // Loads data diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala index cb6e2492f513..94cf2866238b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object LinearRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("LinearRegressionWithElasticNetExample").getOrCreate() + val spark = SparkSession + .builder + .appName("LinearRegressionWithElasticNetExample") + .getOrCreate() // $example on$ // Load training data diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala index 50670d7b3841..cd8775c94216 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala @@ -27,7 +27,10 @@ import org.apache.spark.sql.functions.max object LogisticRegressionSummaryExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("LogisticRegressionSummaryExample").getOrCreate() + val spark = SparkSession + .builder + .appName("LogisticRegressionSummaryExample") + .getOrCreate() import spark.implicits._ // Load training data diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala index 896d8fadbe67..572adce65708 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala @@ -24,7 +24,10 @@ import org.apache.spark.sql.SparkSession object MaxAbsScalerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("MaxAbsScalerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala index bcdca0fa0440..d728019a621d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object MinMaxScalerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("MinMaxScalerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala index a11fe1b4b262..0e780fb7d342 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala @@ -30,7 +30,10 @@ import org.apache.spark.sql.SparkSession object MultilayerPerceptronClassifierExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("MultilayerPerceptronClassifierExample").getOrCreate() + val spark = SparkSession + .builder + .appName("MultilayerPerceptronClassifierExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala index 1b71a398905e..e0b52e7a367f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object NGramExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("NGramExample").getOrCreate() + val spark = SparkSession + .builder + .appName("NGramExample") + .getOrCreate() // $example on$ val wordDataFrame = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala index 8d54555cd3c6..90cdebfcb08d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object NaiveBayesExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("NaiveBayesExample").getOrCreate() + val spark = SparkSession + .builder + .appName("NaiveBayesExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala index 4622d69ef98c..75ba33a7e7fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object NormalizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("NormalizerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala index 338436100c5c..4aa649b1332c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object OneHotEncoderExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() + val spark = SparkSession + .builder + .appName("OneHotEncoderExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index e2351c682d75..fc73ae07ff6c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -109,7 +109,10 @@ object OneVsRestExample { } private def run(params: Params) { - val spark = SparkSession.builder.appName(s"OneVsRestExample with $params").getOrCreate() + val spark = SparkSession + .builder + .appName(s"OneVsRestExample with $params") + .getOrCreate() // $example on$ val inputData = spark.read.format("libsvm").load(params.input) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala index 14394d562468..7927323b4285 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object PCAExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("PCAExample").getOrCreate() + val spark = SparkSession + .builder + .appName("PCAExample") + .getOrCreate() // $example on$ val data = Array( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala index 61b34aebd9c3..e5e916ac166f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -30,7 +30,10 @@ import org.apache.spark.sql.SparkSession object PipelineExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("PipelineExample").getOrCreate() + val spark = SparkSession + .builder + .appName("PipelineExample") + .getOrCreate() // $example on$ // Prepare training documents from a list of (id, text, label) tuples. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala index 4d8c672a55c3..94b17a3cd706 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object PolynomialExpansionExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() + val spark = SparkSession + .builder + .appName("PolynomialExpansionExample") + .getOrCreate() // $example on$ val data = Array( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala index 0839c609f1d9..1a1651559416 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -24,7 +24,10 @@ import org.apache.spark.sql.SparkSession object QuantileDiscretizerExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("QuantileDiscretizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("QuantileDiscretizerExample") + .getOrCreate() import spark.implicits._ // $example on$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala index 699b621db90a..9ea492014644 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object RFormulaExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RFormulaExample") + .getOrCreate() // $example on$ val dataset = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala index 4192a9c73704..ae0bd945d8fe 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object RandomForestClassifierExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RandomForestClassifierExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RandomForestClassifierExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala index 5632f0419a80..96dc2f05be97 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object RandomForestRegressorExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RandomForestRegressorExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RandomForestRegressorExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala index f03b29ba327c..bb4587b82cb3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object SQLTransformerExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("SQLTransformerExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index dff771950715..3547dd95bdce 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -34,7 +34,10 @@ import org.apache.spark.sql.{Row, SparkSession} object SimpleParamsExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("SimpleParamsExample").getOrCreate() + val spark = SparkSession + .builder + .appName("SimpleParamsExample") + .getOrCreate() import spark.implicits._ // Prepare training data. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 05199007f015..c78ff2378bc1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -42,7 +42,10 @@ case class Document(id: Long, text: String) object SimpleTextClassificationPipeline { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() + val spark = SparkSession + .builder + .appName("SimpleTextClassificationPipeline") + .getOrCreate() import spark.implicits._ // Prepare training documents, which are labeled. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala index 55f777c6e228..4d668e8ab967 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object StandardScalerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("StandardScalerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala index 85e79c8cb38a..fb1a43e962cd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object StopWordsRemoverExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() + val spark = SparkSession + .builder + .appName("StopWordsRemoverExample") + .getOrCreate() // $example on$ val remover = new StopWordsRemover() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala index e01a768da988..63f273e87a20 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object StringIndexerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("StringIndexerExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index 910ef62a2670..33b5daec5978 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object TfIdfExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() + val spark = SparkSession + .builder + .appName("TfIdfExample") + .getOrCreate() // $example on$ val sentenceData = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala index 4f0c47b3c84c..1c70dc700b91 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object TokenizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("TokenizerExample") + .getOrCreate() // $example on$ val sentenceDataFrame = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala index 56b7263b192e..8e382ccc4844 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object VectorAssemblerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("VectorAssemblerExample") + .getOrCreate() // $example on$ val dataset = spark.createDataFrame( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala index 214ad91634ee..afa761aee0b9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object VectorIndexerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("VectorIndexerExample") + .getOrCreate() // $example on$ val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala index 716bf023a808..b1a3997f48c9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -31,7 +31,10 @@ import org.apache.spark.sql.SparkSession object VectorSlicerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("VectorSlicerExample") + .getOrCreate() // $example on$ val data = Arrays.asList(Row(Vectors.dense(-2.0, 2.3, 0.0))) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala index 292b6d9f7776..9ac562360729 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object Word2VecExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("Word2Vec example").getOrCreate() + val spark = SparkSession + .builder + .appName("Word2Vec example") + .getOrCreate() // $example on$ // Input data: Each row is a bag of words from a sentence or document. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index c2bf1548b568..7651aade493a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -189,7 +189,9 @@ object LDAExample { vocabSize: Int, stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { - val spark = SparkSession.builder.getOrCreate() + val spark = SparkSession + .builder + .getOrCreate() import spark.implicits._ // Get dataset of document texts diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala index cd4f0bb0de60..781a934df663 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object RankingMetricsExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("RankingMetricsExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RankingMetricsExample") + .getOrCreate() import spark.implicits._ // $example on$ // Read in the ratings data diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala index 22c47a694d34..abeaaa00b5a4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -27,7 +27,10 @@ import org.apache.spark.sql.SparkSession object RegressionMetricsExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RegressionMetricsExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RegressionMetricsExample") + .getOrCreate() // $example on$ // Load the data val data = spark diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index b4118b16e291..94c378ae4b91 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -26,7 +26,10 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("RDDRelation").getOrCreate() + val spark = SparkSession + .builder + .appName("RDDRelation") + .getOrCreate() // Importing the SparkSession gives access to all the SQL functions and implicit conversions. import spark.implicits._ diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 2f0fe704f709..9aba4a05a89a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -93,7 +93,10 @@ object SparkSessionSingleton { def getInstance(sparkConf: SparkConf): SparkSession = { if (instance == null) { - instance = SparkSession.builder.config(sparkConf).getOrCreate() + instance = SparkSession + .builder + .config(sparkConf) + .getOrCreate() } instance } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java index 9475c853a03f..9def4559d214 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java @@ -34,8 +34,9 @@ public class VectorizedPlainValuesReader extends ValuesReader implements Vectori private int offset; private int bitOffset; // Only used for booleans. private ByteBuffer byteBuffer; // used to wrap the byte array buffer - - private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); public VectorizedPlainValuesReader() { } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index b8dd16227ec1..70b4a683311c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -19,8 +19,6 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; -import org.apache.commons.lang.NotImplementedException; - import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; @@ -29,9 +27,10 @@ * Column data backed using offheap memory. */ public final class OffHeapColumnVector extends ColumnVector { - - private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); - + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these two allocations need to maintain binary compatible. We can // directly pass this buffer to external components. private long nulls; @@ -230,7 +229,8 @@ public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { - Platform.putInt(null, offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); + Platform.putInt(null, offset, + java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); } } } @@ -276,7 +276,8 @@ public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { - Platform.putLong(null, offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); + Platform.putLong(null, offset, + java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index b1ffe4c21049..7fb7617050f2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -29,9 +29,10 @@ * and a java array for the values. */ public final class OnHeapColumnVector extends ColumnVector { - - private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); - + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these arrays need to maintain binary compatible. We can // directly pass this buffer to external components. diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index 0457b3781ebf..de066dd406c7 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Date; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; @@ -35,7 +34,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.ql.hooks.HookUtils; import org.apache.hive.service.CompositeService; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.SessionHandle; From 19a14e8417cf4ced0dd0fce863d3f4a0bcf414aa Mon Sep 17 00:00:00 2001 From: depend Date: Thu, 5 May 2016 14:39:35 -0700 Subject: [PATCH 093/313] [SPARK-15158][CORE] downgrade shouldRollover message to debug level ## What changes were proposed in this pull request? set log level to debug when check shouldRollover ## How was this patch tested? It's tested manually. Author: depend Closes #12931 from depend/master. (cherry picked from commit 5c47db06570e65d3f5544d6f26bbdf893e275b94) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/util/logging/RollingPolicy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index 6e80db2f51f9..5c4238c0381a 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -113,7 +113,7 @@ private[spark] class SizeBasedRollingPolicy( /** Should rollover if the next set of bytes is going to exceed the size limit */ def shouldRollover(bytesToBeWritten: Long): Boolean = { - logInfo(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") + logDebug(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") bytesToBeWritten + bytesWrittenSinceRollover > rolloverSizeBytes } From 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 5 May 2016 14:40:37 -0700 Subject: [PATCH 094/313] [SPARK-9926] Parallelize partition logic in UnionRDD. This patch has the new logic from #8512 that uses a parallel collection to compute partitions in UnionRDD. The rest of #8512 added an alternative code path for calculating splits in S3, but that isn't necessary to get the same speedup. The underlying problem wasn't that bulk listing wasn't used, it was that an extra FileStatus was retrieved for each file. The fix was just committed as [HADOOP-12810](https://issues.apache.org/jira/browse/HADOOP-12810). (I think the original commit also used a single prefix to enumerate all paths, but that isn't always helpful and it was removed in later versions so there is no need for SparkS3Utils.) I tested this using the same table that piapiaozhexiu was using. Calculating splits for a 10-day period took 25 seconds with this change and HADOOP-12810, which is on par with the results from #8512. Author: Ryan Blue Author: Cheolsoo Park Closes #11242 from rdblue/SPARK-9926-parallelize-union-rdd. (cherry picked from commit 08db491265a3b50e31993ac6aa07c3f0dd08cdbb) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/rdd/UnionRDD.scala | 18 +++++++++++++++++- .../scala/org/apache/spark/rdd/RDDSuite.scala | 17 +++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 66cf4369da2e..8171dcc04637 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -20,6 +20,8 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.collection.parallel.ForkJoinTaskSupport +import scala.concurrent.forkjoin.ForkJoinPool import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} @@ -62,8 +64,22 @@ class UnionRDD[T: ClassTag]( var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies + // visible for testing + private[spark] val isPartitionListingParallel: Boolean = + rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) + + @transient private lazy val partitionEvalTaskSupport = + new ForkJoinTaskSupport(new ForkJoinPool(8)) + override def getPartitions: Array[Partition] = { - val array = new Array[Partition](rdds.map(_.partitions.length).sum) + val parRDDs = if (isPartitionListingParallel) { + val parArray = rdds.par + parArray.tasksupport = partitionEvalTaskSupport + parArray + } else { + rdds + } + val array = new Array[Partition](parRDDs.map(_.partitions.length).seq.sum) var pos = 0 for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index a663dab772bf..979fb426c948 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -116,6 +116,23 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) } + test("SparkContext.union parallel partition listing") { + val nums1 = sc.makeRDD(Array(1, 2, 3, 4), 2) + val nums2 = sc.makeRDD(Array(5, 6, 7, 8), 2) + val serialUnion = sc.union(nums1, nums2) + val expected = serialUnion.collect().toList + + assert(serialUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === false) + + sc.conf.set("spark.rdd.parallelListingThreshold", "1") + val parallelUnion = sc.union(nums1, nums2) + val actual = parallelUnion.collect().toList + sc.conf.remove("spark.rdd.parallelListingThreshold") + + assert(parallelUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === true) + assert(expected === actual) + } + test("SparkContext.union creates UnionRDD if at least one RDD has no partitioner") { val rddWithPartitioner = sc.parallelize(Seq(1 -> true)).partitionBy(new HashPartitioner(1)) val rddWithNoPartitioner = sc.parallelize(Seq(2 -> true)) From 1064a3303e72d92db02cb94eb2bb81245ac68fc6 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 5 May 2016 14:44:45 -0700 Subject: [PATCH 095/313] [SPARK-14893][SQL] Re-enable HiveSparkSubmitSuite SPARK-8489 test after HiveContext is removed ## What changes were proposed in this pull request? Enable the test that was disabled when HiveContext was removed. ## How was this patch tested? Made sure the enabled test passes with the new jar. Author: Dilip Biswal Closes #12924 from dilipbiswal/spark-14893. (cherry picked from commit 02c07e8999dca545849cb3aa758a624dc51cd1e9) Signed-off-by: Andrew Or --- .../regression-test-SPARK-8489/Main.scala | 12 +++++++----- .../regression-test-SPARK-8489/test-2.10.jar | Bin 6873 -> 6865 bytes .../regression-test-SPARK-8489/test-2.11.jar | Bin 7039 -> 7030 bytes .../spark/sql/hive/HiveSparkSubmitSuite.scala | 3 +-- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala index 10a017df831e..4fbbbacb7608 100644 --- a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala +++ b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala @@ -15,7 +15,6 @@ * limitations under the License. */ -import org.apache.spark.SparkContext import org.apache.spark.sql.SparkSession /** @@ -33,15 +32,18 @@ object Main { def main(args: Array[String]) { // scalastyle:off println println("Running regression test for SPARK-8489.") - val sc = new SparkContext("local", "testing") - val sparkSession = SparkSession.withHiveSupport(sc) + val spark = SparkSession.builder + .master("local") + .appName("testing") + .enableHiveSupport() + .getOrCreate() // This line should not throw scala.reflect.internal.MissingRequirementError. // See SPARK-8470 for more detail. - val df = sparkSession.createDataFrame(Seq(MyCoolClass("1", "2", "3"))) + val df = spark.createDataFrame(Seq(MyCoolClass("1", "2", "3"))) df.collect() println("Regression test for SPARK-8489 success!") // scalastyle:on println - sc.stop() + spark.stop() } } diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.10.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.10.jar index 26d410f33029bb2a9d2b65d9f6985aa159e924a8..3f28d37b93150ebdeec4c6d803351f8c9e1f6cf2 100644 GIT binary patch delta 2816 zcmZ9Oc{~&TAICRZW-R8&Mkp3(bBt7mOwQcrTv2WjQF4_=SYl{aArh*MxsoHa1T!6Rf}AFbaH-XVXaN8K?ce9$c!v^O-GR~rz-b`jihnTpI}iZyXZsIC z0kR(Z#~1mfTq4XqCuyPrAIfA4;ZQ3V}ve^&b%Faw^Xy$@t&{;y{vkNKX*yV2$iD= zMps=WiIk9>U%Ik4q`0^An4z*zk#c9;-M9(B!fWH5^UMkGd%hZ%BwW^+M;O_jVQDiF zUk%C5&IMDgPS|JjDBZ|vbPNcuEsykgid1hZ>sB|I5gWU+_)t=B`a~D{ux&=AYoT=X zK{V8bv!2c2;M!xIQM?{ibiSJETOQWEVXx;)Qx0X-j(==-R1N<64U%a&wYSLQMMt z>DUQI8iJgN^DKJ@nTgT~81mB0HxUs@rIU8a#`iLjQPJulGSFQE+C zSk?e3i1-3div(0$cbUHaQx{JpblM5b@!6h~bv-MYls2W9)-iLniEF}1u+1Zo>lF3n znQ^A*l@N=|mdXX4KMyThec!^QHLmIprg5(|kJ?%!VpMs+1x8jQYy7WOV;u*yw!--X zhTEA+#0Bp;LEcf|tLVl}fY|ga$%rVnx2R?7wmAAp(R+c5jn0woLM;m>ga-4-PV1&- zXiFD+qN@>jT^9weNa3Kg-HIC+u=AdsjxBB*Pn?QAIYlWDDDsrLz5OUP*KIAgjCjQL zBDP4;*e86h|3YsB&hog3uo$10+_$sWt-dld7&B>h2-ic!@GBqPkRW@0F0LIr6;USd z00&!Ws}Zqh@Hf=~|8{PY`(=5XrNK-rO`cGr-ai1;`^ zx6O38T-{Y|;S^BcLm#N|^z;L=hZiEVt(z2HDnV1KCwJd7xDO({h1pFXz&}C>V7xA) zjrU}ES8~%V^ag{fB}awTL_>SG&fq7E8>}PayCAg&CGG4gPU|-0@+yEkG`0q+Ei>qpOpWd%qc5qMCefZ>u&~0+nFDrqWvqR+UgjjvRV`AG2p>3 zReITH#q%LRc@`l=Og=CFdL8O@ZyU{Gtia~YkXpdQbQvIfQ>ZV*sicw}Rw&B9t z8{cj7dP)Yj;b{veg)4E!$V*@NTTCjFTOol~grm8b`qDmzLK~g^6rK1Sw4<;S?dwW~ z4Ele|)PX*u>_#cs>`bcMl;J-npPwBcRCZj}s4XxR}9y+)T;t`PCy-AGfZua+1Z-U%nf14(&%Va^!P?$`qyUT4;|SyW8^P@L1y8)%sPC zu21VN)N@y%)b0Cu*G}+Gh9@qoR49cwva&EdvL2{$Os0bCL4I2t*&MQJ>t-izP_X3r zv}}k1=aM}gpl%!z31v)bT#B11CeM%uYo*4rCD51ZM-{S-TNPPc=Dh1I52V48chE;JRjEME~GD6n|?z#X49lX)W>j4gj zDk~RH5Bo~)9=inOf1>YlTWLiPF}vvZs!W^H*P|mt<%Y*!KRkj6Kle68Yt_DBn$sE{ zI(x|&%cMvn6^PFRt8%>zMhfO0ht*D@VqUwS)bV+jBP83~DU>`P{*umA))|YRC7>;P zxA^*Be|9%OOOK#c{OEC1F&R*TH2lLaio@NcY}yVz8c)3JNJI?Z(`<3G(Rk4>1;;LM zrDk39{g^{#(aS%LLu1|`>?xagU5;sh+8k26)xX7ajw>cz6I0f&I|>yTr0kGVbyE3<$|eF*I!s+b+Ypmb>>VuyjtYPiHR!G9qw3|lk>9G ztX%MqNh%F1FLt0p=To)3`=bAZH!s6C#z`)$q=FfhjVff_F1>CxPW72@wli3<)Rn2a zMKBrd5%l!felK-At%z`K6RIKAx=zEiAOuUB`>oA!h9=RCN3#8ZZL%L9jAzsY@)dr4 z@NxA?(7Up@XMu-Ip^nm|M8aAUIm3gP&}96uL=J55TknyX z&90$4iFLOFvG|6s?~h5elSR&ZHNZ=v%6c9qJxJ;Npuct+32!2ZVg)&aI7!Rtq`l~0 z0+dS%VTpOWD)l>ah#Vv)IRv_`Jvy)XbS=>UHsBUxrY-cX%og9PhB8QbIXg&b2@#7YeVvba*y`u zuMG@gucXYc#Nw99(>>~ByTtP*tfx_rO@vwNm%4Dp@qO{ew|uqm)4$mFb!OOyc#32P z0HitgB?M#>0Q}!tQbZAF!N0ww{=S=}I3lor9{aA5@(iJHXg~A!#G;HMP=E6K!lm#E v9|k5f|RgQq6j|NdGq$Yy)$?2+&kx-xpQXz|9m4lQ#xRKYjzG%00;yE za1z4O!TRjShm{3)&KG#MJ54D)R~nwvDxPQx007P(?Z3l!@nPvhT5PAd8aaR;fpF|~ z$K~%p0N@(iKZ2-0b}3@M%f71!3f9I>LO-|=a#L2R!hy}GC9#FKX?&4U_@GcMq}dww z`pO&nEZ8b|Zu41e$9h*g!_#Vrr#F1MYv`baKnM_7&bj+0Xz*?SM*m)HM92QXOg{1O z;AITjEo*ETfdOU;bdx zaVUX4;o^&mPt|WR+^)lR{&nuISt6)u_p>JadyAgtvc*L6nhiw(Q|H$meF@ULw|!!& zA!2_J;~wW`QcAq2wp8}~$EaJYwppzVT~lQOj*suW!mxz0DPoto(jyEcRt5?+bAjdU{2he3BS%ZsOsac&4dJDDO$aqy{R{ zNUKlC{i$=jI)`ty)N-r2(w(Z-0;Qr<0crH4{L`Xj>>hR+)=V$Mr47_LX55XcU)@nO zZ+Gn#nCV@2;t}C{Y3&tb)*4TUzBe2%LrQ2TdoXPG%Fz>Jo$qD!@8HvBTTaSgC*a*4 zB+O6hQzF}k1O4cw+ntMA%jYm%r$|ghZ~tyzx?!r1Eoi%23Fe~s#cfdQ-sq^F*=DuW zZd>k*aQE1&m8HsgJmK{{&-!pERwgYM*ejKtT4ka&tsymGIo35ZlZ6emU}i%7rOp=n z^DAU;Rci!2y(8YA>8oK|;p8&=rtV#(knN)I6QZk{50c+8k7Dc-oA{GQ79Bg9u`uBRQCQ&DFk+U1^F|z4h?tx@S=H&mn4hFGi*UuzL6Tz)#b}+%9hDsHbPK`zKX81U4^9 z60=^&KGsbs>EU+HP*FS6$u&}|cwNzJcpewVn#-?14V6L=jFrUbK-dJ%GB#*@$VUe} zQ~4PSy$y}XXAckaZcD1pQd`{otPo{DQm*c}e_=z%FrvdJ6<3#Eu=KEBA>94>6(red z1hax$c)XRIQ60eP;@tRwX<+TlVgKEXh`eG9`(+U!{mZ=H&REU&<`7)OTe-&*Tw6*V ztlEHpE<6SUq%pa;TOBAUgqJ?@+4;gGh`jxhJST1Q@50=H_$o|Hl)w9uHY;>O zp*ydIE-?7-Jd&y`dB zm@379#zkqI2C7PiD#JS`2JW5C=5u!qnV6=~zmCMv*f{rJ-67F}~L*P(4KzgPN#c8m?V^32r~6JflF;oPS#<~O z$Gy-tA1$UFO{i-}?fAb@Mh)jn2EJlCvLBemt*l)li$S;&ImH{P7#^N^)H@~`jo0Ud@e zd4VSUavEY`rI&AgGmS>P9`^A3?QVE;sWt6M)wpb&t^~suRJvZ1eCEw5{IIRjgN4kY zVUZM_IBZ}IW%`a9*P|VW8g*7y_{DR+ZsgySYbO!ya>6l(P*6i?E^e85wfm2;*NX2i$zI}rwmKjd9o=DsEsNi*q|ICef((Mhy3&3nCk zSvL=+FS)O{@I2XLn=I^sWlHs5-q`Iy5520!3%PaVM0wIDKfNA1iO=J0yS$AJN#r&~ zxr}`yL=h=@wPiJ4$TGXLWosL8oKdc>4|5lO(lR4k+KE%TKXOz?AJHqkE(}0Coy~xU zcR&=l$vXFd>f77%58ERlH8p8(|D;9FHYGiZ6tDwJE9}ZA+d_K1Gh7xH*H?|od*XDr zGo?icYl*>fK!$9w@YKc_+w$(nG$Hj6i^Vo1Fo>qz`KNX74s&VvatDw%6K)8d#asm^eH&=E*l9MdB; z8?G*Jd^s|#|M(Sk46c0Q`00OjD{+LOf3H012?^w1HI8LVwU-cM3+1Dd_(Z5TB=~ub XbN~7bHBABvtfH1nAh-nJ$Jze|s16>C diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.11.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.11.jar index f34784752f69f71ce56736d3493cfd55bdc32b4a..5e093697e219a249e6f6dedb1ab9f567c58a2fa9 100644 GIT binary patch delta 3351 zcmZu!c{CJk7av26WsognpD?l;#y$~~-AJ+&NyZY{*NkY&GK6fyAj@l?Y>g#SQd zQJ)=#`m{F}A+4E^q;vrYrb&wd1pol9p6-932|n})AT4_hiN2l+!Z#FisgDW(5C#4l z^_H3smuh)v#csuMA}}U`lHl1disn|#P0X*e(5eVyh zLNvrXN^Q?>eyudCdNR``zvTLY>r?6#N+x0ehkrb*{K-%4kw`PSo;)JXx@cI8?%H7n znd{l~f`$SP2U-1T0?hO7M1XP1Vn5b>f^rH9_Jt^+G1Ry#0lPZLneq4?+?^iSd=WyH*RAvxq}|j_)HmYy9tr zVKw0E{5Ji~i5#;C)}}y{IQP1#jt(5_R|eo3u&dCCi8SxycN^wATD}$pMds#@#xOtm znX*Hh<*CP$!!cq;FX`XNNJ%{jH+IfLCI_nZAbpjIi4NVpEx@u_r~RXdRWOESuRnC^ z>T3@YWe~q|f02!qpQX3MQA%5M)Nk`iN?k-{No%_$9+|*w#K*hi?jFNDtVm;B9f6SOZ0+|RAHT?NLU55 zojWbQ%y`gR@MnOTqEKai78snsfZz(`Q!Nd|nDlOU$Hl>;tm0A}<=KGft;oou@+@Wy zXqEa&mXicidGvReE59dp>P#w}?ZK8T~EPd}|oaHi4-c=w`^tz>Sbg z=R8ql-FlH>es45T?IJ%9l3(@nfwI*!y2av2Y`=U{OjR%S8_G0U%EK@F;N0cBS)LrO zkykFH{!OWsRu=s|?i`;XdoOks%|2`?O%w7Med|GyUP#{zK7OTMywVsh*%?g6Vx8oJ-#hB3>Sa(+xzicBu3n3D&NZOmLXb&4QW_}J%Iu)MI6)??gu-G=bC z8$+t+rE+w^cFD&r3TfDutmcKD+wi;KsqFwob+N9AJy01=iqC5Q2+{(#kQ6Y}Tvm0o zuGUP05KVk^u8pXQi6=b2&$e7EwRF&PRsK4dJ~IEyw-)0)>jEXh962{!>&xa^`$@6N zw3T8QXwz<*u9xU%j{C}Qj)v|;Y1%WkB!@k|`7~GpxPiuCgPa81#QM|q{8W@0i$xOJe;j*Y8-gwEgR;37_laTFk%EU0a{;Vg zhg&7&*IUdpUaj1d^W_Z3Aw>-dEFGQ0{a-OGz);MH;Iyq?`S zpmj;}ms#yWq=RI;n|q_3_r7GjWp?h5m4oMRRo8K-%BiYDT;Hyps$KH}h2{OV)Fv;9 z_1^+irbcuiyt#H>;5h(bf$?80I;l^Y`He+H*lmt{4vC>}*(nkP4k|cKX_F{-os(Fw z@`&Q3R45)|#kCVgd$Ah}wvTN8tm6z!~gtEzkxn*p>CS!-zudZg3xWY@g5(7&jP4Qi$p^j7V)4Na~m(w&lHkdEuq%^2J0e8!;*C$@sx=uw{oM+4TXrIB8M*Xc8d&98Z&;ijFs zbDZ{+Vb5XPu|ZUAFX}^9vxl+7wHpzu=vss^`_D)co7mh%zr%_WxwMIi_p7YVa(Xx4 z(8`5*B2rD=szOm4>bht!IRkGFhdUB^7td#_nlkHHaAU#QNmi|!T+sKkWo zr!QTmk4P6_btbRmHY48X>vUBevE|8@;n$$U$wM45hs$@pIJ@8v!ks5lYc6A=aNM)9T#duJT5AqnyXBb@ENkoIHFasi19a>cI^@N|8X*4Qk3D&3)aF{W!I`DaX-8%Ri zUj}10yG$;-MidY2UuFD!>$6%eV&P7Xqn1F|@=63P*a6o8C$s1f_iKz@JtDabkG?@? zZeV*9ECyPat>zk!e3F*sC^9m<2GbVNhSQ3JDilLZrdMo;>@+;56)SUi%rV=**0&Xg zOH?qnlMihGvFQ+G(*N#5^nkdXr#3uxFp>yAmr{Z3M;deFbW#l!;ml zncLP5FrjBUVBxY(UQVyHn47`_x2u>)A^3IgIr&TY;pZ=;7acdP1wYH9YRn{DqnM0l zGW$78W{udA7Cd?G-@sjY%YQ}iXUV%{v%2WkvL<~|LfRV>a;Q3I8OB%M!vDfY)B2Z% z(PQs-lNaOeHjk1aY9xcV?VpGhRU%FYK2r(TO=W?ebk9PU$Elxd-)p=s?wS?MM}$Al zeW5Jj(9sWbWK#d?NX@=t^RU51lp3Wc?=8Dge%vTYpp^C%JLVG?ed`fA$9{Vfb znyx;AL^Cnhx%cpn(ktpo6>-nIRGQPI5<__TF!!io( z{GYrFWvi%2{J8x*(Dqe~<`Q3yglG0hrjVLGUzAEkb5TjnN;(&d<)XMc%zx~>l|e|I z&Q;-`69w4LwW8WxzR<{#s|_*gIMmf_D^E}!`0gj>0Yg8Wrv4pmURXs5^gU>Oz$WQ# zR1>JNYUJ9%HTd2+)B?9`matiLHdL2Q%)lIe<3oW11(Wpli>iN8FjK--1n^5S)SmQKjKvo+dJfEgco=i2i)l!Ex^zKY^(6e0t)-NzS)};|5Ggzhh{FZC{~p)_mbAA5=4J zjMpIaFmv-dlGMKIQUd_?bY~Ss1>^wyZwer^KvY1dV}Sk{^CwXAs0p1Ze-ixDO8i5b z0fb2&?ptS|KfjpMU*-?_cv|Ubdsj~p83%WJKfiyu)c>;?@ceNN00;slP4W@2JWT%$ z>8ueDGQiaR2}S delta 3411 zcmZ8k2{aV!*B^s1SqfQZ#*$suP_jf~EJK#;`@UrCyA-BjtjUszk#&TOUW70qX=Eom z$!^FlYE+i$<9)yH`}cqEbDrm(=bU?=d+#~-`JLbWcJ=pFrdv0uY1jZ@Fc_eW_RnBa zrao1A2oP=)`py=Tf#ZdyXR}Jc0RVvJ$^MTuRg`~b99ZEj(Hs+F$5zZHK6W=dOCYls zEQ=~wka8{-m+X=`4(szlnEt5h=Zav`#C?U4IFyjQQy%SuNm-AlUCmy_jtuD_#8>v2 zbr6`WE0ps~2Cb|jmoG>>Thd?lS>#^zYJHhUCiS9IsTk!PUfP7^zQ`dQZxXz8*Lz8N z7zFJibYsTPnpUGJyCL40bYsElb{R#Vgo+gDor$(|7?Mq{5z>}1gj6*2Re5LoE=Euj zI~WH!TgSmrjERUqHF&x{tZwIH0ZT6Fa2TvN@E$x zE-aj`4vu@v?R*fg2jc*fTe-q!LRB0=OgqZXVk3Lw1#85F4*r=A*0dOuOJD~|M8Y_@ zbM_|8Krn#qC|zds9xGT@kgwliQauhCl#!g}dLXRfsLeDD|4W#qOWAo*4efnL6SqM% zvWqJU&A<%5G2z$$E?F7BidmG{7f!MBy<}Oj#T#_nls$W!8;ne9^4%fCzBZTanqhhp zY8s{#E5G@HC7SPmm+iYk)$=}7XDRaMEj9ND{Mfzql7j6GtO)10Qdv(33Z4}ab+NXk z(A}xu!xR7R5-;oZ$CKQq!UlguYwh%1;JlkK6&g5&0NV14qbK#rTjEK+tx{i92i2yY z1s2?lR%=(Re#g+Vd|{b!)KY(528F@qQGGdk&bK_UG}VE;rbcJ0m}C>F_rZTF`#;y zde7%k7=9I|&mG?#ZZ&;Aa``?h9roF%bQR;Ow7t0c%UJdGSJr(T2GOU~-;#6C>c7+2 zpWie8`*@G@zTvU>tx12Jg;^}KRS0ShC$j`?*Xv4SD$Eu$f>++JqWL(B&5)&e`OEdS zDC^PFW4#~Av%t5igcM}X+)rSA%|s?}|K71PL|H@GYlx2-^6W+1mk&AbDh(bLxZ14y zl?G+|t3}Ub2nJz2Ip`(zXRc22XsdE0ya2PHv(S0&#*I@MNS&rCr0qaK$ZOOm<7uQJ z@4P~Q+jJFAb66u?4Hi*1)7ckr1WZ!WtkjEQfjN!q&G0o6FFO8L5AXDQiW}o3YP@Y!Ksn~l{n>AP zyl#_F&FZ$D-_RGpF>2*+OGRyuF0VYU!W;2L_Q=*!2Za=pA(_B?F6=ZCC7~*-&~_+P zO>6rv?!Ef;85W_jx9h8fg-DN)1V`Cmb{}5sM3JjLqRYGtxeZC(WE_V}?&w{;A{5a} zFOkY^E)ghtJK_@&m2x?jGv%e5oME}Cg?;y6FM-)`$+_0@y&OA;(2K9v9PNIx*UmVY zRLm%EL{2dr&?(Q!Dk#*CKWb=SGcut_zF!9~$Oc*W6vsQGX%#5v8)OA|(49dNal#*A z=&=lvr!|cpzZXO>pTmAffv2l%W$pua{O9EWGbVZq@RABj`bwLowv+j*j|*-VXJN=g z?XvM?L6gvqqB-VN4mC5wTH*%)7pA$^4u4-gwrgqkQbD=S0M!jtldVj|BL$%8XJ%v4n!!tZ-WL=K?X=dMG1%Yj{oz_0jRuwOPt(mbg zv`r>sLoM&3W_}=5o+@ryqsPaEw^P61@%_b;S=M;*Al!|IxvW;dP2#uM3Dp|xOqv_0_&e@jdTqjU4ab2VR({3!VMr*qwQ zR=1c@%$vgt&+T(bkGgABV7G^^uvwqCR*3WDm3=e2(HL&@v;hs7JXIhKglqF@P+>J;<3hE+fwvWjtd72fR%35st7B%-9+wFP9q|1h-fMRn|s z{KTMuLhMcWC^4_gG(u&P-IH;7uU9N?L%taG@kv)MeI!%s?R(__sQWBumNeY{2 z+`w*G6E{K0-P>}j7Jzx>cHZ`VLw;aeM2d?OxadxtD@3kdRXa2Z{n0aj@kT@Kp0MQ` z6_>(F&Jwc)Z^_Qa<)7Z%zx(%qzbtkezl{p-KKMd$JcV#*^c$(@6Zke`{1=303|;}o zL+WU$h*VHYgZJSe6#yW7g7yDRJmfeeQMOGlB@wTZXV@N4fY=8LJ_HvkTH6W^-9sXl zT3Wh!AnYbaZCS!eW`EwZH%SK;Cxs{Cq1luHza=DLh@4 zQR=DKV?8UNt75-}UUHdGe}y06o=VLN>l*!1Hz~{gN2RbsAMxhrTM}mshQnEDQGI6n z`OEc^p*fG%OL9xw49the;O?em-T1}d%UKJqYs5&im_mNDZ0C`0&gnr19p|P>W#G0^ zRL;HQdE&^5DjOuoU#RKrg3**Mv@$(*x9DM{WV=`)UV`;CV=}%}+Uv41ByvKho-|Z~ z?fXd8ZnEqVqIGFkIH@~WfvKJ!#W#Dlm@~@E(VAeTxn%BCQrjb8_iXalc7?gTQcgwF z12>rD?lF_$N!7!Q2s_h)ev%-Y&xq{U1+9dqznl#pezBTPM$RaIh z@E4`SudtA^HJC_;_GPWx>4VtX5n(j`aD1N@r?bh^Da--VQbf5e}rqFATOYXUkOV{v`s@6iD9VEB<4za(K!in%c zxsjRl-f>HN$Qv$j?jESCqcpToNwAn zd@d9{El|wzb7vU1@@@EIQ2d-rWMpytH5B|^o~0iczwg?1c0?l>rFO$XgG*pwI^78p zzFvuVVc>+B_dFH_ib8(IyuF;lKFBQ9X~ZI$!>2nYW7_^yu-PS}sY@xO;4WL*9tJr1 zEW-8j${7yfetvJue8GtL5(dAKm5%V318tYc>?EWkpUcS3lt2WSg10)Rb4bb5PtRL& z7gWW6g~7X>hR?D)kyw$uwzeYM0;;@wwBe)uO~N1-`Ih9Uj^jXhK3W2{7Cd~n#-niY4Mo9MA?J)74#(Ck%;>L`kNTTT&GkS7w5b6AXS#oXGJl2yD#p6+3(VeXN6OeidJ-z)8K Date: Thu, 5 May 2016 16:34:27 -0700 Subject: [PATCH 096/313] [SPARK-15152][DOC][MINOR] Scaladoc and Code style Improvements ## What changes were proposed in this pull request? Minor doc and code style fixes ## How was this patch tested? local build Author: Jacek Laskowski Closes #12928 from jaceklaskowski/SPARK-15152. --- .../scala/org/apache/spark/Accumulator.scala | 9 ++-- .../org/apache/spark/scheduler/Pool.scala | 9 ++-- .../spark/scheduler/SchedulingAlgorithm.scala | 13 ++--- .../spark/util/ShutdownHookManager.scala | 4 +- .../apache/spark/ml/feature/Binarizer.scala | 6 +-- .../org/apache/spark/mllib/util/MLUtils.scala | 5 +- .../sql/catalyst/planning/QueryPlanner.scala | 9 ++-- .../spark/sql/execution/ExpandExec.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 12 +++-- .../sql/execution/WholeStageCodegenExec.scala | 6 +-- .../apache/spark/sql/execution/objects.scala | 2 +- .../streaming/FileStreamSource.scala | 2 +- .../streaming/IncrementalExecution.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 49 ++++++++++--------- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- 15 files changed, 66 insertions(+), 68 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index 23245043e246..9d1f1d59dbce 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -24,16 +24,17 @@ package org.apache.spark * They can be used to implement counters (as in MapReduce) or sums. Spark natively supports * accumulators of numeric value types, and programmers can add support for new types. * - * An accumulator is created from an initial value `v` by calling [[SparkContext#accumulator]]. - * Tasks running on the cluster can then add to it using the [[Accumulable#+=]] operator. + * An accumulator is created from an initial value `v` by calling + * [[SparkContext#accumulator SparkContext.accumulator]]. + * Tasks running on the cluster can then add to it using the [[Accumulable#+= +=]] operator. * However, they cannot read its value. Only the driver program can read the accumulator's value, - * using its value method. + * using its [[#value]] method. * * The interpreter session below shows an accumulator being used to add up the elements of an array: * * {{{ * scala> val accum = sc.accumulator(0) - * accum: spark.Accumulator[Int] = 0 + * accum: org.apache.spark.Accumulator[Int] = 0 * * scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) * ... diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index a79e71ec7c9b..5987cfea2e9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -26,16 +26,14 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** - * An Schedulable entity that represent collection of Pools or TaskSetManagers + * An Schedulable entity that represents collection of Pools or TaskSetManagers */ - private[spark] class Pool( val poolName: String, val schedulingMode: SchedulingMode, initMinShare: Int, initWeight: Int) - extends Schedulable - with Logging { + extends Schedulable with Logging { val schedulableQueue = new ConcurrentLinkedQueue[Schedulable] val schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable] @@ -56,7 +54,8 @@ private[spark] class Pool( case SchedulingMode.FIFO => new FIFOSchedulingAlgorithm() case _ => - throw new IllegalArgumentException(s"Unsupported spark.scheduler.mode: $schedulingMode") + val msg = "Unsupported scheduling mode: $schedulingMode. Use FAIR or FIFO instead." + throw new IllegalArgumentException(msg) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala index 864941d468af..18ebbbe78a5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala @@ -36,11 +36,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { val stageId2 = s2.stageId res = math.signum(stageId1 - stageId2) } - if (res < 0) { - true - } else { - false - } + res < 0 } } @@ -52,12 +48,12 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0) + val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0) val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble - var compare: Int = 0 + var compare = 0 if (s1Needy && !s2Needy) { return true } else if (!s1Needy && s2Needy) { @@ -67,7 +63,6 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { } else { compare = taskToWeightRatio1.compareTo(taskToWeightRatio2) } - if (compare < 0) { true } else if (compare > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index bd26bfd848ff..93ac67e5db0d 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -170,9 +170,7 @@ private [util] class SparkShutdownHookManager { @volatile private var shuttingDown = false /** - * Install a hook to run at shutdown and run all registered hooks in order. Hadoop 1.x does not - * have `ShutdownHookManager`, so in that case we just use the JVM's `Runtime` object and hope for - * the best. + * Install a hook to run at shutdown and run all registered hooks in order. */ def install(): Unit = { val hookTask = new Runnable() { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 898ac2cc8941..35bc46a5f343 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -104,9 +104,9 @@ final class Binarizer(override val uid: String) case DoubleType => BinaryAttribute.defaultAttr.withName(outputColName).toStructField() case _: VectorUDT => - new StructField(outputColName, new VectorUDT, true) - case other => - throw new IllegalArgumentException(s"Data type $other is not supported.") + StructField(outputColName, new VectorUDT) + case _ => + throw new IllegalArgumentException(s"Data type $inputType is not supported.") } if (schema.fieldNames.contains(outputColName)) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 774170ff401e..86ce9705a312 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -104,7 +104,7 @@ object MLUtils { val (indices, values) = items.tail.filter(_.nonEmpty).map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. - val value = indexAndValue(1).toDouble + val value = indexAndValue(1).toDouble (index, value) }.unzip @@ -119,8 +119,7 @@ object MLUtils { previous = current i += 1 } - - (label, indices.toArray, values.toArray) + (label, indices, values) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 516b41cb138b..8b1a34f79c42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode /** - * Given a [[plans.logical.LogicalPlan LogicalPlan]], returns a list of `PhysicalPlan`s that can + * Given a [[LogicalPlan]], returns a list of `PhysicalPlan`s that can * be used for execution. If this strategy does not apply to the give logical operation then an * empty list should be returned. */ @@ -31,9 +31,10 @@ abstract class GenericStrategy[PhysicalPlan <: TreeNode[PhysicalPlan]] extends L } /** - * Abstract class for transforming [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans. - * Child classes are responsible for specifying a list of [[Strategy]] objects that each of which - * can return a list of possible physical plan options. If a given strategy is unable to plan all + * Abstract class for transforming [[LogicalPlan]]s into physical plans. + * Child classes are responsible for specifying a list of [[GenericStrategy]] objects that + * each of which can return a list of possible physical plan options. + * If a given strategy is unable to plan all * of the remaining operators in the tree, it can call [[planLater]], which returns a placeholder * object that will be filled in using other available strategies. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index c201822d4479..1be41ffc072c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit import org.apache.spark.sql.execution.metric.SQLMetrics /** - * Apply the all of the GroupExpressions to every input row, hence we will get + * Apply all of the GroupExpressions to every input row, hence we will get * multiple output rows for a input row. * @param projections The group of expressions, all of the group expressions should * output the same schema specified bye the parameter `output` diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 0bbe97042070..b94b84d77a50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -106,16 +106,20 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil) /** - * Returns the result of this query as an RDD[InternalRow] by delegating to doExecute after - * preparations. Concrete implementations of SparkPlan should override doExecute. + * Returns the result of this query as an RDD[InternalRow] by delegating to `doExecute` after + * preparations. + * + * Concrete implementations of SparkPlan should override `doExecute`. */ final def execute(): RDD[InternalRow] = executeQuery { doExecute() } /** - * Returns the result of this query as a broadcast variable by delegating to doBroadcast after - * preparations. Concrete implementations of SparkPlan should override doBroadcast. + * Returns the result of this query as a broadcast variable by delegating to `doExecuteBroadcast` + * after preparations. + * + * Concrete implementations of SparkPlan should override `doExecuteBroadcast`. */ final def executeBroadcast[T](): broadcast.Broadcast[T] = executeQuery { doExecuteBroadcast() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 15b4abe80667..d6f7b6ed35db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -105,7 +105,7 @@ trait CodegenSupport extends SparkPlan { protected def doProduce(ctx: CodegenContext): String /** - * Consume the generated columns or row from current SparkPlan, call it's parent's doConsume(). + * Consume the generated columns or row from current SparkPlan, call its parent's `doConsume()`. */ final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { val inputVars = @@ -212,8 +212,8 @@ trait CodegenSupport extends SparkPlan { /** * InputAdapter is used to hide a SparkPlan from a subtree that support codegen. * - * This is the leaf node of a tree with WholeStageCodegen, is used to generate code that consumes - * an RDD iterator of InternalRow. + * This is the leaf node of a tree with WholeStageCodegen that is used to generate code + * that consumes an RDD iterator of InternalRow. */ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 56a39069511d..bafbbdf65724 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -158,7 +158,7 @@ case class MapPartitionsExec( * Applies the given function to each input object. * The output of its child must be a single-field row containing the input object. * - * This operator is kind of a safe version of [[ProjectExec]], as it's output is custom object, + * This operator is kind of a safe version of [[ProjectExec]], as its output is custom object, * we need to use safe row to contain it. */ case class MapElementsExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 8e66538575b0..7b4c035bf3c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -95,7 +95,7 @@ class FileStreamSource( val endId = end.asInstanceOf[LongOffset].offset assert(startId <= endId) - val files = metadataLog.get(Some(startId + 1), Some(endId)).map(_._2).flatten + val files = metadataLog.get(Some(startId + 1), Some(endId)).flatMap(_._2) logInfo(s"Processing ${files.length} files from ${startId + 1}:$endId") logDebug(s"Streaming ${files.mkString(", ")}") dataFrameBuilder(files) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index b89144d72751..e9052a309595 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -47,7 +47,7 @@ class IncrementalExecution( /** * Records the current id for a given stateful operator in the query plan as the `state` - * preperation walks the query plan. + * preparation walks the query plan. */ private var operatorId = 0 diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 7d8b8679c594..6ececb1062ae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -84,7 +84,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) assert(framework != null, "Checkpoint.framework is null") assert(graph != null, "Checkpoint.graph is null") assert(checkpointTime != null, "Checkpoint.checkpointTime is null") - logInfo("Checkpoint for time " + checkpointTime + " validated") + logInfo(s"Checkpoint for time $checkpointTime validated") } } @@ -103,7 +103,10 @@ object Checkpoint extends Logging { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds + ".bk") } - /** Get checkpoint files present in the give directory, ordered by oldest-first */ + /** + * @param checkpointDir checkpoint directory to read checkpoint files from + * @return checkpoint files from the `checkpointDir` checkpoint directory, ordered by oldest-first + */ def getCheckpointFiles(checkpointDir: String, fsOption: Option[FileSystem] = None): Seq[Path] = { def sortFunc(path1: Path, path2: Path): Boolean = { @@ -121,11 +124,11 @@ object Checkpoint extends Logging { val filtered = paths.filter(p => REGEX.findFirstIn(p.toString).nonEmpty) filtered.sortWith(sortFunc) } else { - logWarning("Listing " + path + " returned null") + logWarning(s"Listing $path returned null") Seq.empty } } else { - logInfo("Checkpoint directory " + path + " does not exist") + logWarning(s"Checkpoint directory $path does not exist") Seq.empty } } @@ -205,7 +208,7 @@ class CheckpointWriter( // time of a batch is greater than the batch interval, checkpointing for completing an old // batch may run after checkpointing of a new batch. If this happens, checkpoint of an old // batch actually has the latest information, so we want to recovery from it. Therefore, we - // also use the latest checkpoint time as the file name, so that we can recovery from the + // also use the latest checkpoint time as the file name, so that we can recover from the // latest checkpoint file. // // Note: there is only one thread writing the checkpoint files, so we don't need to worry @@ -216,8 +219,7 @@ class CheckpointWriter( while (attempts < MAX_ATTEMPTS && !stopped) { attempts += 1 try { - logInfo("Saving checkpoint for time " + checkpointTime + " to file '" + checkpointFile - + "'") + logInfo(s"Saving checkpoint for time $checkpointTime to file '$checkpointFile'") // Write checkpoint to temp file if (fs.exists(tempFile)) { @@ -237,39 +239,38 @@ class CheckpointWriter( fs.delete(backupFile, true) // just in case it exists } if (!fs.rename(checkpointFile, backupFile)) { - logWarning("Could not rename " + checkpointFile + " to " + backupFile) + logWarning(s"Could not rename $checkpointFile to $backupFile") } } // Rename temp file to the final checkpoint file if (!fs.rename(tempFile, checkpointFile)) { - logWarning("Could not rename " + tempFile + " to " + checkpointFile) + logWarning(s"Could not rename $tempFile to $checkpointFile") } // Delete old checkpoint files val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)) if (allCheckpointFiles.size > 10) { allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach { file => - logInfo("Deleting " + file) + logInfo(s"Deleting $file") fs.delete(file, true) } } // All done, print success val finishTime = System.currentTimeMillis() - logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + - "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") + logInfo(s"Checkpoint for time $checkpointTime saved to file '$checkpointFile'" + + s", took ${bytes.length} bytes and ${finishTime - startTime} ms") jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { case ioe: IOException => - logWarning("Error in attempt " + attempts + " of writing checkpoint to " - + checkpointFile, ioe) + val msg = s"Error in attempt $attempts of writing checkpoint to '$checkpointFile'" + logWarning(msg, ioe) fs = null } } - logWarning("Could not write checkpoint for time " + checkpointTime + " to file " - + checkpointFile + "'") + logWarning(s"Could not write checkpoint for time $checkpointTime to file '$checkpointFile'") } } @@ -278,7 +279,7 @@ class CheckpointWriter( val bytes = Checkpoint.serialize(checkpoint, conf) executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bytes, clearCheckpointDataLater)) - logInfo("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") + logInfo(s"Submitted checkpoint of time ${checkpoint.checkpointTime} to writer queue") } catch { case rej: RejectedExecutionException => logError("Could not submit checkpoint task to the thread pool executor", rej) @@ -295,8 +296,8 @@ class CheckpointWriter( executor.shutdownNow() } val endTime = System.currentTimeMillis() - logInfo("CheckpointWriter executor terminated ? " + terminated + - ", waited for " + (endTime - startTime) + " ms.") + logInfo(s"CheckpointWriter executor terminated? $terminated," + + s" waited for ${endTime - startTime} ms.") stopped = true } } @@ -336,20 +337,20 @@ object CheckpointReader extends Logging { } // Try to read the checkpoint files in the order - logInfo("Checkpoint files found: " + checkpointFiles.mkString(",")) + logInfo(s"Checkpoint files found: ${checkpointFiles.mkString(",")}") var readError: Exception = null checkpointFiles.foreach { file => - logInfo("Attempting to load checkpoint from file " + file) + logInfo(s"Attempting to load checkpoint from file $file") try { val fis = fs.open(file) val cp = Checkpoint.deserialize(fis, conf) - logInfo("Checkpoint successfully loaded from file " + file) - logInfo("Checkpoint was generated at time " + cp.checkpointTime) + logInfo(s"Checkpoint successfully loaded from file $file") + logInfo(s"Checkpoint was generated at time ${cp.checkpointTime}") return Some(cp) } catch { case e: Exception => readError = e - logWarning("Error reading checkpoint from file " + file, e) + logWarning(s"Error reading checkpoint from file $file", e) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index b494ef0dd966..7ea58afb53dc 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -158,7 +158,7 @@ private[spark] class Client( val newAppResponse = newApp.getNewApplicationResponse() appId = newAppResponse.getApplicationId() reportLauncherState(SparkAppHandle.State.SUBMITTED) - launcherBackend.setAppId(appId.toString()) + launcherBackend.setAppId(appId.toString) // Verify whether the cluster has enough resources for our AM verifyClusterResources(newAppResponse) @@ -168,7 +168,7 @@ private[spark] class Client( val appContext = createApplicationSubmissionContext(newApp, containerContext) // Finally, submit and monitor the application - logInfo(s"Submitting application ${appId.getId} to ResourceManager") + logInfo(s"Submitting application $appId to ResourceManager") yarnClient.submitApplication(appContext) appId } catch { From 7dc3fb6ae44ba9863eb59c2724c73201c46e5213 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 5 May 2016 16:51:06 -0700 Subject: [PATCH 097/313] [HOTFIX] Fix MLUtils compile --- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 86ce9705a312..f0346e600d23 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -119,7 +119,7 @@ object MLUtils { previous = current i += 1 } - (label, indices, values) + (label, indices.toArray, values.toArray) } /** From 42f2ee6c5d981cdc8bd6b3845f0593a87aae48b6 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 5 May 2016 18:49:43 -0700 Subject: [PATCH 098/313] [SPARK-11395][SPARKR] Support over and window specification in SparkR. This PR: 1. Implement WindowSpec S4 class. 2. Implement Window.partitionBy() and Window.orderBy() as utility functions to create WindowSpec objects. 3. Implement over() of Column class. Author: Sun Rui Author: Sun Rui Closes #10094 from sun-rui/SPARK-11395. (cherry picked from commit 157a49aa410dc1870cd171148d317084c5a90d23) Signed-off-by: Shivaram Venkataraman --- R/pkg/DESCRIPTION | 2 + R/pkg/NAMESPACE | 10 ++ R/pkg/R/DataFrame.R | 4 +- R/pkg/R/WindowSpec.R | 188 ++++++++++++++++++++++ R/pkg/R/generics.R | 29 +++- R/pkg/R/pairRDD.R | 4 +- R/pkg/R/window.R | 98 +++++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 36 +++++ 8 files changed, 364 insertions(+), 7 deletions(-) create mode 100644 R/pkg/R/WindowSpec.R create mode 100644 R/pkg/R/window.R diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 7179438efc1d..963a1bb5806a 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -26,6 +26,7 @@ Collate: 'pairRDD.R' 'DataFrame.R' 'SQLContext.R' + 'WindowSpec.R' 'backend.R' 'broadcast.R' 'client.R' @@ -38,4 +39,5 @@ Collate: 'stats.R' 'types.R' 'utils.R' + 'window.R' RoxygenNote: 5.0.1 diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 73f7c595f443..1432ab8a9d1c 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -216,6 +216,7 @@ exportMethods("%in%", "next_day", "ntile", "otherwise", + "over", "percent_rank", "pmod", "quarter", @@ -315,3 +316,12 @@ export("structField", "structType.jobj", "structType.structField", "print.structType") + +exportClasses("WindowSpec") + +export("partitionBy", + "rowsBetween", + "rangeBetween") + +export("window.partitionBy", + "window.orderBy") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index fcf473ac7b76..43c46b847446 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1749,8 +1749,8 @@ setMethod("arrange", #' @export setMethod("orderBy", signature(x = "SparkDataFrame", col = "characterOrColumn"), - function(x, col) { - arrange(x, col) + function(x, col, ...) { + arrange(x, col, ...) }) #' Filter diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R new file mode 100644 index 000000000000..581176a6c091 --- /dev/null +++ b/R/pkg/R/WindowSpec.R @@ -0,0 +1,188 @@ +# +# 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. +# + +# WindowSpec.R - WindowSpec class and methods implemented in S4 OO classes + +#' @include generics.R jobj.R column.R +NULL + +#' @title S4 class that represents a WindowSpec +#' @description WindowSpec can be created by using window.partitionBy() +#' or window.orderBy() +#' @rdname WindowSpec +#' @seealso \link{window.partitionBy}, \link{window.orderBy} +#' +#' @param sws A Java object reference to the backing Scala WindowSpec +#' @export +setClass("WindowSpec", + slots = list(sws = "jobj")) + +setMethod("initialize", "WindowSpec", function(.Object, sws) { + .Object@sws <- sws + .Object +}) + +windowSpec <- function(sws) { + stopifnot(class(sws) == "jobj") + new("WindowSpec", sws) +} + +#' @rdname show +setMethod("show", "WindowSpec", + function(object) { + cat("WindowSpec", callJMethod(object@sws, "toString"), "\n") + }) + +#' partitionBy +#' +#' Defines the partitioning columns in a WindowSpec. +#' +#' @param x a WindowSpec +#' @return a WindowSpec +#' @rdname partitionBy +#' @name partitionBy +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' partitionBy(ws, "col1", "col2") +#' partitionBy(ws, df$col1, df$col2) +#' } +setMethod("partitionBy", + signature(x = "WindowSpec"), + function(x, col, ...) { + stopifnot (class(col) %in% c("character", "Column")) + + if (class(col) == "character") { + windowSpec(callJMethod(x@sws, "partitionBy", col, list(...))) + } else { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "partitionBy", jcols)) + } + }) + +#' orderBy +#' +#' Defines the ordering columns in a WindowSpec. +#' +#' @param x a WindowSpec +#' @return a WindowSpec +#' @rdname arrange +#' @name orderBy +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' orderBy(ws, "col1", "col2") +#' orderBy(ws, df$col1, df$col2) +#' } +setMethod("orderBy", + signature(x = "WindowSpec", col = "character"), + function(x, col, ...) { + windowSpec(callJMethod(x@sws, "orderBy", col, list(...))) + }) + +#' @rdname arrange +#' @name orderBy +#' @export +setMethod("orderBy", + signature(x = "WindowSpec", col = "Column"), + function(x, col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "orderBy", jcols)) + }) + +#' rowsBetween +#' +#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' +#' Both `start` and `end` are relative positions from the current row. For example, "0" means +#' "current row", while "-1" means the row before the current row, and "5" means the fifth row +#' after the current row. +#' +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rowsBetween +#' @name rowsBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rowsBetween(ws, 0, 3) +#' } +setMethod("rowsBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rowsBetween", as.integer(start), as.integer(end))) + }) + +#' rangeBetween +#' +#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' +#' Both `start` and `end` are relative from the current row. For example, "0" means "current row", +#' while "-1" means one off before the current row, and "5" means the five off after the +#' current row. + +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rangeBetween +#' @name rangeBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rangeBetween(ws, 0, 3) +#' } +setMethod("rangeBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rangeBetween", as.integer(start), as.integer(end))) + }) + +# Note that over is a method of Column class, but it is placed here to +# avoid Roxygen circular-dependency between class Column and WindowSpec. + +#' over +#' +#' Define a windowing column. +#' +#' @rdname over +#' @name over +#' @family colum_func +#' @export +setMethod("over", + signature(x = "Column", window = "WindowSpec"), + function(x, window) { + column(callJMethod(x@jc, "over", window@sws)) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 3db1ac07666b..8563be1e6498 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -339,9 +339,9 @@ setGeneric("join", function(x, y, ...) { standardGeneric("join") }) # @export setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) -# @rdname partitionBy -# @export -setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, ...) { standardGeneric("partitionBy") }) # @rdname reduceByKey # @seealso groupByKey @@ -533,7 +533,7 @@ setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) #' @rdname arrange #' @export -setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") }) +setGeneric("orderBy", function(x, col, ...) { standardGeneric("orderBy") }) #' @rdname schema #' @export @@ -733,6 +733,27 @@ setGeneric("when", function(condition, value) { standardGeneric("when") }) #' @export setGeneric("otherwise", function(x, value) { standardGeneric("otherwise") }) +#' @rdname over +#' @export +setGeneric("over", function(x, window) { standardGeneric("over") }) + +###################### WindowSpec Methods ########################## + +#' @rdname rowsBetween +#' @export +setGeneric("rowsBetween", function(x, start, end) { standardGeneric("rowsBetween") }) + +#' @rdname rangeBetween +#' @export +setGeneric("rangeBetween", function(x, start, end) { standardGeneric("rangeBetween") }) + +#' @rdname window.partitionBy +#' @export +setGeneric("window.partitionBy", function(col, ...) { standardGeneric("window.partitionBy") }) + +#' @rdname window.orderBy +#' @export +setGeneric("window.orderBy", function(col, ...) { standardGeneric("window.orderBy") }) ###################### Expression Function Methods ########################## diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 4075ef4377ac..d39775cabef8 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -205,8 +205,10 @@ setMethod("flatMapValues", #' @aliases partitionBy,RDD,integer-method #' @noRd setMethod("partitionBy", - signature(x = "RDD", numPartitions = "numeric"), + signature(x = "RDD"), function(x, numPartitions, partitionFunc = hashCode) { + stopifnot(is.numeric(numPartitions)) + partitionFunc <- cleanClosure(partitionFunc) serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R new file mode 100644 index 000000000000..7ecf70abc6b5 --- /dev/null +++ b/R/pkg/R/window.R @@ -0,0 +1,98 @@ +# +# 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. +# + +# window.R - Utility functions for defining window in DataFrames + +#' window.partitionBy +#' +#' Creates a WindowSpec with the partitioning defined. +#' +#' @rdname window.partitionBy +#' @name window.partitionBy +#' @export +#' @examples +#' \dontrun{ +#' ws <- window.partitionBy("key1", "key2") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- window.partitionBy(df$key1, df$key2) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +setMethod("window.partitionBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + col, + list(...))) + }) + +#' @rdname window.partitionBy +#' @name window.partitionBy +#' @export +setMethod("window.partitionBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + jcols)) + }) + +#' window.orderBy +#' +#' Creates a WindowSpec with the ordering defined. +#' +#' @rdname window.orderBy +#' @name window.orderBy +#' @export +#' @examples +#' \dontrun{ +#' ws <- window.orderBy("key1", "key2") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- window.orderBy(df$key1, df$key2) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +setMethod("window.orderBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + col, + list(...))) + }) + +#' @rdname window.orderBy +#' @name window.orderBy +#' @export +setMethod("window.orderBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + jcols)) + }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 3b6a27c3b86a..0f67bc2e331d 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2118,6 +2118,42 @@ test_that("repartition by columns on DataFrame", { expect_equal(nrow(df1), 2) }) +test_that("Window functions on a DataFrame", { + ssc <- callJMethod(sc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, + error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + + df <- createDataFrame(hiveCtx, + list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), + schema = c("key", "value")) + ws <- orderBy(window.partitionBy("key"), "value") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expected <- data.frame(key = c(1L, NA, 2L, NA), + value = c("1", NA, "2", NA), + stringsAsFactors = FALSE) + expect_equal(result, expected) + + ws <- orderBy(window.partitionBy(df$key), df$value) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(window.orderBy("value"), "key") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(window.orderBy(df$value), df$key) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) +}) + unlink(parquetPath) unlink(jsonPath) unlink(jsonPathNa) From 1ee621b1d949ce8e1bb41ef3fe19dfaad4a90ab1 Mon Sep 17 00:00:00 2001 From: Luciano Resende Date: Fri, 6 May 2016 12:25:45 +0100 Subject: [PATCH 099/313] [SPARK-14738][BUILD] Separate docker integration tests from main build ## What changes were proposed in this pull request? Create a maven profile for executing the docker integration tests using maven Remove docker integration tests from main sbt build Update documentation on how to run docker integration tests from sbt ## How was this patch tested? Manual test of the docker integration tests as in : mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 compile test ## Other comments Note that the the DB2 Docker Tests are still disabled as there is a kernel version issue on the AMPLab Jenkins slaves and we would need to get them on the right level before enabling those tests. They do run ok locally with the updates from PR #12348 Author: Luciano Resende Closes #12508 from lresende/docker. (cherry picked from commit a03c5e68abd8c066c97ebd388883070d59dce1a7) Signed-off-by: Sean Owen --- docs/building-spark.md | 12 ++++++++++++ .../spark/sql/jdbc/MySQLIntegrationSuite.scala | 3 --- .../spark/sql/jdbc/OracleIntegrationSuite.scala | 5 +---- .../spark/sql/jdbc/PostgresIntegrationSuite.scala | 3 --- pom.xml | 8 +++++++- project/SparkBuild.scala | 3 ++- 6 files changed, 22 insertions(+), 12 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index fec442af95e1..13c95e4fcb39 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -190,6 +190,18 @@ or Java 8 tests are automatically enabled when a Java 8 JDK is detected. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +# Running Docker based Integration Test Suites + +Running only docker based integration tests and nothing else. + + mvn install -DskipTests + mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + +or + + sbt docker-integration-tests/test + + # Packaging without Hadoop Dependencies for YARN The assembly directory produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with `yarn.application.classpath`. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index aa47228eff3a..a70ed98b52d5 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -21,12 +21,9 @@ import java.math.BigDecimal import java.sql.{Connection, Date, Timestamp} import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.tags.DockerTest @DockerTest -@Ignore class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { override val imageName = "mysql:5.7.9" diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 357866b87ca2..c5e1f8607b33 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.jdbc import java.sql.Connection import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.tags.DockerTest @@ -46,12 +44,11 @@ import org.apache.spark.tags.DockerTest * repository. */ @DockerTest -@Ignore class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLContext { import testImplicits._ override val db = new DatabaseOnDocker { - override val imageName = "wnameless/oracle-xe-11g:latest" + override val imageName = "wnameless/oracle-xe-11g:14.04.4" override val env = Map( "ORACLE_ROOT_PASSWORD" -> "oracle" ) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 6546d4cfd7ce..79dd70116ecb 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -20,15 +20,12 @@ package org.apache.spark.sql.jdbc import java.sql.Connection import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{ArrayType, DecimalType} import org.apache.spark.tags.DockerTest @DockerTest -@Ignore class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { override val imageName = "postgres:9.4.5" diff --git a/pom.xml b/pom.xml index 852136a998ab..3e783fa56e76 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,6 @@ sql/core sql/hive sql/hivecontext-compatibility - external/docker-integration-tests assembly external/flume external/flume-sink @@ -2469,6 +2468,13 @@ + + docker-integration-tests + + external/docker-integration-tests + + + - org.eclipse.jetty.orbit - javax.servlet - ${orbit.version} + org.eclipse.jetty + jetty-servlets + compile + + + javax.servlet + javax.servlet-api + ${javaxservlet.version} @@ -356,7 +359,7 @@ true true - guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security + guava,jetty-io,jetty-servlet,jetty-servlets,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security true diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index 719905a2c901..be19179b00a4 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -71,7 +71,7 @@ private[spark] case class SSLOptions( keyPassword.foreach(sslContextFactory.setKeyManagerPassword) keyStoreType.foreach(sslContextFactory.setKeyStoreType) if (needClientAuth) { - trustStore.foreach(file => sslContextFactory.setTrustStore(file.getAbsolutePath)) + trustStore.foreach(file => sslContextFactory.setTrustStorePath(file.getAbsolutePath)) trustStorePassword.foreach(sslContextFactory.setTrustStorePassword) trustStoreType.foreach(sslContextFactory.setTrustStoreType) } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 14244ea5714c..7e93bfc45ebc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -17,13 +17,12 @@ package org.apache.spark.deploy.rest -import java.net.InetSocketAddress import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.{Server, ServerConnector} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s._ @@ -80,10 +79,15 @@ private[spark] abstract class RestSubmissionServer( * Return a 2-tuple of the started server and the bound port. */ private def doStart(startPort: Int): (Server, Int) = { - val server = new Server(new InetSocketAddress(host, startPort)) val threadPool = new QueuedThreadPool threadPool.setDaemon(true) - server.setThreadPool(threadPool) + val server = new Server(threadPool) + + val connector = new ServerConnector(server) + connector.setHost(host) + connector.setPort(startPort) + server.addConnector(connector) + val mainHandler = new ServletContextHandler mainHandler.setContextPath("/") contextToServlet.foreach { case (prefix, servlet) => @@ -91,7 +95,7 @@ private[spark] abstract class RestSubmissionServer( } server.setHandler(mainHandler) server.start() - val boundPort = server.getConnectors()(0).getLocalPort + val boundPort = connector.getLocalPort (server, boundPort) } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index db24f0319ba0..6854f7baf716 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -25,11 +25,10 @@ import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import scala.xml.Node -import org.eclipse.jetty.server.{Connector, Request, Server} +import org.eclipse.jetty.server.{Request, Server, ServerConnector} import org.eclipse.jetty.server.handler._ -import org.eclipse.jetty.server.nio.SelectChannelConnector -import org.eclipse.jetty.server.ssl.SslSelectChannelConnector import org.eclipse.jetty.servlet._ +import org.eclipse.jetty.servlets.gzip.GzipHandler import org.eclipse.jetty.util.component.LifeCycle import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s.JValue @@ -243,10 +242,16 @@ private[spark] object JettyUtils extends Logging { // Bind to the given port, or throw a java.net.BindException if the port is occupied def connect(currentPort: Int): (Server, Int) = { - val server = new Server - val connectors = new ArrayBuffer[Connector] + val pool = new QueuedThreadPool + if (serverName.nonEmpty) { + pool.setName(serverName) + } + pool.setDaemon(true) + + val server = new Server(pool) + val connectors = new ArrayBuffer[ServerConnector] // Create a connector on port currentPort to listen for HTTP requests - val httpConnector = new SelectChannelConnector() + val httpConnector = new ServerConnector(server) httpConnector.setPort(currentPort) connectors += httpConnector @@ -260,8 +265,9 @@ private[spark] object JettyUtils extends Logging { } val scheme = "https" // Create a connector on port securePort to listen for HTTPS requests - val connector = new SslSelectChannelConnector(factory) + val connector = new ServerConnector(server, factory) connector.setPort(securePort) + connectors += connector // redirect the HTTP requests to HTTPS port @@ -269,34 +275,27 @@ private[spark] object JettyUtils extends Logging { } gzipHandlers.foreach(collection.addHandler) - connectors.foreach(_.setHost(hostName)) // As each acceptor and each selector will use one thread, the number of threads should at // least be the number of acceptors and selectors plus 1. (See SPARK-13776) var minThreads = 1 - connectors.foreach { c => + connectors.foreach { connector => // Currently we only use "SelectChannelConnector" - val connector = c.asInstanceOf[SelectChannelConnector] // Limit the max acceptor number to 8 so that we don't waste a lot of threads - connector.setAcceptors(math.min(connector.getAcceptors, 8)) + connector.setAcceptQueueSize(math.min(connector.getAcceptors, 8)) + connector.setHost(hostName) // The number of selectors always equals to the number of acceptors minThreads += connector.getAcceptors * 2 } server.setConnectors(connectors.toArray) - - val pool = new QueuedThreadPool - if (serverName.nonEmpty) { - pool.setName(serverName) - } pool.setMaxThreads(math.max(pool.getMaxThreads, minThreads)) - pool.setDaemon(true) - server.setThreadPool(pool) + val errorHandler = new ErrorHandler() errorHandler.setShowStacks(true) server.addBean(errorHandler) server.setHandler(collection) try { server.start() - (server, server.getConnectors.head.getLocalPort) + (server, httpConnector.getLocalPort) } catch { case e: Exception => server.stop() diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index cc934e9e6ff2..83bdd9031903 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -46,7 +46,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -83,7 +83,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jcl-over-slf4j-1.7.16.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index f85aab3f93d5..121e28261837 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -86,7 +86,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index dfcd35193c4e..1d5ad271326d 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -86,7 +86,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index a29f6700bd59..909b94bde20c 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar @@ -94,7 +94,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8955b0e97727..750759974021 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar @@ -94,7 +94,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/pom.xml b/pom.xml index d71913c9589c..9e9aad223a5e 100644 --- a/pom.xml +++ b/pom.xml @@ -133,11 +133,11 @@ 1.2.1.spark2 1.2.1 - 10.10.1.1 + 10.11.1.1 1.7.0 1.6.0 - 8.1.19.v20160209 - 3.0.0.v201112011016 + 9.2.16.v20160414 + 3.1.0 0.8.0 2.4.0 2.0.8 @@ -328,6 +328,12 @@ ${jetty.version} provided + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + provided + org.eclipse.jetty jetty-util @@ -2264,6 +2270,7 @@ org.eclipse.jetty:jetty-http org.eclipse.jetty:jetty-continuation org.eclipse.jetty:jetty-servlet + org.eclipse.jetty:jetty-servlets org.eclipse.jetty:jetty-plus org.eclipse.jetty:jetty-security org.eclipse.jetty:jetty-util diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 3b57efa38b58..37e4845cceb9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -37,8 +37,7 @@ import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.TServlet; -import org.eclipse.jetty.server.nio.SelectChannelConnector; -import org.eclipse.jetty.server.ssl.SslSelectChannelConnector; +import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.ssl.SslContextFactory; @@ -59,9 +58,6 @@ public ThriftHttpCLIService(CLIService cliService) { @Override public void run() { try { - // HTTP Server - httpServer = new org.eclipse.jetty.server.Server(); - // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests String threadPoolName = "HiveServer2-HttpHandler-Pool"; @@ -69,10 +65,12 @@ public void run() { workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), new ThreadFactoryWithGarbageCleanup(threadPoolName)); ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); - httpServer.setThreadPool(threadPool); + + // HTTP Server + httpServer = new org.eclipse.jetty.server.Server(threadPool); // Connector configs - SelectChannelConnector connector = new SelectChannelConnector(); + ServerConnector connector = new ServerConnector(httpServer); boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); String schemeName = useSsl ? "https" : "http"; // Change connector if SSL is used @@ -92,14 +90,14 @@ public void run() { Arrays.toString(sslContextFactory.getExcludeProtocols())); sslContextFactory.setKeyStorePath(keyStorePath); sslContextFactory.setKeyStorePassword(keyStorePassword); - connector = new SslSelectChannelConnector(sslContextFactory); + connector = new ServerConnector(httpServer, sslContextFactory); } connector.setPort(portNum); // Linux:yes, Windows:no connector.setReuseAddress(!Shell.WINDOWS); int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, TimeUnit.MILLISECONDS); - connector.setMaxIdleTime(maxIdleTime); + connector.setIdleTimeout(maxIdleTime); httpServer.addConnector(connector); diff --git a/streaming/pom.xml b/streaming/pom.xml index 7d409c5d3b07..e7415863e356 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -77,6 +77,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + diff --git a/yarn/pom.xml b/yarn/pom.xml index db7f3e51d3be..11df2b3f4f76 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -102,6 +102,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + org.glassfish.jersey.core @@ -147,6 +152,15 @@ *:* + + + com.google.protobuf + kinesis.protobuf + + com.google.protobuf.** + + + *:* From 31ea3c7bde94f5bcca1db601f9c16c36c56cef73 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 12 May 2016 13:56:00 -0700 Subject: [PATCH 205/313] [SPARK-10605][SQL] Create native collect_list/collect_set aggregates ## What changes were proposed in this pull request? We currently use the Hive implementations for the collect_list/collect_set aggregate functions. This has a few major drawbacks: the use of HiveUDAF (which has quite a bit of overhead) and the lack of support for struct datatypes. This PR adds native implementation of these functions to Spark. The size of the collected list/set may vary, this means we cannot use the fast, Tungsten, aggregation path to perform the aggregation, and that we fallback to the slower sort based path. Another big issue with these operators is that when the size of the collected list/set grows too large, we can start experiencing large GC pauzes and OOMEs. This `collect*` aggregates implemented in this PR rely on the sort based aggregate path for correctness. They maintain their own internal buffer which holds the rows for one group at a time. The sortbased aggregation path is triggered by disabling `partialAggregation` for these aggregates (which is kinda funny); this technique is also employed in `org.apache.spark.sql.hiveHiveUDAFFunction`. I have done some performance testing: ```scala import org.apache.spark.sql.{Dataset, Row} sql("create function collect_list2 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectList'") val df = range(0, 10000000).select($"id", (rand(213123L) * 100000).cast("int").as("grp")) df.select(countDistinct($"grp")).show def benchmark(name: String, plan: Dataset[Row], maxItr: Int = 5): Unit = { // Do not measure planning. plan1.queryExecution.executedPlan // Execute the plan a number of times and average the result. val start = System.nanoTime var i = 0 while (i < maxItr) { plan.rdd.foreach(row => Unit) i += 1 } val time = (System.nanoTime - start) / (maxItr * 1000000L) println(s"[$name] $maxItr iterations completed in an average time of $time ms.") } val plan1 = df.groupBy($"grp").agg(collect_list($"id")) val plan2 = df.groupBy($"grp").agg(callUDF("collect_list2", $"id")) benchmark("Spark collect_list", plan1) ... > [Spark collect_list] 5 iterations completed in an average time of 3371 ms. benchmark("Hive collect_list", plan2) ... > [Hive collect_list] 5 iterations completed in an average time of 9109 ms. ``` Performance is improved by a factor 2-3. ## How was this patch tested? Added tests to `DataFrameAggregateSuite`. Author: Herman van Hovell Closes #12874 from hvanhovell/implode. (cherry picked from commit bb1362eb3b36b553dca246b95f59ba7fd8adcc8a) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../expressions/aggregate/collect.scala | 119 ++++++++++++++++++ .../org/apache/spark/sql/functions.scala | 12 +- .../spark/sql/DataFrameAggregateSuite.scala | 26 ++++ .../spark/sql/hive/HiveSessionCatalog.scala | 16 --- .../hive/HiveDataFrameAnalyticsSuite.scala | 11 -- 6 files changed, 149 insertions(+), 37 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index ac05dd3d0ef9..c459fe587859 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -252,6 +252,8 @@ object FunctionRegistry { expression[VarianceSamp]("variance"), expression[VariancePop]("var_pop"), expression[VarianceSamp]("var_samp"), + expression[CollectList]("collect_list"), + expression[CollectSet]("collect_set"), // string functions expression[Ascii]("ascii"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala new file mode 100644 index 000000000000..1f4ff9c4b184 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import scala.collection.generic.Growable +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types._ + +/** + * The Collect aggregate function collects all seen expression values into a list of values. + * + * The operator is bound to the slower sort based aggregation path because the number of + * elements (and their memory usage) can not be determined in advance. This also means that the + * collected elements are stored on heap, and that too many elements can cause GC pauses and + * eventually Out of Memory Errors. + */ +abstract class Collect extends ImperativeAggregate { + + val child: Expression + + override def children: Seq[Expression] = child :: Nil + + override def nullable: Boolean = true + + override def dataType: DataType = ArrayType(child.dataType) + + override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType) + + override def supportsPartial: Boolean = false + + override def aggBufferAttributes: Seq[AttributeReference] = Nil + + override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + + override def inputAggBufferAttributes: Seq[AttributeReference] = Nil + + protected[this] val buffer: Growable[Any] with Iterable[Any] + + override def initialize(b: MutableRow): Unit = { + buffer.clear() + } + + override def update(b: MutableRow, input: InternalRow): Unit = { + buffer += child.eval(input) + } + + override def merge(buffer: MutableRow, input: InternalRow): Unit = { + sys.error("Collect cannot be used in partial aggregations.") + } + + override def eval(input: InternalRow): Any = { + new GenericArrayData(buffer.toArray) + } +} + +/** + * Collect a list of elements. + */ +@ExpressionDescription( + usage = "_FUNC_(expr) - Collects and returns a list of non-unique elements.") +case class CollectList( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends Collect { + + def this(child: Expression) = this(child, 0, 0) + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def prettyName: String = "collect_list" + + override protected[this] val buffer: mutable.ArrayBuffer[Any] = mutable.ArrayBuffer.empty +} + +/** + * Collect a list of unique elements. + */ +@ExpressionDescription( + usage = "_FUNC_(expr) - Collects and returns a set of unique elements.") +case class CollectSet( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends Collect { + + def this(child: Expression) = this(child, 0, 0) + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def prettyName: String = "collect_set" + + override protected[this] val buffer: mutable.HashSet[Any] = mutable.HashSet.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 3e295c20b6d9..07f55042eeb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -195,18 +195,14 @@ object functions { /** * Aggregate function: returns a list of objects with duplicates. * - * For now this is an alias for the collect_list Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ - def collect_list(e: Column): Column = callUDF("collect_list", e) + def collect_list(e: Column): Column = withAggregateFunction { CollectList(e.expr) } /** * Aggregate function: returns a list of objects with duplicates. * - * For now this is an alias for the collect_list Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ @@ -215,18 +211,14 @@ object functions { /** * Aggregate function: returns a set of objects with duplicate elements eliminated. * - * For now this is an alias for the collect_set Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ - def collect_set(e: Column): Column = callUDF("collect_set", e) + def collect_set(e: Column): Column = withAggregateFunction { CollectSet(e.expr) } /** * Aggregate function: returns a set of objects with duplicate elements eliminated. * - * For now this is an alias for the collect_set Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 8a99866a33c7..69a990789bcf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -431,6 +431,32 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { Row(null, null, null, null, null)) } + test("collect functions") { + val df = Seq((1, 2), (2, 2), (3, 4)).toDF("a", "b") + checkAnswer( + df.select(collect_list($"a"), collect_list($"b")), + Seq(Row(Seq(1, 2, 3), Seq(2, 2, 4))) + ) + checkAnswer( + df.select(collect_set($"a"), collect_set($"b")), + Seq(Row(Seq(1, 2, 3), Seq(2, 4))) + ) + } + + test("collect functions structs") { + val df = Seq((1, 2, 2), (2, 2, 2), (3, 4, 1)) + .toDF("a", "x", "y") + .select($"a", struct($"x", $"y").as("b")) + checkAnswer( + df.select(collect_list($"a"), sort_array(collect_list($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(2, 2), Row(4, 1)))) + ) + checkAnswer( + df.select(collect_set($"a"), sort_array(collect_set($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(4, 1)))) + ) + } + test("SPARK-14664: Decimal sum/avg over window should work.") { checkAnswer( spark.sql("select sum(a) over () from values 1.0, 2.0, 3.0 T(a)"), diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 75a252ccba56..4f8aac8c2fcd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -222,20 +222,4 @@ private[sql] class HiveSessionCatalog( } } } - - // Pre-load a few commonly used Hive built-in functions. - HiveSessionCatalog.preloadedHiveBuiltinFunctions.foreach { - case (functionName, clazz) => - val builder = makeFunctionBuilder(functionName, clazz) - val info = new ExpressionInfo(clazz.getCanonicalName, functionName) - createTempFunction(functionName, info, builder, ignoreIfExists = false) - } -} - -private[sql] object HiveSessionCatalog { - // This is the list of Hive's built-in functions that are commonly used and we want to - // pre-load when we create the FunctionRegistry. - val preloadedHiveBuiltinFunctions = - ("collect_set", classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectSet]) :: - ("collect_list", classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectList]) :: Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala index 57f96e725a04..cc41c04c71e1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala @@ -58,17 +58,6 @@ class HiveDataFrameAnalyticsSuite extends QueryTest with TestHiveSingleton with ) } - test("collect functions") { - checkAnswer( - testData.select(collect_list($"a"), collect_list($"b")), - Seq(Row(Seq(1, 2, 3), Seq(2, 2, 4))) - ) - checkAnswer( - testData.select(collect_set($"a"), collect_set($"b")), - Seq(Row(Seq(1, 2, 3), Seq(2, 4))) - ) - } - test("cube") { checkAnswer( testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), From 0d24fe09aa5bafe85bf694dc4e2e2ebbfb3af250 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 12 May 2016 12:36:18 -0700 Subject: [PATCH 206/313] [SPARK-13902][SCHEDULER] Make DAGScheduler not to create duplicate stage. ## What changes were proposed in this pull request? `DAGScheduler`sometimes generate incorrect stage graph. Suppose you have the following DAG: ``` [A] <--(s_A)-- [B] <--(s_B)-- [C] <--(s_C)-- [D] \ / <------------- ``` Note: [] means an RDD, () means a shuffle dependency. Here, RDD `B` has a shuffle dependency on RDD `A`, and RDD `C` has shuffle dependency on both `B` and `A`. The shuffle dependency IDs are numbers in the `DAGScheduler`, but to make the example easier to understand, let's call the shuffled data from `A` shuffle dependency ID `s_A` and the shuffled data from `B` shuffle dependency ID `s_B`. The `getAncestorShuffleDependencies` method in `DAGScheduler` (incorrectly) does not check for duplicates when it's adding ShuffleDependencies to the parents data structure, so for this DAG, when `getAncestorShuffleDependencies` gets called on `C` (previous of the final RDD), `getAncestorShuffleDependencies` will return `s_A`, `s_B`, `s_A` (`s_A` gets added twice: once when the method "visit"s RDD `C`, and once when the method "visit"s RDD `B`). This is problematic because this line of code: https://github.com/apache/spark/blob/8ef3399/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L289 then generates a new shuffle stage for each dependency returned by `getAncestorShuffleDependencies`, resulting in duplicate map stages that compute the map output from RDD `A`. As a result, `DAGScheduler` generates the following stages and their parents for each shuffle: | | stage | parents | |----|----|----| | s_A | ShuffleMapStage 2 | List() | | s_B | ShuffleMapStage 1 | List(ShuffleMapStage 0) | | s_C | ShuffleMapStage 3 | List(ShuffleMapStage 1, ShuffleMapStage 2) | | - | ResultStage 4 | List(ShuffleMapStage 3) | The stage for s_A should be `ShuffleMapStage 0`, but the stage for `s_A` is generated twice as `ShuffleMapStage 2` and `ShuffleMapStage 0` is overwritten by `ShuffleMapStage 2`, and the stage `ShuffleMap Stage1` keeps referring the old stage `ShuffleMapStage 0`. This patch is fixing it. ## How was this patch tested? I added the sample RDD graph to show the illegal stage graph to `DAGSchedulerSuite`. Author: Takuya UESHIN Closes #12655 from ueshin/issues/SPARK-13902. --- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 47 +++++++++++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4dfd532e9362..5291b663667e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -286,7 +286,9 @@ class DAGScheduler( case None => // We are going to register ancestor shuffle dependencies getAncestorShuffleDependencies(shuffleDep.rdd).foreach { dep => - shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId) + if (!shuffleToMapStage.contains(dep.shuffleId)) { + shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId) + } } // Then register current shuffleDep val stage = newOrUsedShuffleStage(shuffleDep, firstJobId) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index e3ed079e4e16..088a47608621 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -325,6 +325,53 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) } + /** + * This test ensures that DAGScheduler build stage graph correctly. + * + * Suppose you have the following DAG: + * + * [A] <--(s_A)-- [B] <--(s_B)-- [C] <--(s_C)-- [D] + * \ / + * <------------- + * + * Here, RDD B has a shuffle dependency on RDD A, and RDD C has shuffle dependency on both + * B and A. The shuffle dependency IDs are numbers in the DAGScheduler, but to make the example + * easier to understand, let's call the shuffled data from A shuffle dependency ID s_A and the + * shuffled data from B shuffle dependency ID s_B. + * + * Note: [] means an RDD, () means a shuffle dependency. + */ + test("[SPARK-13902] Ensure no duplicate stages are created") { + val rddA = new MyRDD(sc, 1, Nil) + val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1)) + val s_A = shuffleDepA.shuffleId + + val rddB = new MyRDD(sc, 1, List(shuffleDepA), tracker = mapOutputTracker) + val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(1)) + val s_B = shuffleDepB.shuffleId + + val rddC = new MyRDD(sc, 1, List(shuffleDepA, shuffleDepB), tracker = mapOutputTracker) + val shuffleDepC = new ShuffleDependency(rddC, new HashPartitioner(1)) + val s_C = shuffleDepC.shuffleId + + val rddD = new MyRDD(sc, 1, List(shuffleDepC), tracker = mapOutputTracker) + + submit(rddD, Array(0)) + + assert(scheduler.shuffleToMapStage.size === 3) + assert(scheduler.activeJobs.size === 1) + + val mapStageA = scheduler.shuffleToMapStage(s_A) + val mapStageB = scheduler.shuffleToMapStage(s_B) + val mapStageC = scheduler.shuffleToMapStage(s_C) + val finalStage = scheduler.activeJobs.head.finalStage + + assert(mapStageA.parents.isEmpty) + assert(mapStageB.parents === List(mapStageA)) + assert(mapStageC.parents === List(mapStageA, mapStageB)) + assert(finalStage.parents === List(mapStageC)) + } + test("zero split job") { var numResults = 0 var failureReason: Option[Exception] = None From 54c04aa5d0a6012eb58efd0e7cf6d1d287818fa8 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 12 May 2016 17:50:55 -0700 Subject: [PATCH 207/313] [SPARK-15202][SPARKR] add dapplyCollect() method for DataFrame in SparkR. ## What changes were proposed in this pull request? dapplyCollect() applies an R function on each partition of a SparkDataFrame and collects the result back to R as a data.frame. ``` dapplyCollect(df, function(ldf) {...}) ``` ## How was this patch tested? SparkR unit tests. Author: Sun Rui Closes #12989 from sun-rui/SPARK-15202. (cherry picked from commit b3930f74a0929b2cdcbbe5cbe34f0b1d35eb01cc) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 86 ++++++++++++++++++----- R/pkg/R/generics.R | 4 ++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 21 +++++- 4 files changed, 95 insertions(+), 17 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 1432ab8a9d1c..239ad065d09a 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -47,6 +47,7 @@ exportMethods("arrange", "covar_pop", "crosstab", "dapply", + "dapplyCollect", "describe", "dim", "distinct", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 43c46b847446..0c2a194483b0 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1153,9 +1153,27 @@ setMethod("summarize", agg(x, ...) }) +dapplyInternal <- function(x, func, schema) { + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "dapply", + x@sdf, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + if (is.null(schema)) { schema } else { schema$jobj }) + dataFrame(sdf) +} + #' dapply #' -#' Apply a function to each partition of a DataFrame. +#' Apply a function to each partition of a SparkDataFrame. #' #' @param x A SparkDataFrame #' @param func A function to be applied to each partition of the SparkDataFrame. @@ -1197,21 +1215,57 @@ setMethod("summarize", setMethod("dapply", signature(x = "SparkDataFrame", func = "function", schema = "structType"), function(x, func, schema) { - packageNamesArr <- serialize(.sparkREnv[[".packages"]], - connection = NULL) - - broadcastArr <- lapply(ls(.broadcastNames), - function(name) { get(name, .broadcastNames) }) - - sdf <- callJStatic( - "org.apache.spark.sql.api.r.SQLUtils", - "dapply", - x@sdf, - serialize(cleanClosure(func), connection = NULL), - packageNamesArr, - broadcastArr, - schema$jobj) - dataFrame(sdf) + dapplyInternal(x, func, schema) + }) + +#' dapplyCollect +#' +#' Apply a function to each partition of a SparkDataFrame and collect the result back +#’ to R as a data.frame. +#' +#' @param x A SparkDataFrame +#' @param func A function to be applied to each partition of the SparkDataFrame. +#' func should have only one parameter, to which a data.frame corresponds +#' to each partition will be passed. +#' The output of func should be a data.frame. +#' @family SparkDataFrame functions +#' @rdname dapply +#' @name dapplyCollect +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame (sqlContext, iris) +#' ldf <- dapplyCollect(df, function(x) { x }) +#' +#' # filter and add a column +#' df <- createDataFrame ( +#' sqlContext, +#' list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), +#' c("a", "b", "c")) +#' ldf <- dapplyCollect( +#' df, +#' function(x) { +#' y <- x[x[1] > 1, ] +#' y <- cbind(y, y[1] + 1L) +#' }) +#' # the result +#' # a b c d +#' # 2 2 2 3 +#' # 3 3 3 4 +#' } +setMethod("dapplyCollect", + signature(x = "SparkDataFrame", func = "function"), + function(x, func) { + df <- dapplyInternal(x, func, NULL) + + content <- callJMethod(df@sdf, "collect") + # content is a list of items of struct type. Each item has a single field + # which is a serialized data.frame corresponds to one partition of the + # SparkDataFrame. + ldfs <- lapply(content, function(x) { unserialize(x[[1]]) }) + ldf <- do.call(rbind, ldfs) + row.names(ldf) <- NULL + ldf }) ############################## RDD Map Functions ################################## diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 8563be1e6498..ed76ad6b73c8 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -450,6 +450,10 @@ setGeneric("covar_pop", function(col1, col2) {standardGeneric("covar_pop") }) #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) +#' @rdname dapply +#' @export +setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") }) + #' @rdname summary #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 0f67bc2e331d..6a99b43e5aa5 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2043,7 +2043,7 @@ test_that("Histogram", { expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) }) -test_that("dapply() on a DataFrame", { +test_that("dapply() and dapplyCollect() on a DataFrame", { df <- createDataFrame ( sqlContext, list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), @@ -2053,6 +2053,8 @@ test_that("dapply() on a DataFrame", { result <- collect(df1) expect_identical(ldf, result) + result <- dapplyCollect(df, function(x) { x }) + expect_identical(ldf, result) # Filter and add a column schema <- structType(structField("a", "integer"), structField("b", "double"), @@ -2070,6 +2072,16 @@ test_that("dapply() on a DataFrame", { rownames(expected) <- NULL expect_identical(expected, result) + result <- dapplyCollect( + df, + function(x) { + y <- x[x$a > 1, ] + y <- cbind(y, y$a + 1L) + }) + expected1 <- expected + names(expected1) <- names(result) + expect_identical(expected1, result) + # Remove the added column df2 <- dapply( df1, @@ -2080,6 +2092,13 @@ test_that("dapply() on a DataFrame", { result <- collect(df2) expected <- expected[, c("a", "b", "c")] expect_identical(expected, result) + + result <- dapplyCollect( + df1, + function(x) { + x[, c("a", "b", "c")] + }) + expect_identical(expected, result) }) test_that("repartition by columns on DataFrame", { From d73ce364e301a54e60b336f0fd2d3836d5d0f25a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 12 May 2016 21:35:14 -0700 Subject: [PATCH 208/313] [SPARK-15306][SQL] Move object expressions into expressions.objects package ## What changes were proposed in this pull request? This patch moves all the object related expressions into expressions.objects package, for better code organization. ## How was this patch tested? N/A Author: Reynold Xin Closes #13085 from rxin/SPARK-15306. (cherry picked from commit ba169c3230e7d6cb192ec4bd567a1fef7b93b29f) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/sql/Encoders.scala | 3 ++- .../org/apache/spark/sql/catalyst/JavaTypeInference.scala | 1 + .../scala/org/apache/spark/sql/catalyst/ScalaReflection.scala | 1 + .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 1 + .../main/scala/org/apache/spark/sql/catalyst/dsl/package.scala | 1 + .../apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala | 1 + .../org/apache/spark/sql/catalyst/encoders/RowEncoder.scala | 1 + .../sql/catalyst/expressions/ReferenceToExpressions.scala | 1 + .../spark/sql/catalyst/expressions/{ => objects}/objects.scala | 3 ++- .../org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala | 3 ++- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 1 + .../sql/execution/aggregate/TypedAggregateExpression.scala | 1 + .../main/scala/org/apache/spark/sql/execution/objects.scala | 1 + 13 files changed, 16 insertions(+), 3 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{ => objects}/objects.scala (99%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index 3f4df704db75..fa96f8223d17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -24,7 +24,8 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} -import org.apache.spark.sql.catalyst.expressions.{BoundReference, DecodeUsingSerializer, EncodeUsingSerializer} +import org.apache.spark.sql.catalyst.expressions.objects.{DecodeUsingSerializer, EncodeUsingSerializer} +import org.apache.spark.sql.catalyst.expressions.BoundReference import org.apache.spark.sql.types._ /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 92caf8f4065c..690758205eff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -27,6 +27,7 @@ import com.google.common.reflect.TypeToken import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String 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 79bb7a701baf..cb9a62dfd4e8 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b7884f9b60f3..ce941e3df3e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, InMemoryCatalog, import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.objects.NewInstance import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.planning.IntegerIndex import org.apache.spark.sql.catalyst.plans._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index b5d10e4a584f..2ca990d19a2c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 5d294485afd7..f21a39a2d473 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, JavaTypeInference, ScalaRefle import org.apache.spark.sql.catalyst.analysis.{SimpleAnalyzer, UnresolvedAttribute, UnresolvedDeserializer, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, NewInstance} import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.types.{ObjectType, StructField, StructType} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 33ac1fdab4f6..ae842a9f8779 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala index c4cc6c39b047..502d791c6e85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.objects.LambdaVariable import org.apache.spark.sql.types.DataType /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index dbaff1625ed5..e8a6c742bfe1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions +package org.apache.spark.sql.catalyst.expressions.objects import java.lang.reflect.Modifier @@ -27,6 +27,7 @@ import org.apache.spark.SparkConf import org.apache.spark.serializer._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index c00e9c7e3956..1b25f5d7d0cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -23,7 +23,8 @@ import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.typeOf import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, NewInstance, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.objects.NewInstance import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 210ad956ff22..02dd6547a4ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.optimizer.CombineUnions import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index edca816cb1d1..8f94184764c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedDe import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index bafbbdf65724..3ff991392dfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -24,6 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.{DataType, ObjectType} From 51706f8a4dd94e235cf4e2c0627bc3788fec8251 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 12 May 2016 22:18:39 -0700 Subject: [PATCH 209/313] [SPARK-14541][SQL] Support IFNULL, NULLIF, NVL and NVL2 ## What changes were proposed in this pull request? This patch adds support for a few SQL functions to improve compatibility with other databases: IFNULL, NULLIF, NVL and NVL2. In order to do this, this patch introduced a RuntimeReplaceable expression trait that allows replacing an unevaluable expression in the optimizer before evaluation. Note that the semantics are not completely identical to other databases in esoteric cases. ## How was this patch tested? Added a new test suite SQLCompatibilityFunctionSuite. Closes #12373. Author: Reynold Xin Closes #13084 from rxin/SPARK-14541. (cherry picked from commit eda2800d44843b6478e22d2c99bca4af7e9c9613) Signed-off-by: Yin Huai --- .../catalyst/analysis/FunctionRegistry.scala | 5 +- .../catalyst/analysis/HiveTypeCoercion.scala | 2 + .../sql/catalyst/expressions/Expression.scala | 27 +++++++ .../expressions/nullExpressions.scala | 78 ++++++++++++++++++- .../sql/catalyst/optimizer/Optimizer.scala | 12 +++ .../spark/sql/DataFrameFunctionsSuite.scala | 6 -- .../sql/SQLCompatibilityFunctionSuite.scala | 72 +++++++++++++++++ .../sql/catalyst/ExpressionToSQLSuite.scala | 1 - 8 files changed, 194 insertions(+), 9 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c459fe587859..eca837ccf0a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -165,13 +165,16 @@ object FunctionRegistry { expression[Greatest]("greatest"), expression[If]("if"), expression[IsNaN]("isnan"), + expression[IfNull]("ifnull"), expression[IsNull]("isnull"), expression[IsNotNull]("isnotnull"), expression[Least]("least"), expression[CreateMap]("map"), expression[CreateNamedStruct]("named_struct"), expression[NaNvl]("nanvl"), - expression[Coalesce]("nvl"), + expression[NullIf]("nullif"), + expression[Nvl]("nvl"), + expression[Nvl2]("nvl2"), expression[Rand]("rand"), expression[Randn]("randn"), expression[CreateStruct]("struct"), 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 8319ec0a82f5..537dda60afbf 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 @@ -521,6 +521,8 @@ object HiveTypeCoercion { NaNvl(l, Cast(r, DoubleType)) case NaNvl(l, r) if l.dataType == FloatType && r.dataType == DoubleType => NaNvl(Cast(l, DoubleType), r) + + case e: RuntimeReplaceable => e.replaceForTypeCoercion() } } 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 c26faee2f487..fab163476f5d 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 @@ -221,6 +221,33 @@ trait Unevaluable extends Expression { } +/** + * An expression that gets replaced at runtime (currently by the optimizer) into a different + * expression for evaluation. This is mainly used to provide compatibility with other databases. + * For example, we use this to support "nvl" by replacing it with "coalesce". + */ +trait RuntimeReplaceable extends Unevaluable { + /** + * Method for concrete implementations to override that specifies how to construct the expression + * that should replace the current one. + */ + def replaceForEvaluation(): Expression + + /** + * Method for concrete implementations to override that specifies how to coerce the input types. + */ + def replaceForTypeCoercion(): Expression + + /** The expression that should be used during evaluation. */ + lazy val replaced: Expression = replaceForEvaluation() + + override def nullable: Boolean = replaced.nullable + override def foldable: Boolean = replaced.foldable + override def dataType: DataType = replaced.dataType + override def checkInputDataTypes(): TypeCheckResult = replaced.checkInputDataTypes() +} + + /** * Expressions that don't have SQL representation should extend this trait. Examples are * `ScalaUDF`, `ScalaUDAF`, and object expressions like `MapObjects` and `Invoke`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 421200e147b7..641c81b247a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{HiveTypeCoercion, TypeCheckResult} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -88,6 +88,82 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } +@ExpressionDescription(usage = "_FUNC_(a,b) - Returns b if a is null, or a otherwise.") +case class IfNull(left: Expression, right: Expression) extends RuntimeReplaceable { + override def children: Seq[Expression] = Seq(left, right) + + override def replaceForEvaluation(): Expression = Coalesce(Seq(left, right)) + + override def replaceForTypeCoercion(): Expression = { + if (left.dataType != right.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + copy(left = Cast(left, dtype), right = Cast(right, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + +@ExpressionDescription(usage = "_FUNC_(a,b) - Returns null if a equals to b, or a otherwise.") +case class NullIf(left: Expression, right: Expression) extends RuntimeReplaceable { + override def children: Seq[Expression] = Seq(left, right) + + override def replaceForEvaluation(): Expression = { + If(EqualTo(left, right), Literal.create(null, left.dataType), left) + } + + override def replaceForTypeCoercion(): Expression = { + if (left.dataType != right.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + copy(left = Cast(left, dtype), right = Cast(right, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + +@ExpressionDescription(usage = "_FUNC_(a,b) - Returns b if a is null, or a otherwise.") +case class Nvl(left: Expression, right: Expression) extends RuntimeReplaceable { + override def children: Seq[Expression] = Seq(left, right) + + override def replaceForEvaluation(): Expression = Coalesce(Seq(left, right)) + + override def replaceForTypeCoercion(): Expression = { + if (left.dataType != right.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + copy(left = Cast(left, dtype), right = Cast(right, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + +@ExpressionDescription(usage = "_FUNC_(a,b,c) - Returns b if a is not null, or c otherwise.") +case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression) + extends RuntimeReplaceable { + + override def replaceForEvaluation(): Expression = If(IsNotNull(expr1), expr2, expr3) + + override def children: Seq[Expression] = Seq(expr1, expr2, expr3) + + override def replaceForTypeCoercion(): Expression = { + if (expr2.dataType != expr3.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype => + copy(expr2 = Cast(expr2, dtype), expr3 = Cast(expr3, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + /** * Evaluates to `true` iff it's NaN. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 928ba213b593..af7532e0c03e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -49,6 +49,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, EliminateSubqueryAliases, + ReplaceExpressions, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), DistinctAggregationRewriter) :: @@ -1511,6 +1512,17 @@ object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { } } +/** + * Finds all [[RuntimeReplaceable]] expressions and replace them with the expressions that can + * be evaluated. This is mainly used to provide compatibility with other databases. + * For example, we use this to support "nvl" by replacing it with "coalesce". + */ +object ReplaceExpressions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case e: RuntimeReplaceable => e.replaced + } +} + /** * Computes the current date and time to make sure we return the same result in a single query. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 746e25a0c3ec..73d77651a027 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -152,12 +152,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Row("one", "not_one")) } - test("nvl function") { - checkAnswer( - sql("SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null)"), - Row("x", "y", null)) - } - test("misc md5 function") { val df = Seq(("ABC", Array[Byte](1, 2, 3, 4, 5, 6))).toDF("a", "b") checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala new file mode 100644 index 000000000000..1e3239550fb8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext + +/** + * A test suite for functions added for compatibility with other databases such as Oracle, MSSQL. + * These functions are typically implemented using the trait + * [[org.apache.spark.sql.catalyst.expressions.RuntimeReplaceable]]. + */ +class SQLCompatibilityFunctionSuite extends QueryTest with SharedSQLContext { + + test("ifnull") { + checkAnswer( + sql("SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null)"), + Row("x", "y", null)) + + // Type coercion + checkAnswer( + sql("SELECT ifnull(1, 2.1d), ifnull(null, 2.1d)"), + Row(1.0, 2.1)) + } + + test("nullif") { + checkAnswer( + sql("SELECT nullif('x', 'x'), nullif('x', 'y')"), + Row(null, "x")) + + // Type coercion + checkAnswer( + sql("SELECT nullif(1, 2.1d), nullif(1, 1.0d)"), + Row(1.0, null)) + } + + test("nvl") { + checkAnswer( + sql("SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null)"), + Row("x", "y", null)) + + // Type coercion + checkAnswer( + sql("SELECT nvl(1, 2.1d), nvl(null, 2.1d)"), + Row(1.0, 2.1)) + } + + test("nvl2") { + checkAnswer( + sql("SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null)"), + Row("y", "x", null)) + + // Type coercion + checkAnswer( + sql("SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d)"), + Row(2.1, 1.0)) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala index 72736ee55b99..b4eb50e331cf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala @@ -102,7 +102,6 @@ class ExpressionToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSqlGeneration("SELECT map(1, 'a', 2, 'b')") checkSqlGeneration("SELECT named_struct('c1',1,'c2',2,'c3',3)") checkSqlGeneration("SELECT nanvl(a, 5), nanvl(b, 10), nanvl(d, c) from t2") - checkSqlGeneration("SELECT nvl(null, 1, 2)") checkSqlGeneration("SELECT rand(1)") checkSqlGeneration("SELECT randn(3)") checkSqlGeneration("SELECT struct(1,2,3)") From 7b925e500a2463b42ddb3ad539343ed633eb76cb Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 12 May 2016 22:31:14 -0700 Subject: [PATCH 210/313] [SPARK-13866] [SQL] Handle decimal type in CSV inference at CSV data source. ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-13866 This PR adds the support to infer `DecimalType`. Here are the rules between `IntegerType`, `LongType` and `DecimalType`. #### Infering Types 1. `IntegerType` and then `LongType`are tried first. ```scala Int.MaxValue => IntegerType Long.MaxValue => LongType ``` 2. If it fails, try `DecimalType`. ```scala (Long.MaxValue + 1) => DecimalType(20, 0) ``` This does not try to infer this as `DecimalType` when scale is less than 0. 3. if it fails, try `DoubleType` ```scala 0.1 => DoubleType // This is failed to be inferred as `DecimalType` because it has the scale, 1. ``` #### Compatible Types (Merging Types) For merging types, this is the same with JSON data source. If `DecimalType` is not capable, then it becomes `DoubleType` ## How was this patch tested? Unit tests were used and `./dev/run_tests` for code style test. Author: hyukjinkwon Author: Hyukjin Kwon Closes #11724 from HyukjinKwon/SPARK-13866. (cherry picked from commit 51841d77d99a858f8fa1256e923b0364b9b28fa0) Signed-off-by: Davies Liu --- .../datasources/csv/CSVInferSchema.scala | 50 ++++++++++++++++++- sql/core/src/test/resources/decimal.csv | 7 +++ .../datasources/csv/CSVInferSchemaSuite.scala | 13 ++++- .../execution/datasources/csv/CSVSuite.scala | 15 ++++++ 4 files changed, 81 insertions(+), 4 deletions(-) create mode 100644 sql/core/src/test/resources/decimal.csv diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index cfd66af18892..05c8d8ee15f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.csv import java.math.BigDecimal -import java.text.{NumberFormat, SimpleDateFormat} +import java.text.NumberFormat import java.util.Locale import scala.util.control.Exception._ @@ -85,6 +85,7 @@ private[csv] object CSVInferSchema { case NullType => tryParseInteger(field, options) case IntegerType => tryParseInteger(field, options) case LongType => tryParseLong(field, options) + case _: DecimalType => tryParseDecimal(field, options) case DoubleType => tryParseDouble(field, options) case TimestampType => tryParseTimestamp(field, options) case BooleanType => tryParseBoolean(field, options) @@ -107,10 +108,28 @@ private[csv] object CSVInferSchema { if ((allCatch opt field.toLong).isDefined) { LongType } else { - tryParseDouble(field, options) + tryParseDecimal(field, options) } } + private def tryParseDecimal(field: String, options: CSVOptions): DataType = { + val decimalTry = allCatch opt { + // `BigDecimal` conversion can fail when the `field` is not a form of number. + val bigDecimal = new BigDecimal(field) + // Because many other formats do not support decimal, it reduces the cases for + // decimals by disallowing values having scale (eg. `1.1`). + if (bigDecimal.scale <= 0) { + // `DecimalType` conversion can fail when + // 1. The precision is bigger than 38. + // 2. scale is bigger than precision. + DecimalType(bigDecimal.precision, bigDecimal.scale) + } else { + tryParseDouble(field, options) + } + } + decimalTry.getOrElse(tryParseDouble(field, options)) + } + private def tryParseDouble(field: String, options: CSVOptions): DataType = { if ((allCatch opt field.toDouble).isDefined) { DoubleType @@ -170,6 +189,33 @@ private[csv] object CSVInferSchema { val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) Some(numericPrecedence(index)) + // These two cases below deal with when `DecimalType` is larger than `IntegralType`. + case (t1: IntegralType, t2: DecimalType) if t2.isWiderThan(t1) => + Some(t2) + case (t1: DecimalType, t2: IntegralType) if t1.isWiderThan(t2) => + Some(t1) + + // These two cases below deal with when `IntegralType` is larger than `DecimalType`. + case (t1: IntegralType, t2: DecimalType) => + findTightestCommonType(DecimalType.forType(t1), t2) + case (t1: DecimalType, t2: IntegralType) => + findTightestCommonType(t1, DecimalType.forType(t2)) + + // Double support larger range than fixed decimal, DecimalType.Maximum should be enough + // in most case, also have better precision. + case (DoubleType, _: DecimalType) | (_: DecimalType, DoubleType) => + Some(DoubleType) + + case (t1: DecimalType, t2: DecimalType) => + val scale = math.max(t1.scale, t2.scale) + val range = math.max(t1.precision - t1.scale, t2.precision - t2.scale) + if (range + scale > 38) { + // DecimalType can't support precision > 38 + Some(DoubleType) + } else { + Some(DecimalType(range + scale, scale)) + } + case _ => None } } diff --git a/sql/core/src/test/resources/decimal.csv b/sql/core/src/test/resources/decimal.csv new file mode 100644 index 000000000000..870f6aaf1bb4 --- /dev/null +++ b/sql/core/src/test/resources/decimal.csv @@ -0,0 +1,7 @@ +~ decimal field has integer, integer and decimal values. The last value cannot fit to a long +~ long field has integer, long and integer values. +~ double field has double, double and decimal values. +decimal,long,double +1,1,0.1 +1,9223372036854775807,1.0 +92233720368547758070,1,92233720368547758070 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala index daf85be56f3d..dbe3af49c90c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.csv -import java.text.SimpleDateFormat - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.types._ @@ -35,6 +33,11 @@ class CSVInferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(NullType, "2015-08-20 15:57:00", options) == TimestampType) assert(CSVInferSchema.inferField(NullType, "True", options) == BooleanType) assert(CSVInferSchema.inferField(NullType, "FAlSE", options) == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(CSVInferSchema.inferField(NullType, textValueOne, options) == expectedTypeOne) } test("String fields types are inferred correctly from other types") { @@ -49,6 +52,11 @@ class CSVInferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(LongType, "True", options) == BooleanType) assert(CSVInferSchema.inferField(IntegerType, "FALSE", options) == BooleanType) assert(CSVInferSchema.inferField(TimestampType, "FALSE", options) == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(CSVInferSchema.inferField(IntegerType, textValueOne, options) == expectedTypeOne) } test("Timestamp field types are inferred correctly via custom data format") { @@ -94,6 +102,7 @@ class CSVInferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(DoubleType, "\\N", options) == DoubleType) assert(CSVInferSchema.inferField(TimestampType, "\\N", options) == TimestampType) assert(CSVInferSchema.inferField(BooleanType, "\\N", options) == BooleanType) + assert(CSVInferSchema.inferField(DecimalType(1, 1), "\\N", options) == DecimalType(1, 1)) } test("Merging Nulltypes should yield Nulltype.") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index ae91e0f606ec..27d6dc9197d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -43,6 +43,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { private val commentsFile = "comments.csv" private val disableCommentsFile = "disable_comments.csv" private val boolFile = "bool.csv" + private val decimalFile = "decimal.csv" private val simpleSparseFile = "simple_sparse.csv" private val numbersFile = "numbers.csv" private val datesFile = "dates.csv" @@ -133,6 +134,20 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(result.schema === expectedSchema) } + test("test inferring decimals") { + val result = sqlContext.read + .format("csv") + .option("comment", "~") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(decimalFile)) + val expectedSchema = StructType(List( + StructField("decimal", DecimalType(20, 0), nullable = true), + StructField("long", LongType, nullable = true), + StructField("double", DoubleType, nullable = true))) + assert(result.schema === expectedSchema) + } + test("test with alternative delimiter and quote") { val cars = spark.read .format("csv") From b6b2c613847779daf2eec8122efdb5f2188fba76 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 13 May 2016 08:39:59 +0200 Subject: [PATCH 211/313] [SPARK-15188] Add missing thresholds param to NaiveBayes in PySpark ## What changes were proposed in this pull request? Add missing thresholds param to NiaveBayes ## How was this patch tested? doctests Author: Holden Karau Closes #12963 from holdenk/SPARK-15188-add-missing-naive-bayes-param. (cherry picked from commit d1aadea05ab1c7350e46479cc68d08e11916a751) Signed-off-by: Nick Pentreath --- python/pyspark/ml/classification.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index c26c2d7fa5bf..5c11aa71b459 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -872,7 +872,7 @@ def featureImportances(self): @inherit_doc class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, - HasRawPredictionCol, JavaMLWritable, JavaMLReadable): + HasRawPredictionCol, HasThresholds, JavaMLWritable, JavaMLReadable): """ Naive Bayes Classifiers. It supports both Multinomial and Bernoulli NB. `Multinomial NB @@ -918,6 +918,11 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H True >>> model.theta == model2.theta True + >>> nb = nb.setThresholds([0.01, 10.00]) + >>> model3 = nb.fit(df) + >>> result = model3.transform(test0).head() + >>> result.prediction + 0.0 .. versionadded:: 1.5.0 """ @@ -931,11 +936,11 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, - modelType="multinomial"): + modelType="multinomial", thresholds=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, \ - modelType="multinomial") + modelType="multinomial", thresholds=None) """ super(NaiveBayes, self).__init__() self._java_obj = self._new_java_obj( @@ -948,11 +953,11 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred @since("1.5.0") def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, - modelType="multinomial"): + modelType="multinomial", thresholds=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, \ - modelType="multinomial") + modelType="multinomial", thresholds=None) Sets params for Naive Bayes. """ kwargs = self.setParams._input_kwargs From 0076bf085b0fbda8a7811e2c961451dfddcd73f8 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 13 May 2016 08:52:06 +0200 Subject: [PATCH 212/313] [MINOR][PYSPARK] update _shared_params_code_gen.py ## What changes were proposed in this pull request? 1, add arg-checkings for `tol` and `stepSize` to keep in line with `SharedParamsCodeGen.scala` 2, fix one typo ## How was this patch tested? local build Author: Zheng RuiFeng Closes #12996 from zhengruifeng/py_args_checking. (cherry picked from commit 87d69a01f027aa18718827f94f921b4a1eaa78a5) Signed-off-by: Nick Pentreath --- python/pyspark/ml/clustering.py | 2 +- python/pyspark/ml/param/_shared_params_code_gen.py | 6 +++--- python/pyspark/ml/param/shared.py | 12 ++++++------ 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index b90daf16e2f7..921633164bac 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -517,7 +517,7 @@ def describeTopics(self, maxTermsPerTopic=10): def estimatedDocConcentration(self): """ Value for :py:attr:`LDA.docConcentration` estimated from data. - If Online LDA was used and :py:attr::`LDA.optimizeDocConcentration` was set to false, + If Online LDA was used and :py:attr:`LDA.optimizeDocConcentration` was set to false, then this returns the fixed (given) value for the :py:attr:`LDA.docConcentration` parameter. """ return self._call_java("estimatedDocConcentration") diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 12f14534f4b8..c32dcc467d49 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -124,12 +124,12 @@ def get$Name(self): "E.g. 10 means that the cache will get checkpointed every 10 iterations.", None, "TypeConverters.toInt"), ("seed", "random seed.", "hash(type(self).__name__)", "TypeConverters.toInt"), - ("tol", "the convergence tolerance for iterative algorithms.", None, + ("tol", "the convergence tolerance for iterative algorithms (>= 0).", None, "TypeConverters.toFloat"), - ("stepSize", "Step size to be used for each iteration of optimization.", None, + ("stepSize", "Step size to be used for each iteration of optimization (>= 0).", None, "TypeConverters.toFloat"), ("handleInvalid", "how to handle invalid entries. Options are skip (which will filter " + - "out rows with bad values), or error (which will throw an errror). More options may be " + + "out rows with bad values), or error (which will throw an error). More options may be " + "added later.", None, "TypeConverters.toString"), ("elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, " + "the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", "0.0", diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 9ed63e48ae40..c5ccf81540d5 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -328,10 +328,10 @@ def getSeed(self): class HasTol(Params): """ - Mixin for param tol: the convergence tolerance for iterative algorithms. + Mixin for param tol: the convergence tolerance for iterative algorithms (>= 0). """ - tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms.", typeConverter=TypeConverters.toFloat) + tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms (>= 0).", typeConverter=TypeConverters.toFloat) def __init__(self): super(HasTol, self).__init__() @@ -351,10 +351,10 @@ def getTol(self): class HasStepSize(Params): """ - Mixin for param stepSize: Step size to be used for each iteration of optimization. + Mixin for param stepSize: Step size to be used for each iteration of optimization (>= 0). """ - stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization.", typeConverter=TypeConverters.toFloat) + stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization (>= 0).", typeConverter=TypeConverters.toFloat) def __init__(self): super(HasStepSize, self).__init__() @@ -374,10 +374,10 @@ def getStepSize(self): class HasHandleInvalid(Params): """ - Mixin for param handleInvalid: how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later. + Mixin for param handleInvalid: how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an error). More options may be added later. """ - handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.", typeConverter=TypeConverters.toString) + handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an error). More options may be added later.", typeConverter=TypeConverters.toString) def __init__(self): super(HasHandleInvalid, self).__init__() From 7affde25ba75f83aef33a3355900220b57243ae6 Mon Sep 17 00:00:00 2001 From: sethah Date: Fri, 13 May 2016 09:01:20 +0200 Subject: [PATCH 213/313] [SPARK-15181][ML][PYSPARK] Python API for GLR summaries. ## What changes were proposed in this pull request? This patch adds a python API for generalized linear regression summaries (training and test). This helps provide feature parity for Python GLMs. ## How was this patch tested? Added a unit test to `pyspark.ml.tests` Author: sethah Closes #12961 from sethah/GLR_summary. (cherry picked from commit 5b849766ab080c91864ed06ebbfd82ad978d5e4c) Signed-off-by: Nick Pentreath --- .../GeneralizedLinearRegression.scala | 18 +- python/pyspark/ml/regression.py | 201 +++++++++++++++++- python/pyspark/ml/tests.py | 39 +++- 3 files changed, 247 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index c294ef31f90d..05fffa0d97e6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -848,7 +848,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( import GeneralizedLinearRegression._ /** - * Field in "predictions" which gives the prediction value of each instance. + * Field in "predictions" which gives the predicted value of each instance. * This is set to a new column name if the original model's `predictionCol` is not set. */ @Since("2.0.0") @@ -870,7 +870,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( protected val model: GeneralizedLinearRegressionModel = origModel.copy(ParamMap.empty).setPredictionCol(predictionCol) - /** predictions output by the model's `transform` method */ + /** Predictions output by the model's `transform` method. */ @Since("2.0.0") @transient val predictions: DataFrame = model.transform(dataset) private[regression] lazy val family: Family = Family.fromName(model.getFamily) @@ -880,10 +880,10 @@ class GeneralizedLinearRegressionSummary private[regression] ( family.defaultLink } - /** Number of instances in DataFrame predictions */ + /** Number of instances in DataFrame predictions. */ private[regression] lazy val numInstances: Long = predictions.count() - /** The numeric rank of the fitted linear model */ + /** The numeric rank of the fitted linear model. */ @Since("2.0.0") lazy val rank: Long = if (model.getFitIntercept) { model.coefficients.size + 1 @@ -891,17 +891,17 @@ class GeneralizedLinearRegressionSummary private[regression] ( model.coefficients.size } - /** Degrees of freedom */ + /** Degrees of freedom. */ @Since("2.0.0") lazy val degreesOfFreedom: Long = { numInstances - rank } - /** The residual degrees of freedom */ + /** The residual degrees of freedom. */ @Since("2.0.0") lazy val residualDegreeOfFreedom: Long = degreesOfFreedom - /** The residual degrees of freedom for the null model */ + /** The residual degrees of freedom for the null model. */ @Since("2.0.0") lazy val residualDegreeOfFreedomNull: Long = if (model.getFitIntercept) { numInstances - 1 @@ -944,7 +944,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( } /** - * Get the default residuals(deviance residuals) of the fitted model. + * Get the default residuals (deviance residuals) of the fitted model. */ @Since("2.0.0") def residuals(): DataFrame = devianceResiduals @@ -1000,7 +1000,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** * The dispersion of the fitted model. * It is taken as 1.0 for the "binomial" and "poisson" families, and otherwise - * estimated by the residual Pearson's Chi-Squared statistic(which is defined as + * estimated by the residual Pearson's Chi-Squared statistic (which is defined as * sum of the squares of the Pearson residuals) divided by the residual degrees of freedom. */ @Since("2.0.0") diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 0d0eb8ae4618..fcdc29e69b1d 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -29,6 +29,7 @@ 'DecisionTreeRegressor', 'DecisionTreeRegressionModel', 'GBTRegressor', 'GBTRegressionModel', 'GeneralizedLinearRegression', 'GeneralizedLinearRegressionModel', + 'GeneralizedLinearRegressionSummary', 'GeneralizedLinearRegressionTrainingSummary', 'IsotonicRegression', 'IsotonicRegressionModel', 'LinearRegression', 'LinearRegressionModel', 'LinearRegressionSummary', 'LinearRegressionTrainingSummary', @@ -1283,7 +1284,7 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha family = Param(Params._dummy(), "family", "The name of family which is a description of " + "the error distribution to be used in the model. Supported options: " + - "gaussian(default), binomial, poisson and gamma.", + "gaussian (default), binomial, poisson and gamma.", typeConverter=TypeConverters.toString) link = Param(Params._dummy(), "link", "The name of link function which provides the " + "relationship between the linear predictor and the mean of the distribution " + @@ -1377,6 +1378,204 @@ def intercept(self): """ return self._call_java("intercept") + @property + @since("2.0.0") + def summary(self): + """ + Gets summary (e.g. residuals, deviance, pValues) of model on + training set. An exception is thrown if + `trainingSummary is None`. + """ + java_glrt_summary = self._call_java("summary") + return GeneralizedLinearRegressionTrainingSummary(java_glrt_summary) + + @property + @since("2.0.0") + def hasSummary(self): + """ + Indicates whether a training summary exists for this model + instance. + """ + return self._call_java("hasSummary") + + @since("2.0.0") + def evaluate(self, dataset): + """ + Evaluates the model on a test dataset. + + :param dataset: + Test dataset to evaluate model on, where dataset is an + instance of :py:class:`pyspark.sql.DataFrame` + """ + if not isinstance(dataset, DataFrame): + raise ValueError("dataset must be a DataFrame but got %s." % type(dataset)) + java_glr_summary = self._call_java("evaluate", dataset) + return GeneralizedLinearRegressionSummary(java_glr_summary) + + +class GeneralizedLinearRegressionSummary(JavaWrapper): + """ + .. note:: Experimental + + Generalized linear regression results evaluated on a dataset. + + .. versionadded:: 2.0.0 + """ + + @property + @since("2.0.0") + def predictions(self): + """ + Predictions output by the model's `transform` method. + """ + return self._call_java("predictions") + + @property + @since("2.0.0") + def predictionCol(self): + """ + Field in :py:attr:`predictions` which gives the predicted value of each instance. + This is set to a new column name if the original model's `predictionCol` is not set. + """ + return self._call_java("predictionCol") + + @property + @since("2.0.0") + def rank(self): + """ + The numeric rank of the fitted linear model. + """ + return self._call_java("rank") + + @property + @since("2.0.0") + def degreesOfFreedom(self): + """ + Degrees of freedom. + """ + return self._call_java("degreesOfFreedom") + + @property + @since("2.0.0") + def residualDegreeOfFreedom(self): + """ + The residual degrees of freedom. + """ + return self._call_java("residualDegreeOfFreedom") + + @property + @since("2.0.0") + def residualDegreeOfFreedomNull(self): + """ + The residual degrees of freedom for the null model. + """ + return self._call_java("residualDegreeOfFreedomNull") + + @since("2.0.0") + def residuals(self, residualsType="deviance"): + """ + Get the residuals of the fitted model by type. + + :param residualsType: The type of residuals which should be returned. + Supported options: deviance (default), pearson, working, and response. + """ + return self._call_java("residuals", residualsType) + + @property + @since("2.0.0") + def nullDeviance(self): + """ + The deviance for the null model. + """ + return self._call_java("nullDeviance") + + @property + @since("2.0.0") + def deviance(self): + """ + The deviance for the fitted model. + """ + return self._call_java("deviance") + + @property + @since("2.0.0") + def dispersion(self): + """ + The dispersion of the fitted model. + It is taken as 1.0 for the "binomial" and "poisson" families, and otherwise + estimated by the residual Pearson's Chi-Squared statistic (which is defined as + sum of the squares of the Pearson residuals) divided by the residual degrees of freedom. + """ + return self._call_java("dispersion") + + @property + @since("2.0.0") + def aic(self): + """ + Akaike's "An Information Criterion"(AIC) for the fitted model. + """ + return self._call_java("aic") + + +@inherit_doc +class GeneralizedLinearRegressionTrainingSummary(GeneralizedLinearRegressionSummary): + """ + .. note:: Experimental + + Generalized linear regression training results. + + .. versionadded:: 2.0.0 + """ + + @property + @since("2.0.0") + def numIterations(self): + """ + Number of training iterations. + """ + return self._call_java("numIterations") + + @property + @since("2.0.0") + def solver(self): + """ + The numeric solver used for training. + """ + return self._call_java("solver") + + @property + @since("2.0.0") + def coefficientStandardErrors(self): + """ + Standard error of estimated coefficients and intercept. + + If :py:attr:`GeneralizedLinearRegression.fitIntercept` is set to True, + then the last element returned corresponds to the intercept. + """ + return self._call_java("coefficientStandardErrors") + + @property + @since("2.0.0") + def tValues(self): + """ + T-statistic of estimated coefficients and intercept. + + If :py:attr:`GeneralizedLinearRegression.fitIntercept` is set to True, + then the last element returned corresponds to the intercept. + """ + return self._call_java("tValues") + + @property + @since("2.0.0") + def pValues(self): + """ + Two-sided p-value of estimated coefficients and intercept. + + If :py:attr:`GeneralizedLinearRegression.fitIntercept` is set to True, + then the last element returned corresponds to the intercept. + """ + return self._call_java("pValues") + if __name__ == "__main__": import doctest diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 49d3a4a332fd..8e56b0d6fff0 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -52,7 +52,8 @@ from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed from pyspark.ml.recommendation import ALS -from pyspark.ml.regression import LinearRegression, DecisionTreeRegressor +from pyspark.ml.regression import LinearRegression, DecisionTreeRegressor, \ + GeneralizedLinearRegression from pyspark.ml.tuning import * from pyspark.ml.wrapper import JavaParams from pyspark.mllib.common import _java2py @@ -909,6 +910,42 @@ def test_linear_regression_summary(self): sameSummary = model.evaluate(df) self.assertAlmostEqual(sameSummary.explainedVariance, s.explainedVariance) + def test_glr_summary(self): + from pyspark.mllib.linalg import Vectors + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], []))], + ["label", "weight", "features"]) + glr = GeneralizedLinearRegression(family="gaussian", link="identity", weightCol="weight", + fitIntercept=False) + model = glr.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertEqual(s.numIterations, 1) # this should default to a single iteration of WLS + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.predictionCol, "prediction") + self.assertTrue(isinstance(s.residuals(), DataFrame)) + self.assertTrue(isinstance(s.residuals("pearson"), DataFrame)) + coefStdErr = s.coefficientStandardErrors + self.assertTrue(isinstance(coefStdErr, list) and isinstance(coefStdErr[0], float)) + tValues = s.tValues + self.assertTrue(isinstance(tValues, list) and isinstance(tValues[0], float)) + pValues = s.pValues + self.assertTrue(isinstance(pValues, list) and isinstance(pValues[0], float)) + self.assertEqual(s.degreesOfFreedom, 1) + self.assertEqual(s.residualDegreeOfFreedom, 1) + self.assertEqual(s.residualDegreeOfFreedomNull, 2) + self.assertEqual(s.rank, 1) + self.assertTrue(isinstance(s.solver, basestring)) + self.assertTrue(isinstance(s.aic, float)) + self.assertTrue(isinstance(s.deviance, float)) + self.assertTrue(isinstance(s.nullDeviance, float)) + self.assertTrue(isinstance(s.dispersion, float)) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertAlmostEqual(sameSummary.deviance, s.deviance) + def test_logistic_regression_summary(self): from pyspark.mllib.linalg import Vectors df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), From 86b8f8a9acf88998b10ad86e6438c4ebff17f948 Mon Sep 17 00:00:00 2001 From: BenFradet Date: Fri, 13 May 2016 09:08:04 +0200 Subject: [PATCH 214/313] [SPARK-13961][ML] spark.ml ChiSqSelector and RFormula should support other numeric types for label ## What changes were proposed in this pull request? Made ChiSqSelector and RFormula accept all numeric types for label ## How was this patch tested? Unit tests Author: BenFradet Closes #12467 from BenFradet/SPARK-13961. (cherry picked from commit 31f1aebbeb77b4eb1080f22c9bece7fafd8022f8) Signed-off-by: Nick Pentreath --- .../spark/ml/feature/ChiSqSelector.scala | 4 +-- .../apache/spark/ml/feature/RFormula.scala | 4 +-- .../DecisionTreeClassifierSuite.scala | 2 +- .../classification/GBTClassifierSuite.scala | 2 +- .../LogisticRegressionSuite.scala | 2 +- .../MultilayerPerceptronClassifierSuite.scala | 2 +- .../ml/classification/NaiveBayesSuite.scala | 2 +- .../ml/classification/OneVsRestSuite.scala | 2 +- .../RandomForestClassifierSuite.scala | 2 +- .../spark/ml/feature/ChiSqSelectorSuite.scala | 10 ++++++- .../spark/ml/feature/RFormulaSuite.scala | 30 +++++++++++++++---- .../AFTSurvivalRegressionSuite.scala | 2 +- .../DecisionTreeRegressorSuite.scala | 2 +- .../ml/regression/GBTRegressorSuite.scala | 2 +- .../GeneralizedLinearRegressionSuite.scala | 2 +- .../regression/IsotonicRegressionSuite.scala | 2 +- .../ml/regression/LinearRegressionSuite.scala | 2 +- .../RandomForestRegressorSuite.scala | 2 +- .../apache/spark/ml/util/MLTestingUtils.scala | 4 +-- 19 files changed, 53 insertions(+), 27 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index cfecae7e0b15..29f55a7f715c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -80,7 +80,7 @@ final class ChiSqSelector(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): ChiSqSelectorModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(labelCol), $(featuresCol)).rdd.map { + val input = dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } @@ -90,7 +90,7 @@ final class ChiSqSelector(override val uid: String) override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(labelCol)) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 5219680be2dc..a2f3d44132d1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -256,8 +256,8 @@ class RFormulaModel private[feature]( val columnNames = schema.map(_.name) require(!columnNames.contains($(featuresCol)), "Features column already exists.") require( - !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType == DoubleType, - "Label column already exists and is not of type DoubleType.") + !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType.isInstanceOf[NumericType], + "Label column already exists and is not of type NumericType.") } @Since("2.0.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index f94d336df544..91a947f44bc3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -337,7 +337,7 @@ class DecisionTreeClassifierSuite test("should support all NumericType labels and not support other types") { val dt = new DecisionTreeClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[DecisionTreeClassificationModel, DecisionTreeClassifier]( - dt, isClassification = true, spark) { (expected, actual) => + dt, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index c9453aaec255..5a5e5c15fc59 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -106,7 +106,7 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext test("should support all NumericType labels and not support other types") { val gbt = new GBTClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[GBTClassificationModel, GBTClassifier]( - gbt, isClassification = true, spark) { (expected, actual) => + gbt, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index cb4d087ce5bc..f127aa217c94 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -938,7 +938,7 @@ class LogisticRegressionSuite test("should support all NumericType labels and not support other types") { val lr = new LogisticRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[LogisticRegressionModel, LogisticRegression]( - lr, isClassification = true, spark) { (expected, actual) => + lr, spark) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients.toArray === actual.coefficients.toArray) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index 876e047db54c..d5282e07d65c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -169,7 +169,7 @@ class MultilayerPerceptronClassifierSuite val mpc = new MultilayerPerceptronClassifier().setLayers(layers).setMaxIter(1) MLTestingUtils.checkNumericTypes[ MultilayerPerceptronClassificationModel, MultilayerPerceptronClassifier]( - mpc, isClassification = true, spark) { (expected, actual) => + mpc, spark) { (expected, actual) => assert(expected.layers === actual.layers) assert(expected.weights === actual.weights) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 15d0358c3fc0..2a05c446e516 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -188,7 +188,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa test("should support all NumericType labels and not support other types") { val nb = new NaiveBayes() MLTestingUtils.checkNumericTypes[NaiveBayesModel, NaiveBayes]( - nb, isClassification = true, spark) { (expected, actual) => + nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 005d609307fb..5044d40998d6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -228,7 +228,7 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with Defau test("should support all NumericType labels and not support other types") { val ovr = new OneVsRest().setClassifier(new LogisticRegression().setMaxIter(1)) MLTestingUtils.checkNumericTypes[OneVsRestModel, OneVsRest]( - ovr, isClassification = true, spark) { (expected, actual) => + ovr, spark) { (expected, actual) => val expectedModels = expected.models.map(m => m.asInstanceOf[LogisticRegressionModel]) val actualModels = actual.models.map(m => m.asInstanceOf[LogisticRegressionModel]) assert(expectedModels.length === actualModels.length) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index 97f3feacca07..8002a2f4f29e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -189,7 +189,7 @@ class RandomForestClassifierSuite test("should support all NumericType labels and not support other types") { val rf = new RandomForestClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[RandomForestClassificationModel, RandomForestClassifier]( - rf, isClassification = true, spark) { (expected, actual) => + rf, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index 4c6d9c5e2609..4fcc9745b738 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint @@ -81,4 +81,12 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext val newInstance = testDefaultReadWrite(instance) assert(newInstance.selectedFeatures === instance.selectedFeatures) } + + test("should support all NumericType labels and not support other types") { + val css = new ChiSqSelector() + MLTestingUtils.checkNumericTypes[ChiSqSelectorModel, ChiSqSelector]( + css, spark) { (expected, actual) => + assert(expected.selectedFeatures === actual.selectedFeatures) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index 46e7495297a6..c623a6210bda 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute._ import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.Row +import org.apache.spark.sql.types.DoubleType class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("params") { @@ -68,9 +68,9 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul assert(resultSchema.toString == model.transform(original).schema.toString) } - test("label column already exists but is not double type") { + test("label column already exists but is not numeric type") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") - val original = spark.createDataFrame(Seq((0, 1), (2, 2))).toDF("x", "y") + val original = spark.createDataFrame(Seq((0, true), (2, false))).toDF("x", "y") val model = formula.fit(original) intercept[IllegalArgumentException] { model.transformSchema(original.schema) @@ -134,7 +134,6 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul ).toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val resultSchema = model.transformSchema(original.schema) val expected = spark.createDataFrame( Seq( ("male", "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), @@ -188,7 +187,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul "vec2", Array[Attribute]( NumericAttribute.defaultAttr, - NumericAttribute.defaultAttr)).toMetadata + NumericAttribute.defaultAttr)).toMetadata() val original = base.select(base.col("id"), base.col("vec").as("vec2", metadata)) val model = formula.fit(original) val result = model.transform(original) @@ -309,4 +308,23 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul val newModel = testDefaultReadWrite(model) checkModelData(model, newModel) } + + test("should support all NumericType labels") { + val formula = new RFormula().setFormula("label ~ features") + .setLabelCol("x") + .setFeaturesCol("y") + val dfs = MLTestingUtils.genRegressionDFWithNumericLabelCol(spark) + val expected = formula.fit(dfs(DoubleType)) + val actuals = dfs.keys.filter(_ != DoubleType).map(t => formula.fit(dfs(t))) + actuals.foreach { actual => + assert(expected.pipelineModel.stages.length === actual.pipelineModel.stages.length) + expected.pipelineModel.stages.zip(actual.pipelineModel.stages).foreach { + case (exTransformer, acTransformer) => + assert(exTransformer.params === acTransformer.params) + } + assert(expected.resolvedFormula.label === actual.resolvedFormula.label) + assert(expected.resolvedFormula.terms === actual.resolvedFormula.terms) + assert(expected.resolvedFormula.hasIntercept === actual.resolvedFormula.hasIntercept) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index f8fc775676c0..e4772df622d1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -356,7 +356,7 @@ class AFTSurvivalRegressionSuite test("should support all NumericType labels") { val aft = new AFTSurvivalRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[AFTSurvivalRegressionModel, AFTSurvivalRegression]( - aft, isClassification = false, spark) { (expected, actual) => + aft, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index d9f26ad8dc93..2d30cbf36766 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -120,7 +120,7 @@ class DecisionTreeRegressorSuite test("should support all NumericType labels and not support other types") { val dt = new DecisionTreeRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[DecisionTreeRegressionModel, DecisionTreeRegressor]( - dt, isClassification = false, spark) { (expected, actual) => + dt, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index f6ea5bb741d4..ac833b833d7d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -115,7 +115,7 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext test("should support all NumericType labels and not support other types") { val gbt = new GBTRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[GBTRegressionModel, GBTRegressor]( - gbt, isClassification = false, spark) { (expected, actual) => + gbt, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index 161f8c80f8df..3d9aeb8c0a2d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -1021,7 +1021,7 @@ class GeneralizedLinearRegressionSuite val glr = new GeneralizedLinearRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[ GeneralizedLinearRegressionModel, GeneralizedLinearRegression]( - glr, isClassification = false, spark) { (expected, actual) => + glr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala index 9bf7542b1259..bed4978b25b3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala @@ -184,7 +184,7 @@ class IsotonicRegressionSuite test("should support all NumericType labels and not support other types") { val ir = new IsotonicRegression() MLTestingUtils.checkNumericTypes[IsotonicRegressionModel, IsotonicRegression]( - ir, isClassification = false, spark) { (expected, actual) => + ir, spark, isClassification = false) { (expected, actual) => assert(expected.boundaries === actual.boundaries) assert(expected.predictions === actual.predictions) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index 10f547b67375..a98227d2c14f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -1010,7 +1010,7 @@ class LinearRegressionSuite test("should support all NumericType labels and not support other types") { val lr = new LinearRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( - lr, isClassification = false, spark) { (expected, actual) => + lr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index 72f3c65eb8c7..7a3a3698f950 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -98,7 +98,7 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex test("should support all NumericType labels and not support other types") { val rf = new RandomForestRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[RandomForestRegressionModel, RandomForestRegressor]( - rf, isClassification = false, spark) { (expected, actual) => + rf, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index 4fe473bbacd4..ad7d2c9b8d40 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -37,8 +37,8 @@ object MLTestingUtils extends SparkFunSuite { def checkNumericTypes[M <: Model[M], T <: Estimator[M]]( estimator: T, - isClassification: Boolean, - spark: SparkSession)(check: (M, M) => Unit): Unit = { + spark: SparkSession, + isClassification: Boolean = true)(check: (M, M) => Unit): Unit = { val dfs = if (isClassification) { genClassifDFWithNumericLabelCol(spark) } else { From 43570c57630bb03aae0b42a4cae65c8323ebfc99 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 13 May 2016 00:15:39 -0700 Subject: [PATCH 215/313] [SPARK-15310][SQL] Rename HiveTypeCoercion -> TypeCoercion ## What changes were proposed in this pull request? We originally designed the type coercion rules to match Hive, but over time we have diverged. It does not make sense to call it HiveTypeCoercion anymore. This patch renames it TypeCoercion. ## How was this patch tested? Updated unit tests to reflect the rename. Author: Reynold Xin Closes #13091 from rxin/SPARK-15310. (cherry picked from commit e1dc853737fc1739fbb5377ffe31fb2d89935b1f) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/Analyzer.scala | 6 +- ...eTypeCoercion.scala => TypeCoercion.scala} | 6 +- .../expressions/nullExpressions.scala | 10 +-- ...ionSuite.scala => TypeCoercionSuite.scala} | 72 +++++++++---------- .../datasources/csv/CSVInferSchema.scala | 6 +- .../datasources/json/InferSchema.scala | 4 +- 6 files changed, 50 insertions(+), 54 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/{HiveTypeCoercion.scala => TypeCoercion.scala} (99%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/{HiveTypeCoercionSuite.scala => TypeCoercionSuite.scala} (92%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ce941e3df3e8..1313a011c69c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -105,7 +105,7 @@ class Analyzer( GlobalAggregates :: ResolveAggregateFunctions :: TimeWindowing :: - HiveTypeCoercion.typeCoercionRules ++ + TypeCoercion.typeCoercionRules ++ extendedResolutionRules : _*), Batch("Nondeterministic", Once, PullOutNondeterministic), @@ -1922,8 +1922,8 @@ class Analyzer( } private def illegalNumericPrecedence(from: DataType, to: DataType): Boolean = { - val fromPrecedence = HiveTypeCoercion.numericPrecedence.indexOf(from) - val toPrecedence = HiveTypeCoercion.numericPrecedence.indexOf(to) + val fromPrecedence = TypeCoercion.numericPrecedence.indexOf(from) + val toPrecedence = TypeCoercion.numericPrecedence.indexOf(to) toPrecedence > 0 && fromPrecedence > toPrecedence } 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/TypeCoercion.scala similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 537dda60afbf..91bdcc3b09a3 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/TypeCoercion.scala @@ -24,7 +24,6 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ @@ -34,9 +33,6 @@ import org.apache.spark.sql.types._ * A collection of [[Rule]] that can be used to coerce differing types that participate in * operations into compatible ones. * - * Most of these rules are based on Hive semantics, but they do not introduce any dependencies on - * the hive codebase. - * * Notes about type widening / tightest common types: Broadly, there are two cases when we need * to widen data types (e.g. union, binary comparison). In case 1, we are looking for a common * data type for two or more data types, and in this case no loss of precision is allowed. Examples @@ -46,7 +42,7 @@ import org.apache.spark.sql.types._ * double's range is larger than decimal, and yet decimal is more precise than double, but in * union we would cast the decimal into double). */ -object HiveTypeCoercion { +object TypeCoercion { val typeCoercionRules = PropagateTypes :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 641c81b247a2..523fb053972d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{HiveTypeCoercion, TypeCheckResult} +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -96,7 +96,7 @@ case class IfNull(left: Expression, right: Expression) extends RuntimeReplaceabl override def replaceForTypeCoercion(): Expression = { if (left.dataType != right.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => copy(left = Cast(left, dtype), right = Cast(right, dtype)) }.getOrElse(this) } else { @@ -116,7 +116,7 @@ case class NullIf(left: Expression, right: Expression) extends RuntimeReplaceabl override def replaceForTypeCoercion(): Expression = { if (left.dataType != right.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => copy(left = Cast(left, dtype), right = Cast(right, dtype)) }.getOrElse(this) } else { @@ -134,7 +134,7 @@ case class Nvl(left: Expression, right: Expression) extends RuntimeReplaceable { override def replaceForTypeCoercion(): Expression = { if (left.dataType != right.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => copy(left = Cast(left, dtype), right = Cast(right, dtype)) }.getOrElse(this) } else { @@ -154,7 +154,7 @@ case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression) override def replaceForTypeCoercion(): Expression = { if (expr2.dataType != expr3.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype => copy(expr2 = Cast(expr2, dtype), expr3 = Cast(expr3, dtype)) }.getOrElse(this) } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala similarity index 92% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index b591861ac094..7435399b1492 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -26,11 +26,11 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval -class HiveTypeCoercionSuite extends PlanTest { +class TypeCoercionSuite extends PlanTest { test("eligible implicit type cast") { def shouldCast(from: DataType, to: AbstractDataType, expected: DataType): Unit = { - val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) + val got = TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) assert(got.map(_.dataType) == Option(expected), s"Failed to cast $from to $to") } @@ -101,7 +101,7 @@ class HiveTypeCoercionSuite extends PlanTest { test("ineligible implicit type cast") { def shouldNotCast(from: DataType, to: AbstractDataType): Unit = { - val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) + val got = TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) assert(got.isEmpty, s"Should not be able to cast $from to $to, but got $got") } @@ -129,11 +129,11 @@ class HiveTypeCoercionSuite extends PlanTest { test("tightest common bound for types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { - var found = HiveTypeCoercion.findTightestCommonTypeOfTwo(t1, t2) + var found = TypeCoercion.findTightestCommonTypeOfTwo(t1, t2) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found") // Test both directions to make sure the widening is symmetric. - found = HiveTypeCoercion.findTightestCommonTypeOfTwo(t2, t1) + found = TypeCoercion.findTightestCommonTypeOfTwo(t2, t1) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found") } @@ -206,31 +206,31 @@ class HiveTypeCoercionSuite extends PlanTest { } test("cast NullType for expressions that implement ExpectsInputTypes") { - import HiveTypeCoercionSuite._ + import TypeCoercionSuite._ - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeUnaryExpression(Literal.create(null, NullType)), AnyTypeUnaryExpression(Literal.create(null, NullType))) - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, NumericTypeUnaryExpression(Literal.create(null, NullType)), NumericTypeUnaryExpression(Literal.create(null, DoubleType))) } test("cast NullType for binary operators") { - import HiveTypeCoercionSuite._ + import TypeCoercionSuite._ - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)), AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType))) - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, NumericTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)), NumericTypeBinaryOperator(Literal.create(null, DoubleType), Literal.create(null, DoubleType))) } test("coalesce casts") { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, Coalesce(Literal(1.0) :: Literal(1) :: Literal.create(1.0, FloatType) @@ -239,7 +239,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal(1), DoubleType) :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, Coalesce(Literal(1L) :: Literal(1) :: Literal(new java.math.BigDecimal("1000000000000000000000")) @@ -251,7 +251,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("CreateArray casts") { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateArray(Literal(1.0) :: Literal(1) :: Literal.create(1.0, FloatType) @@ -261,7 +261,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateArray(Literal(1.0) :: Literal(1) :: Literal("a") @@ -274,7 +274,7 @@ class HiveTypeCoercionSuite extends PlanTest { test("CreateMap casts") { // type coercion for map keys - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") :: Literal.create(2.0, FloatType) @@ -286,7 +286,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Literal("b") :: Nil)) // type coercion for map values - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") :: Literal(2) @@ -298,7 +298,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal(3.0), StringType) :: Nil)) // type coercion for both map keys and values - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") :: Literal(2.0) @@ -313,7 +313,7 @@ class HiveTypeCoercionSuite extends PlanTest { test("greatest/least cast") { for (operator <- Seq[(Seq[Expression] => Expression)](Greatest, Least)) { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1.0) :: Literal(1) :: Literal.create(1.0, FloatType) @@ -322,7 +322,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal(1), DoubleType) :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1L) :: Literal(1) :: Literal(new java.math.BigDecimal("1000000000000000000000")) @@ -335,19 +335,19 @@ class HiveTypeCoercionSuite extends PlanTest { } test("nanvl casts") { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, NaNvl(Literal.create(1.0, FloatType), Literal.create(1.0, DoubleType)), NaNvl(Cast(Literal.create(1.0, FloatType), DoubleType), Literal.create(1.0, DoubleType))) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, FloatType)), NaNvl(Literal.create(1.0, DoubleType), Cast(Literal.create(1.0, FloatType), DoubleType))) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, DoubleType)), NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, DoubleType))) } test("type coercion for If") { - val rule = HiveTypeCoercion.IfCoercion + val rule = TypeCoercion.IfCoercion ruleTest(rule, If(Literal(true), Literal(1), Literal(1L)), @@ -367,20 +367,20 @@ class HiveTypeCoercionSuite extends PlanTest { } test("type coercion for CaseKeyWhen") { - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Cast(Literal(1.toShort), IntegerType), Seq(Literal(1), Literal("a"))) ) - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(TypeCoercion.CaseWhenCoercion, CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))) ) - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(TypeCoercion.CaseWhenCoercion, CaseWhen(Seq((Literal(true), Literal(1.2))), Literal.create(1, DecimalType(7, 2))), CaseWhen(Seq((Literal(true), Literal(1.2))), Cast(Literal.create(1, DecimalType(7, 2)), DoubleType)) ) - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(TypeCoercion.CaseWhenCoercion, CaseWhen(Seq((Literal(true), Literal(100L))), Literal.create(1, DecimalType(7, 2))), CaseWhen(Seq((Literal(true), Cast(Literal(100L), DecimalType(22, 2)))), Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2))) @@ -388,7 +388,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("BooleanEquality type cast") { - val be = HiveTypeCoercion.BooleanEquality + val be = TypeCoercion.BooleanEquality // Use something more than a literal to avoid triggering the simplification rules. val one = Add(Literal(Decimal(1)), Literal(Decimal(0))) @@ -414,7 +414,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("BooleanEquality simplification") { - val be = HiveTypeCoercion.BooleanEquality + val be = TypeCoercion.BooleanEquality ruleTest(be, EqualTo(Literal(true), Literal(1)), @@ -473,7 +473,7 @@ class HiveTypeCoercionSuite extends PlanTest { AttributeReference("f", FloatType)(), AttributeReference("l", LongType)()) - val wt = HiveTypeCoercion.WidenSetOperationTypes + val wt = TypeCoercion.WidenSetOperationTypes val expectedTypes = Seq(StringType, DecimalType.SYSTEM_DEFAULT, FloatType, DoubleType) val r1 = wt(Except(firstTable, secondTable)).asInstanceOf[Except] @@ -512,7 +512,7 @@ class HiveTypeCoercionSuite extends PlanTest { AttributeReference("p", ByteType)(), AttributeReference("q", DoubleType)()) - val wt = HiveTypeCoercion.WidenSetOperationTypes + val wt = TypeCoercion.WidenSetOperationTypes val expectedTypes = Seq(StringType, DecimalType.SYSTEM_DEFAULT, FloatType, DoubleType) val unionRelation = wt( @@ -536,7 +536,7 @@ class HiveTypeCoercionSuite extends PlanTest { } } - val dp = HiveTypeCoercion.WidenSetOperationTypes + val dp = TypeCoercion.WidenSetOperationTypes val left1 = LocalRelation( AttributeReference("l", DecimalType(10, 8))()) @@ -584,7 +584,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("rule for date/timestamp operations") { - val dateTimeOperations = HiveTypeCoercion.DateTimeOperations + val dateTimeOperations = TypeCoercion.DateTimeOperations val date = Literal(new java.sql.Date(0L)) val timestamp = Literal(new Timestamp(0L)) val interval = Literal(new CalendarInterval(0, 0)) @@ -615,7 +615,7 @@ class HiveTypeCoercionSuite extends PlanTest { */ test("make sure rules do not fire early") { // InConversion - val inConversion = HiveTypeCoercion.InConversion + val inConversion = TypeCoercion.InConversion ruleTest(inConversion, In(UnresolvedAttribute("a"), Seq(Literal(1))), In(UnresolvedAttribute("a"), Seq(Literal(1))) @@ -633,7 +633,7 @@ class HiveTypeCoercionSuite extends PlanTest { } -object HiveTypeCoercionSuite { +object TypeCoercionSuite { case class AnyTypeUnaryExpression(child: Expression) extends UnaryExpression with ExpectsInputTypes with Unevaluable { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index 05c8d8ee15f6..de3d889621b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -25,7 +25,7 @@ import scala.util.control.Exception._ import scala.util.Try import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion +import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -171,11 +171,11 @@ private[csv] object CSVInferSchema { StringType } - private val numericPrecedence: IndexedSeq[DataType] = HiveTypeCoercion.numericPrecedence + private val numericPrecedence: IndexedSeq[DataType] = TypeCoercion.numericPrecedence /** * Copied from internal Spark api - * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] + * [[org.apache.spark.sql.catalyst.analysis.TypeCoercion]] */ val findTightestCommonType: (DataType, DataType) => Option[DataType] = { case (t1, t2) if t1 == t2 => Some(t1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala index 42c82625fa53..579b036417d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala @@ -22,7 +22,7 @@ import java.util.Comparator import com.fasterxml.jackson.core._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion +import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.execution.datasources.json.JacksonUtils.nextUntil import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -252,7 +252,7 @@ private[sql] object InferSchema { * Returns the most general data type for two given data types. */ def compatibleType(t1: DataType, t2: DataType): DataType = { - HiveTypeCoercion.findTightestCommonTypeOfTwo(t1, t2).getOrElse { + TypeCoercion.findTightestCommonTypeOfTwo(t1, t2).getOrElse { // t1 or t2 is a StructType, ArrayType, or an unexpected type. (t1, t2) match { // Double support larger range than fixed decimal, DecimalType.Maximum should be enough From 3727e28593e4d0784b678a050919c9a85ce208a5 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 13 May 2016 08:29:37 +0100 Subject: [PATCH 216/313] [SPARK-14900][ML] spark.ml classification metrics should include accuracy ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Add accuracy to MulticlassMetrics class and add corresponding code in MulticlassClassificationEvaluator. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Scala Unit tests in ml.evaluation Author: wm624@hotmail.com Closes #12882 from wangmiao1981/accuracy. (cherry picked from commit bdff299f9e51b06b809fe505bda466009e759831) Signed-off-by: Sean Owen --- .../MulticlassClassificationEvaluator.scala | 15 +++++---------- .../mllib/evaluation/MulticlassMetrics.scala | 17 ++++++++++++++--- .../evaluation/MulticlassMetricsSuite.scala | 9 +++++---- 3 files changed, 24 insertions(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 3d89843a0b71..840851675110 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -40,15 +40,15 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid /** * param for metric name in evaluation (supports `"f1"` (default), `"precision"`, `"recall"`, - * `"weightedPrecision"`, `"weightedRecall"`) + * `"weightedPrecision"`, `"weightedRecall"`, `"accuracy"`) * @group param */ @Since("1.5.0") val metricName: Param[String] = { val allowedParams = ParamValidators.inArray(Array("f1", "precision", - "recall", "weightedPrecision", "weightedRecall")) + "recall", "weightedPrecision", "weightedRecall", "accuracy")) new Param(this, "metricName", "metric name in evaluation " + - "(f1|precision|recall|weightedPrecision|weightedRecall)", allowedParams) + "(f1|precision|recall|weightedPrecision|weightedRecall|accuracy)", allowedParams) } /** @group getParam */ @@ -86,18 +86,13 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid case "recall" => metrics.recall case "weightedPrecision" => metrics.weightedPrecision case "weightedRecall" => metrics.weightedRecall + case "accuracy" => metrics.accuracy } metric } @Since("1.5.0") - override def isLargerBetter: Boolean = $(metricName) match { - case "f1" => true - case "precision" => true - case "recall" => true - case "weightedPrecision" => true - case "weightedRecall" => true - } + override def isLargerBetter: Boolean = true @Since("1.5.0") override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 5dde2bdb17f3..719695a3380d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -139,7 +139,8 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * Returns precision */ @Since("1.1.0") - lazy val precision: Double = tpByClass.values.sum.toDouble / labelCount + @deprecated("Use accuracy.", "2.0.0") + lazy val precision: Double = accuracy /** * Returns recall @@ -148,14 +149,24 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * of all false negatives) */ @Since("1.1.0") - lazy val recall: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val recall: Double = accuracy /** * Returns f-measure * (equals to precision and recall because precision equals recall) */ @Since("1.1.0") - lazy val fMeasure: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val fMeasure: Double = accuracy + + /** + * Returns accuracy + * (equals to the total number of correctly classified instances + * out of the total number of instances.) + */ + @Since("2.0.0") + lazy val accuracy: Double = tpByClass.values.sum.toDouble / labelCount /** * Returns weighted true positive rate diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index d55bc8c3ec09..f316c67234f1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -69,11 +69,12 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(math.abs(metrics.fMeasure(1.0, 2.0) - f2measure1) < delta) assert(math.abs(metrics.fMeasure(2.0, 2.0) - f2measure2) < delta) - assert(math.abs(metrics.recall - + assert(math.abs(metrics.accuracy - (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1))) < delta) - assert(math.abs(metrics.recall - metrics.precision) < delta) - assert(math.abs(metrics.recall - metrics.fMeasure) < delta) - assert(math.abs(metrics.recall - metrics.weightedRecall) < delta) + assert(math.abs(metrics.accuracy - metrics.precision) < delta) + assert(math.abs(metrics.accuracy - metrics.recall) < delta) + assert(math.abs(metrics.accuracy - metrics.fMeasure) < delta) + assert(math.abs(metrics.accuracy - metrics.weightedRecall) < delta) assert(math.abs(metrics.weightedFalsePositiveRate - ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) assert(math.abs(metrics.weightedPrecision - From beaf703d5472a90f529d5194f5cc657b45871087 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 13 May 2016 08:59:18 +0100 Subject: [PATCH 217/313] [SPARK-15061][PYSPARK] Upgrade to Py4J 0.10.1 ## What changes were proposed in this pull request? This upgrades to Py4J 0.10.1 which reduces syscal overhead in Java gateway ( see https://github.com/bartdag/py4j/issues/201 ). Related https://issues.apache.org/jira/browse/SPARK-6728 . ## How was this patch tested? Existing doctests & unit tests pass Author: Holden Karau Closes #13064 from holdenk/SPARK-15061-upgrade-to-py4j-0.10.1. (cherry picked from commit 382dbc12bb4b06871850a94f88018e76058a9b52) Signed-off-by: Sean Owen --- LICENSE | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/docs/Makefile | 2 +- python/lib/py4j-0.10.1-src.zip | Bin 0 -> 61356 bytes python/lib/py4j-0.9.2-src.zip | Bin 55521 -> 0 bytes sbin/spark-config.sh | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 4 ++-- .../spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 16 files changed, 15 insertions(+), 15 deletions(-) create mode 100644 python/lib/py4j-0.10.1-src.zip delete mode 100644 python/lib/py4j-0.9.2-src.zip diff --git a/LICENSE b/LICENSE index 9714b3b1e4d1..f40364046b34 100644 --- a/LICENSE +++ b/LICENSE @@ -263,7 +263,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.9.2 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.1 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) diff --git a/bin/pyspark b/bin/pyspark index d1fe75a08bda..396a07c9f413 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -63,7 +63,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.2-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index cb788497ffc7..3e2ff100fb8a 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9.2-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 8584b62c0ece..c9853528b4e8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -328,7 +328,7 @@ net.sf.py4j py4j - 0.9.2 + 0.10.1 org.apache.spark diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 8bcd2903fe76..64cf4981714c 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -32,7 +32,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.9.2-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 83bdd9031903..2477312d74bf 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -140,7 +140,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 121e28261837..0181a47a79d3 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -147,7 +147,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 1d5ad271326d..f7ff23472b9b 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -147,7 +147,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 909b94bde20c..92db55d674a4 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -155,7 +155,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 750759974021..44b9b049344a 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -156,7 +156,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/python/docs/Makefile b/python/docs/Makefile index 905e0215c20c..12e397e4507c 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -7,7 +7,7 @@ SPHINXBUILD ?= sphinx-build PAPER ?= BUILDDIR ?= _build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.9.2-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.1-src.zip) # User-friendly check for sphinx-build ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) diff --git a/python/lib/py4j-0.10.1-src.zip b/python/lib/py4j-0.10.1-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..a54bcae03afb823da3b2b69814811d192db44630 GIT binary patch literal 61356 zcmb5VV~}XgvMt)SZQHhO+qUi1wr$&XueNR5-K%ZCwf8;e$M@pC*!#_hnlb-W&WNnc zF)Bw^DM$l@Kmq*gK~}Dk`1i;Eyg&hP031A7tmsu$AOV0^teDkB%b3+&J)i*qLC$~y z0RH___&Xit?*s?{amO5qn}#$mRagK3ATa;{jQ^c(W$0$8Z)|U4V`}VTX>aFD=ivEo zi>=ed>i-)5FN;??w)UGGNZ;4`{m#7-+N%TPoAXuZKSQZ< zZJSrwri-dX8gxfY(xyD2H%&C)2aj4upbYh^pP7Rt+?E$$;571sub?ytt`oS7->Y)1{vg?)^8gvqu ze+{{59`UlBAocpk7BJ6wLSJRbCBCvMToYZ!^m@jCk@{sGNmVSdo`~^OjTA@;SyCTYacj=b2?i1S3Yf*8 z{>-78VE7oXg@}zwM)l63q`@TRKmst83`CgQ|4U{|4rHHjk}OGe257$b;-$L#!KOWP z3LL=2ASR#y7O*!RohudxBgDwY>M$tZNX=O@6O{vfxmqQ;L{y6Wveq5-`(^I|i32FE zfl$b6Y0jF^nMjVcd0020&KDOKy{!r>87R3xiltIQc=D|jRF5bkxf`N1MxQ4ekOERi z;HfQ&ZS0Mp(;@1;|m)YSRo*1 zqDSPxy`TiUb3zdK+RZIk-e|kqPtwePo35+6=>_A8AQr_aOqa;POP4T0f?Y^lf8+-_ zV3Y}U9IPdR5T3n!Kv>TR z7Q8IGs$0bYAh;LB{14BYI^bu|pE^7I`!P^@pd&cGkoK;@v`dD^Gsg8#5I7-~M$33C zs_M$MuTRY4AJxcfA(tn9Ge;~0?K5&9i@8wH$PUfPgJ3#y)qy(iYN_yvXHiVipr)vK z{CNk40bOK^AorU=I>JY-XTq2S0?d_d8t=B3+^;D=6CywbqKWz{Qvi1sAi*;6q5xE2 zv~{V(hV3>nNWL7lLXc+ns^)n9+wO!+FRMqZH&&){I`l9Df$jBh+;R(r;TTIAa-Q9> zGQ#|cimWWme$S;NyIEOplxoY3-mMzg?9JwYwsJwYs#=rLwf>WbvR00U8lE)W8_m1ud2N$rz6 zBDw=>iuU~S2~hPDreb!@?+BP#<@ymXN`jQtmG}3#Gg|-Q#2K=Migs6KY`(Ad#9_*` z;eBejs%_o&`cHr>o9NDfu2DZe5x{y-0!`yrp*CQWzT&f&v2Pz5$^Ze*8YGFJQ&+)~ zg&G@d0-1tsNG+j#U3Jx?wZ8oKQGrRu(#~nJV3T}Cd8uL9U}YW_>PdF466sL2JXHYN zl>4uu$;l%A!b5g*bMez&(Nl4A;O9?ydF!-meSe6-wqCp9HcY!@#Vvtg5JHTm5A%(h zio4Hxw?L$`^Cee5yJWn+IkAwaqRE7kql#|C4%T4&G+#;s2wIxCpVht!FCS!z4BO%C zydQC<2y!!Skjp{%a}F!^LwI@lSnxPwKM*KXxAZ(J-Y<>qBZ7lZ?oM;z9N;MK)OtY_ z9DglU^`prNunOB@h@kg(b%O*d8xhbJipkr2R=iJx1L8-<(n_&E`;NM`;P5-0mcsdn zHt<;fOJF8uEdUOKIS8RB>hVr57Hxqbo2g_iviQ>gTzk*BqPMWK6onMkW`vLx===SZ zZDM06ysqCMU9HTD_jJd)1~#Ac^&);4#t=yuzbdI7RKRfrDQRppXf(X2ABJUFTduYh z(thzm;Q3iY)FN6(vBvY6a;#mnn@Xz)Kx(M6kkOKqA+o_}^O6xmSrF6-gos0AcQ6+o zR3zwFCq||{{*f5r4?lR%Z*51_`HanmqPIeMJ--Kn=}<9;Q5sRV%%Bn$6YfxH%RtTt z?J$T4Y#LUB?lKVH@8| z3qBZGaAz0b!p-LABH-oSbD&yMzEysJ)RE4)(3^Z~b9~_lC%(Rr#PJ5@&Tl_nE50UX zXl(#fO`{)rvisQ^eIe|MFieKj6=&{#)2o#=lrCg;m4A7EVOHgUYstsBiBU@&fG5KU zYB8|M4%3MlM=_jLsV7iJokw;}3QB6x5(MnYoVrUy9o=8FtiF!SuN=5qOg@*AuS_&& zW}sIz_w+$6;~_LOX}AByD8N4d!j<_kK`swZ*R3lnv-+~&eY&vugsBIxPI3u-k~xub zY>;)o>E%|-UNrK_lQm7U9euSFg`P2Fd`C4%C*$)u0a` zzqFbPls}c^bGk~WRb%JA489T^!x|`Ea%0qUx$>T`g(f3ESqWcu^?ne4^-+<{WCvqy zyvw=+kyr08B6=h`tjEv33GIj`t2WcIpRWrE<1mNk80>1Ai;pD>14cXxEF@VoAT0b~ zcjH?$5Za`xi?mc+hgDE{%U&amU&nVmNovOPk^@0ik-TQ3P5OIZZb0m=wohYQZZrPQ za>bgX6jtYXvLI!r^HUq=WR-|6k4k((5N@)i@#nW|dg_yf`hhNRPjSGr_A&bvc2)?x zs+D$H*Xw%W(Xt8lgx*#2bP2s*EA_9thg+odcKiaE5&5P7OS0X`{Sl|vZ}I0c^1*qu z62^pe;E$Wrr`_30{L}7lYnsSu_+VeiLG~r2*LAwCQF+?BP|)IFyEFs-u>`l~va_L$ zAoB8a-Ps(3m^3Z9GBp9UDMDUf91|WtXFng74N$E4{&EH3UV0H^e$(Pf_Kfv7PSFaB z?hWfChy&H}5JTJe>qzbDMBUHp=jIttqP?tCFkv1mp5iMiDRl!b+^-Q1NJ~W@$(aa8`|@y&gMDs?2~crWvI(} zMC9_*aC*7!O*-(iC+qBK1^XGttkli|YcY}=lJ@3BE1-7Tm?0_BG0y)n4EaCi_(uermmx-I5o9Ru4uqib&&!9h>+L`%$EY`=xF* zYc|@nA&!p~aOcoA7GRJH21*-h9TR&Wwt2#q7PjG^guSid9b0(8-vShyCQg zL9fS@;wZsDe@K)tcvo4}3~a?tWSuXEYda)wLeXHd)8?je8fL|LMbCP(ua&F=wbt6} z>iAi_&bGlGMYTUQ0=tMP6oqKMfCIxF4reHYsIbM%ecmXl8n0!?hKip}lB%xO4PW+0 zMq40}*P2}lKIjJNB?8H6HFBJEj2(T{mB0*x_63Ky&xX>HQ7ONyn)%}0xNLvf5!?zO z^);o;J#3N*;gci8Y5!yfR5Gh1I=HzY#(cM9JZcP}A5z-?IuXkl>`vSIwvcPz?n}Pj zayQYU=Z3GrOYOQ@yRa?K0WV|kxaMTBW95eTVZzeEi`miH@pkz*rDLnO{q@f>?ZSAo zs|_9iz=#n50O5bjGzTYp7yG|u+J7j|TClADmS^`}exULn!#nLl3t(N)yKaD^irVhD zo95iU`PyU6GMiRU4382QwC}t2p~VzRiAi>j=qR!xGf~Wgo|3J$+LOG*VJzdw{E(8# zHj(iOu$y3vlM*v8ffQpw+#bbuq4W(?Wnlz62}H)&jeJ^7hsV~ULk<3B@rbYDk~6=L zE8XhGC zm_j`8kGzoKOpA0;d%STH*YHdPUn^9zg58O%=~haNiD+{SsQ z!}EqT>kTQxafgIpStdIW)DTW-Vk{+AXr5mXW%-M!8hxE++3syP$&ngm_)QSB{V_~Q z))Q%7vmjl;5S=9|>SO^I$g+1+-#hZfB-4-3h(zh3-M53M)gk{A@c4zGKlyGCRH)HK|Yk%lmoqxX#I%#gj2JI`zhI7H`<$EAZ1IswU#Y zsUD}t5jLINHNPJujoifunBAQbXG+KZ)rw(sx%WG8I4>Og^z4nI{8%pa;bq4L>cC^M zYW3RTJf;?rt*vI?g()Cp-sNEWWHEzV|2Mm#&n8g9R$I`^&OlDDtV z)^*f8H^^((?6K$9$w;T0K8)R{H2in!+Bx~AMh=Xfj!syB{m)N!M-a zjkyp0+#bWVbbP4}l#Mk{PPfXe@n&bH+~~DIVd_esjh8p8+wuh1I=nc0P=i+I2=kOU zR>@TB9?SJs4h;3wQY+X4Cr-BxfFb&|67RKO_I#liob9CyC{0@;7d|<@pTwq-9ANu7el6 zN_)?)>b-qOqdAjG;H(2M4Q+kA_5A?E^y85&`dGpcuSj+)2hWuO zZB;Y%9Ds?I1c!)Qdu;jjKv-Q#Y6ztj&@=TmzMB9Ywh5aYV@^Tv?nT2cGpbG-^Cxsee?m6FlBozvTGUD;!*XE2Yr4w= zJNhwpfeLZ?$mS z32#}XIo0dHh1SVkfRFVW?e`PR)KyP4(|Sm{)_BU;08;8AUN;D0K2$#%`1Q(nGRiNq zGoOoIDZ(0}T^rEJt@^#41yC|4tjLuh*^dpKYAsDeFhoCNeLMg91aE%c70y&Jrd*T*{@-nD7q1&_^w8E4T zJ+%#pkVez|mg`^)OjhgxJ&DD4@#@OnzNdz5FdV;(tn0p2eUi#^5yGlrPxd)aFwT=n zOm`J^tX9`SUBiWi9ua)gmxH)c-Z)P@)nF?DZ6a;0+}%)UXuP={tp!=7ZexR4XG)jw zT3-tVd1{PMK+Uz?D>S`RSyf1?uVmYdZ!;9K-fCWGf-MU7IuEh+?$@t-`o<0 z#}DlWJ-K9bN!L+6=0nD%ehi5)p&>W~@tSHbB))qG-$PDND(1V15d8?nZOkVST*T=N z;EPuZ1~u@6`sOyBs~K4nF6$ttQNdl492VqWqkO8@58QpXqEb`V_>4+ICYij_uo-nz zuWYqMrqqhElP-{5PDOcit;j-`gv#NMfg|MTBcsgksCy)TqA^Wc856+fiYEy!%46MJ z>;j-TfC1Y$Vd1Q2;W;oCdC;LunkA3akO^h;aoZ0X8UE~N$wcYapN^zT!ay7RS^$Gz zld(3-k)P{!O9oYVXd(UF2SdIDC+7|p+?mHdVE(hzxm_Eu2nUW<)2AQ|o((?E+)`*E zzrBDBQ#M%)4Wd0~fpAz2>_ed}4B1ihL8EVA2rN89zyh;lhN^Hi^asvPb)vi%oHgj& zCzCPRcVra(NHZtmDAwePXqWG8`dX?L(^{g~e-b%zJcc|F{bv_hP$EAP6#;c1t^fwN zc_|Vs+BueBXCnoGtK@lKqJ$=Cj@SiHIL`7jA0sH@y3uNPNj?Q>)TdojA=*6KVv;O!4$tdHVQk+xL*eLY4r9 z>9_ePM7%Ub@ErFLzRZ#FgPh#XjVugqMnId)_z} zF}yL}L=_$bWy`Etg|VEDltF2I&j7Tb5W5Qb{>*W7i635B%v_((KM4EOKq=zr{}8d~ zihF&woQ@|WAq#?Ge9*K9am?=G8VQKyqzp~!7?b3;qAcV_C)#GHi?ExSpb9E4BXFnk z35Yq^6$CEGy_FV|bnE=yV^vD!aHPd9FcS6F;O~mHy3mTJP$gLzVh7;)5k-6kt*4GG zNUXgtOaq!HT*Sbd!hnf4eWlVv<5*zm?vz|lEph1AeDl)#TZ~zR0RE1RJ1eM)s?rgZ zh|KkuT0&xNd;a3Pec_Ui$DI(|QidVf%>&&Tp7rQeyYF9bVvyn5Rz;r~s3O7Q>ckmq zrwl9y`{{tl-gJjjKtm+LX<(66@^tLPpW(L}-bzIec7m?#g=Q8r_yS_r{qw8+)xzff z1ETAxVGn%hog%`|W8+7ox7utHmWkqh%{aqwI5}(-3QGV!N)@*=WpZ`};6MsPATD;3 zq(v5E5G zM@-KGl0f9;C$g9C|tbun_jOr!7is-NaEI;_&xp8?~eIn_xj_@h`YD*H|a2T1Jd+(Qd92m z2%4uvqOibz>L$^R1Za=)Tq&Xoz6g|b9$u(DsOnEVNMJg!M&0!4g7cc7l%{b(>=2aL zSTl6|``Y;5@WV8wO=5?!KXqs`Vs_7Eyl;?2)m&`59eU;|sL>W$| z=rnyV6;7F(+D@@=g-+pv&A)w`n735}&abK9f(fFi7oO&mf=I~eWdDWnMj1+>S49qE(lrQ)!zek&uan0 zS*M?T*|!$4-{r!q`7FQe_Oh;NT1(iibKm{>sq=p5>{oEqQcJScqLSL?el7JM&y70V zw|R#Kf_n-K?%79QYs}VIB#@V2<{D>fJqdL4<^6ix$Pb*80+~!vsfS_z|vjU)2XR~j21Z6!e0-4JUQ;0Wj ztQ>@=Z@S2QWua?XLR_uvI$*aqt$nuQG&KzdIA8Z=0`A%CIlZxR6T~qNOu1f}!{Ulq zb|bc~16&r7bW&Vc-NR9Ulk%x0k4^1X5ibHK*_?Nvk)D#QY7~mWsPe8#6@d*j1@#~r zCfvc&(p0n?)1*dn`s2hE3OM*MXU_zAtMLG!Ut}Cu79-yk zR}}QFiV2b(R}Nel^`}pRea9E^ZSwg*L0irLUdJ1zGv0{%I~70lcQctuSr@i$8Ol_6v1O4iZ{yIvcdLE1DlQ?b$(M^QhoYTc$;v%>M!8jc!cSg3 zV?>2 z+q}^KD*(8dI=eXk|7iARY3W)5fdc^i!Ug~!_@@zL8%t9=7ky__CpS|k{eKz$530hI z#^0td8^XVuzFhhweW;nu9}9HDN~@Ad$HU=0bi&#g%`UAON`+!dj@|mb#FXn^jyCLq zVQA@!EJ&X2-K(dunG%Re3M5$9tmei82PzV})k7==%JDTmu2`*Pp?ov#W)&!e6vK&@ z{Y&oDpe++4mae4I=_~mzlM#~Wb#_78UgF})m4o^%1!hE?HO0E*W3e1T#YhFn+%Dd1C z_M|V->Skbm$x8(ks>qb!i6TToOgv+JBsWnNXUAE-x|*y&F_z)`J3C^P`S2=VM?%Fh zm;9_&HO;Tjj}|HZSiBYp!we_#=)6Oz9s*SCBG}`pb4fxlSFGVi@8uSw?q4ozGz|6fiu0m51zg)c4)ugM!ft;A!BWJ~7HD`q;ODH;FzkXDsOkHi;OO-sN=(W|1CT~w^iU9G>o zO?LwA!1X>lISK9VB}!8lcJgucUBFJ)C#z7OkAzVgz++UfxrDf0m%{{h`#!XHGOVe_?WAl7E z3!g0Un_AGEiICFkrwwZ_!-9tqZlrt^jAd1iPK{5CVGr@bDwmw5ABNVpw76!Cu!YrFm;ap~*0Q+O)LFCoDY|i0@AQbP5J3IynHS*)9aS zLwI@?Gkz32Vgld~ZN*d_)JX1buyKKpM-0efjW`d@>VQk>*Vm9O#1Ku$Z)5cG@KDrz+51um+u*h7dWn9;e!;I27n@; zA49wwC6YP9lb=d>oXawFeKy9!2wQBN`?7c*t2)$%pW(XFKJH2lkLSBWyvJ=H-G_&3 z%X(|7SJYaKf;p>KyPE@pdIhWl0PU=y9KxvmK*yP6^<%n@siW|)g`Gt7??E&cry_?9 z0pik8{*cN1eMX<;D8GYTE%ws5qUfxZ}5eRN4!Q7-x7;|KkzDfXf6eFJ~ zR>l2}&C!Rxv1#9CP7qk&szE*H-u*^6ymd`!Gi;A|2E<_INLl}?JuGfUTxXBAxjEM; z+g?N0OvmgWmV(H+0Uy(v3WN=ZG#OYGy9PrR+f!HJn$9;F)Df@Hf`>jB9q0vdf)XWGxjzvN5q!l=;m0S%A z72L!c-GB^#9^f$0gqx4JiY{yR-$wL5m{Rnjd8I&Kop4Q+MTM9h9}*X=r0JBFoT>ae za+9^n*_m zg=L|A(-}Xb25A_7Ldi$^lI(D0=2|h8%i&uA6Y^1=346JolEWKbe7hn1Y>p`DUUnDW zCqH3U<2|32jj*qqYesMLjy*!%k90t}-8MvLA)eaCEZ6iRQW`wJC6Q=Ax>PDsn7sn zWwhINv|P$AR>PYIY6u11v8L?`hTQv%dqT7E2MVflV>m(qWa{OV89e#p4fhBKdT1Ib zGj&TcRWw$4So`tk*6AU>tLkU0?r4uuIlqzz zn28kVWv$`Q(FIS0sWeBce6dXaO2>1(AuaN>iNQWorpU>Rf}CnJqx747YOgPbyh!^z(WK-iQQ;fMly9FJu z?JOwJ3UC=Aitrn}TGQbSUeWS{SjQAq-LSwqwz#@ahf+kXoZq@U>dl5s(chM8@ zN$)qrqLfB(!45~;cPtVP40ZI%-18x~G85`rhAz$sA#uSysACw5+L4HE9XCOzTsY*{ zEuO4kN;w|>%$K=1s|i=}4j#$%bO(e<6LyLd7qcZMmg^0x5V$_7FIlf~Ojw)lbu_?F z*{qfJ#v_0TzV|}GwLwtx^Jj=vM6Y>8n_}ffi4@N!R|)U>(KP6(*UD&i*VRZA+9U_e z=9sAn;EmF`d@vg>b=4eU^VU^3t`CfFn&nN!jy;GLt>UeB=JU$!?N?ZcI^-l^zs8=((xGaFSvSW>bX<@oxd|4}$VS)?^zWvdjDvx-M(| z-IFEntuV*=oBiRW=hPE#6B#NB(*m?BlVAe`fmU65#7@*0KC?9*q7j^?!7Aev&S?R4 z>Bo3J#UcU5n1(ilv3`=tvPA8- z9F)wJ(PW=&D5uGLg%}dAW&IRzJRk}n&Y^gXgU%nZ69*8>{Gi@>I|y50C=wk(Mr4i5 zyw#N~icU?tl`7QB^L_*L(*t$YE-VF2V8(7wU&vQVuBG50ec2su;~A9GiIQWS0}P%S zYC_c0oqSeTH2sz1S^I?sHoNy74_r*`X4SZL(n%(3 zbm*)Gw(&EwE~(vh#U?AUm&SGaakuS$>U_u%OUD8M%$Wq+6$t9hZL)5`6Iu9D$ZB0t zW<%JlDGd7Li<0y$HJR!7+e)@7z*%ic%4%?@QQtH2pX#b$YXR3GmMS{^~Redkr*^=N&kLX zxhkj*#c)chv;a_~5h%`T&G>O^!7P*jY$3&j;jmr*tyZ}SgR35k!V2sMZj{Hwh%g%c z8N0N2v-SerH!6Spej`%&A7SgG5Swd(EPD|0_2Zn4I_QA8k7h%JDfbitl`6e6f<^Pn zwLgx3#tBq*7&#usrJuRnt}v9m$;`>O$W*+>({P{H*|!J$U)Ky-2_8uqU4d518n}6+ zNNG*yU(;z9CHJsm#5Wy-yPHisB1(^Sub}c;OUoB#fmZcx8+f8PQ_e!C8U@I*ZG7T2 zGvg}X5emx^H>8j-CG|>gi%Q7kolG;%q2&sG>kvFSo9Dsg+28h=ii-p~ zU6eBQW1bEu+{~SS-X<`YYRk0GYB`OdK_OsOktN2fJ~E0E6g2hj)iC{z0E~0%%5*cY+oC)3N;yoACUC*h_d72po1T1a8B{6<62Q5v1HdA z1n4`qDP4#&&45l}c^w|pLMUQ6Q>raE;<_$7^VZd*Funo)nLiSU!?#m^Swr+M;$i(W zfBwxH|3eo40px7sWUc>lMd-~pst`GPsN^UeEr=L9#vc}o8!_GEA=B?ZSG9q&Uv&~( zYxzRxJ#~1#1chFdGAZ!!MBVq~H0toTYb?|yxt8U=RMu2jrav50xNQSaVBV+tcT1M3>tD zS*P1980-G&pG7Qr3fsT<@&x^V#}{W)m;VM|w$*l{vIG$PUg|LZL3`lnX_)HaV103(`0o=8L*b=e~;G03n`kS_<5zT2Sk(2&IgLl1q z`wPtDyylb2xT?F}Mf2JSszh)B8xMnqtQD~|QtPHCqX97%mye8*YH`;c)k+Uc)iIkh z`Mq1LH}ow|OcGJu#1MYGLTJLwWIO5R8%1usuW$6CFM7PTU$wiCn$f}C{z6&;8q%1+ zVW}OBkaZurM(hr4r=9ele94ueBb-MBv%1*X_(3$Y)r%> z<^UbK1Xrow+|VW_r$6&RoUft8x%(P3Ex+!tX zw&JvoEXyv)M}Epk>0zh>oJLKkhH`)_TA1Rflt`$G=|G8tmE-`;)Fh@5?@)*p%@d_x zyxXO4G|rGw>k!Y*&x2XxrMk^*v{25WLDNPQt>|EVFPTA@3jvV)J_-7Jv#DRIB9SkA>)-ST_}enWY~}+L{^ioHkaq~ zys)0vy4Q@!>p@e2tazlJd;h{CF9eA>hdx;Xuwt%1tUsLPx~L;)Q(2$J%-rlL6`qx+ zhl*>3j1#==pSu_2hfKs(4O6eckyn(;4h!Z>OJ{c7Ep6ad2r*R92()J;ESL~CPUhsG zCB52nuc5RH^*<|mz)dU_641DP8>IUg8|51GslaC|7CXK1Puzv6A}^Z*W5Y&YJ=}M( zC_4e#k8@oO^5WJg|KM3GTfB1kfZj)T!InoC%VJU7`njX+2*iICGWa@#7A4KOwJTt4 z-U}jS>08Q-R!LV}xyJn=sKG@p$;uXd&;5!_>*w4^TE?OKls@n~^?7`HyZ15X9fkWS z^XVB~T@CkWp-a?@Bioka)7_gNRR5C4LYcF8|EY3^$lHhgXB-t_Q^-91jUyEh006vy z#*vYyi>bb|i<70D`F{h3Au5xxSqv!K52$V9V3m+ob2OC_>&>#%uJdYErCExmoDPs8 zBc;!z$b1#M^Z1vnAVLeH1ZdAsoTGfBahx=IbOfxL6))5tYrplopHm3hN`VOPL$RWS z1T>dNkj8SDD+y72`5|VW#L`60EY6fQ4dk#qyxbH&&uTX|4t?E4y67Wlt%#7ZyzMoKrvq zsz{#h$x6Sb(1bOZwm8(Ie(Y$UGibREV4m#7<6X=ZOw113iCvGdR{!w2V#Mrm+C(h? z{+Ix?92N&ajh~~ANj-_rp04hA&yfP~IW&))b=`<-+s2K{ddo%|9iEo~Q;-Y3TkJb`l? zY`K_Ww=XRnG9N!w1eWVFfUJMFuQ9yc!9jj)#5a-kn^$s+HiIKcDr3DQ)uUmY5}LcO zF0Tn%UW;0Lc38Rl)VJ(2O9eL6&NE2zJ5bUN>Cibem5m>(LMUV-YAeA0u}NgMKZ%m( z4S#DbdKq8e*IbPLdUJL!Y4ihex8*JI^b^HG^DWF^8W2ixTnU=-vdy)d^!J4257De-lY1YTAD_U=+XAI)bIJv2-#P!8RaJ78=*47Qq#dh$ce| zsKC-#8!nWJ1Qn#858R6biPT1ybiJv`ea>C?TDU=mbZi4Srb5&=P_-ixGV;2(Uoe4r zn#XJ#-NbpIy~Mxlf`bS|-nPyH6*ARpnFJ^iMJr;6Q;>Ab1xP0_$!ulTCag3$^J7Hj zx@oA+MJ!pstwXfjXVIr6=j?1-sKUt0cRB0r=nYNVmgHyMT3MB^cWjbqyUbl;{@|_E zGNXWk^g4IHwRD7!XG?Y%eO_Zt@uq_^R|Ak39%?NG7GWO2I4bW*DUEO@$8st9*Xqb< z>B`%i^J3t&;`$eSTZCKOt!6oU`Mr-)>SojNj|?^9ly9rhEy6>!A?0U-_Os_l=#h_- z^XJ?62M%$W;e?2*B_ow5rEX-Jv3Re}ZF+Ovy%*9yRs0CWaYReL^3va(0_-JeOgFF^ zBsG~i`uaL*r7Ag`1gq_@p6)lVaPyE#cA11LxWUr-#tu|_#xK1nHdTEs)xsJ?QYjT$ z(q*(9u}sFd>)8f$tz?fTpHCt=uK>r;jPZ~Wk!`Bwmr2CmjIou=WBc+&ECMQBj-`H9|rOe)dy?n-z>gK(FU%4cu)drm%_e=U;PKwru(nb_i@S zYO2Qe5#Lr`BZgDIALVR}x;Xb*|8E%k~I%k(}BhrC_#S{)w+V|w( zwe#O^4lf_yJNbDECOm)L{Q<@0>n7Tr2g)+4viEaqDONY{-Sz3@`nX+i{ta0DC#H1< zSqxTMxnDZ2*bLJaR(g9WH-_Wkws+2U6zLpUxuAWqMJ?c6tr?YE3`&Ycgv0D6qM3wY!^Kfp%Y;684fS}{0PNE=4dCfZ z08|gC2#sOHN*eICE(r?+mw`>+ObveCknXpWEf=2}(NI?WjrZ_xeG5 z*^)cdXH#}5NpW*usD$MjhZle87p2z${=d-a|Ma^4bte(k{}uFKHx&Oh(Edqv`udi3 zmM;4G|DhlvCQ$HXks?s}+r}bL_$#^@nwvuX69Gf)NV&)`000KO0RZs-`Hz2Z%gha3 zOx+DV{~Hv%;yriT6kGUtLNPuQ7YGxQP-w(mE~`^Zpw!mWrJj&$r9p=x2}7|#$amB* z+p)+q*04P^^p$s)pBgBb$s=Dlgu}*iLi*;(A<@GaMR1kmDO2r&L4IYv?r8i*kw>%J zo3R`!D0MlY)zJ^^KKf-B&VP#hX1PzciesGc9?=#(02yP<5IDqhDo^5r;e5uC?Nb;5 zvcc~XD;Jig%f!X~+Qr=2jd7jl0vbIOy?CcomNGZ&bn+%tcZ4;?qspP*2##6b7Vn?H zU102fF9}3POfP}+&Z-OrlS9Ii`m9si%CD8=(BQ%xFbxaDpMX7OndWilk)psJxrO@O zpd(lW2JC}Iu9#**e?@mUXs~12{v%;2#SImdf!_Z6g*(ROP5e3 z8yeNMn?P>MKq$wdG=bBd7`=QELg%sX$(Q9hTuk4h`1f13SD)EI7~|UQ@vXzp)m@3R zpPS>jWdV#bmvetjb_BsedlIBB862BgdLV+*H3LNZXs^_^MP(+c4;dsOkqdTCUF$AP z5d}!qp7@gV+e0!LyWyiGPk=&S`Q&E0V!q%$Wx?H$t`d1ZJ3;~ASu$9O_#F_@_C5+X zHa2$A{&3em=wjv{L~nBj3>zId@w5wM@59f$P5bPJsQvwU`tAFzq zf13h$^G@!ZySim8&BgQ{4RPRH{*piv+Hn}=^XV)-vI&MK${a~SCc|s|z zWY~+iPIW(dB*9QNH|}~iL^GZ#6xu!7tU&nC6EMQmIkc8Zv!7-M;xUDEf&g)6V(U|G zx(_jsxJ>qT)wVk}$?W9iCJ8r7rw2^?uWThK@8NS;_`l3mxhqP zwn5`0pyS+6+h_ZHGfIp3gkmZ?P|*hVE$r)f;f0LuS(WqyDr4*Te*TZPz4A1{li6_| zW2NcSzD(oJYIU;*?X6t|nnYPLeWGUb#&mV(I2F`EqGhK>H3YrU*eHt}Lps_uIaCdX zqLc9W5NxSQUW$Lf1X^_?jp>9H0IR@Lg1-<9w0+q3VQ()h2fwu zlkA!Cm^2;aGa`pJ6rnO+(bND4P~qv2K=Kq7$?2?6lWYazg zSl$bYEGpk22T}G?RrrnP$M;04;MVDHrE-MWSPa-cv<-`gY$$Y|C_#`>TIBQmI{W2? zsy+J)m=Bn6QZ)s;)v0o2Wl{P8LZb=65J4;(-7!m4_SeU`pbfa~4;M4X)imys-ZB-a z#nxHfPWbb?#1%V-9O=NYqJ7G{!kM+!exejH-g;99Qgl3P*$iBrS&kHBd|@M6XYeMi zG(`vloa!$^bpn-74-uG628+9~#`GK3$QI*A(#dKKhpZCX7;{SaGC~lc;e0}U`GrHH zkqo5Ph_!vcK~P-{9*;?5jYpR9Tt!X#)9GUS(Y!~fpkPT^5XKT&NAMH_Rlp5}i+LB? zz&GEabNKwv;9U$bup)f`cIw6Fq@DXc(*hkLeWdI2p^)fS1?CK(c+ln^l??&!(gF+H z!bz`QiOx#W~lC#sTWDq?N{-VB# zdX$+ex1-oPFc}MN7WLce)rf}YFRQi3s;jsOQy*7-w8nTd7_Fi%G7_mSYd@HzBFd3F zg+Hjw5Nw?bhXVg|##@VY?LEAtppaktsEG9tNayP=WLhUkW|JEltsad&Ze1xgfd*&W z_#Rb|*%=MR{Adn~8404;i3Yk#!`}gwSVaK-8)}cA;rEP06ix|ptHZe0q~riF;WAW>}*mp`wzje=<}iB1U8fn4TTSm4yc49q0nqC zT9Uhb};ab~c^cn74@f_fnp?n|TsFhA!Aj9FB^*WB+y?e3i1t1sAm{K>+I4 zbrT-y7igk|I@GuKO#E;6w#S_kkwW>4lJ0mRpu`w?ZsYmkMG5k}<4uS(L1;Ck~ z&jW*z+2S*?I87|$oj_bqc#N9lt*xcYv361?9QdRx*3E4bC#@}LU*lwb8WD-HKkE^R z1_#ojQKC^sP)7J!s>~d6NrGLzM6Tp}1v{#$GBp{}2<1?iQzLtO-_NLfnfE;vXk3OU>q@i4qa@`;927WCLY@CMdf7M4vuBo)*yj~mexVmenYR4TH7LsrX1hu7GL?$gN>`p0O zhafWE>=zLD%Q+FQ8s>}3uOpoieF`};qtv=G?GcbvucJySpou&>Ux;4pRd=(fFtAd8 zc?8ap=##(joz^*t3OeE?S`NIR9S1f$blE^^s)=`8F55;ZN{=1{$~HBwYj+@#V7~-+ ztPUo7wIrPsUf~xnRo#DHWHgj|-Nu^Y!cjjLc_wK6Ka9O&bZFnU?V0R2*|BZgwr$(C zZQHhO+qP}&7&}(wRJD8G`=7d1?|hiwS6g$nHRl+;_uqKeX-=9;_Lp>@8t4=?E?Qu# z(%06sGOz`|gr%2#lt%OTy`sN1aeKsmYM**si-4G-4vFG5*-DnOVW`!FNAtax&~e<^ zzlA~EXZ9t@k-4wY6=34uy8pY<9$6bi6(YCM5I1@v4erwyy(4lcA&!wDvay{xal==j zTv)5|#CNF9rwV*ClqY3ehg>Se(0+6QwOt;XZObZ>HR!oIBk>* zsNPdqXOJszUXe{`oRb3X$OznYb(U1NBJ-G7Tf`}#Gud{YPuP(+q#4Tnt#>dK#p+?6c!_XWO8M1G?884_iRh;YN(YqN1gt8nepBur_&LE8Wpbo0?AY?>oAF3q(PY>7RLRD(0AecgNIcPlc&9EZf> zIP+qbDW_|6s5Ci&&hHl%$AD*^V(S%$nxgUL$}c*w1|Z7*w_gfPl9_R-<-3Tm{oI3c7S%3H`kus zZ@mbNnnt_SX|vkxm$#v15*wEBeO?*NhE9*7AtiCw;5_Xf#8gJU_^9)!S%mPMNXRA0 zL?P+Rq!*gM$ZI}eX9CR{+A5qUOIgO4qu97y-%BC*tjaUDevD*A1?#}AYbO#GOq=s)`PACPSiCu2@s4>|u@19- z1Y3EUX(Yfu{$%mhUAlvNK0VS`RZ7COgfW>`KQ~T`@wVtcV=nnt6$N@*#z9!m_a17RaGi#_R7POTygRAt}8!XLePwJb(sCK?O2aY#@7^ z*wn-FdBoxHy^900YtaPnXt#Fm8b_7rY2DFdY&B=7%%mXOMA}s2rG9<7uQP}-P0RxM zir8E4Ov6suU&cWvgDZB~yU?{Cu(4!&eeO+@Ge*_B!aq@vAjr+phH(`qIAI$S##&D~ zup_ZW-BWcj0yrpvaUM8anJ`_sKJ3RvJ-xg)ZSi`#@jaGZ_XhUn5d^;mqrRJY#78*r z5e==U)VnO90i__pTQDw0JY85L#BZw(*p%MW2ku$$-F*i@%VP7<0kZAgo!MJIr5X^q z1m1W?(Q`+|X`8o5l1ZZ*E?V|ja7vuEjzy;@b6Ka?05~x<=7&c(2YRKQs*b~9wH;*o z<|e`E`Fq?YxfhIbeFR&EtBERCak&4gs{+Bj2Md)*Ke_a5AJ{u7Gv!eIB0lsxEIVJB zsm>rDoM7K4D1w`5843W>7m>NjCbBt^n$Q`p}8LBp=qCS_C_9tzXfa zKdLNFC5-biMTe_@8v%(S(s)UnB6W^U@pA>9ghdCD!T65cp95K#Ame84(QC}C^{EcB zJjq*b!kcnkhm7b>63M5)->Et@FM-+CyoG2T)eDEO*{d%B+em}bTin6FT!Kmgo#W#< zV=N8|>TE;dbB6n54P@vt0Ox_5hWM~z#PQ=)ICYJa^H%m7-P39_I$K&6#qks$xb!)g z7=On&M*FgK1;*&Y?tQwA%o?br8}wb6@F@>IbG;PMvlAT?deg6aF)Vvs^KI1HT6@6K z`V_&WF}f9rFjX!Sx=Wdsy0IFC>>f+sJI^YC-ep#ttOkb>+LdemX`?}%otR+mMxTgA zAfNQ$m(?Brg~}_=F8Ref)T&OpVR0FfoeObRbv)I&!4~aqlkpE@M+9s0G0df0QE+lE zXe*EyeG89NWzHzz4|2j37%|~vGicn{InM;V)5pZx$O-D$@^9CXnw0_<5Mfqa2)e7{ z$nrqIP7EP*C_Go#qz5Z5?T5}*ChZSNhf3-UH%xn}VL*zpbyp%)3ypWpCTb`&vprbL z%c_FX05Kc&iPQZgLAhHl{zw@PRosKUg`f()?KITXwXjiK;e) zrdyEu+GxWU#1TZE8xOzJK1=kA+qt#`Owo3CpwSN#Heh-^rtc(Ne|&`Jd>+nn<%o%z z6xwN0r;e-yC$BNT7c)_w9Fb^1&DkaaipQBdw|-mK=#;`ejJm-iafK*xxE9juxDcio zmA4#L;^8;~7A@Hg2+ZCu6q_(q$t^CgD(3c*@t^ z_!To40HHfc*-Y2lZ#%7m3#!N&Qq~I`_u*@v8Eq2M!%e4gDKo`@bZuIr6k}~qd*L!o zJ7{U(hX?(@3H1Kjh^j3!&cV*%ymN)yi|v9l=H-Q0RHvIpg3|dZko2o~+x(K@fUreM zU7E-x{UFM=8yONC_*Y?5 znxGkg3mfcRiuNbUM?u<+%#Qot5;845+gp2ij;<+~sY@?@B*XfESqT~h)*#3&=T$ou zcJRWyW-ot}dEXl`ZAL)X9k6=8`)W{P)76$V-frRqRE^5B4l>^dMMeTBAN zGis0{G7k{S*9^3aKTEd7$q=3eS_)w5nHN4S^ zikUkcGT6B|8Qox8ObTGf*HQdXAExEu=E7h_R_zxD(i>L@yRy*;6xB&7KP1PN68>E5 z-`%BaA8qEC)>J#+%bxCCIytrbdX#Tr-DbQ^-ZYyCZx6Y-S%WFAmh(0F^L%`%LIdj7 z1&Yu=y}|f!hPIWupASp!YKIHjH23?1=!syzuliKKxP{$r1)qd#RAuZ)BTo0rRTH1K zno{Lp#8w#@Gu=<*Pha1sU^wVWBa?h|JCz4A58)GJea+Z6ly!>H(+syb5zj@max>D2 zdVR;uiFCiF?Wz{&BO4jCxy3dLRM1cilYTnClsh#u&UJKRHWULgNz&--I}{xx0e}54 zMW8PDyJa8-GVkz}L%G%ob9VNjAYF)(>fH=mUZEax4%xu~#OE6=Fm7f`Cs*HDi#8HE`Cj)$Mt0e4 zs(b%n<*Bc918zw;1Cs3lPxL-{%k(HYYISW8TjhIH)4Tot>TZe(^vwK$JxR;p$JAXE z-%Ah==XWA0B|LLJ1tWu7znSyV$23PAk&1Q#S#L{2W$_ms+>hi#o93@6hA?vM=0z<8 zt+Ru{Z6DM6`&18*oHyj!8t z*7^K9(oL?unasJ53^ZtoR|&Pm%58Ix*(KV#`C7bW7Q!<~`gZ+ie^IqpV6RZ^AX{&3 z$S-Eg4pAXcPP~EO3bYUp!07chh~36R&UKUNPo_0y=k=GT_59cuiwcX z!_l1;kKrXY(o@zS=5K(F9C-teLs~llkZ^q*`dALhcDT_26pY|K@eI?AdjaD2OFsb; z6lO6*hZl1Obu}d9fO-t`M5Og>V`>ZzD&vREdj#!0=%Mfh?fsQ9-;#~diY0!Il0W;w zl*~lA_8w8iBEVlEOsQmev%)ov+!%;-BTooX`!~%Yo2~|L(m7+@+=2ai+7Vc{W6&G8 z?Y43_7Y5}rwemBpvDP1XAe}RWgz%2ODAfTyVvBbrn^f_dzL9yl4V4(Ne1mi51z{(# zP@*SVYk`NnFiUp~Em;$DRW@_XTgxKGkJ3l#1y)W1@5r^a$aRLGGPW{P8iRnn>e*$A z$4q=9-bbXyp>+&^-n>gskE(zRxTpzk zYy97xhSL2$MxmIsB*i{B!h*OQyRFUCnn7a<=k4bN>>feE=Lz@l1iJDJAoi)d^HnPy9ih_bKe zQEGvXbR1!I$iubIWd@YUsv_KEKj_!$eDPS~3F8=sm~erI82bf?^6$7%Y|Pw=wwf=V zYP8b257Y{HnCxk!37R5Z+mw9X5woRwBiS(^VTwegGGY!=OqNPXIK52=LBb;722l{< z0E#5=Xo3;}0zyNTy>O}0^p>Ojn89igCvbLIydI(=mY!Ov+7e2(^USHb5QF~c3Mc)- z9VW?gAyfkO0EsJiywN6}WR^*U5CM%)JbY@BMh0vxSo|p*V2kL#Ur**F$_w&~-B&0(&DTh~{95AP1L71o~r!0&>=aT@IIk z+dgEPkKENK8F}|7xlDxLhG5oO8!wS$jZd}9vPpXlbY?Dlda91+ne6Q?pm{*;G12l@ z>Kb5HM|W0GJy0OqS?~^&f8u7sDwY-Bofy$!usCTbYefgLL<6ZFdS{Njv*<`mX&zl2 z1iI7%?Z}9Edm*XV0Uo_pt{vMoA7%G3fu$Z;^lJ%D2=)r{Zfv^t?&$ooF73HrR9hO* zNEy~{eV3TrWbkt9mMBJ3_@vI@KH4$CoUS^H*wnh|IB6ZKn=C?)B0fchNAt_9IJCYq zx49y5*(P$`zp?VGeQL%u3H?4->so%FK@HY<=(@Sxwd&&J?&;d_r0sa3Z7}oL#JrPi zbO0xpkgaCG4lgCb6;5-XrcacB%cD=SgG=it0`D3$$ivomN+f>GRNn>O5=5f4uT={s zjXEDD3<8rBCy>dFYv)};C5W|9E)00vJvjZ`59Kvnf>mT4Wp2TD+ki=XVQ5y}56JNthd2L3zW^PgmAHU2w0+o5YDh4|4> z!^aBbh}9~+$|Q1WjlJQQN4%j=5RM_t{221}j9reJm~gbR@bx`>~Gmm7<9*x zn4X75h(fF-mNYzd3OJ0Ps1!%%46k0yHAOVsE<}opZk%rvzQ-f2h+;dj|Ik&e*j2J# z5jQS6?&yb!b}zXt=LC7MhHLmGv+utcVZ-IS~-ysx%dj2M;2kpc*S( zJ_M+ZxCihaOhnQnix^HENj7UV3y_ajih@*?Va4^J+w1PHE9H~3_iFUIB}}fan3ZG7 zu%}Aj6_P|kqi{$;%oA&PE;(|jQ@Z#HE+?~#`5Tv9A5N$ocoo1yJSm7SS*Lu_GoiMQ zA*|IH1s*9xTk}x? zKoc;}Ga>pm==jCiXJJ2G2XjRsT9zgBwQgSr41zdoXgl9zH zX($JkP;j&a9|%7 zak|`fZ%|@|ql5c}Ux)L0a*qmrPe-8Y+u5|j+2Ejf!Wu$`>A&i2Yr+Y(8zaI@v%p%9 z`_+8?qw;I#hClr?Ov;bu`%$7ao>=OS6)p3BZ9?`K5&*AbfETZA7}Wp-CM*R6`zFn1 zK#5`HA=ax*{^t|H zNu0aw&6n7(P+m~n4xBnn_%&C6Lf6r~6x#JLiwWn)dd)0#YaVTTfn!VUyV zh$?C=6?4NzCPylgiUyKdV>^Xq@`DkI%?6{A5d)(BWp|-IaFcea}Fbd@el0VsGmYN;K^gjJnkOqpLy$9IXc-F zZ(C-vHuuxC6p~B=2;60eG#b-_E?tv#fwe?yl*c|LBm4Hd8QHd4>*lplOvH7&BL5a= zq6!ckQMZ&A#kbyJM~0;)bpauux#dnMDClSt6HeU(>#qzPnTO1b|oS3Rk$_#0BzL0@61 z)}(B8Q^bb7G#Ka*(9P;MavREV#UBr9N}+r#2jI+fiBMk+X^yBU7~ZRynQ!jq>2}q) z#I91Ip5u5XwIp#QLtbQ7++B(v823sm^Ydn3+rC*#ol72klrZAX1YmxX`lW{~Pv`!+ zFRpB~w27g2F4u*ypy0z=U32v4_@Qf@8KBtKVYo)iYv6~W@YdU^O8M1{7y~cTzWKG- z0G8=->3uO35zQpX)~|+qPoJBYc8nWXqv#Bc-@B9?v(Z)gNT^9DX2zbsIMlkXaoy6E z(-HPwAriw~fRjCh7pVa!9eE%(6T|Z2szqLGAK&D}nb7UH)LLmN|AB&X&fN_2BGEdI zZLa(Snr!`Alm8|CZ54aj+H$!X*ocx!`E)hZbJ70!Q}9FW47CZfo9W>317|IT)6lq) z%w*TvPK?7wjaP~wJM#$t#`q*PkTb-kD)JkAP;qZNUirC&>J{{C%A%MYY}ft4mj-@Z z{{DEGTA3bQo#L=5`p`{E)=AiHh4S|NqTO+h)<{Tz{te9zG4j@T_4)97>paRD^qnpC z<>KuIea^d$Op#Z5M9(;C+16S+ig3UnsHNrx12^GM00yACJy0EijfC&CJ8>|bxvi7rU?WM3%N z5=6v0-Mbr)Oe)hs1U0F48VM(P$}M_B_=iXyDIODOi(f`r_=bP+>jDQ`>#-KxebSOu z^(14a&E7Fiu0|{3s10j9FUwpsM&fGI;wkn|+2!%jQu7e$A zO89XOvUD0?y~G~I3bR_ZP&lJp!Etp2Q-_<+)TJMn-IEVivd6;k+U}C&w=;paPAB5* zha6=D#PBFb)8t-ws>ZKF4MuAvP!wo&h&)6AF_QLXr-+Tt+C(2LETYwDp3q-^z?=Hj ztoTv5J7`*9Ze`_)IPJ$xR(gx83+*XMGyj&<1Q=x2$%Q~tRPd4FxRiVpMwe-AO=r^J zOo9qi$j9&0Rs}LlTNvg!D9R5AiIH1n5pjlN0N*i!1}6R%@PB{3LB_Sh&-OWUZL2=? z%Y3$YjdCLk{#-~Z5LovgD$ib}d48Mm@{DLnF>KP||=KhWs^MW(dQ~QNl!i!corMvzyKe_2e@Bu3@EcvZ#$5jGvNQ{bX zC?9A2mW3$7oKa zg+3BqXdr3Vxp+eS_OGusY;P!2P%`*_k54jEL@IAHCs`HKi+{$szrEFoC^%-5xq$|m z((@M!*crfr)`gMVd++`0(R-Lx4W6B6DYXu1uy*MI14c!@o=2y3LRAVELbg#9#E?Wf zpGAciMF@&Cx(={^5jDgfSa*B&2$K`VKSF7w_~+W|YI>YxSjtkezV?n9#+GfKv#YX; z=eEODco`rWH8t%3+72}RmkhbSk<0<4TRZ1BULR@_0!N&%dpRF)Z`Fz=Sw#iP7S2I# zl)bUQ+JQ8NVAvwl5%)Cyq2cT;!hltBBS^cX8$GDld$*1)prjK}RAmyqtIU1SBWVLQ zq}B|ucCw&rA&&+^1OXCp*ufE~#0|Qw5zZl%V6B75PTUa++513wdva9{S$98r5_KdE zquzIs4d`7%RnlMgdCB=D9+*~L*BTPN3X7X-@1ar!%5B-9CVFO?SSo}WsOv>qrcj>M zl7k6=#a69mP;KRffimb}%|tSJcBtiTjYV`tqeV}@YB)jkveD9W#4T;42{P(Z&K!*J zEMb^+*|W1fe8JMDK^3OI@(*#?ci~76qr`U4;IKW%T>e_ci9OzMAC!hnxr_pjcV4)Y z8-1&-nCPE6GX-m&!sI*RUlgUIIOC)PIyY!zGg0*$c&S1|h;gg04iHP7g>25Xt^|HU zh{7AdhxWQo8Ya{opPw%5qCN%7rU@S@1rSbZ)+&W2!HJ3@zmR9=H~50B>l%< z1TN06HHs13Q3;l|&=6Dy-hqf}h%0t~?X@?8g;`?D+2!{EF$IEVn|yxHmI0EHO>71( z*~=wxE@45zVC?O=-KsZUFs?u2d?S5^pX$ytPub@H8dQ=Ae!vp8|B5e0vq{=AdV*nf z>s%Gl+O3=>;V7f6q2#m70e&WhB#OGv#F7gYC9J<#T?k6Ae}R#Vy6gCQf*2-MV2oypQDgCO@u3F&&VJU}d)A}l(G_pbV{MT_D* zCzQrZ`3Ma7x&0H$!Q$}xF9XG}@bkjLi+L3??ey8$Tz^`PWcjM9CLMOeU02XC_x@g! zjRoSsOwRff>v^kli}2G$l(jH)>1=pR=@AwK5yjAm=uIX@3{JvR*_kVQ6FO(%OZ{?T z*HA5$(U*auyPRzV(fSK7op>&tJmek_A`88IF6u!S7;?ey12!=CD0Xv}LadpwLl(~J zM{)DdEyr@w*c;iGUS@Brlk0{}TCygGCJ!gRMtJY&L?QpRTmp+vZafUyBUPbcN^|SC2DM=ME5k+FDtp~^Tv0kc8sfR zQATg>JH5%FHmCKK3i;J@q(hewhL%Zn8cC2U5TceYiYkg!K5s_?6sb_!?3f;Aa|{ae zV*M>XwBH~HR%F!tjfjcGm75W$#J>*S0u9nU@YJT6|C-7DUJtOzusmzr&qM15v8h|V ztzeoN+mVUX^=9hPg=>KwJF;&O?w1rlzm~l)-Z$0%asJ#L4Ce-`%Mq@4?0S0Xgfc?y zIvM0EU$4RjR_bV8BS)X&WoHLqF;sN8_b?qLtgomgOU;F-2P&?`Y0|k`wqzUl{xEpl zR4T1)0>1ZxNvE%F-i-h^Z#>Z|pL%^!F`U}onMa4`T7iQt+#+^p<=HK}aJKiOcE@eU zR}Z*JTYkMpa!nL~{EpbgzbUdY2=o#?TP3wn6dwVg1vNX1q^2_PD*avN z?P$(D%NVMh*xWxi&wqG9*6;MJdiWValVe2HtG#hhXJxg~;I8Ju_0XoWxk+(}RyYc%v*8rC%IK&)d$CZMJYgzHhLB70O`kC8Q7IG&fIWoh| zom8WMPU=>q%WkaLWxy`5kmz^zuO{Dg(Itnzf8(mN1h;Cp|4hc3h5w)P+T~YMXR=#fSjDq;`F1F>K#I5wLLg#k3*+F&`&+_q`vkzC5F6YK zupxGyuNhY_8Wy8~RPe(p`DlPBcxp=;Bnyg+TO_(dNML=3n6c!eDtW(bVK^fb({A<- z9j!#-p8sCXRnq+j~F#{dc$*XU*Hyq`J#EC-6?IZxS#0(kvz z(+%ZuIl@3%4b_Nfgj*cq=y+%VmXPd&$Sr1BlObO5>=60e`rG?C210Xlz52xyk`Rh% zM<@q+4{Jv}hqyhe2C1snyw#y(X_RvWbH0U^gs!#R;eoS9Y_w`(qN%}rAQFSo8$_v+ zp3PvUaBcNxMukB|W5;8)^rl%OIDGhhr|@cehIP+e=p)FAHC5vn<9rprs{>}j34P!u znE)xNSwVCOc_-U(Vmcys0oKSk>wMo1khyYB^J~e;5EK)RDknp*asMhpYSBXR2+LX% zU{RWpM;KUMkGdn{=lR zVJ7X;(yRgt>i*0kr0e5b3}i<@)WvE4x12yP)V?XIfC0-5`!vep$(x5V%_V0INOI(S z^_*K!6RJL@L_Ti1&7S_1jj%p9Bd^AUp<1UIJ#EQuS!C?#w>Ib?YV|hjYAm#l z%RCiTrm-dmr&YL&5a7W2NcB~BRm*5YX3mI>w`=e4{xKuLE@~b{nV7a)=q#Gf^Jaej z;i_GxOI31_{=0X-lu|17g*>4%5ruWV>@U!zrvy+InYv>*ZlML77n&H0>E{{yS2*#) zh+MSsdwCUmq~9bB=q>Ep>;75=NhavvWk?cjUxIlGU%(hE&kfYKQLF;UH3PAr&qH>y zTUYt%6Wb|=c6^o{oGF7(XFa7Y<6c2XE%;hI0Wu$rXIQcC?Ih#SWKZDDW=O`deeZz(EYI@y;My*KGS+85eO~H+sq&i zWERP>bf8H->}1|(`O#kjLFOw!p%^|ohPF%7piE^$qfkwXB}_N0<&kxG`hL*P@`Deq zTyDk+!_v(G_W-CdB(PNSU)p;Y6VasCXB=&xSq!PSlVU!HmD@Y;We6(gZw-mk1Bn%G zTaFhXnGLq=@S7mNf1bz=K^05Z-6>H4JjCV5$_?%jB`5bG6hr%qDnGc4-r;;y@8b%d z#Y>JB443qGNsHxfBA%Ql5A^hV)j}_e66Q87Cpb%`GiE)nosE;ne#xYZP=Kn4gt({p z8{tI+;9jDxQzu3!=jBHwgzKxuyIbYw19>dRXut~S=PYY9E1cA=lvm!=(^7mCdJ~LU z=GDk^;^!WJD;VEe#Uj1b(7evPPCTao`<5~@<&Lkhp#$QoNI9`gSZe~z4MsO;E@}I6 zkrVUn+}$Kn6i3&qIk3`i0NRuf(GmftyV1B6Nv9e3n1>Ytxfn!pnP;?Du2`g6ei8~(w>+(zHZ z+~YsPZ7!?s{uKN_1QT8SE!0d9#LR&_IJNPWP=406j#Bv`t6a^}hGdC*f#=m5r0=AQ zX7Ns%YFq?y*N*pV&h2p*?NxONJq!f`35OXXcV5DIt}M^< zjH>yuIGXjmUArI~x{SEKp&OEQwFqzja!&45Ay1F#rP=8vxNIgJvXN`Z(j)v{H56As zMmE*Rv&nfQh3Pk?eo*ja+nxdC0E`*$1c8{B8c0_-YC`oO(BUt@#2kStW*-qLFZDB> z?r@)Svb5ko=`JpQ^&AQVWM>Q;S-D4wKK_#g^>LL#dIqVM@dWIiUEw~L&OB}`V{HSoUgqX@3W8BItilZUjix2ORKAx z0}X#FH-zVEP+$b2*B+tJ*aggPG89HH^k53rflYQbf(?G?5dc+t#qKY!#sC7Ohru}_ zT2DTz%xOcs6I=ywX#H#*)CG}gcA3=x*U$Wjy&@nGBSvncUW@q1F)udi0)KOUh`$-= z%})}3k_IA%5fRhw0J2!pe!CS{LtWA%65 z%3TYISLQfzfI!I+>lmLvI^SO+iHkj|KG7p6qCBKTSE7(0Wyr8Gu?04!=%S~x0K797 zZDO^SEJI$jPV@vUnfNYfFsS;E`W@1cEBI~p?)}}Fkuv~$vhH%3&1djlR=Zm7to?U* zE>z(*(ukTZZLjphfh*v(5pI-gbvn;crd{}Jr5;&!-wX0u+o)6jwd8?Q(=Cd={@^#r znt8KSPbQBS0`I>Jo&||2fA&^4F03;`t&(jE_O^!HbRjN64se zEdv!=O?oUtOpwvP|4pBBji%gS{GpKX{|qxl|L4Ek(b&qw(cIL=*oel?*!ur(l4*C! zI!vWl!oMIZyNkpwbj9q{#F6XI(=Rc@ety=CEiiB?P=Wk0|UBQCkuc|~CtugkXzL9_^R zdKPia^4yjJ`~n@O3Cd{lfR{dKs4O7b*Bf#b#cf_a83eoQ(m_`Bj&rKA2s{mnia}Wk z8}SzK@(??1ecx2d8i~_UWei<+q16?^f2nnr%!(!F21mnMefGc-5JPT5aBo!seF_*y zK4-ylO5T9zvFF@4d9Ko)WL9&!4{S@eN$?vrT^kf8`N}FI22H%K-2a<@Lzyz|x+-%^ z@AvUC5i0__wt^I8?uPi+g3LO8MTzL9duq6c0W5w1w7TS=c|Hak`HjH=Im}Obec4|^ zN*zIkXQD>o`CdlZSdHPfG^6yq@xdu?E-LAf^)_;{Uj}_8RP1TU(M_G zrDnw|Hbe{V3}ZE|^$58Se#4seSn;&RC(>?MR!H0q?F0O5B$v!(*wuG=pI zXIJxRn6YqtGkZJKteW@TK$ekP_BJ(u`?;aAf+eK`+V7(bNLb|J6=CA{Yb%W zL8tS`UUofG{;lXw)O!;Ip9*E%O7yMQUk;VsSwny#>S*&_kF0SmJ1$dxybEZqZM1*{%cH zxoKnOe96LQLhv?3^DD{3IYVh@@UFwT-{Ao<5Xde*Z#Jt+kHS4i2!XJFywq}|CJN0J z5VYcavR`Q^G!kNp?3l$^U8qXnn=NjBTLtd-8V=czrY9La?W!xBH(5uZ902|RC;WY( zz`KisQ;JU)AE`|tL^dD^-T7#?63qi)EhL_@8jyr2-!d#El3qF1Fgy>v50@mpX?>ko z@aUy+OhG(~9XY~QKct!BoN>S7CMK2=>O)~y?|*@o*yC_gczjZujO`p@HvaZfB}mgx zc0(^?5=jKZ1~wD0?}D3wlAWhWNqE$>TE6?aN{Q%rC;>;}JL?cA7&|MT<23lfiN)+y zT>}D@BBlZ^3A=b@Q$GVhZ%a?9=vD?J`T;t;5ISsmi43_Q{ zmE7(fIBV0SzliURx84X#7~0A&tNuArj+y25OIwO_d>CnHAOPH0!R0{dZ|J#TY)1KrD*v46)RToB#c2EH7e;8RR#& zw*B`vx{JobTlzFF+?VJavv#j@*y*?Nxv%<4>^`355agHF!%1vzw(2L7V6C}E&f#}? zxGn-aZAXuT;!CaR*WZYIGJ%r^F0)tU-0kXN#8@5jyp{%K3#r9R#(rr>(kxa$-lj6l~x2V zBLo@oS8zJu=Tp(sl;O3vZ$K+3^rq@rPJ=NyTP?ygYUuUzWZXD6E;L`=3D<6>Zdu zB`)R?Q~90-Zbi3qq-1~N_Z&4D*-9$ngcj!;A)mw_=iRErbvL>~e+CNVTu;xTGZuJ zsrg_1(|jJDxOEv9n7W={xSN;wZ0FsKwV$+?dB!zM(k%|{ytpuBb6o2O%J=LMPd?U2 zrvlvG^UMjTDLnGz0l9I1gRkb1Oc_Clv6xVgc=^TRXa5S#GmVIm7e|dB3$%6fA1cEO z?6hfkh-N}#=Zl>4q##7DH)IOE z_lbfv$@++E3876b&NwrNJ!YXY<}*GLvfYXK`ErYokwH3Z=n!yRH^OSFo{%CVR$yJ; z>cWuRNjLt5%4Kn57xUwePM}72;%`JnpSB+4Y}IvAKYt1CTDjmw?rad~63(5mdEi$( zo15-=1|p&D}%dW@KIlN?#O}QN_-nxG?4SWi7nWk1fo9`$fvl!0~biM^E+zomt zn@=&>G`K>A6$ZS!sskJeLdmIL$hS(NqlpAZ)6G~M7i0#U`L*FT9~6)CNL!YTypeju zjSqaShY9*@WFIV6<<3NtcO^eB9oB=)++7N!aGubWmjs%TCC_n9b3Sbq+0%+K2FY`O zlQ@YZ4tgY(5ZpMLRv{F;(do8@if%Bfn^O|Qb3BWL`$_{DPq0XDGLl!}M_1A|7Zp$C z4s%{m>ae2{>rKy!G}6&tf3A{zOkheViYWh9u~gnD3B;sh1au)K6|J#FT9u8SxMD3~ z<4k-!6wdSIzn%Zg>9}D30rtPP427>!j7Se?P5OtlWBarId$i=gb9S)A8CCza{>>1q zr~+Qbbf*9=CN(1?C3`GEIWemo7As3hO-?Nl9jiJvMLkAKAulFHF*8d}Egl_ui$C#K zufe!ncsTvlT}$ikPL4#blv+v+R{V!V_P-wMUl5>`35?ht5C8xejQ{Pi{sqvsvvv4C z&8ApZlZ^eDP4Q!jha1;7?t_oO+AxquJ);z!BOr?d@c{o7Oc}j8B~c`x&{^r5yD!Kf zEZ@Sq4O}J@!t`1C-G+>}263z$Pcb|62(ZxePt!JQvr#!pdZZVN<`5AWKff<`xO z0d^$?9@j9A55;)=p2kMwLHva%%ZNFb7dqb=Mc?M`UdIY^@X!Qf_#uv)peB&5p31Q}K z4B7&*HL|o=76J~a25tg(+_`BJLs-3NE9q>t+LiFOw2&>B0{W8YE$6HHThIXhxJ8FT zT4AWIxpTCptd^5xX5XA&4}8@qO|k?@s#3`zg;)z*Q9N>I?cdE2826}0`o++M+XA)_ zqX_=+)d+U=q~IUd-U2qir8z>;1Tyyyi>=~*7#8@93-+s{$!TU^9%dEk`-=$<@>oHG zV=|sjvle)dTTA%R&N~q-nzJwA)QgCV@9RV2W9g=(Md$?4*xDa@7Xs-8L>@=5Mt?X4 zh1A*;g>8TKo7E$9%G=y@mTqPa1WTqBfI45uoSg1Y8Jyk8{?gn5_!d~P#o<|&Q&JFtB76Y7po8{ggGtY>Tn{~8qW_Nox`pQIg3U21g_z|lM>1DHZ zQj#&ci4$#DSmLG(tw-t){NlKW!JS)GGx#Ddf!jI)gF<2@Z3t1rgHKwjc+}w#qY$($ z|7nlJsR3xSisrxmZDbJ<$zwODn3`hxm`KAp72fiC;8UgL-h*t?BMwHJY>ZGr@n~|4 z=$m^Y7jZGuLDYv|!_>hON@qP8)-r}xA!hG{5LqX)vCj~G`y{?!<=bL&N6D86S_xLr zmO-Q~ae>t7Go;P7sl9o+XDFD+6So`mV@9Z9$1T2Q8r!g$6T%koVd0F;6j`GgJj{+U zkS=xg=e^SA?$d(N`~1PyO&w#v2q7_<(A zPaAc|>{W`UU{%^2W@r0Us`~rtGC3_fI9O`F}1F zM=YT)4q9ZD^T6eD(xKlbskVR%3{R3|>w=@Wmcwn0+NywuCKY?R2NaQ4{U8}Np2((B z8;#f;mAvE1Ftl<$+Sl)^Vnvu%WPJ!uDRI55q%U!#zsY65mzSOt1*G+5mV-ruI)uAb&k9N8>-%rx`i^t+5W zeGx%oV`2ux3KSc^16@fxfNq!EmRX_A#wQNMPmLg-BuP#J)gdo)*BrYRW3ps(RAv#} zd9v1|zDyPWsuiKMJRDu!M6P!pKB#xv(V;c#J(3q%Ijmgex-7|NzqcBol&*6f^ zlq`xweY;6L0smm7+YUOHP5*fgD83a2E&eo67vKN@;Q#AJ@!ze}|3lyXN_9JOQv}X; zRhM21*8)|uhDYs&pEraqk>Dv#HB7YHEj)O9Tv1E!$ z46PMwQpxujKBc#)aLuVB8OKf10P0(Xyi~uDEvr7-fLBAc0ywWF<7}eAy&+cbh zXchE(g7-DTN{+_JRV-JAu2fpvR5`Q&>9R{|nUj_w*^_dBnhw(V*Tf*zjHt|u2bR=O z)Kj-Gdw*)4ygryTdA-xksO-phJg|0D$t02#ENa<(gHt=i4M^AQ8Q7j_06oixs@by_ zxPosBwjo;12oKk=g7~X=b?Zl6?{+Py7m8*e=jH$dGv^vgx_Gn%Or~x{7IPvS&OCm4 zWV++Gp^Cr+w8yEm0=1$_4rWKEH=Ckl|7{RRS>}L9@i5~PopsrTD*aq1b8)y2K3TWa zP7lbmqk=}-OIpG5`2Yc@uMok&^|k`btqAwBrm1$59r9wXzEw6_U>D~9>c14-AAFCD z&<^SE-M5;XxC%J76oe{Otf3H~k}{|VM7zWjl9}AbLNS4HYg`I2l@g-&<_O%Ift0L* zbVnHG6!S6mIKiz5Rk!L$tT<{%hvo8v%;WRQXzPN>&Y20?o$DQAU0g@e6$pZ=rcm_) zZ`z&#rbp||MomH9z{31wQ|yoCKrknrphLULtv1M9A(`^t^EklTWv>pa4ENy@VBmdT zHbMEq9oku7#wF|i#f2b~zVyGaa*RuQLHHSo>Vs^j+XNoBs9y2n0lPzBDq{tU_O8hO ztT}yZv;~;^GeZ&$;3j&G1{m{Ht2!H15y7t~t{_}iBc&?i) z=2spR^2bO>jABh^GJ6qb%$n_mjlVS#GGW2`HZ|jDw0Tw75j;3D>}kNs<0m1TCw?Md zPcQEH_gp;o^C}vC&gl87EOqjkw+snJCcWQeJ~n8>yrn?JKo)o^csug=e{BiFSBuS1>uoM9xmtiHb_&tuYLDG$0Cpr=W;!VlHp z2Y&nrvI+diWB7t!X~&7pf@@G(KOr}i*Z#yJ^Y{N&iT{O)Y*MTef5pT%MgLd0{wLu7 zJ3IU{m^oS4+5UHKAhKju`%nC@+Eq{Q|GlTDN5D%!&Opcb?;-ymn8-GmB5~#LowJC4 znP*k>KRC+&NALbGSK5E6(*Ao9O(P1m|HS`V%pWp3l6|8RrOI8N zWW!-KQ5y-@ZZa!Sz)_qK2oMZ@Vte`L)Ae;4;0Z`%GSUtHU;u64A~)C9P3{QO>u1G@ zjULUS7puu_DQ&KAj6CST-d&X@GrH-6tU{x6*VudJsnZZrt9Xi({s8(*^$|pHqf`Ap zddA&n&Vx^UU2Z~W&{uYd-8S(#seVh(StbQ zC=$&n*c%rn)uh4Hq1*bh^w=SKn|3mj)g#jUkOs7uL<6_ML)c-$;L%CSy;C1?kxF2TyieM%nF{|Z1`7xTp?-hux==mA5x~Ms zQ*os@1#T8_*h3NH0`TNpuQVI>x?X=m4e%+Kgi}4k{HY8yu^)&L2<%?J<-DCf)*)cH zNhgpFm{KQNBq4|=u1;0V`7fwm;BJ|R=2*ji#fLJFR$wnZn*QR-Fe0m@f#G--sf^C> zRI*vrgcJiB9@g8r+T#91vvs>RTriMI3y3}sAlsNmg1dsUN+zMPJ<=p1=}v5p0P&0GM61XXDN}Rhd}=_QnwniKDC?Sk zQLn+;6E=hioHRX3@q3KS4>{T(CClCibbt*BDv;jnA!m~P-N`I|$ z#;jnFdCepGAN&D(_mE9U-R(ngNoj^4nsWo$bC~lN{)4IFiE*Rfo4Y;V&zdfUGH@g3 z$2F1(D-^nG)oEYgWSWe*4*B)0Msrp6;WS%hm-0^{e4vUGw27I>!`2OkLG1>B^W9cB z-F+!D1iUDot;P-2BaC#|NSK%$e=(9dDhbRTC3C00trL`DxX^h*7mMM7b{2AUpF zMfoX5gv{f`Q0pY3*AR8|mP*mujEfIK7brmjg5(ig3Re1mJM_ng4uk*bsW`x?jHyH5 zL2|<3gzRF7-|Jngb7^s3MMl3CP-&9LueybZZAnHd_S9=gYeT`iH*r@)jG zmE3yi`cGfd+yo%s2z?tnc&u0mhDBFANi z$~8R*%AhAgU9<-0*r%z<@%Q2>T=VGs=r|k+u}-p3^RTOyoE^d0wiAyyFOK;KqNCtF z=;*}Or>3;RE&)OuAlBdFii?H$C0BE6k8snA_-=UhXKu^tR^q@l@X-P;u={OFAU*Z)Dt+Ll{VluX=IoL zv6&n2A48ozAqKmhgJP?aYz-KoNO-aOt z+xe^D;qTd zq19(wo#46{2o#s2MLV*zwbws_3BxS>$*$S8C0Hobg<#W7;qT)vVVw6A&OQfY_=lrD(@H7VcYu`X< zm+D$7oK-lU{Y5-j=KGCcoqgVE$|Q)V1qK#Vha9$)M{L9>aP0&X{B550nS3jFOnC1H z3%&a5X28uPauRb`40(t{O@BkvHH zILwnrst-??8m0-$p}4kRW3*COX`O)8672KO4;d-0WS@EPrK0{L;rjqtlp-hK=jaiV zXU5VWVBI;U4Uy~#48<$e)V!M+u5#phNGkPhjTi*Gfd{~7b%6Jm@iSZ`%!Vw((B`bZ z@=V_`qqPM3VneyA@KbyxRlUBDDo0+y@>}?J>2rxMVLVYI!2@l<$;l+UL}kXw-j5Bv zMUNy(dvH+_l&3xdvd9WNk57Cl<9yV_;8BH~OwzfH7xY937BuPg5dK|9H2VD9k{3voe7mP$jXdEZRQnMA{xNd=;Cem9j~xRY zMn$)z^@aqTD0*$5%-HkVV!D<{gupQhDn%$`%oVS!ZXtI(-gX z{Nst%eyaNt6**uba4u7Y!5R=$Si;S3zbrg1eJ`p2x?*)`{>lxtBYuS*x|0RpqBKA) zQnp$Uy6}<6KToY2?Y6_Z+140$0{Uc^WynjeY#^$JRTH33XyR6vs+R4$-v>w*Vv>N> zU*iHhKe8NulMc^^IWui3Dhb+EIm%;CCLs_#3_vY#XVEcRax_5iQjPPo#a&tg&+QhZ z0;7q)_kE6`kg||VUO4(=W}xf>$-32<9>M2;4ft3-COqRr-G*r%djy{3@@c?J!=yhYD+JzGxYLqR;bNn$ONidfhSr};d zKU@>@BZW)AOqKPi@WEnTJUP8xJaKcfb+xg&Ik8_Plc6E~4+kS}YG$Zm^|xUvg!V)O zJyH4c(Muz~TqMcn`y)c^I*xr2&l_W6cDv&ZggaWF6*scjb%f}{j6*M-P}QK&p>bOR z@z{;K@r8cgqGq-YTQ6K_pR%u)=a1kBI5;;mSxZM3g5ZCj+T7j)%>=cCeGqu=F`jCj zUc=BZD$+k2%jKXShM^ui`7@HDK!QtQo$<7Y3)7Ug+YNuUFI9dfa+q{`DX!lB1`mr+YPR%Bk4r zhDWPI^;=2Z{DU41c!KRYGsf}?tJ){x9kSo&&;u94=-w`?c2SV#TjKI`o} zYIWcleBGjWCiY_WbEiy#+}UnuOkDb||HY^8|LN&3Re~TLz&iu@(ZCahQ_D{@>wWa9#LRJ31OHGK(n1bnA%{4d%a)9b@NXo;0Ni6m4qE5i;Hu)!s={A z9%3clz`Pu^f-55}7*zv*67d|kgA7DoD-VT8xTv&ZO~@_tNt0eT3=t_w=$C>!izxPTBTxJ~HM#@g8zxH+#Z!{az_o z-&NH=!?ZgILVIU*-$%HOw=TJhPA3_zWT)tW8jt}!RhAT)-}+jqMN{4HiHgRj$&3Iw z%Qc2&VuO~!!%?Ynp|^<0l3ql@CdKH=ig9w4@2uDfxSc|ca+$&j%zD`}Qjl$-LxyyOnHGZue`^w0_1&^YTKNg^l1e4RK!!SHN!TXU^zt1>-!D&Ppmep zl-S-hXk5O|qkR6$?g#khilf_s-^%pxqM^k$FaEvhx4JAu75Z|Q$HNP*Kn*q7R(GM_ zpPP`%td%l$RIoomGi%E$T?UrT*=QT|3sn|wg5%Kdx;%K)#nko=d9v^$i}gMQ%|2QL z_tPFR-F9!i)~flNcas#=Ws`vDRI93 zhqm*9qeNUg0+x1H=wCyX1n~d2HU6jf;=kJ(rApLl|B3(I*6^`%+Zs*Wcl(BdX*vsj zo)6E-+Q?amZ6ua*EF4bcNz9gBT>}puFAYuy!2n3Eo;dBZtXKhK!bEv z>#MA)s=RuTczm8%GS-}leDq}ZaC(2d;6F;r8PK;^oLPlVXjV~7t|@=A$l6A|kx|t7 zrYN-mmP_J!C+nL@x{K`8Ig2_b$=gLK5kGZlk$%isd%clS`LJ8c$+>&%9Irzwl-SJb zmPsA?E0ZOS?%^~<+ssb%e%h5(Z1JSYmYFuunH|xYq1v*Jsn}HZj^vqxqkK|Bu?3Ww zsUo4#B0*ca;z&7(>Dr4duH=w4A46SDuzynMPLK9Ve10TI`=~mNh(H1!iUw29{^?Ubw3>)hB%~ zXXRBT-qM=V!mi3+uXkaVC2!J3#qEK`)cFOg1=Da}56&r7e`2G+068l_3I)@)cOYW<0wkp9y$1hSMqv99H^jC68Cjb(%fGULI*v^Z%`nD#kb#J%#Z~f+`r^A`4G_kXdk%wR2KutZK z&$QnTC--N^-)-qPuk%A7JosF{pT0%2`@S>a@0Y(D@1)%yUwJ=|X?^Ko{Mx=g zu9-c2{GW07*GPKjNZ_A!x;^XsZ){y0T)%KwyZjs+o<12mczHNEJp+fo%}*a;^t}^w zy}Z6TxY<3PF3$EI-ma28;k_94?^93g9R8xsLv2A`3oxS-`*^<=7kQqvV0ZfM-0d{- z5UM`8b?E+Z?}iR3a(ia)&Qz*4t7uYy=}gXklPb{5GsvKDo8^F3 z{>uhEQ%yrnJHZin@qd9e&I2a_$d=y~0Yu53D#0Y>Ss)D@GP@C$ym!Kj{b6?j8nm~O zIow&KOs%%(B%ragURYOxe&dDDCYXK^^-*P-&_G(MZxvZVcA26sp<-ihQbR1QdjfBl zCphyWP^|$;K8BD)xn=dSTP%%Y9KpynxjkLfTQIcteK6#OQauSy#2hd@VUp zgvt1%Uv+^d&>dqUIb{Ms=dU|VoMPOcpPKaSOQqBHuQLY-Th;qDK)GEsb_SU_0#FIx z-rwni`nYDw7%HZ^W1>xjPe7c{i<+~@bf<#Y{So;6;Nsn#FIIV-Of$MBj)_9}g)$!) zU0sS#73AtMA$bFtv%~-d`sS{XQq9Hxx>rODGu+_(xOYw4LOZah4zmOqja!!*d9ETz zH3q^0l3M|dPm;81L|8D=>x<^ejdOj-Lpla_pJ>?l&2 z#FgCyP^*Z2E%=&RH_y9OpX0s{gK@Y9#33(3b%cGZgB=4TZ$Nv?(ReY^1=d>Fg@KLL zT?K7iL`99ig5JK}i75yD6@CGyI{b%)Y_8K$j5@(Sf8KJ7Zk!+OR^Mkh7YC~ZD0AeM z$6hmJZ=ajO;kci!dvP}6wPc@B>vx$At>o(a07=b|4UGDqQ9_6^Y5RDo0oC z@jtQ_OiDpCHg^RPv&r-w{Z$}GT;mtoUbv}(n=fZ`CNx+4TT%zzsaQ=-Pz!LYAGWQ5 zD^hw*JR|Cqv=-QjkOfgD@~`adtOX3nTP4r~c)DHPo>xJj@?S%HIs|sk?{WzY>;^nw zmKP2*7w5$RfU152c&@=bbc+gVc|q=NctzLeyQ>uGr7L5Pw1!R4z2+4{Qa3{7x{N5k z)N7YqskC|7?fobF_?m`F0E`rh)8ueeYr7s>uEqkyDV(1SLp zMGjQD_6NhJwbK5%n4y2%WZtYw;9 zePc^m(tr})=2MVD^eY*<0}P@3ZOw`@+u#XViO)lu{jh0 zpwxoW-@ldfQU+*Dm%HAe9oW!k35c(s8mzLv&U()K32C0|s4uK2grUON$j%-;$g5Vv^raK)0b+7A`j!B%={ykL zM;Aa%KW-cx1kB-15^9S92nOU}RN8V1@lK^Wn2yr{CO;FF^KE*evsdLatOmneHNcj2 zi3xq|hqoJm3EE2FyvsjqzwfbcMr7mnD=Jka@;NBwjymdY?Cxcz(_|Z98tpbSyZk;- zBe{bTo+@g0BJa*B)x-T4rBqN%tc+8{VsgyLYFAeU#^Mj&3TFTh%$CJ3+SG)ab1fut zmUaD%Wi7n7zJs&@B(`g1)(r3SBLxL!3R=#<-*$XBJ{66kLv9YHagDS54lLqoKgX|Gb@+dV)XBXOgK`9Z4zfAJQW+(&S7KO&j{qf z6Br2YHvF_UOf#dl*pgYK5Vvc_tpkHN6_+#V+synUl%~Af7(1oqX#$xGYqSw9XHaq>3x)aVC1UBJF zO(%v?@<<*(6d9PQ31yqx!^-lK6N>wp^ytH7WXRq-YueepoOm!xv}?jON3>EgbiuAJ zTxGnxqp7e8g(XJYLeQ3B-p{Ib!72Ubvh z{Jbb5UV#GV-ctxiW2Tj{5XWt5iPB}8;Eqobfx9kgw9PE26BEnLWwA&9V%x~5ckj^_ z*|P;xzqFwrUVt39*s-SG4`z$Tj;!b$fCSWLPilwO08`ID;?kk~3OSB>sz~J3}o&$EyCc|NNqFBVjlXX;EPSPE^ zejSo_L?@Pc#ZUE@m<=kilZ+e5t)})w z#F^Wqyw&%>mYQa{(M7{Y0P2pzl3;ol^Mq8fUeA$4aTz&lDXy_(Xzp$^7fvdeEqiBv zuYux?uQEq!b)IZreSeqbV}?m)kK)ygzJP#4nl%|K`cUSp0T6g?1z|(!kE0tVQX8mv zMvM3i-QBujY?e}rOe4R7I~WM`HI(@~ z)kcBu$7tf$B9^mypAWCt(>C4@UeEe$FA!*`c6KgqH6>i&4rI>e&zx}=;TASf9FBDVYIB28Py?|QEO3i(EAA!iYNUXt_`V(VDgg$S2KUjl6VHpHOu3<>=^ zTajv3kAg8mK9weKf>2G>$T>4tMf?8 z-~wo;@#yLqgxL#lJwT1)PiG^jIT17&+l3QoSq2@u z~NC8f}Qxv}& zkW%U8M`$xU2&O}Nm$}~>PIRCpI90lj&+~8R5=#82L1a$P+v$0}n(PStGl9z%{-`v_ z1M>I*l7SdDG}af?jVUVDcUP7KWkZA~L*e&r z&uHuhGAsa1C7R{s19CZ>GOSUxAo|-d#dhOgQ(Ig>U?-ny%i9GY3Q*{HZYd+>sWwlN z0H#GgnVpq$PR$&p6>pb0RhR0?KKQfSlY6Obc!Gpt`!>1pVNN1sUO^C0@MOe>4^-2R z`a+=fKbF6b z=bzvB$#G*~0J?oqU_I56Og;{7CCq)eEx3z_ObHQ%669j!W(O9aE{%1o_JZ`q z)jTh#F%zi9R}X`(BpIJYCJ{v+FYUnN;sJA6u>P}mZu~@tB9Fv@Y159ZzL%62U}xIl z(A3xxrKn;WRcOI}r&jk0zg@rB3H7^fe)Bw#vn`i+mvXU=A{Mb#rGhRCi+#zqnEZir^qL*je#S;Sl|M zBP8svgvD5ywaRRiRrlg_QiRJ2>&3v8KIdoz3eXD=`X{5t%FRaY-=pEqbd#2N;JDqz z;`&wfRutBQ5fMRntC9v)p)42KOiIM`(l?v#%u4lvOfZ@!XFEXMj#zk ziMlmpUv{)zFT)JJle}k3nK5$l9nxgz=yh=I?8p_pdAs2kpVy_9nl&o981A%t$i$x{ zUP5Y0)m=D^G~9+IIl&Ad7gLY8r@8~qs7^0XmKq+?YUzt?Na^&>or>srtKPwUeAuRg zxd2N)1u&!lL;3Ph20~!)>*&F%j)76!DhGz)T_(;cy3!o04`C3fM>$*Ey%w!o-Hj*5Lf0g;Dka9yodcsI4OO4gyY&30wCp<&S zzi$u^>LwwVv`xn zuW}pB6VjhXKrUA@*T0KdxuRxosm`EO{Q)PO802g+P56*=>2c)>3lbC*WyC59L%kyD zJjb}VtgW_emE@K%t1JD4p&sDYn_iTvCf{t5Z{U8e zKk!nbOC%SR@#UEB=-`$mr%)*>G-za_qlF6=)y0clDa@QeLK=@qxoS;X0A2l!9?*3) z)L<&)G25K(<3_yzmHuH0C#eY;U!r_2Ujb~FwkIO zZ#Y`y!;S4R+QLR+3VI3dJg5EEZVo6gYAl`m4nwdHHZ*@Miy4xYwCpvK0bxv9tk8DZ z(F}b)QgY$lx>xnPqcAjz)!~UUm>eV754$>f%q80CAmNrr26IBlVc7>r!Y(91a$?skw@Bw< zrL%<-`GRM|SuU$Z(r99%-`&1o-3%M8F&;%4%fsZhTt)Rv1*|)QvXm4E8uWGT<#s~n^-U5%X z;)B@=lc_>%>yb&8F*dk)Yz;u@)7fZ?`t05To;mwYKUk+##HU&&S*(C-= zU7zFH|AJ@wYqAp{-W6m_z=B`iD$C!voBFx|%RG@boC)86*Df}@7042Iasz5)6s%~J zFEaY*oB=LCLy9lbdBWj*mt|SjwK%8?dUaNl4wIc|a`ZW~Se!ea8*_}A1OCra{2`hg zshn$sdP0>0l@lN?=&Y9fqaUrA@dj0trIzFb3}qkUsV0YYI#HEtoX*WR9i_`U0lQ>J zHUh)cHX?dm@MBWafg;+7HUn%*V_)RA=wOXwlm&DVrQ($6L7OrJ4h?SF6H(T95t^3G z5li}=LO4q+dHB;{QB^V9BjZFFk3z6wsE9hjrCqy)@@YZ<(|pjK z?rxk8stFwbwfq3yqX)<9j2k%@SHelJxOSc)M;*?!5&KAR-EPK8NxP}PnEAOMjZmvY zIHAVdq##$;nhEn%gsCyHn|$NgG>ptTjkjT35#oS*u&3Q7>2?nAub`_ulWKP4DqgE( z6PCR&oSn_`-{Qyggf)N($WoPF?!2=RA|@tX+Oh=hfLFCVpx(ne1&S-Y!`T|R~v!Rva8XzzF- z<_$vDT_=}=P6V70oE(vho;%*uCtA&KVg}N_bNinQz3H=LixLsD9xJC!EcRJ{{^rc> zGk})ZKd|IH;A8>~0EscjsuF=tbk@=KYZP5jBKPmr!?J1KgaR+U(R{dWfYlRv5EX64 z+_K+Tamg@u))b)A_Z#U{?ph3%!F@Rx<$SR+s_3{^=$H>nb@1@ZWs*ikb+w6q};p(qkf$v?Ojo6{_m1K@0;zAgEmtzi{wo`OE!Cz+3%uPtUFC3-<* zcqKwh?7xiPn3wf-RX~_(iSN=*!K{8qlgykcUXHRSb|*Hyzf9bGyq!2zZRlp{*C&r( z=D))ZT#ew^2^-GXrRtt)XI0XzXl4#?C+d>-qi=E@a(^uWbu9j}D+BKqoU~4lhA`3) zt>V^7XVcf2dN``Z`S3^aS}otwytvPj#J;WSvEBF8o0uu8iYzm}bw9iwrWCZFw78n3 zLNhP)av%@4NuXF9LD58*BE8tJ7c^trtg3Qo>I3U_>8RJxKD`1Fzrcb2UCGx{3x!+5 zv!Y0PzfQve=BOwg?wuxi1vg4wI?}2P#}Se32n)dK8fuM4rKyX}sgadOc>`3mj46tE`+SRn@t z{(x8F&!Q=Yy>5gOpSrpLwo>KqeSQ<6A4mhsgdbr#7T8Z?d<<oM%3vCT&X;P zaVnx!-`X&zN5H3Zo1NDZMWy5AJ;9agfqSLeFPKRsBnCRUA{D%^Dd>lXYk5!m9c^cm zt`n!dXzK<7vljb;yLAOjc{5gDj+51;8)A%{_VkAhr)S6K<4fncK51y(t!xH2G@2aq zy!wrEZ;5q$8a_iY8(*iFlYd55Aa?8blA^F>QRj@_F7K}Qsr7Nq8FH;YJh98u49+p# zDScRT{5wXJtzV2r?WPA@s@0YBbdgc*oVB`%3 zmOW({LFYogy220=Rs;z8D@!O7HT6z4WspIj|o6(RR7J$$OG`(ds^5DTC+HSc6t^D)jBUih4S!- z+Ilmr=qQuTT*k!jdCNrL&Fy|NBdDX+!l2#E+$Ul^(^|C3N45dqAO|PMgHqs7%6-*k z1%9*RL`z55uPkE!$4kMCd5$M!uC!w*R0^fi@WG&m^o)Ty-97xstvO+Ck;{Lwat(Qk z!4#;Q@Z~7Vco>Mv+u?@fB4`M(1*6!16FuBavfxC~C7;2OUPEe~y=YwKlgxvHe)=V| zPC8MH)A&7XoY)UR7W6}9&hQ#~OSM611EA21oTI?_lg=R5&489P3qK7x<|sh%uvbJE zEc7X#u&wwOOZc33nkqZWxMN(%x%dOZR97f}^SdH5k8iBB0f+AS+_C!>B3cjt;?!bP zpB!w9t5svEG|T&W+!hr#PgaYZI9yFL(dYjZlcOab zjK%#Ki`?4faBg^7qCTX#H2O0fL(DF%1@8NJXHUC063 zou|sHulBQ>HE$yvGG0H-80~xx}X2IHgjtLO|hII-xdFld`YKo#gGd=yO!| z2bf!Sk4Axl*^{5Ox=N4mykX$}cH>X{1RlwRaV+@@8Da6161UP+XWG|S)a&GNfj*Hw zHq8_gh7h}!N)_uFd-QIN3l!~}2mNi4B_4lBctq4WVqy%diTg0bH=~u|N72jok?l?C zQja?RsDG!IZW6ZbgYqz3zK@OqRhd$-Rn@Fv0foU~HAgbT4pmiriw3E}gYZbL>k;vD z=ovS2Iz*o?bu7*)?YN&*iw^U*)v%^EJk0p1xhu~d(&9RRTCI9$fFJ&YrYvNGG|nV& z0r861X#`EQv5xef7>@1kzd1yEt{VJj881n~_`(!)AZL|Tk{g|r>eatccUa$iy!aI% zvZ3t`;#Hepd5G2KErSRh$)x!dWhyd&Z{{t~vfhX4stC;_3f`PzVZ#IZ@|1P!XGO;R z1H~xS!pWbhI9A3Xh#drlvks$(C}}_?ELe_OpIAF23buQYhkC*e9q?9a+BI6X4&>_W zy$lW$UitPIA7>dfeQt45VBY#>bU#nP(<6Y+yMfR z`g4+VKMi)DX5I{Um7Nxx3U1PmxCMnN+Dr2^;*&~pwkW-FP7xB&t(0bkksLQGD= zSdg=&st2^OSGdx8H1zQw^|>i-rDCXK3X%Ka9BEWweg=+oBN0JMN`0YYODN z^ysE4=`(8u|J~@1HcLN*7Ra zVPj`KjY%Kri^&+aM%wOfltL+Pdel3GsUPh`YcH7l*K4{REXk(Po z)gVbDiPWK6_Mb;fI-z0nu3dL=B@d-1@*hM~Y-tatPn?tI{{&l1rEVKkCN&qQWIQ+H zA{Luls&h{a%ik0gjQJ|>|4v}32repl&l<~`1C5;n33`C(viz@u;p-gXNNy?tyE@5 z^l-c(!k+y*oJhh1IXT{tQmz$!`ZBQO-+I=s5+2h1?gttyhJup( zFop8x*sbZQt(W;aksbxSh>0+5drU zTox|mO1JP+vNf*69ZveYd)w<3oQddu(+2D@4_6-K?GES7R;BiSVl@7~Fr_92o;GGy zrp3^#p8lpW@eNpN-P?05m=xyJ+y~}xjVaOuKT$@e_sB^olhPDQIcs6q_FFB2+vM3LS}*f6ecu;Th5pCA1XWCuD1~S|20Uxb{fwPU z4VG$=W@V+OtX|s)5pS6rqQI&aD!v-QMt(bRWs>`l7FQ`hlQ6{VuJg7c5<@fkE7bb> zU8*N(;Cl)0J>?4MJfwh)hE4gF(S3~ zAVuGKm)zW2SFHgrwq{N#P3M$yt%_s~Uz#e7Mz>vh4TBh;8NzKN!ga7e`{i7J$X zF)FWbzeS}v>GysuBv})TNV~@D7}EpMiubU+rgPmP`L%eIn6z%yNPXuum0K_G2jIkE zA60XmCG=h&4u>WOC$f$w$NT2OFd?f6FUsf1q$*7g4xgHrhHNyxWrtKrDfX8V^$K2z zOwf%>*K}kC{?+kO`6|E)hIbyFSn9ka1m-GexVWo#`;tOxpY`&Xc;+{oY@U+9{tPCXo5j{?>P0Y~h=sk|1ZdFJa1<8FLHoZfhktNzYha zq-U^e1zbHU0R8Vb%1XdlTGt(0gie--S#ok)eBw#7`UVT>79ua(+X{%#A7wWxlyqdA zVQ9eg-C|pZM=;Fu`9puQOr=q1GVnu<0T{d~e zIj&XIOnwQv;v13^)cNbm-T-XX5-6-{5x=fU_lzBL%NYJ>5x!L{Z?=M z-Fxph_sR@2%M5eQ^PJ!NJn!!Cat_%Pra1u>ii+VE_1`@6Qk$H>hoTmhng#RYblW3#Ebn7l@DF%z zlj;SOwMs5OMhhS9mGLxL2GLEQdc`%3UK*&qDj)YoDDmX6*Q_{nQ_J#evp%pv%k-yQ z4yH{x+HDRUl}+4T4`6G(Z>HkSmQy%O!DBxDsb@O~EK7|!E_jc=B1O_JU%(+cpqD6U z{*mJw)nc|G$-bOUJ()gBa4{VbcOP30&ojnBA|T_yT)(GoJNF9KyqL_YEu+xK#fYJk z&E%tnL`H=2=ZcwaQ}X8nwRKS4qH1mSJsw^P;Y(#?*-6_u1B+SrZ1QnF$C--YMWdx@ zHW~jRgq8{rxI?cN7<~)(AbMA26QpQ^xY$TrP1sA$C_fb92L|?1ms;`S7)}j5H#+ugbp{a> zl~2?(o~cC&e3_fmubCM&+Cz@DymqXRujE(AVRSFqD92U*STWQCvNE1~{2HZAy6v+w zI#_6qJ1g43UMUsOOvv;?lsa=4&{&|PMB^C4+iJLOB2t{rP&F^AwCdGoH~H>8Z_zIo~wtG(~&~N$Go?2m3pF%m`Y>9FXFNbxhD+zP4b_{c3*k3T9%wD8FL z74Ih5H6Q15{`;m}>LuB!IB1@qRJCc&`$r}oU9h~(2y{3CFmxfO#|zSoK1(w7F11m+ zL=zxj+RHjX!P*7UVMfJ?2J|xoO6tZnRkGPg&c-A=BU!8$(1#s_Sj^f(7Datyg93{_ zcD*!f!Dfn;o?sm!5H1b9_T1JcTSjjxZSpE7cKHosKaQQsbb_YjNr+jG5yU;#lQcQ02tCfH1B{-sARmf)4c} z|EZ*#oxTV&R37voA#(q~JyyA?$sUi~Yo14iSu}&!@b;o1bX*NI6=Nf&^C6}pw$+3F z+<7}y)j{Yo;pU#Wvi&I$$9ii|%NfPS&T0{KhGGlc#vsPl^X+;y7MSVp6<1w_t9tjI zo%YAu^^ux^<(2&4z&D)j1$3rLSV7YXCo;~AiG(|QxKc`xWU7m}{d$|aK7{-T1Vi!X zSliu_3LDHrl4%|$mKu}On1XxU__Cd*j~-nduZ4bQFyALT0tt`3zg(Y~ap8gnK_`x4 zhq>d1>t611ar7zxgY;u)^-F?yoEt0Lql}-Hf#{RoXlf>X-4uxo>3x+lHy7m+dz%Zo z-Wd~{AB~c$Wo_+om-VW9Ol0t5ka4p@A}4s0JonU8D9A524Q1Ed^h9TnzB?9I36@nL zCK;A2+FoHBazLLtJ-0Dd>P_TW6VOoaVpOFDBq@Do#r80=JI}V`rh!^>L2aldN z<^kFcCYyU77-i|>fNz6TUJgwWb!B461;Mb~EzkS2@yX3bj-ODRnQUpHeJ=~R1~Ao0 zEFFaSgO+){*vagPQ9Z?9H8_UX>iUpve8^*sos~$sGBzS!lfaMA*I4S_PTt+XYL050 z0f+ICQN{*yd8ep_p7P1eTN@5erPp`vgkL(BrC)E{?(O|t)#3Vjr7;lgl-pPC&4gw3 z;C?FI>*)Oc+b5v!D53t#{q-eB?&(;p3eFkuYnT2Ow(Wut+rFSMLO$JT&1#|4 zii|f#Ywd!^?rWoGxg-?3TK=+DeSPwc2=?J~_hw6K5kub*;Q^tjp2I77n0V}*5H@Oq zcCi&L)$OfOQRC9F8gWbEZ*vG(A`g2rGK$hl#v4zG}ey9vqFhD zE2`l4Qh6h9JDXuVTnkEZW6JT?oXUUJwcd5W{D|G|>)@MBHDTiAbmlO7ox> zKcWp~#_Dy$g5^~N;rL~3W5QJ9OO}Bg0Q3^$L;`I~>1_#c;8tOso&`XbWBABUR<^9@ z-VO$b@>R5=?4vN&fF09*M+>1Hip63V@&dyw>IR}fD=KAo=|bCs-u^^#S8paWMnO~g zzG*EK$_+Nu%ox%R)gY>E&dM?On0e?@fHN|tzR`<9tfUAjc$0A^01+gaV^*UN?! z(A`#ipvck(+@)1i(}IckyDGc>QxC-T%ep;r4K_iwq_Q*M4$H6WOD=W|3!6$!D2P=P zt;K`g^&M$8%^Sqw>;uvHUYd}1pMYB+iWqmW9EF+ z>!sDozZ8hs3Go~~Hxl<&S;rg)i)pV%=uwm6Qe*T}f09@!XDFI$50E0${f4z_m7+uW z`KaC4NVP&~;Fic9{^jTP4olYjFInsLJ&Q)5(pcaA;(!AI9n$>ND&yS-!;~&9Vh$Vi zh4XBu=_b%vkwV+2lRiKM70XeGK{uy&o}I3yb-|QQ^1+>Bg?PvLT7>?zYJZVNrmPWq z@`MZ;C;70~R`*{=xIK;ucdp6~>u+e$1wfs=-n|_kt^OP`^cnogj7zlmNu+17F>qYT zqugNI+F0IAf0HqQ+hdUcmpcwpOl8EH$}@T_!zh7QyWGCbx%>$t2m0X(wA#= zlMuu_xs~j7NZt_Ig+0y>(VmuvvJ#RDkK6b?4Ldb@(zWZ8xB|_&WcI% z*E*5oxYA|?oIrtyARi+b1YkFIYTlUqe( z65B#%-0H@jQtQ>M=?;pxte*8a52TSOkn0fD7|y0sZ8<+Jxq5Yyau1wAiiMrr(Gb@c zTVvcuiNO7J=KIrJ(sPkGK@fLb+}?|!t84m4v>Q*PO*A*;Sljz`bmLgj+{Ib~A)7NI zR%m;wGBw@%;oaagIlm6wG&4Puh$B8@Vx|mipQ-7$druI*35TR~wn>r)Ljlt->Zy7V zacbCDQJokX0^JTY&jZF=GnzWh}rd{`I?^m}Dyx5udT>ImLBy8hXJRH>eO< zw9m|*xpH`k?)-ks<1IdYs4PGiL(L+W(>sX`JpC5`P4cJiB~Ti{TjTq_g6{%7Q@R5p z)mq{-O7sOtM$F9Z%7Is|d>kz7Zp7wtsI5F?Kwg4f`I{L!djaEev#R#z--fd?LTLe3Xr6lrMCb4i0U&Fi}&@F3Z|0x1IMFZuB*zyBj+q^;q z+26uv{1^KaB!ZpwtOkseo@(b{*EHhapKL5Wk$2H zgmxy@Jv$zjKsyP=)mh_XWm7UtQvVp7wRw&mjA}O?5g)e|8J7dOWaHBilYG@#|A2Yu zabm@aBbTJwg;v5if6~*}TtyiVC$eb*Y$zK`j4f*Qy=SjpNhXNu-Qn|V9`Xb$8xPj>32I?B(ju$Ol{9ffmh947!8HG zTTqkvB8ho&ASuyL1Dq;p-kFXzpTQnj@&{*Q(QIhR73b!RVh>PP`Y2#r+++6i1Pv8MnBCv z;I4#BkenCeX_YnUoSQY%Znwm{bN;y1=byj^oNv!&BoHeVBc?lRMLsqY} z83O{?ZZxC&3F6et6&|XxuMyDCFkj|B1gh`2G@Q*r924Ivw14P-zUfZ;qU$!=2?$!x z&`A;FWIx+t)Q9iW){7{&!RlJ8nCenp}egMUq#*tZqWTp2oH=hX;>o~k@m z5isAU_0KSAmJ}6+`1t26NJ=1ME}s$tnbb?~7Ze({CvyPAxe}+c%j1>xt(kD97Lgjq zYWQrLRXvLG9?qq+rUhpY+n6uS@N+)ib$F-_SWYlh<(SpJ(0<>qa`jRAk@y?3kC$J< z>Sx)Yj=KK8{mL;JoeCiRi#KP0FYrF12T+ULCLO*0$j-s*|nl6JRG?yS) z>q3y!62^4Ahl3bRHE!^+p9VdTqu(}o{*2M7_X-yoDc46+kPxs?fX#Xa*6g&OOP>^A ze1Z_v=4y50wx8%%PL=+_0bq~UX+v*bxnr;!>lVl`dwnPZdFlPiXcJ}|;^N6LJ<12&&>X71xy!MPC7yucacB%w&MOS zvV$3i&_k1v*~FW#szrFRww1G$-SjK0;S77a)n;F4Nh1Jr5@~f_eQA(cHJs7eKnN?t zbjNgffkCo30e_h>p2*OgVmoW1O-saBO;7jpzF56`$42TsGXq*Ogblcpc$V4N4I-1G zOj-;N@Fs&J(zW6x_H>bY@tPc138d~tD`rW>QR1zjl&&ngspJr99dFYDE^);kX}(R# zCA^=c;}MlnVkf3XBii$NSY84}`HmPyH94{yRe=NDo%T#sRp!dn*NkpOZ@gku^dr!_ z)=`P`9*hH)Q8%%;Jpsf%`QcgG+4$Syp}QuS8^lF#X;={EW8RMfu8rSIRb@qrO37ix z46jdc0W#cdQMuziM$d<1b6wO32qTu_275k3RUjV2sjm7= zMa3W$z)V0ZYZ7fhI}O8s-D~QmbbuqtGU(izg zjE#(b@-4}vr~tG{T^Hc$dUnOvBrYi;>gTZ&gC{k>S6tV7u}XwpRs4|Eq=7*>n6b^d{*;Ur%zEnUGrnL~PFc1lI$Ko3=vJXb^xqQ!UNirUb#$RU04S z@>yxxwTrAgR+3c6fM5ZK@R+`DYWCN~_v#$&B4ebTT7<$vDPAh6cjKz!~z49 z?u2w^&Wn@mR6h~A+|KP$*RBaOZl|L+v)?6DUsJT(-w@wBGokD5E9IFlvSu1LjjxewsZ(V+_n^+c!7+{(C$7DeRI}9q*EbA7Qa;1(D9Sn0H#eyJeJb zD+bz#;bg~_MQ@;Q404+!?L9&kc}_-4yZBt8I$yk?H!LJNf5@lm%xW9)lrHivkIXT{ zn88_rU-v$;zGm-uC4pJ2__yu$IKY0(spY%;z*uc|N>MdLDGE=#2A&5I06MBNJpkk`PTfo*#+}W?8k(vEh&J*JLm5^JcJ2pyFlgQ zX(M&>q=($;tsYQWyMr$f|0x))JQz?no|;yyXcY;{g38OnMnucpG6O84s_;n+)g zU?o6^6jGR@re;j&LoT1C^%1X6qSK?FE?@4sJy5kE1zwl!tfkIR6c@7ZSddUPqROLyjFEJT38g#jjYbcWV;A$OI?F1Y z3fZ1cXvT0Aj0y1Xo|B}jvBxJl@V`o|LrX|D2|UL0nBJoOoUoN^5FHiShY_R4{EZ#L z$m9-XLu%)EQnIrGIqGhyO@5pRSWeN(*O=<}LN?xvVHSrs*VTVEw^vRw38PT42` z2qt)~H;Qrikr07cfH0(dcGNO4?AjeGRTSqQ5>>9?UQ{#eLF(;WUC+wWJukwFp!SO$ zp9ILEX_~Q{53Q|g`cm|v;+B3GR&0;0mDqEDkJy?a@}w!WtryHq7oRR_6Nq7sKiq&5 z9f5YzO()GnIqWK}5B>ADLtZ>=UfWt_(i+7un*ptrK>&qu>a>i2KaMNUH(GcD=8xC0 z-e@obW-C~@*6lwnP+go{E&t7izgBP2d4h+ibwryAEVQFTKY1tkt}pMM6(o#6={2MW z@rjvbOIl*kEKVO@vnQnMv%@YEVC)F9$ziY7)AjY&A`(G5HYO%6IeeEN z`?1x%+aDnNFllubjy}7OlC&byi%qal<@Zqf=?6sEoxEqi;0ag9!zwLQV+l6 zo~xk+4(-&DxfM%P%ew^jhA`n_c5Sw3>y>M*yVisn3S`>OI%T8m77@t`K%P(0bhc?G z+XUJZ61$4msWz{k&GCcpw29H7A8Oi(Di11fSvpYX@yZ8!kReM6?Xk}~-!XZ;a_&xu z@Yp1@+6PR?#L;pNzTl90wg{0!c14w4szShuVR(VY$394FJ>uTh{It$>6&Wx+vSJoy zXdL>5wO|;dkKV-y)w)7-NS~iJuoh!aq$M`95O|llBHma3`hj^%EpdWT>-oE`XFM_Q zSq}&E!!U|lysSV?cF{d_+L3||l=*&SCSE1y1w(`JCv(^L2ILfJ5KWgs4(R92_$G-k zpL4cQMpgP4%8L6e-=YY=(kc&nr@qTt+i{ctKcBEtbhqpP9CGO@9^aYM^9;_!7Pq=M z%v`C({Ay$wa@$pGx6qg&E^Z@0s|96-d7tTq0|)o27PIiiGi_k9oDi(l78_oz-=5P! zlg_42TNK-hTX-*T4tn-f%yVKG!0pox8o0E&RQCmSJ-T~RjQ&kKr{COuqMcoOwRh5U zsX(UsRrQ&46Qi=y}2I$Dk^z$tCxhTs3D~$k-=k zpcaW~r7eZMtMO$W>KCri7uuwmk&bmb7Dhbf?YPcThq7%o5q2vNUp3Y}J&p3`=8BS) z+=e_;XJxdsu!}4Vj9$&&uftUv3ud~5?wrTf@5<@sh~q|O&v8PWNtjWwdb4mD<5Ch+ z7c!z{5l1nF{TyQmEnL+w6bpA8=*dxL8uOyk=QYunkEy||J9H8`r3AXdhh#02#go8E zeTad_s8}M6|1Rv|dfM}ls|UxhxD7-k8sy(RiwU($m0?!Je}17Lkp2D|58*)O`^)b! zDcrg`C5yLo$x4Qhaq`fpgw(Sa)F`$ zE9yVDj|JEM7WMC_AfN}x<3Ed#VPtBD1WRlUphQ5RgDaDR1`hBy-)8gQjGN8Bll*Jh z|C5>#rz&9=CTbi^0Y*5Ma3(kw_?(;H<8AnUfvMJdGglMFQUuG;VSr;AW&U@jUm#Lj zBevRukx;-kJ0%H4dhDdTq^55hE4fOSIwN6^X+NAN?lpOb+8+XXs7 zq0Z_AYv3H~uMI45z~TRBzyk>S)duc;)_-~eE8ZFVdwT276OhCGcLH0W<1b5;Psyrw z4r9={g@8Z<$FL2Xx*s$4+ca4DfPm&6Aa^_GN59&I1SwC33Jf2XvGbR678HQv_#qPr z7BumnX5+V!{=SBYpIo4PpOo+4Bydek2*VMWo7*|tfy~W+;UW!jW)fwFDc|9TfItg} zEhGYm{iFCtmLQ;)rO$tB;D5tiFIG!T`F#>l5fG^1NZx4voy5}J{nsd{rAn16T>2r0 zutTtiEmZhb&8h=OVFh%ywso>}ci{a)B~j(Snb-eyRlBpJQtZKYAON-@;CG;{AsqS- zm8>lt9lu8@{i;m;^M#M>U^0PVM-MMkxg{Kn6W9@C2eNesS_1ylrS!j(cs1KN=?6Q7 zIoJk*pExG~9L0}DzNd)&Y7_TD>P#D8?`KEw|2hql4u59=*t!2I0ep)1&r3ti1&#pz zPVvu~;{RkMj`GjolwWNMK3?}{26QkS!(YR8|62p_xp6%D5_ezk~iQKLe)1KV)b8 zjQn@g%Ab)1Gk-_^`5y2`&HrFu`7`o&ec?ai{)}w7@<+}8`@sAEof!&jZGGRqbO;|{ LH`>be??3$?_=WdZ literal 0 HcmV?d00001 diff --git a/python/lib/py4j-0.9.2-src.zip b/python/lib/py4j-0.9.2-src.zip deleted file mode 100644 index 881bb759d78236931569f9fee632409a38013be7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 55521 zcmdS9W3VXAwk5i3+qP}nwr$&c*|u%l#$L8iGsq)jN4+Zz}dSlC*nT4fp;nb}?C=H=xk zVJRD$>}4w^=Ve)4ot>Q>oEce_Uz(WMRa+mN9FKdjuchIHH}8$$1gI>NZ2X+L5l&W4d9`Wdz8 z91%r4s22oK2zAWrlvJ^hQdiZlynSINQN>pFZIE)gP?pb=uj#p2NlL2U_H>eE?(}{- zyxz|?Pt>&b+JvksCaZT!%mb+FlJoJIR`Ex0#f>+^vCTa@&~lAp6uK`*rr$$eEu9-d zfqOJ50v5VSnJN9KgKphII6dI^vzzKi_Agv*6YABBoe>wyR>!QU(KjILneLviP$BLl zK6lsm$KH+GiGPkmefoNJ5PhomM|kJ7A-NCc?@lNDXss(zDD{f;moPre#Z6NpF5?ue z;0HTgftfn!UN_qjvXle z7PJe$nBXmlv=F2!x3mW)D?oH_R>NSJwgSJBfr#&B9KiMvaK2yjfIZG_lE+18LD$ty zuoeJKS8@YS=jurls6vmt_BV;VLpm1NJgRwEhEOv^fQDgP6ivX)V_!*`Pa?38d$|eJ zw@@N#7rXpu1Y=ELz<2`zVvczmFRgC?#n6l236?nRp-jQFYR!c(Jl;Z%hCcKUMtqXL1v!G%6%iz_$?SU2Eb#n~7q zM41VegBi)F9ai%|zwwRsSYR5fDkaLgjYVH5u||;|$40~j-N;Mii9a`OW(;o-VU9MwTlKLnv$Su*4?5 z6AhG*dBJgYG$q~Q+smRdV}CKxNf9S_XiU!AW!8$|X={lP)^#V6O?&n=f_4#k@nd~h zaxBA~ycm-x23Pmf;8G}~ki_c<-sGHXNJOI}N!0%Ddb4|lP5D}y&oa!+f#S%u0niqR zSW+^amm@fw>@Uq7Kx}~*eFWv1%V1NJl-CZF060!+?Rmx1=iXrMZ80G@wp;I=TX;wQ zzFEiUVfA$MV5~~gq~c|*O89MaDZ6a3PEIz)Fma*_4^P^hsq;kpNmvs9Ftl^4Zh=_L zBXnC&WK=||q6;Z*eDFn2lYl-FY7&aE6)^3UG&KloQQ7hs&`uEbFm zTj{H405Mfo;XA}3``gK6lY<#LH~~X}8FO<_>@q%fI+*tGdxSP*LglPC(^k&dCe-4C z7&7~0Htq$|f1m8vcg42E+)>IOB%MSXSnFW&*7zXWj2ZG4`?S7%{WCOd)QQ^-#xWE0 z@Z;8B7FyfzSrejGh~W{;EmYZ~nS89yu~2UH4Htc~mY&nZh@z}aGL81TaX}}flb>XJ zFvuJM2b&|xX$_cd3c|)Wc3TxX{!BUtqNh#zV~%RYQ*f&|FJIl-q9bm8V|ak44sn;O zb1JEVE06)70BYGRBe;+qx~{9K8}Jm;rD)_$XO_{LQv;D4#n*@Gv>_nb=P$;p3 z)?Qig()_WrYaoXU?5TSh#{*L-8UbI6q>)RQi$hk~6?_PJ+zgnv$?B~TLL-x8Ir)p24hEAoCsr_}g9 zHnM;4u1fe_YxCH_`M1Z{7PG{?-XDm0X`xK`CN)#$L{RFv_c>&)pc`UQTT8vK| zNS>NNJxi0F1ZhJ3$y;;oUX0C_&sCd6^5n~2lleARe$*&NYkfGnx`|rvI(*RVa$vw{ zF?gaZvT<6u%5z(ivzS(oRYhZ_zMjJejV)aii~jMDc?NMeux&r+S~mZe%&tse#r1*$ z0Kj4WPnrE^E%+zR`wz14-&XVfRTuob95A-Gbuj!V#cNU9wp(OC>77-^ab-VOAzC5~ zrt)GW+yjCP5I6vmO-RbF-cjOi%AxgS)bmF_T${09bcKn~Tbgn2eKEE3`u#B_d@I=-K;Ero!OPJkDCDETG5hAVoH?G&>FXpQ6#__1DK!v zl6uXKP^WJ1$q5<2S0}X5(-oPNrsv7^m4-il+fNwS0ZeG80RTFNI1ikOdJ5lo)~T;M zu+3o~*komPPcKYUJM@d9cm{He$X*M~xRz^Lof$u%1@e-!CpNppl!~l2!At!^R==kO zHBqbgX>vH8A!dW(da5W4vZBY`$fe6N`Zs}cB&#)ckvE2pd*xH{^i;zA`Ul&Duoxbo z?yie9#8Wq-PSRgWp$sLFB8tJTl_vj*LjJ~=nDR)U%sDu>;dM=3G!k<%95lJ6Qx&@< zTvu}o&W7PJdcDrxYk~UKG7N!p;Jzr)CXS0c`lZwZx=Qwk^pge_{=EgNb?TvM>&67 z-)eDyT3~;4$!(`tu(2%jL&9He(w&)hH>Sl#e}=m>qrhmRLU`3R<~I?e>%gr`1J2Tp z(`tc?orT~0vxKPK4?SI!5Z{?!ygUVop@C&&NkH^|?w>YznfYK+LI}G;x^V|uM;lm294nf8)NJIe z+WpV1w;21%n6{8vC4)GuWte2hY4C6<%}{9g1yeVgYTcN;de1tPue zj%$pbrHUofC3Qay=Q-{qJf90opiBIxkayWz@PARq-AAkkI4}T!5#)cO4$OZjm;XP% zuK%{gf%sphurQ@|F5_)TS81uT_0U9egWv?PAh7B91fNNkIZj!XCbq z18r$yYv(H6hOWrN;Ln?!uXxh+`|Xa^Frh+nZa}t3iH5B#KEzCY2TE{PUrcK^&bknNa z3LST>9aXZ4Bn1oF_Fv%Cj&TDrHM@p(rI3Zd!_>;-P%TY_zfR?{NGHLM{1s@~lO zQGa&2<~0h%GLUm~fPtBFjig*XTLLChHzSKVk&UMBKRh!%@Y_&DVFKFYR9k^sQKbg6 zqtlzsP_myI1X7kbAW}Td`NU>icc4l?*2r8PZ-bB5taQ=?GVQ6Lk#>`ov3%b_!09VQ zF>rmX!E!6Yy=`c!-Q|Y7nX9i=jOW=!_&@tE#P$Z?A|rG{`g`}RXU8uCj;sWsN|kCT z1gNA9>jBX&@PuV2wy;o4VcZ)R1I(m_>3ui?cc&qxsvtcOhB?K3O+1frD?&A_I}$4n z+tFdUenIB(d1thB!DQ!52kp%Ej!k2;HZ3ySO&0T`01EkSEG$m3symUr05fXC z_RPlL8VQ*&Z*!fRarnD=MZ^g_I5O;Ez}fRVA)6uPwxmW=e_&0Z`SblBdYgNAg+Gh zFZYgIw-&q(4e~RFk>IlW{*pY;3Dd+$W*ED$)6|k6?o~X13k+n8*^o4T`ocRAS4eMlH>! zD0e{p+_yV*D_X3FLoT;TqW~;w>tx1}jmeiySi)mW@8>Z>Z!=J6(MqM(5l)0gpQ@=>uZlI4ESfwV zGUiF-9UW(yS`CFr85w&((~=g-ddBdYT>}O8%yBqLx>hh6deBFmASW{WxCWNyLjgrX zI~a>bwyG4@tn>5Oz(0;At@lb?5PEA5$W?lP_Ij#wUpO`Z)__w7J@AlSx6II>leqkA z*iG|@m+b_p&p)<+c}Cnp+Xa-YXWyC`tMh*Pm3XREe0O%(h(=&Ag`zM7ph9`sFItbr z6!Y0M7Zo!-%CVxCduW}{-U$6NPY?c{XVJKS3!NuNO!~IiY8;e zib5`pnxu>X1XeQrYFaI16<6q0VK3pH{Op=&GRUcQ9}v>t0MK0Ox9lDnL0oj)BgUML zXxA#-K*Us0hXKCM8@@pSiKH7aHDG762z7O{Dwio|FjX~%Tp%Azlubpjz?826(AALL z2@$EF+8~IUVCI04WSvNDmMA_jUP=E%KziI_dw8Khq{)gg=G!q=Ioqc5V5a-S^VkG z9I6S1kMUZF*qCHg?<`6hOi~Uc08`08gt-I1WVYl$_6aA+l2m7a=KC&Qs=FU-+B2uX z0bC4X0t#RO`_j?5VsS7+jBKn9gYu2koHa91InbA@Rgz0YrN}R9-BG_^_8yQpfZ`em zg}j#LtO=coeVIyqtw0lwj< zmt|LVt2hV*_oA5p;dxUB{OtKtXNP}321*Zf1jiTB-ZhkV$?$l_xc&(OC#2G78IMI( zUAgx4iCO%k8hI_`^2BfEh=rhiMh;{#7YZ8Lp*eXFOlPh-Sm#|W6+ZDSiYXe@6g7`O z@4zsqi)<0(eltWz_^9W;LZXhSSDT+fC`MZ zE|u7@-6jUfm%~;F((GQ<9M6B-osj8e^=S3R%2ZB=9%dl0y&jHRZlN#|V@X5KvpZHs zm_JdGm4!LrxpZVVE9;F?ZMo66RRf#7*&NVTF6dTOYcjevP)p#+zGNHWpo-0EsAKCD z2kXcTMpoXsLfQ%46f4{EC@y$lE3xNpjf1xWC?D0fwm-0mxDt%lpW)REW)EU6QnzPI zfp!NC8S6_Q@R) z-GMbldw%%@sQL+0F}vn>1k9{*{ckTyf|S*j_xHIoTK|#68M1|nc2{O>zOVMg5z4fY zeQLO>ZQb_zPk<|%=+1zyQ9nKrzF&S|n>lYB;bsS(*=WgZslNp`Lh=}@&iRRG$Q z`>&$O$s+#3Lw0j>@zXxhQ*m?P=TCWg>$Gcqe~6*BKD**JOuJ;oErDPVLX4&l^NpH{ zyU%*JK%}$tC09SYWW4@4v5=^u$%K-lif+UX)?oZJUrGZATAI0^)&2@EA7qLQ+mY2rTG`y%EMr2uAuC^7@ ze(^%!`B_8MB3ehW#`BqStX;I5N~;J!YN)c1(UO!QvcYKck`Y5$5Y!2Th(ly|Fc%(F zB0$$!d2dX9ITjd8x9qF74y~(#W#}|%p;_C}Z9B)wW{PyFu;%j1t z)&?-uH2UEuyPv($7s9Rx!(>QZapvwfy;@mA=|W~#`Iq+>W>pTjmVAtx7`4PfcruKj z76Y5?FrAoj6vJ7SdIELSd1Tk5prjTpLBO8Osk=nf(fvis>g&k-%E7C}J~SOcX?Zj5>^SKjlr&}8H%E8)wo-VfrhJ}R=A>|m^o zcUgBJ^6LFXM2|#A^!V8~p&ij=)n+>O^K~I%9Om#GgI!H?@v&rKz=&sog(PbRg@qsN zZhVUdLz{GUk(P?-df*NzHg(av-QGlGkjsNqhI@2F31b`!%-ZHskLs zSFAZoVRfD-3sQDEKecgAR*C5HsKh4(;U-%ee|lZhQ=cr<4|IWhiUXdtkJ+!VvqIQa zt+dm+Ue^ncmQAoH^sbtxOX&Stsej!)+#;p7;}^jEmTwBMB-@?b|LyepE&g0aJ~WS3 z!kDlQ{Bd*ov^#r=f7<0g5$0P_7`{M=yfRZ(2Obp0OUsDOzFC zySoZZ8_y*%FnD_gOIqc6WSl0|dTrJ=~y?jc^*O!qs9 z{ID?0`!~FSZX+ANV_D=zXc@s1;o#TAg*-VcG8+2ODabJ z#(qAm7@t&PKOu!EO_QnR`OpJR%^D^xhIA+{_H6EWR%qhle?NsD0A05|4ph z4WKjUtFHs+R)x}+lJoYh%NsMB#4**Vy4z{D0ehgG`1uLI4ej|;XY-tR_Q|;RGSuZf zB69g@IK5o=CLQ?MlXdpAg8htRR%&N~wHV0_Nqh666;L~E%rF&*DoIcw2krDyGb6{S zx-OE~YMIu7tw+Dz^x<3UY9H_ill>|SKeb-UZpn*4s|TYQMWpe=j!k@r{V3AH{ZhA@ zH5+Z(Fy`Eyml-}kJD;z$pA}h|1a-hp>3wX- zR10Uig5@E2zn$vYE#VV(0=w!^v!vXSM z(Caa!I7%?k9}*=D-c=Sg16#2ZS?9}<+78K^P&AnAw7F@VhFNi5(X+nnYbEPIt+n>L zI(`+C{`%M6+)YE8 zmntj(0Fc;!@}$K7;7R{~xVHbc))eLcu%_mQE~f5=p8sS^eXZR$#S+ilzo2NGXEhh3 ziQUM&Z>ndgx*A2urK(3IoVq*k@DYfzkcf~B02CV8*nhrOigvdNC`ZOk+|rU7B=Wo7 zRdsb$b>n6ePiJ$_zUcUPdJsPg-nP=#x}AQC*oUtQr=}%;fJuAGtWoC%6?`L0MR9 z5*w|Ov}7trgp!s2_>jk!9$=Y@`=)9Zas z&dk{TwYbY&&$LOkt5P(Tbi_zg5M0d^u!*M8Gr*bC zK2*|gS>-{Mnu)Gqh?RWV2W}C%$Wmpw;?O193Q6(dwU+TPYvqXAPwMcJ`4J0lv+6|a_dit_;BnJ}-hFeQo0Mdy+ba{vVkz~j) zg=njyz#0jvRc;-F8~$^wJQ!rtKm;br0p-T)0h8&Q(yG2pv!ZmlWB*5TD)LV6$%cex1R(x#{)zetEg%>f(IuA}+n(kM#ZB zV;-8n{A#nOYx)`*dYB&p<+<+pJoqW5-}^=H{eHE6{zlsUevtR`SjL|b#;@b&?3I|4 zllMIe?-oJt8WHrxZm)NR?~|q*J<}H!`T`FpW}dcwJb!ogNQ?G!n$v{e`^nbzdVNO2 z&EflC=HlS#;w97*-h1ZwHu=!T;V;(w_0j`A{}?KH1NdWkp65mCg>TT#-A&+zAL8sJlSY=rkCDq)Aa>76AlqyX4O*a8bW= ze7=8fHz)!>_8vgvB4`SLeBna~K%Ath3QS6o8C=gHs}~{JYbB!84`%PLF@q0*!;?w2 zhB;g05KC)C3GEhtHr2-4cKEu-$bo95|q!4SXo}l~1QMUXD zQY!$`mmg#iE?IqyR!c)DM{zZUVcyAN4_@_}&Q`=*uF%2Id{vu1{L@N89u^#ELPT6r z&#J(Eh)$7_n6d#Ni?`jp4l&->=K*>)u`#9sCm`-S$yI5@s?!ZE!SJj;012LMr|WFC=7}9shs3^oo|#X$E^fK! z60(h$P~6_s2_nFpyQl6T(o+QSBc&PW)Y!5x6Wj7Ahp?7w&?{IOICrd<8q(zu7#KSTkU3Rq^N&**sr$x`GUW+exeWmI zirAOl&&f55ylahF?%NL-$1^}2GICTW*w;GPQ9$wrwAUQVXJb8J?fIQY*jT+4(8dK+ z)c8y2t?RFta?l@6m0b1aI$foxW9+lX5BTWDx!=|4hYV*DV3h!6PTR`Zt48c# zrGA<80p8iqs#jxGm6CVv@5ZXh@1xD-j@FM)jpgVL7}J*Jm;*t-!5kDy&rd(`pZ{@phY|W|e^#qWjHXRVVYDimLD%7aw=$`AgCGi$?fax6xASBS)+rHWkQoJ;>?sG zwSUX>>~?|FPS_-~vw-Cn9oBhb7^4{l=+qNXj}VR^Z$oS+8U|pEDDVH_7Y|UB9M)0` zC?Uw8mEGA6Pc&#bARe<5NX+N@^Vjl{{n6OU;nAobE)pbz%?oPg$t%Q(&Q9`Yj;UmJ zt(2V|Rsp={PJSFc{J=Vz=3Bvu9t92@?jhk&004ozY~jH#e|b%9aH2CSjc4WbQkWw2 z-a5<0&FabOFhqoR92c@?f5DTG5J$(2oQK33V&9B@t`6K^^y&=LfO#S3}o5Ol=S zRIQ<(YYGXpYU%6^V8-N_C0wQhom~ii)Aj&q@|mRuX8J_J)E0m}Lfaa`ocBqyq{!_D z2z&voqChYuArrh=1mV_J(`mL~Ily8Av`Q%tL72pD5McGEl7?}vCm7mKaNHD3CH<+C?Y>9zw64D!AsHwxTHZb_A=S1cO)pF?lQSZtcl15^9QK z&RS)q1A}PA-{g|tgho&NXyeBy-lB(eRNQs2+>sFYMr0fn|@N$9+%dNdiZe|2E zrKwA!t>Sq!E}yA#lBsd9mM}nMLj)v*5oMH_5hS@!UPI0ofRk7Mw!7OW6*F({zyO_XI^|hkoeHO~nkW(Kt^XkX1uc z;sBe{(Fb0l_EIMAYKYDE1MuCbO#FUTm9IoT*E!~KXMJSVi{Ny+Y$I%=!)9jBuWy`4 z-ms*{%33?oHy=$J(b@A-DyTwo#z|oD97a_2s>@yzi8~*~Q-C{WtKw%JYE9+Y77}@@ zx_-v86ryXtLD~QkyVWuqhPSzq-U4%mf*0NrTp$5a;V=(Q#4m$5B;*JvPrcGcH;VrA(jEeF(~R9TaiSIrxWlV4f4w`AFjy%_8q zZ*JIvt)1I249_tNo^M&x7yz?JWkFJQ3P1*EwPcO91OglD9V???{~GOq&~3Y%YwzHoo+*Hc8>DDaVQzQ6s*!HHR9dv-XIdLzzq+Hn zxFUE;3WiTAf5!ORQR2`&bbrU9{Gic`C7 znnC6agG(~9B|`b_47#P=Q^?NWG(BYrotO>z<|+J$=X3UolxV( z2!Ny*Psxc!QICz%(9cy~k4e8mkRpV#MsDS=$$B{{)ZiwO+frLNNqOFlovpFNJrMb2 zp+U7!qFqI64(RqrmoWtk4|qivXgxx=CQU`l7S#62%P>TFJ*@VA>5MLy-{bWNa?^{{ zSEMQIs7#g5+?M((sepxsr(depgK}RQSF89qpiZxWL}?Lub8o7_V-W5}Q#TfBhz)0Z zw#8t<=2^J|l?HFd%fR1t$%KD`*|T^hqc1lgkw#6{nm&~IascE-M^VH`=KXM+h}0G; zp3$-`U5}4uM7x=jB9oYNYEg3+H$Ldb${K)lpb2N$)AtkIRs{C@2XvRpRbO}oK}WEQzFthJ+g9yhs@k$H4$ztyjPNulv6`BZ|AUr<5}DxUJ1D$50M32zD% zFi7(P77=K!SaW2xWX6bvof<*yW|9v()@lKj4mwq2|0kElPvxDF`e0yJmJS9&18q6( zI^_=F(-v*)QshEb-|KFAbMnvU?e<%pHWx^gl)HOpmzq+Ju*c&13ujIjP6`eOsVn8p zd=_?dhM<&;0TKJ{Vxi{D1Ne>CSA5>V%5doz4Ci>=nV5PeDWM`|5f?z4zRd*4>@7lK zsz~jY!j0DOIML8rFz4bpO<;r(N!wS~0L6#8U`O#yiD`}$?&1t~N}=)O!G{vM$wl=d(EeW5*EP)Lhmt$yVQ=QJNk z8Qp&E)b5~EpK^QE=p)$a2f59z}4H_ihO zANBZJJ7ZO0Lx~nEtrB-Cm*C(-jxak2V4BlUTMiaSA&ja^glSDaisEDbXe0xt)y{+4 z3rr+y`sA^h(+krjy}&SJ^$0r97LYF5i`ma`(Vke~l&+mu*^7f)AFF%{z$>+V`AsFl z@dx10A^!c>H-Dfm{}0wQh991NsrY9a|3H7XZ>1%#N7FCIhUIFZqMu_kB`KT2Ygmgu z@45!#XAxk4sLGH`uTPLlVAUZE>p4(9MksfiqjYU?fmt2=s!Z=@0I30>qnQPa<@*{u z$^7VMd1ZIj6`Y!x3aj2u^C~X2lFC0R2~Q3I)xJ6J_+bbkC>k#cjh}f@Mq!Ef((1UQxu(kfx@O;l=HRxUg%? z&NYf|0>GdNhJa5^+3jojERl=EGRA;63%Qj zx0_IjG(D((77v3tRJzd(>8A3MDvO^>1ZY0|4Iv#C-0 zJ+K8icr@81Ui3Kas40qdV0)llH-9bOE9LKY^DOw}Cz$nnfjox9!)j4(&@9$Q_|V3N zA_AB)%H$5D+U?CYZF+y2X{S`D&Vjc{TMV$~!79P7)Y)16JvqXL##%&f19$<-c{`ho zl^q~PW0=tTO-d-Ij9K|K&W(U)W+9jSRXp9iD)<~@)fzAbH-cs|U5eSx+lX_+i*#>e zRa1Z&q9evcxEnL;S_wPv3$e6mOYr*pf-isF1)Cyuq#ID7PO8#+naKJxy1U~p)|Gys z&KH`u`3WZciTBb$Fa>1YGjR9}Xq{!%H*7SYE>rdlJI7`x1V{>qya?7E=+gV1`F5Iy z-A=95O|f#qMvvv{xQ3w70 z%7$Ayfi*K`OblO7jOzl9q;hBVm0G@?`O18;?oSm$f4e*DwZpMb1sswd0|&V+IUQo`0G~{Cl%^Uq=j|D- zER?nOF>owkK2H${lmh04HAn7n<>W=zlE?VW-BtEy08E$*1?ejI{U*t@rYy#eR_bMh z;gy@uT7mc!SUDPlwK9&>kBfoa7(YKO7E{yjTx6DOM!6A@5duQJcy#sm&wS8wOA+j4 zHPbrLgb*F6$_uxaqsiM)96`Khu!Mb7m$KlC=_+t*ITGqp*c-6oGi5wnq}))D9{r>K z>_k+qR!huv@+wjysFuh0Eff^hpr^VRC0i}^2Z1>Bw;D=e-G)$3a`*Uk7!OI5t^;OC z47At$^cKrvXOuUltdZv5H%(<1UOxe`ka1-z+1_*TVm58$-Avcn|YA|=410U zf;7OPshZs5VKu^$^iQYMXfi z>R_;RR(XJ+u+rk?Nre`ha25Y~ZrI6WT!X%h9=!=xojJAx@Ej3m)NJwTFw>cFLB~mJ zqISX9`;fsw)KliVHJGVw2s_$90aq_7(bi;PZ(trH>k!}3JcET?=6nwB+?49nvcP5^ zwUWXwRRib4)Lb$SZF~0?NA&*sbD(L{(^}{sRyA@EAmrDc;Es5!`kB<;_H z=m5_N2I|yK?P`b3;;51(C76MoWiN7iD$8dqHXIs4!8d<*o+}T^nT=_26l}TGkcWL} zKXk-Y)CQoXws+Ojf(bbTRHI+pV*$DbUqOSgL_BVWP=@2UKqi3!6yPo0d1?Zb=+q+N zwoWvMIFO8{I(lQQ4Sl~gQ(~NIx>GK|QN^sL7`8S%Kwo6;I5p`Gdc@E%eiLlid4B&Z zwJ2eKxw2&z;#_jsMLw%VO4##vzlTHbMC({d37vN`wG(jlx|Zw2dr=l#fgyoNsF&|< ze~yjscUO4AFM$2gydPa#DHrPDg-GE1sR3+TUh~BEvD=eZ|3X&~-k<34lT%~T1HC<{v4pbxo4 z%o0HJuy50r<|W(vm1_uzb#W_OMqU6r1ZkxxQVFrXB~ z@5UUeQ+`;oYRnWNW*xkDT{jT8#&F0#tRY_Buf+BN3f-irEU2%m!yS1aQ%l(4jvD;1 zy^2nBI~=?BY4f(}YgA&iWo@tTHt3|P8Xq1OorA2oE=bUsRfctdI+KO|ZnSb56$`Yk zH^`$v=S29`%Qq%FCIm~srlmTYq8P4`s*k-ytw$Hew^v>3goPO9>rQ4^j514gdsNF33i#YBuo^ba?8ZIDB1(|ECEV07ha02tbE;nK4S!nYv zx10U^=kp+KJ5T?cX)Xb$^Mr^D7js8^)KfhV?h7^<7|ynybrMMs!t&LcLfwkq!vJ=~ zdOu4V(^#uX$XcfI6KK15?_~iUBstORiNrTR5oxVIw~)R*8HBC8Tg8xg8{I=&}@wjhqzNkFbB!^hE zY^6q-{=RK4q`aLnfNt2y(D``|UmWzFT~WS$&1nzLu8r8F)XsQ&zUDbEv2fLn@`~;} z-DI@7?<44?YgXSc9M8=O<>R`zqbPnq8EnVwBlK|twrQ=ybe&Km{P$iI%Ra&VJ=}1W zKF1~c#{tlh3=G2ZWr&kenI0-XAZ%)qV7M^2 zu(Hf}M2OdQ1zYh(;Kv=t3HctIj&HR}xr&vdLg7o+b-U~aNZIIABuZAzt#E{YKf0ik3Itu zY!BPI`)v?p^k7*zbWmCI<(_H&H5uD5M}%NTY`PMgKn&?dND*?35J01$E=kT&Nn`cM zGqLSE@u_ZAWme^ohWX$Fak_--eifdJx?)ndZwxiZl6_E{qp4qY2!KboRdZ&Xv#+X^ z#cw-p8HF7IS@%3okFwSTQ4tz`MWS~tY1;{y#JWUuyI)1y4tB8+Z@OhQ4r^$Ghxn&C zZQ_sIKE)WGOD2yTf!)e4z6eA^`Dh`p&Fm=`!{FAfUm67I+77lf@oRwlsE%B{y6JGt z7SkSLtrzc6WwMwPL@;(P$Z$P}b{4Um)&~^I#T$p|@bWbWb}o*}dj*?5h0HkVa32+O zFo(k~w!DWXpgCr_iKs_sbg|FzF1AIw_Ew%FMPB`Kyw>!{t^4{!?0}t_FrJ*Ez=9TX zWij!(wNj+Ki1^uL`a!`4mO{zfq!Q{mRxYSYla47M0kYj8p@%l5p(!WWTpI8yZo|;{26~n-WfoawjVOL9DwtEUi_b3Mcp%G5Uvdj z^zp(7bqo%wUuhY(sj%8wG)WcjURP_~4v8N_PqRof$29b);hfKC$A0QJbeX@bJ#2O0 zVNCYIvvPANA`<#(DZnaDs~BgGlH=X6xt|DKir>qn^QFBs(HHgNGkkXnYJeTMaKRpB zvoML{3sbyA78=+kz4b{aReOe|q59+d^{84LCF|;BJt+bhfW~CzQup%p-xGHk` z)q&Umxge?r;lSHy)r?(YV)#P&@3#|9hF4jVlm}>qyl~XQGx#=mY+MXY;O}JMsp5!H z2D$<;;F8c#$W-5RI6AzWB!yWbIwBl!T@;^!S_=uAMw`!x%MDl(hZuuH!6*Kp*R~=1 z4+G;D-RBwE{yOg52z}&(MzLY}l{Ye#sIz~5d*)N~rKJA3=}NBH;SZWEe%X{+Pi|*?>CxTU zeb(ZYn&SuRmf4fv)da8CGT(G0Wk<2KWA@;f6oV#BhfzCh8$aK+sRxX0+sC&Gmc15S z34LRvBc?rHUQzT88>P09S6Yn{Z-$-S;9@KeU1^W)6HlD*wtQkED8JM)h3>m-y8LIb zRQIk|S8#XI1%ddCqXy^b=?gh6BEspPNDv=;EL0m3pIJB{;N54}DV9n+$VD9adbuDZ zjG4~;%q;R`?sTmo2QYBl4#*CyE>0Rpe=U$Qy+yHf!Y(as-T3Y^wW<*&r5k3~LknkQ zs<7^yQsd6Dn-QCgwS@~uD`@?Sw%Pg2vty15U{rkZ))p2Y*mos)6&JVj&u(L{-!It} zz-rj)b1~#JiYe(lkt6_#je_KvY+P+^Hoc54{o~fkRi!C%o%vPh8o|rKjllXf&1fguv>+g6ut+paF#wr$(CZQHhOProN-?%X?Z-g9Su zJo5+S6B*wp_llJ}_g`C?Vk30GUkB7I(lbySh2Sdmp6i7rK%2uA9gvsV)^O=KN ztjlUXEO+5s4)wdzQI%1N)J+)gAM0jlo8ir%p!wn9ylVL2T@7J%s%ctZHY^Z@In1ZK zAFr}!5J0ZZ8FyP=gHad`81hD+_b>oMzx;ksRwM=UjA`@yR|hh3Cx2hyNK{0ZlGGXKyZvxJy?9;fi8(Z!?wUtucaO z7vk~e3D0ScEa?V3HSk_kM*Hxf$+H@HK(%Jp%ci;?rGCq^lde@SF74lzZ)%QfDHt}Q_=BER7~Lk%PN>-GxgC24!vd2qJ-T3P+l(6V=fCUD?5BZHlF1y zc82F&(m)oM##_ZR(Owk$W8|^taZin=J6W-t6M&&E7yW@)8aTcps z>_%s`?oN@uWtI%FFN6ano!JM?C<;@4(YZ>~Wf+$Y zNXNDzLrjvLB!wy#%7*po!*OvLIA~OdDj%=1J8qwa6-D`0%Jo#zc==MAp6we=J(=qt zU5+A&l8=kW3B!Ifq)d{A$u94tA{~krUn!KIhyKbV0a{FdSTf#k(<8H1{!s z^2($UNtzJ{Lq`V;5p{BCnU_oIwpbVzP5b1Q&Q{*zD`cfvz9~*~&VIcns^3?}M5hz9lI+aJ8|@ zBeZ6>Jh)@y1jkIQ8<%W>@3P)c`EMXz#)|~H1~O|wtr}Jz(BUbv$YfxVQ7)xiO+=0f zs65KCRb&t}1+{A~QAWGh4sE?I-8(a74_#2L1jKHW$?d4-GnbY!4^;H97>{4I$q(qm zTi$3EoZ-0c>$>8*Sbs)SElp5o#SZ%e8qI^Az6OJu%oG!C_=g*3P7TDKU^v zo0!1f4Aw%O46Fs+Bl-guam%Xz93%ii#XjDzDr*k7^TXg`trHK5qIqRgP9UXUXpLkl z&rC&Ao!hle^*^M$BCY#m#+@Ct1%Bs>J>GUBZxdG&bu5_5TR^AKnz|c2^m06L3O%D^ zz%P$8)ySnW4|St)Rj0j2Oe&Lm6s&@xJFHNEu~*O_2kiZ1*8q~u#xf4`dp^cmHC5}S zofveZB@+%B1vRhrXg&P(Oa;!XA*PtA%iAhFYKE+D!Ue82Ts6}p$HfW~P%>Lr+oSlC zld-^PrAiAtT@jgL5{Qq}z(-r9jYF?|jX~IN*h%=Yz>b439?^0WcA@FSX;XQZ(=R@V zO6SB_APh1t8bf?Z+sOdYAJ^c#^sM#)-jEijRGq*YcZ=LAdxhcxN8zJvAj4QKqww*; z)1ZBQU?zXFlWJoDNBA4#;FtT#DwPpz^-dMSId1WAxOxi0Y=rK#9uu zqC`E_6X;&75+jpO@BBiJhAW<6@~t9BV$J+_U(uVLC6&n56gviNCL9qFsN3T;$m6l?R$!T<&wi%_euyzSq8Li+ z-4{E;yvxQ3_s-x2nf;3653o~Ij;)-fZu)E2tD#wvt_!5yk~%xt0M>3*o$>M1_V31q zks?fng@Ws4=n7x(k%FUcChom{9M~a!d-+p3ccLD=i~BSdVX3fA`Gx7UlWe6UdhYmW zO=&vKQCa)CcxIfnr7E-np@cI`V`mJ_YNG9iH5V{ICH-wf&$=qqYf-iwqx8+c(&vRd z&W>2&X#c))Tj8(sDF&<)l*x&aFij~P=&(0-kR zekS1Ock+@Ow);gicPb>lbF0u}WD-w#Yvq&*`y?iCtCVn1HTakC$7J_oTDSGk9?yh(hDk_-nN6uj`hsduX5@nuR#K(`lWs--)!t;F6RY=vcIQ+Jum0o9WAf_kt}u`2ke{sZ_DTi>v^^WsAbJ_+BUdKF% zop95FmD$}Lcr(@Ez^S;X%P?gm3P00Wl4}|w94M4Y_O`g+!J(TR`uL2W^0z&NwgOV1 zr@}=1`)zw8E=HYnFLJWcNKTh6uiX7#Xs`3qwv$1n>JSfyBE+mXwZM4+U}K=-Akvhq z{H@#c&Xj0o$RL#5#jO@03JSL)N;rodv!Z0sVtzb+c!|5przN~S8IV1Iy31us&_47>(VY3QBxHz8*l}zGILZj>L0~o$E4;1Q#`m_X)9|plFijM{hn8kdkFy%gSP}&Ws5kmvaVjIThK(0fYs89kKZe?c+Y!gEMTsD!aEL-4JlSTqUpX6^2H z09!~GRlqMTL0mwGaJ_I%TFJb%A_j0`Z&RlVgrMs@ZLRVmHmT){$uM?Hm|41f? zX>TV2RSaA5O5scga#q~SKK!F)RF3E!-QRkn9AXKp!N74k__MNcea1jnKwL>%bC2?Mp7{krpfxyNpJ8Yi&;TNT)jn*0YzqL6X5=#i*slwkhSgKoAc^NmWO<~1W~ z_DAL?Pe+lUSDC`7lt(%$rC$9i@ZL`2^h&L^-vH@4<<6|GLqVgJaN{lXA|@<)m3y@l zGd{FohK!Yg6Ra0!MmcCl%R!L~q|wxCA=(_aOmFCzv?C!WDCe#z9}5~nO*@XwP*l*= z?`>dbU0v(riA0I4vowyB;7hZblvGbMC~_N0>{Hf8U4nbNhqJJWk^VnTM+-W;J&o`! z=1KI^G6PKK9F<9`>-PP{t`;M>;^>vt62(?Cf9z|JuLhOWwPzL7`E^37c7fU^%axE_ zmuW1<2C6DD>qH@i`>mC}N@*y8*BHRs6$l&Yy(2oMw5hQfNil^p#QG9iHj%dQl>Zd1 z$ny7HmW8UX81j%Y8vqEKA2|0;{_)*1p|JobdQO%?ngd8gXZM>6xghi;+n*@-0|XrT zXQq1G_k~iUS>rEeZOugdq!BgWBWLVsNa__mAxekdhj@a?TLH9u9W$V8qSwCft1a; zf9wNmKYhPdpiaf9dGFbwfile$;W@i#WERT!@$Ok2*b3M8!!9t074(sL%!D)?3;ag< zn)eL=M5N~#S}<*d)D)~LR$0=mY3o&yOer=~phFptdzsR5qJU+T%`uN=@D>s!S=7(v zO#ZMPZ6e@ArC~g@ER3;Ig$DW1RIO7*0Os37}Rn8NJ}KRSZDdPJZ5 zt_QDC+nd>`o-Dn7vM2}@DBuG9!N&PQ+6}5r0|U+4Y5MUqSrUP%P$miqGrh z?LCd(Pe`UQyEg~WI34!Wz8$+i5?i3gre&26?K$;|UAcbd@~J=T!EL1O5tit9H32U~ zO`p(N#oOM`6r{Lzy|5iYa*Zu0Kys( zFV2d^iibs8W}44$qCm5(9w+HOdLoyaYTJqbmdIPSH`*gQ@iP#3+l&muI#+cx%YfmJ zt8n)2QBgQXdMnowGsH22mpHN&sBy4^S27J)w~5(5XzCdRGoB!1X$NaTx!|3!mb@aS z*u`n8`9a1auOkn3EgfYh>jky0fLEI4Kyf9`wj*zFB{>Bxf%Jrhma>L@BEwi3P8?W2 zvWtb}fV3=_x}%cT(eIkO(6xdo$v%#@jj}}ycU>fWm}Isj4K}kR!;KGce{Ig+XAmg6 zXajd5SEmj)93l*mhH5*_{?kaD6cSmF5!6;B3wI1=LSVOt58W1{^#}3!0kMS1Rs)k* zHmN&Vvgsu_23FgTR9dw55ta`LIRj&}soN`tj5!wQ@iM3-hbZmILBH3{JVRIoBHS2+t1tPNe6%=Z%xv}hj0EKYV!YQf$$0YoX-5x)^{nyv z$-O8jbEbw;y=OCgk%Lbh5UAMh4qHVg5BZmo3k@N^XDsiI5E-cxU_KB0$tk>!ft|Tu zzkX#Cr^wtce)T$!4z8Z;oIK9@-T^#B2zrq6CM0rJ(!?eigKN~qzPa&mu)U?+%&*Z2ORPAW%FEdSkzb^y2_y>JDkx?oQ0N@|BzRf+IP<*EQ!R*H6? za*|Ve;{yWUl7ohTHUcb$BE(lKiWWK&m5Tx5$k(1@MlnJmrpA5Zo)z?1%6zIKY6mdo zaW!DN|AGvwUd}kGG7^brCMm>il3u-ly^keQlbo?H`td6|Z!|$zlk|3BA(8r(pqY6H zWT{_4W7+~-Og?w=RF#>il`_^!_W>FTt)AlDpQ)o~SLkT{PaeWs++B9pN8>lgFAj7q zn8o$*dH?IWKV7ETuUhX*Gk<|-gbxtl;a`GCDXi#zc6cVo`OT<=oG?M7<5D1TR`x1F z?3d4$88~T`^dV|Ad6|*CwCxj>x}7oWWrS6o>%QhVv}Y|#nqqlflkGB^OlYGm&9v5* zuSHEuxj-7&SyCGGnDU$9hiK=e>z&y;A9e5d<2oQ#9;YJ@Y`h1XMO$2+C%OYEoOGOn zcKv@=>p7=zs{$Z?_$LKHr(*y``C-+nlawEJw~7*tUyW{*drmF&JsJ57!8{0{J0dj+ z6G>GWTBXcy`1P13F+m8cZJ$X+u+Y;K2|6_}d@MoGbipR2C5{MX1%cfgj!fn%gT#KB zMC1%fE+kl;52{~ULe;hh5!@Q||6b_5Z2$z+uDRO4P6p;6nT+`2JG!c|KlkO(3wdHki>h(oanku6ERXdPgDjwAhMV`^iZ7Bi6j}{&qK&RfONf%YRDqH`qDm z72FdD36~r0U&GyKEkqPJo@+3sPol=p)>rk{^TTg^!NlfGc!4?FZzTT?O>a5L(t~6B z>P9$8aH-Vnd-_lNbeDy!{2yfTd5v5{u)(v@QjYxi#fv2z2vBnwMi9}#VntIwoyAkv<{kq8pS@fJ%8E_gu7UV8>7kX@bX+p+m^gu;y>%6MUfDsJ95}q zbHMC(Za26-Vof#;^wo$Y0m2y5uynWEQ|=5WKw=qv$T-jae0()k>KQPDWW;2de0{63 zv1BP$Q*g|0TsvB|AHk#7iv!*`E~q!U)^fMvOlw(co3!!K=w~5_vV1+qEnB>^H8HD$ z0A}C%*%`k^`u+QDxjP-^@B(gADhmK4gXsrB}PBqMG8Xl`hD}5mIoNGtQ?p9 z(N1b>i92#Sv1zUL=ZnHNd^3BqQ=NE=^e8ux3E#>?eb#})-Av&AOq2S0jyscej#@%j*gULFC=b# z0Z~)aGSkgjQ;!x;Cd#_w2*(yyt$ptJh0EtZEl||Xdo8|wJ8IRAwZZUJ1W^lyQ_h38 zLGN_vXo%a$!wp4TMWbOM(M8hl39JV4Kqvgk%6fo%VKMY)ND>*0Vetwi}^Bwd7T%x@b7hnpkR`Y$M|F{4)GA@0Fn~Vt0?m=C+ z8lIU*CS9%+a{CBi*dj)U6*Vfq*+?GvGv`MBeMBU!>Oy#XY1 z(v=|$QYnPRlcq^j)u>_^p-npSzKi5R6@Xl__#2n$RTS zb`szDPL-`Hi~=zPZIBm+uSX1>Zs&7QD;_y4jrbwXho0UvbQk2uQP^=ly0Xo9LyE}K z5P<9Q1doBTdOD7y(dGp~i9M0WxMu7a7-TdTsH+}x9~n>pZy^}~95UjP5)n%&*A4pO zbD-Vh!#x@8N+h&fJBl#WfTW|3f4~$f7{U6~5^EN7TtdDI7rYtW)^){^G1Q*?@e%&s zf`qGKe5;dk8AZ5IEV+Z-#52gdjvz#YGe-q%%-l2{PZOmEl&Rhp7ZunOE93JhQjOXZ z$pP3=PV#fJ_GE^3ta{{ZO*svhGCb0pxR8+*wC_(+YzTzNQY{+i3{rjII#^ z&z^`p1ZyM)Tmw)^6pX(UB~C@D4lBufTqH(dnRFf`?;)w*QblXf`HKg^o_kA+++|Qq z!`sLi5uGe`8z+}1P>$b`UJWuPhee;wwgsTVpR^#K9}QX_iL8SG6$U=4vE&ml-*Aq~ zmFg?pBuX^*rL_fOl2k{pduC~*9;4D&nbvE@fnVoTfzHY&GdF)0!i`b4u`|i#E>9i~ z_>H9m{mtdJkeOR~;K~iMI--|80OIYUVG1CC6WchwDGxg(kE#PqFNORQ0fKM80sjhV zNRtr+H{b;TFhv0T2jbwLXD{RaT}Z><=NJ6j1~mK!bHK*V+QIrycHln=XK2JQ|C1Sb z*Y5RG=6!Ihg?|pD9dg?Va9B>=8GGI2muHT~NQ30M*(2S9=o$6fwslYenS9*5l`Sfg z)BySvlu6KioW*)mlDjaBX&k8+LLAvD5r64EVFyoZc^K;0$F%@34g!}J2fVg01Jo6%fR zSh1mski&OF0S}Czo-&aoA?YhMHzK^$wHLSpcZ4~IVNp#%^CaVfdSmXNy;Ly|qQA$O z(A)6VmEaU|@(NDrOpYH0?%P(JMt@9(WDls-=#htD5AZ`Xjkm%n!yQvanu;z_Jw3xq z@#Ip}dEWJ~+*z=bBGgIp7{Y1P(2t4M5v$#>BA!7J9L31%qyrU5v2{~F+Ve%m(T!4X zAo*Wb{e!zaDNoav=7r1eh` zktpao+O87HOw&$Lm&-unB|eQJOYz0U2#MH9^>Q+eP+@W!r<1_nc38%ulHYS82nG{P ziot_=)#+)Y9sCy66gayDWw@h!r%{v0PhK?Hk!oHy^TNdL4gCq4)2jkvH1pU;&<+pS5)OZiYOxI;=Z(1^Z+TqLCb$4 z9qtS6hi~}vJ-XLSBg@C&;UL?C@B{SEny=P)oi?;KY{z=9hBcE~-0{Kb;9+%gH`n{& zoeDUHl$CJ&>>YN`g&fo}?MQSJfYb)6|KUla`3`3(Av8$~XD%XUOKbS9?vY!6vOjy( z*CQ&`m*57Q$g~I_wR2rY_r0+K+;F;fr(LjwwL@e}t0i=CJUCTE5u!Cq_SDAfU1sS1 zI!eev7kbA%%GF2XFVwL)XW*r8XyP5a<0uQ19{0Q;~Jm z%s)m)GdM>uobLEn^6-!GigITO=UwXUNj9veUcC)eJgHWsl_eJr*W9e(CP(JXsLftM z>Pp|$hbyy-vP9@Q+$bwhgGPr)lay#C@l?xR(~Z_&=*qto)-fiIZEoxU1GHKs9$P?d zg~QL-8!Fd(CAs9+(>5Y0t7X(?bp>84z2N?@RqS{##Ui>71#gw+T z2EB17L6Mtaw0TpV7YsFsQIZ`goqP2taALF@UWT4Q;j=O75x5E} zS>)jN0Sixd9zQ=#4^`2Q;!qA!-QYTlfy3XiG;k$7zd_oyi8<{MW9DCxaOO1UeFB5O zQU{d-?p<`}vfF0z!G%BR!fvZq)`J1-!eMT#;R85v_3=$G99*MCrU`HXL^ZN?UR-G| zKhVtJ9c4zPE}-I6*oV{Xvp^aPzZU7 zI_Lr=LR2q8Iu{rtzg>)SO!@4SM0a;cHn1gm&R6|YM3-ufo3L}BuQXKZk`K18!zQX-i zy!HZ4E1bzL%+a31WwJGlcn{DaQ>kJYM*xPC)Ta#5FWyIx#@bZZGK^ikdMKBhJ1cHq zi8|E~B!al;h#RB5-?~bAS^J?iG$~AqzK0VZ5b?7%97A;OK(@otHrNG0P4sKFpyrLU zO;8~OGR=E}#8DLq((L4U3AU|2kF#(c+?q2sZy_OBy+^}+XgkD}28FU6_%SM36FjbB z4YDQUQXGYd%acWr*SMe|hvZ&$q`^9+*G}V(Rrd>i)*o6aSu|P>GIaFgwj*fK)~`8@ zylrh1EW22y=d>`hDk|w?ww=&&D(x*3RfzXDO|H}BGFz!-jy? z_%GSOLOY|~c_mg4&%oG8CEAdzN&MbJ7**9B0%r;^6%x5R$MumB6pwxD@I~%KX{z`0 z6O^(i+kEF0heruHB!9V1;4?=z=ubb&s*!cnB}E3k)YX{pKLYVOFE3S<(z~UVmX4w7 ztpyl%QYmXPMI=`bx01+}T1-NEaQu@kNfgMEDI5%kr5lel6RPqS@0L_2W^R}RjV*@I z$3B^MdFly(WUC9rWtE1dih*-WpXXYYFd7UlL^Z^Q$^H9=Z(m0Ji*9Gz*Q$Pd;U-C?%t*NOMfPrej?vSI%NeW%%+9*qq z^M*A8nSW(6AbXFP#u%yRKpe#$T@&i^yi4Cqv|wC{5yej?MKVB<0jB-xCiRP1p-|>k z1mg)}a9EJ1#9)9S_H!{;0Xj)oBSlZFljPsdxv#vWkHkebpqtfF?v?sfB-fktKCQxw zH05WCGlN(#ooXJFvLx{bBNFqkU?%wv5fKt)cOskzUtk#1yV=*6xi~x!-&JKE(*M2W zT1}#p>6pCT%KT<&@3Sm-(2gta>oUtm$u@10 zjcvDgr!y1MG}q?KL{neQqzj(lv{JhfKUe`Oc^_sUeFAa^BZk0zJebEYLl=tgbhw6J z7{$8}PQobnlQ&304Wv(Di?)9o)FQK%dxcTD{0jnbIbtK!Oo;J{x3oMQnPLfb!uBF7 zMoAA8_37-_B&ib%f~J-xIezRJpI@sUua>o{-?}n75fmzKq0-QwnE*#u^RtjL0K$=Bf3`;7aLxW?Fr)YO`FS4Fu_9HDj2!nE{k=6bDPKkgfczr!WUm&;LEi(OCW zU+aad7pBf{59a#{*a>*dheVELnJfZ$I`oIB8Un>}?GHw8v~R6PY(U_TwpnVF64cE? z(iR$*kxVcQUVbIIwqKffy^ZEFGtB%lU#)y41|^8 znUtVw4NeISfsaur$SxCp5EJhn-dR?})(wQ3uYw^{uJymIK2{c0{S1{C(wFL}Iew)L ztDyxbAqx_38w=MCiG)cVn$aE7^`Yv2KC0`94cc9l8f&5ogq#|4KIMny5~75<*>n|z z)zZ>Dgy5Hd09MRPDrs%Ree&&IcJ;fEg&~lnkMZzEd8hFjdT)$_oyrQ-n>(#)vq8}& zo}HOJsbzpZE0s;-frj>cC6DEBT_CK=XZs+;9N& z_P*@}xO)eic~iaNx_ZM4?ebpxmhG-E9*1Tids{I`)f-F*9)`f+g$-B2Zc7}WUgF&o zN8^o-(v5rZirphc&AEg^!U7cDEI#npp2paH7ken)3Vked}r~t;nkavwgDI}@7;S)C(H8_epORn!!kL!APyiZKU~0m(16rwrBvjeR^}(x zkMdy#B1J_Rqw08zwLpb&-tPpW)=Qj^@ZQSY-&$K0>=jfJ?3E~_*16wO zpveNb{iE9#=nyyufYAQk^lb)Q&3Qsu3C1o_wq}bzZr?n<7LT`X`OERVt5d|69hee$ zlzEw;YTXtZR7-KA`3?9=@3!w4r`bl$GXiz8^eJ+@D+1u1{ptQ)B5ORFa$n+>RLYBUm= z!BJ86Adr2`O8hv2+D9VzJ8lDTP zbP!nyN;&R4Y}$7R!X0QTx>0^LtXt+FEUd5o4_FMX4J~c?FD>fKOuKTj^~VtMt}De8k`wkF;!GArL`3NyqCSw{>2UU#x+##% z42N~cBA6c;g`XO2^?&$)Zs}S_31Pwer36}DcYfEb=C~qfZb0W^%%&Lh`B+`xW;oNg z(n_>>>|WaWdxmgYW2jJ>MG$ASO&xUxpI5nYA(%?&+Oq^K4y>L)pOKd%+QN)R(>p7T zUyy(GH=OS39*X|(8)51HRS4xjV@v&u*Z+Tm-o`(NXF~q3@JRwM>g z%iIrz;B`$wl4c7&c8Lu{3UoTp@9eu`qpJ5mxaiB!S0MvXQ#$nRSkr*!I$edtF!F*s*4xnt9VR>D_9%oX2U zj%blP>=xm3fZv7yjr#?@5TmEaBjQgXOR!GzDZMcE{UHm#Bgt|f=-s_7hK8nG5JlmZIQgu(vB8jw^AZuix!sUqiEQ-Af|Tbk1ve}aoo>^%Tb6f zQvPgC{TRc#2{4^cwc3@C2%L=>$Op;t=tI=G-BlgjY-J<8(&rva@8S~MpiX6r7f)R+ zOme9mC5QagQ=MH8DWguQF+HfzaqLY8zuov6cl?3$TkSPicM=dnZd4wU{=C_- zgSczV^c&KDh+KIHXE`%|O@S((n&ZoeP84*3zzk;Y=XsF`@sUkZ zXo=+{91N<9N|iCl2tp(rV-W3|39D8BS2NA1_uFDjmJggh|ArP{W?}8IA~LMx!{TVv z^>prlsOvD~{D!Ve)X^lodH6NHQ-M4=qMKr)8|Sp1u+K)WE<=w1wQL|JkBn@ho@15u zMheqwLVYjq&bB!P!T}gH+Wr%KREf{BieaxWn^GS_kz zk5@N*=~_&N-t47|Cydc;mlqag%j7BClQ~ywC)R5huX+5Zu5S^95D%@6LKZaqvFrez zvwoi8PuPvQCIaP+AKXfsE4T;o}4=eq$ zBHjwD0NA&DwhZV1Pyc<9Q3uz{{D{58&mSd9ZlzX@_{cFUI_w00eYTIk?(fM*62PuH z%iFC5X}LYyZR1Jn0PYf+ki;0D&aIIzxDZHp(b45eawe(Dc6MKeB>H*Uq~8IExS>+E z@8NQPt6MJwCuNr^8Q0JU#Jk|+j#l4DkpqTV$#>ae(HX?xXre*cWmI4JowIb?MBHgwct7h=>li`oz5D1yIx0k7xb~bPc#WG8gfXC7=u%sVtJ5NyZf>W~&ZJP)6 zW*yg>MC#u5Fb;P69NjV;+*)uOTwo(TWt=j91FU7q>AUUI+VKAj)x)8W=8$NG8}38F z2;31%Gg-UiCw{;1;U__17DcpsF=bFwMMCzg#W0OWTFo?~#^9hbx?jIT(At3>2%Xc~ zT`KV|S{p7~;NvJd-3_EOFO{yAn_`W&{1eO5K1E0fZ|{v#>DMJXe_OOp6|3PL zo}*J&juFkr#5?49Kx!0R!vN^Xv+(q&0yw7=pqY0^nXe1fasjZPbLEb7 zN{s?ZTTySlzZq~+7Bsm2T1B`jUy2J0(7~4;P!wMr3KsR?4WBhcNY)WnCoMer>Q?Js zfv9qC>Z(wm5$ycgZdot5$i>(u-awi6_6;)b;3`4L(V>wfaffzJ<0Il5%n?X zNhSmdfjB_LGw7+!Ug@p=>mS6omS2s?m_Psk7$EkoAQVH0`7!A0 z8M_oUK5mamt2rS=nl7(0*dw6?tMq0LPybRS^Fcs{Y&7 z^+k`E*gS$j-RDu?y=wxT8Vo$qmTdmBUk9Pf4=AlzOY(gOI8NG}W3{c_yjPTwFu;%Z z%Z&ROAq9jWh%vt~XT^z7Joumd@+#3%r2~Lkh&ur9fkY%-GKitX;bhZ>(*U`6#VAM> zX%?LKI^C|mI+9*lJFkYX8$#r23K>}@3_B|H9YF~sH1hit#N5#aXA(pE+Qsv)V6xKN zm{7RndT@fJz{>z`VhI6siQ1*}?s3&M3>n1^?A8wDZf$1RTBwfQPlzuiy<4HDiKt@= z)=A!}L}DWzd3es#PrGPd6nLcIEsY0x01d$0&xGikAfx9epZUFXZOmo)Xc^|vS2{gy zFbHA@nm%NDT!Pg)-NM17CyS#9TwLs)EE;)-;BU7lhY}4Q+yeLW@uwEGydpA>=XOo1 zeh(J8hp0SK#r3PX$_Y8Pt)AE^4g!ZQkT7pL5$-t$vx9^FF%;o5xo8v|smrjGs7 zjUBt&^f0wpGc)gTIN$RYuK0+aG)ROF*;l|Z;+z-!~MJYU;DGVvXAmUPX{1so0+si znP4DzLh6DBsX(>XRiOl%^#-#_dy(kgtPb{^E3g)>$ z>kwTA1i-5pV1+AdhE)LmaSMKd-U-ubkfK;Qh_x%SCX_ScoGZvoFd3R}luRd$Y8H)* zXYP)9xv5_O9F~bnV>=CHa$AlfPSgm{VKR&>1bw53=>AF(=!TSNs0k3%L5^N~dBAZ{ z{%amJCeW+9D%M#zU^S9lO#yS@Kxh3|JIOO0NNTlqmUmRn0zdrg&=i)|q)ZgnYr2c5 z05%sNw1L<`&Y26G=y@#%FSvr zH;iU@Og)Tl&NC&>G;V(q#8Y|2_k1KUj&r-Y{u2Ea%maekhEszHzv2v#?>xMdM7tVd zHs<(PtC68*$*pC}e<&linSC!+>DT`VUO}y?Y-&)?WKTs>R!1^zWFx<5w0{o4Qfwlr zP{2N9Otc3#Ru1A)|6~?EBzC$7y$HLwj$d2~qpG5*nOAO)K}e>cLKVdS88&J^kV}M0 zRt?DoTUqiaH;JH2c>EOWn}Jqn1P+{cMe6 zk=<$H>=T}E2-;l6xS3|^jh}TNJZr~twwNGTQyqiJ*qWgc$LXKhYD#|~u%<|r z(#WSoc+YMpBilxE&8!xRv6xOr_(NejDnG#ibyI0UZ1XL4ct~&Zh|>3`C>s#D3lIm z0UVhw5NazSOc50XLc2B6b4^{`T`ubv*pPQtL}IveFtYp50#)FI12^PGVptwr)$sGp z!|SXVW4bM;Y70%JQ%Fe1?DY^263xTt#`05;M9bHz+%Ktbi|C8yri*3&dX!|!r^|t^ z^VUyj@mI*Lffgb5-`aS5z!?jnG&C+GQ<>G)W23MUqvgWLj@&{}7@ws2vIe+R1wQ@v z%C0SkOXAV?x_MpeGAPD-n>DBSQoxUk-ybg%OOwOP6C73rA36z%+Hu>>ke=>ev|Emm z>Tz+9I$jf(FBxT5m{seA7{$v3ZK)Qz`tCk!>?XAF|ojSk>kOGNF7|agTT?P0Yf0ZpNmnPXoi* zVe@GREe#=kw8{Nvg`(grkcMU21`NvwK-tkgw#~C(CZ+sMQ593FJiH;7FE}%!u#4^&g6`ml=k{J}-#GJ`9d9b>3 zQ*SH90tzq#j(Rj)^1I$tHfi4zT@!U9g7qL;(Oblm1|YC%K{PZog7lIKPF`awjHGR| zY!TrydH(so_LO29(G?cIpfOl8e~<wnCLdE||#|KOFp`ix#)LJ>$O={bWiBcREFUXaZV(~FlK>}za zEL`mw2U^?mWH4R~^1>Vff8*^!69g1p^y#z3bH*B3Oy6<~zi`d1@{Dh26@wL~Bl3smzsFlb77s<>u75A=+5m2E}q)X1{i28z@O=?3ehZ1kdN z3krU8!h~2)eZ5H-u5p^v-;(TRLhaW6c1(xbE_B+|QJ5t_%n;lK-c5!dKL<8M^Ld?q13*aHV!A9?T8Ier8uy0m?=@iK%(O%F{Ddm~q00 z<)l%MfCM*GD;-|aZ*_;M;m3`*JxH})IR%G{LdyMPT0hjBm2UJXu*MT4Bif#K(7$x= zZYoEs_)mW(@n805NdC3%{R=7hx0SrVCY=9I$@`zf&3`L#FF~3X_DB2j&9Q4kB9E3V}a2>D|Fz`i@}T8UC_P8;?h4sUpc24ElV!b8z;NfABi^rk))Vh8^yP`89gJ^9bz{V^F9FAXx2KK43;s9{df~tvm$9eq z+(o-Lig^oX!{3{1WUF0S}hWyERD@&-o;LezBmth%3MM?`~D;AMgD#Rp}9( zFl1LhX^`j8k^JOspzH)>mO+{lc*6Vfbc{gvG~{p)$s^GtbP-%C(QOn7Bvec;Rlto} zekrrWmPXqlscY}wBXbeRxJK;7@F(~u+`u|e2Zi!vq2>?4j_E4yqHvy=4{53m>iJsY ze{}biVRa_U)Z;ZCS@?df6Nrs1_sL!LH%g091ieZymz0MEn?o(dmW|O2 zlWNWe4gHO<`QlCN{}uif~ERX8}kL|yK{O-@4KHq4Tiee8qq8MfB>J&TPI4Pi`5(3 z9+Px9RPM8u!i6%TRo69UjVuO7!%9hoGk`@FvT{@L`}dRQ^8$EcBd#~^zS3$98@u9m zrgow-xttb%V#J2P!AL<#2yTRM5ORf0!SdegFFneuC@ZsJt95@-tG`d|)-k(%agE#F zIw|Y1YOa2xcU^Stcz@gER`$aENmynF7)wJLpUB&uM50v58&2@g@n3nc#JmV4q0yGX zi|4nk1L6C>Nt_ez7{Xp3F_OMul`X|^Wpj(!-y>oIi6Bd}*D3eJZqXdJb#zVE7#d(g z;9^GTz6s7k-WtS(INJ7)IH7e%8aEIO&QnY29(Hx^VfHlzH@Z974)})TwncqqiyXV5*jw6v}f%_pw4_*-~gQ~isjblN+SQ+>cont znx#ZRFSKof8jF0&%X3DH4$N~;2ZLr?sb`%YLndd*HQ5vBS&bzI&03*5=eC4Pe3~B-RFQ^er@l;|>%c6`Gj{S&)tjLR_Iz3t&a3fu-=u zlOD}U62EOn(@Tq=;O+713H@6X-2s^Dc;uLv#va>CP3oo1kxNo6{%G9v7hcWTZtc!5 z(NmJ?4xY@b;GRrORJM`$4NN&pjp|r_Dq+1LtX$yKWGH17O9kkLxSo(@Zjd99IUI}YM;$NB zDiGETAekpQMB&ze)W_)@7&^wzvGIp%B^EwhBvoVg^oDmx*FKe>W<$l1Y-oF#1}B9w z(wR~gM4A~X@t!}14)grosueeM!TI^S{nMH=Ol}X5QNpbj*+H0TNV6GIqckjm+C#7(-W+4J+qKnnVpHi_!iS1$)&w=5$r)G{C!>{V9UO z7&`WnEMB63^b_F^IF`J0*dh}`cZw;Br)hn8mQ)K7_AMVzNM2ZK4Rk|k#b|2dSQ#S; zPj5Tin4v+Xb?=&GX4Fm#sUI7le+YeNJ3Oo)@3WUqO9NM$Nh#C$`64JieZ;GtMs#~X zq8?g1iM|37yCR(&&L5jGB33LqiXJapKO0Sm%>f(T(0CZ<*}9vY9|i>Q=G-QP@m(gU zCWMt2vQ0zcCrL?oRnE3$QVkN$uTdM3%K1RJkGAOAKy?)}l={-o=wQ&#iiHGjxKCH^ zNwYlhDUzDBt?|2{vAp}Acn~*;7AWvguris1WUG%cpEbuk7wdhcK$^XfH!(m4(f5&| zimCum&SbcH*`W+jg?}H`DWw-LU89McjA1M(av|`E^2MTvu>A-QOrkac(%9GxX_|PF z1PR6A#oK`3t(kUEmWh64OejuFaLsiA`y}OEh+|^H#geDmQbR)(H1zI@mTmDVAJ%$r zuiFrFom!*X3C^nUhGHbVcc_Y6S7Zw6QztLY`@$72A&|Fgf{;8!t@`D-QHzsEUt%aV z2hPmLmeo}FgV=ldD#+Z4wDzR?Vbj>+NzO*enl^?NgN@OopQUZhl))frl)- zO*nz}4Gz6{aYE9k4({d0ys|fkw;XIsV!GE$6lEuy5>J?pw1joMCp|Adpt}Fz9q-`s>rpBL&EH)g7TXk?9v>P zqm(MVQ1bINl@F@d!KyG{`i!%m%dX_hVW+T$8YrtuNDlb6+(EBJS+qH4HY2plb&-{D zhE*f$--c(VwK}<9yvctdDgwXy46)c-4S+dRSb&B#_*PRox7NKlO-0*+GtyMOdwRv8 zZ}ZEHSmqd|@ByQc(+6c78&N}>scOTo-w>&)j?AWRWwxedHgvAT_2T!UUspjFAP%hW zYt&8fE(-;d9ne*}S%6qHp$}GCI9TX*c%%S08)*u3Fh-22@iO%B(V=Z2eJd(Xup@UY zxw78Jh1!sA;B)8Y9psKU5=-A3&5R`NKYjDfEmvt?>@A5CX`EeIG}DrzaGTfj?nXxH!quM8b?u(e>_oE=h+q_GnTv?fvK2+%U_6GKB6 zdLefw#2Z$SdJIz|>)cjrb@Q~-!{`(Aq@hOQ&ds@^?BU4{yLmBB*^Zh0g!hid4%FLv zSN{)V%?A2*NqGIWDnwg#py1W2?V8@2g&qr$9$?z9Se`o_&?k&8I` zJL{qs!n+uR9G$=-?i$}UCLSuBbO$Krer)H zpTJ#OAXbRYNla4p3JCL??DXC-R<5I%i>qH3{@C`#8z0yLUnwcbas%V~8b2h2ZtFgL zQR~_?w=ZB;F1tf&Ikq1TUL2g2l~$U5Lo6}kI)?6IzdF#kPPHZYRRJG8HqS2S6JJ_R z_BxUDHCjhNl`Prp#{+)`YH5%+@udqOpHxSlR5SCqR92n8(DbuG@vB%Xtpr@?WG6ne z3RDYAgmjw6snoM-o_qV*v+lN@7a_4g4jGpTd#}1_K65E%5@&>FZcpVtl;k!m&BWuZ z;svUa6k%BdnTSx{h-=;Lt9eAd3)}ssHkV!Yj_SRps>YLYadDUC!6nRjXWv9y+ZtN- zsMK1x+(Ee=Ce=_d)7CAv^qUG3ya4$Hon})UwOt->vXUWE>-*!;nE-o7M&wE?Q}fRP zDf9|NtxZ%>n2Q>eMvE$JPKsu$D<=IiUggv(d24)4+Q*F|8-wm56vanW-}?DCJgfz{bqBXmOm%X@F|F{f*~vDjpTrX(aLjj#l6uxA8!x*03UT~M zRL>$43->h*0l*Q_Aj{e>F{3=hFOl=n=u+S2${_B@F)X;cunI)t` zn7&pLIM7hJx_r|wilxXgl+DEvewo(uIiTwb;Q;FXvsN6z+sFkP$H9D$4rcTxc|@BF zk?Q$o$(vhhUzkQ(38%MI_EkpRhAbC#I->kAFKL&)p(F2*(=H6Kg|<;Q=}yD>$^~QM zQWJFSG-P=Tcq+w7iY9qNd^KlOr?b*(=_cfhz;dD^W0mCHPgF2c^UsqT0k8W?NuFIF+TZM~otNMb(Mm|DMw_2htpyoQ5SKR3+= zQ^V^t&eWMY=1ocEhR?^5FuL_&qt(Ls8MU}NRptE9{K`zY`0}fLHvyisC+q{;rN3^iTy}uG|e1?_KXHMsZcix{sFo?kz+>94>FZ@u*@fAS=*&2wo` zNM1m)pDMBe!KpVvx!P=jkuK2v{`_|PB4ImPI66uZx@m}(<}it-xR8DSzMk5$rGNU( zENv1meWCd)kOrNOy#1+N6ln=!s@(BIZ| zy?p8rA+WCXe7IZIz>?~ zsrrB#R3TxSU31WBnH@E61WePz=etjg{sY_!!< z!2#10arIeXi;(*(v1q2ym)`pkGEBP-&Y0D;lJ={V?_1OYQE{q<01(=u_$LJ#w0tRE zbw_D#s~8127iNKm+B1a^n>9uQT39j*J+G|RqlFTSKG3W60IG7BRtP6zxRr(Wyt=m! z@hDH~`^mp$&&cOUJ=k9zIRg7WnSrik3@uMA?I(p$vJ(Lk{iP7w`uEYdgVlKiVl|S#hy1HsIBZ*hOP0 zR5w$fa-_sb+^9PL1Vhw0@ow_|^S<&XPydl389cBE=!KgwmdcRh=%B({g$hoot2cF^ zH<2+u2n?hypbf6rZ-T8~P#xAHt_TB;maN<-yAS4gFSpy4RV_4umx)EmA&2SoFX4Nj zDNu}gVtpv!S`;s;WO7q7O)k-YeUTw!Ax$P#>HH+P%MVWejIQ_il z{pKsO_`dGj)iqt+aN*41UF{?cCJ#tbA@8GVR4eRIE8~ULoP=`*1-5=!4FM%lke+F( zsMB*u2=sA}P9{Fp=e}o@0y>9Jt}{^>Yc@sCLQC0K)yDz}g-QuUjr!9n#cue%^?W7e zxZPirDdBF|sv7GtVvx81PG=o|Ks;W2jyAJTJgizZ+c6218qoB9dIbnNKiJ0*u_@o z0=T$)nnb4iY^ObLHIrw34qXer^Ts2Wkro6kWNaB%_)s`eUvsA<7x$#c-KK89aBuIP z=d*a{gomp+`d9=<%k((SbY8NLWG{P8iPgND*;-y@!~Cd=wG+Uqvqs?_=6Pb?Mi9?N zL${dUCYtwd!~_t2%^{>y;K>HxGstEI^4!cEwG1QVa1;oXbx$^91`whAG@4htN+pB-w_ z{5-k;Uy|Pc*aZI{3cP)NQJ52-t24RckkI`t} zu*bl}zD5;IFAxz~E2H(&@imWBHC}bsq|}&9(2e9PO)LeF&G&r@{+v*9#s+wAvc(ld z)nUXSvy;JwiXx&(7pX)uM@iy#zF9_dmJd`R$V z7X|OjoUy#8#7uM2#HflAg+?-bE<0wUs$W$?QX4VeJk39)9r<=kK$!s^vJLxX0>qT1 zbggu4n0u#$sOHV=Aa=rhV~)_U3?ocf_0uqcqt=nH0xzCM;41_u<>Sf7y-Eu3<%^AD zaahVJ(@l-{{_7e~8kTN6W7@p_mre{%}F?_2@`5 zC}V?D`VbysKpPXc8S!FexNj_bkZDZg2;ePwCOt8A3|1Ko_mP{PuzS{$MXfr=*!gS;;Xy zf)(c|&!>V?HjaxEGhx!>A8|YeWkbesAmcUaRgfAAV{LixoobO(a$K;JpH;`P<@Lwiyo!*7u=BcecTg8J z(E6z8b9qv50+U1#W@WavmWhROE87)WPvuU+@s9)1PP)mhFQ5^dHl732vm={4kZx%8 zSW{U9&yL**`{!DwYjCk|HHR{0+(f7sU*6H=b`$hcwrL(>^$Z1<%@M^4cc6w(U)esL zXvpMrC~+e35=Ir717EYzvR5?>!Rj;_&u);#Yz#WqjM-g6?6~fZ!TL6Xv4OH}^MOdE zc4%bOT2unidnvfPbD)}y;(|#sv4Dl5)OP)ZteE`c64-KG1=Kq;d|B)CmM5$}kjaL( z_rrOhR8)QQJ`*;w5j+!kun0KI%g#qC6}v367XjB7%~HTGC}GHeSFPa52%e{9eW>PV1Kw*Tw+r(z zZWuk`x*aQ**K6*PT+dDB2o9C_-^`REfFK?L4ZoJ{EnLVSxC^_n|%=jI;+;uMW?QS#KiGkd^k5?drU&5f5c z-#GG%Yk@6+UgL?`Id>Gb)jH_l7WDNL7qm3tNQ+iHsVpx_WUr7$k==fSb}*jrEuKvC zE+pl}anY%1wfra#tm`>;ina#ue%R%`(BUPnsrK>%E1+qs-q%dG6mBjRx zmk5V(T}e11(?_nEH+2t=-q;5czR-4yNH=b^+N0FwrOV0)m%RbmuI`_jJ|1R zpN`O!^|&EQ6NcLzcEwo_wSxLI$@ z1hcJ*l{BnI2+PvxqmQgZJ&V%swLDmxw2-P1d)95rMOr0Z?i|SM8mt)UIItHUuXd*T zsmPK}xh}c6YX-2uF4^m*D;!KC`V?rY9iWG75+x^l<30OW@6yD&9x*$m94$HGui_&= z6RyVOWLBN5>!~i?@2ps0Vd;Ip>_<7ce|W7AI)CVfRdC+yiHT!hd(GC}IZ^}eXX+fZ z$SB8U_KD#0O7K#zRqymFFJ(2@)%)jt84z$mOT>p+mbyOfVq4Rcrt&YlKo}v7x8Bp! z>AL5_<+;20arE?at_<3l@D8&CC`4AZj*JSd?NYVcUns5J`ZJ{H(=}_%t(Kab&y~B# z-=s(auUpH-nAXBgnW7kXovy~n?RJIi=DoA$vY4e{L(6 z-zas#Ya|1+=pX$)!P$jUHG@g=Nc59+Pqt&5wNEA)TryatXS~>iZHwpkjXZaze!>md z8Zk@tf7r->w!i&jH}bzK?z7PO;d^kW;*!-ku+ec?OQ%hOMnNvIO+tixq>Eo%+|4W3 zw0z-9J%yC+nw=u#-2Kq(OSOi5b}8}YSnkd1jydm6p^kQAs~GL9|bA7O;N{MTgLPJ6*;}Z=w%%J_{(LS5V3h1-%pkL&Ve}_8v8J*)j4@%!(h`|7G=-hb473Y z50vv$B$>O1t0aVd{Dzc~8!`6^1^cCD&QWGIg^aC1{5Gs~`u%ssj1THN=daT$2+I?w z3RoD$%XHCMU-A-^&KAyp>vrU#1Yi1!jaQguh^xrD&Q^I;v~%okr{^(^Z6Q5D^_EmB z*JG!n7y`ag%6e?c+^FJ(7mSXp^nOqfqR!G>7tetbTe-yzJF6*|%mPHHG1 zISKw6>t(A86NmZmmyHvgS%D3&MLmb`d3?(u$$m4neSz&3;N z+)TPi&X;Jwvpch~n+tq43LFdLu(91_nFG$|rb9b_2aV}?6F~{)9x%Ri|E)>zjDn*+ z8%;eEj@L^qY7>q^*c)!OoowiFSlt^v(J@@mWi%G~FQLV4$*cUz#bcgPd?1fuH8cdP zWqL+SAK2AyBb_o8p(B+1YF{fyAV>|(?7%qSnte^Iy3tA!mGjd=dRJ1d_cpjvuOy>i zo}>lAFzDN0gXi9FwTqvjAGiJ>;; zIHb~8eVfKTI1=tDkNM67!hcU`1G`TVN>3v`4B)+8qIPuCWGB&4*$;3!1uF3ZC9|L5 z%dVG2y{1cC%(c^q7%rXDq)%@$Xz_7uFlQwkP6N-E46+~(ZMi?Rb z{H5xxfJr3XrI+D}pLvZSjy9g#K^y)|TWB^#{FB4>r=^EA6 zy^&h^ zBb+PD1ukqd5se7H7X%rw#0gQB5|xK?JOsCQn3=bX1T~(;OHWmTeLr~#s`4vKU_S|n zsVje4a;>4uVO_IHuGQjTjo-2fe~*^uBfb7)dn~G-K7@+zZl#%QA21S&E$__!jI_%L zq)vlqKjw}HR$qVfeiV%B(w{VCMR|yld%`FD2n{J^L9F|0QX=z-9pJ{k``BrPyEn*c z1w!lW5`iz-n^rjVC9&`P!0IuKybAWI%=Zm{97l%A0_eDBu>S|g{fR;UV;%PojmH)a z=JqD`#&YRw`^o`B-|wTN=S7y!x)8?c3_&TTR!t9{8h z2Ah{Ypi@fMlfm{YZ@x#8*yI&EB;6*HL;HaHWI;e$LF7T^$x7}*x?*VJTx0nhi_zQ zgPwMFb655+2ga&`t*NqDyn)1kS9zL-Dkx4-S7jeG*H9Aq77veXI)pC(Y#X|=RoPqs zgSVcc8sSJrHjBrm*Kn#iyC~C!mNM})r`TIJu~gO{hUOzbC4pnkO=d`*`odr$13^Ee z2#wr_#gamwXM?7CX}0omZNlQz=1F8cfjHpDERa5taDspnBye~XXyBbG%ZNn?GO8lw zn^(f12_3_*T%We4Lhr7oQtF(B3LSgxqhBB2WX54zD%z|^$&V^R%U~N{(0&{(dVN(O zy=BFaFA|7FS!KT}zeSc}zQA5T(frvO7QY2!0kbhIJwFF|wiN-L(408=?Uqw*(VAD{ zh3QqWGnL=%SX74KETAJld79zk+mO3kP&HfOZh{!bVYy+In@DWm4Tzpi)e?Jhl^VT? zCCL1u7e}vCxgC!nBw^`LRYR-}lf_XL?Q@xW*$nP4^htihvLGpbaOe1z2)DhW?%;hy znoBjYVNaCY!bP@Ru8Be~3A&m}?l5y+U}X+y&EAam&%2Y{APRO$+-q4s&q3BB$3)kY z3%B@|L@3hwo7?uzXB(LS-PRtc2J~aWfFHLp|95l#$2usVNu z8~la5PqeI*L_0rH<6#Mw?JH(qb_EzeuPUfk9~f6u752dn%Y@JH6^TC>&_d`3(2pGGKv_qsD1?J2TaS{cK~6V-IXDD;Ch%ZAIz)=V3y2W2i3 zzP%<>4gbi#zTB#J@dT|^@r?q7V*q9r=~x9(`UtT~*Gw_WR2#)`cBE(k1+C4dp&8^` zTRI{R8McV6m+uvj0l{g(%i(-DA(^GPDlMQ{eh7D>4aD~FCgIi)8}TiDcXVpS5$lFc zstg!~xdC3Ec`$3zy1+vm24Bf0ncdTGhUXwz7yC{mLHSHi zsJN;qU87AA4!>T(C3M%f(~5>e`2Th?=@sWH~ZFO~!SgtV(l)!In1k5s=cv z3T5}aa2NTOx~*?;5z;@H-+vGBeMk9hHy0rS%mw=b&GW~m$?(JDlwbY8{;>v1_6r93 z>@O@-O18nWoe6H?4BeN~?RX;JD;G|LN1vt~4WgCP-c$+ECnNIPA*l$pE`qQ1`sZU- zacia$MtPo5zaz;$B~=}JJ0!m_-$wKvtZ}s&R#6LDR|Q# z_QS26qe>Z8i6i?g<@$Dpv-M48%ci06-4!kpFTRf;S!#Ay9I?R{@c?`>gLKG@BvQ-u%>=-1da)}f^xH}H?`7v;d(V>Lb$)!67TxNC7xY}@iI z5$B#s6U1yRA@Cr|aHW=vMhqwU^p%oM3Grx#l{P!YB+rA!3|Lse>uH094zv6UCcHa0 zlWUA)R(gviWLZc$yRE_4sv;}ON!uc;WlC@!Ocl3B%*>{Ue-+c>{4fB?^< z|MkVcFYx`B-+C1ll%aq31X+pikN=OjT^zS>D6q*B1gL!qw15YX16cm^_#+ihXpryz z^!?{wNPn%)`SbzOtw012>GwtdPUb%Xoj-#>cm&b=5aRbqJ;wQ;N%`TbehvdZu?YOZ zeSnh%fB;6%->LkD^9Q-lKhE!eK&Sd}!kQpJKoD^LxKjH-(BH7WD+gIy+5SsS^T$X( zrgsI@8u%lQRDK*%b!0|E2|00M!{Ah0lmVtPW?fZY&{Yo4pe!wmZKw|&b zXuli{ABw~hJb>N@AUQl*y}wk0{k4yp`T=ehK*#Avl>g=s55*@R#*lr0lJg_#zlV?i zDPHlw`v6j}KNJ5wb@hAN`PaD9K0wX_n7{ie_rEj9LoJMl!Dk}lsl>M3ZqciJYWBu5^ z{VIRT`WNvZ9^*YG?OgR|(vQyT{x_ugYJW=l7x5n+VmyxE%yo}Q|J7;5uhD)S!Ot3g zMEmcJ@G!;w@NISu!eZc$X|2~O- zWm11_7K{(jvjEdAe`^;1ZJ>vFK%55y?E}VJ|2&O{3BO+>PyPTi3vki)Q}TZw9{myg zcKaXUe^1{1_?`Pi>0nEtu| diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 97df433a0b67..5f7bf41caf9b 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -27,4 +27,4 @@ fi export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.2-src.zip:${PYTHONPATH}" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.1-src.zip:${PYTHONPATH}" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 7ea58afb53dc..3f6d7b28b540 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1138,9 +1138,9 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), "pyspark.zip not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.9.2-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.10.1-src.zip") require(py4jFile.exists(), - "py4j-0.9.2-src.zip not found; cannot run pyspark application in YARN mode.") + "py4j-0.10.1-src.zip not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 7df11ca7608b..c4656048455a 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -197,7 +197,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.9.2-src.zip", + s"$sparkHome/python/lib/py4j-0.10.1-src.zip", s"$sparkHome/python") val extraEnv = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), From 6c5768594fe8b910125f06e1308a8154a199447e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 13 May 2016 09:00:50 +0100 Subject: [PATCH 218/313] [SPARK-12972][CORE] Update org.apache.httpcomponents.httpclient ## What changes were proposed in this pull request? - update httpcore/httpclient to latest - centralize version management - remove excludes that are no longer relevant according to SBT/Maven dep graphs - also manage httpmime to match httpclient ## How was this patch tested? Jenkins tests, plus review of dependency graphs from SBT/Maven, and review of test-dependencies.sh output Author: Sean Owen Closes #13049 from srowen/SPARK-12972. (cherry picked from commit c74a6c3f2363f065a4915fdadec5eff665fa02e7) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 4 +- dev/deps/spark-deps-hadoop-2.3 | 4 +- dev/deps/spark-deps-hadoop-2.4 | 4 +- dev/deps/spark-deps-hadoop-2.6 | 4 +- dev/deps/spark-deps-hadoop-2.7 | 4 +- external/docker-integration-tests/pom.xml | 2 - pom.xml | 45 ++++------------------- 7 files changed, 17 insertions(+), 50 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 2477312d74bf..c3be6b2fee99 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -69,8 +69,8 @@ hadoop-yarn-server-web-proxy-2.2.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 0181a47a79d3..61ed4c0889b8 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.3.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index f7ff23472b9b..fb014921765f 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.4.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 92db55d674a4..0baf4e84fff0 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 44b9b049344a..8be218cd68d9 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 53a24f3e06e0..3169a0ebaeb9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -85,13 +85,11 @@ org.apache.httpcomponents httpclient - 4.5 test org.apache.httpcomponents httpcore - 4.4.1 test diff --git a/pom.xml b/pom.xml index 9e9aad223a5e..21b1becd08ea 100644 --- a/pom.xml +++ b/pom.xml @@ -149,8 +149,8 @@ 0.10.2 - 4.3.2 - 4.3.2 + 4.5.2 + 4.4.4 3.1 3.4.1 @@ -410,6 +410,11 @@ httpclient ${commons.httpclient.version} + + org.apache.httpcomponents + httpmime + ${commons.httpclient.version} + org.apache.httpcomponents httpcore @@ -731,18 +736,6 @@ guava com.google.guava - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - commons-logging - httpclient - commons-logging commons-logging @@ -1448,14 +1441,6 @@ ${hive.group} hive-shims - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.apache.curator curator-framework @@ -1807,14 +1792,6 @@ libthrift ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api @@ -1826,14 +1803,6 @@ libfb303 ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api From 1390eca2c9c51ff862d0d9598ec07d68466b11fc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 13 May 2016 13:51:28 +0100 Subject: [PATCH 219/313] Revert "[SPARK-12972][CORE] Update org.apache.httpcomponents.httpclient" This reverts commit 6c5768594fe8b910125f06e1308a8154a199447e. --- dev/deps/spark-deps-hadoop-2.2 | 4 +- dev/deps/spark-deps-hadoop-2.3 | 4 +- dev/deps/spark-deps-hadoop-2.4 | 4 +- dev/deps/spark-deps-hadoop-2.6 | 4 +- dev/deps/spark-deps-hadoop-2.7 | 4 +- external/docker-integration-tests/pom.xml | 2 + pom.xml | 45 +++++++++++++++++++---- 7 files changed, 50 insertions(+), 17 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index c3be6b2fee99..2477312d74bf 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -69,8 +69,8 @@ hadoop-yarn-server-web-proxy-2.2.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 61ed4c0889b8..0181a47a79d3 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.3.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index fb014921765f..f7ff23472b9b 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.4.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 0baf4e84fff0..92db55d674a4 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8be218cd68d9..44b9b049344a 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 3169a0ebaeb9..53a24f3e06e0 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -85,11 +85,13 @@ org.apache.httpcomponents httpclient + 4.5 test org.apache.httpcomponents httpcore + 4.4.1 test diff --git a/pom.xml b/pom.xml index 21b1becd08ea..9e9aad223a5e 100644 --- a/pom.xml +++ b/pom.xml @@ -149,8 +149,8 @@ 0.10.2 - 4.5.2 - 4.4.4 + 4.3.2 + 4.3.2 3.1 3.4.1 @@ -410,11 +410,6 @@ httpclient ${commons.httpclient.version} - - org.apache.httpcomponents - httpmime - ${commons.httpclient.version} - org.apache.httpcomponents httpcore @@ -736,6 +731,18 @@ guava com.google.guava + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + commons-logging + httpclient + commons-logging commons-logging @@ -1441,6 +1448,14 @@ ${hive.group} hive-shims + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + org.apache.curator curator-framework @@ -1792,6 +1807,14 @@ libthrift ${libthrift.version} + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + org.slf4j slf4j-api @@ -1803,6 +1826,14 @@ libfb303 ${libthrift.version} + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + org.slf4j slf4j-api From d3110d8b943b1af2cd44a6408036fc93de1d1aa9 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 13 May 2016 09:04:37 -0700 Subject: [PATCH 220/313] [SPARK-15267][SQL] Refactor options for JDBC and ORC data sources and change default compression for ORC ## What changes were proposed in this pull request? Currently, Parquet, JSON and CSV data sources have a class for thier options, (`ParquetOptions`, `JSONOptions` and `CSVOptions`). It is convenient to manage options for sources to gather options into a class. Currently, `JDBC`, `Text`, `libsvm` and `ORC` datasources do not have this class. This might be nicer if these options are in a unified format so that options can be added and This PR refactors the options in Spark internal data sources adding new classes, `OrcOptions`, `TextOptions`, `JDBCOptions` and `LibSVMOptions`. Also, this PR change the default compression codec for ORC from `NONE` to `SNAPPY`. ## How was this patch tested? Existing tests should cover this for refactoring and unittests in `OrcHadoopFsRelationSuite` for changing the default compression codec for ORC. Author: hyukjinkwon Closes #13048 from HyukjinKwon/SPARK-15267. (cherry picked from commit 3ded5bc4db2badc9ff49554e73421021d854306b) Signed-off-by: Reynold Xin --- .../ml/source/libsvm/LibSVMRelation.scala | 2 +- .../datasources/jdbc/DefaultSource.scala | 26 ++++------ .../datasources/jdbc/JDBCOptions.scala | 39 ++++++++++++++ .../datasources/parquet/ParquetOptions.scala | 7 ++- .../spark/sql/hive/orc/OrcOptions.scala | 52 +++++++++++++++++++ .../spark/sql/hive/orc/OrcRelation.scala | 46 ++++------------ .../hive/orc/OrcHadoopFsRelationSuite.scala | 18 +++++-- .../spark/sql/hive/orc/OrcQuerySuite.scala | 8 +-- 8 files changed, 135 insertions(+), 63 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 68a855c99f1b..39bdd1afadcc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -32,7 +32,7 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, DataFrameReader, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala index 6ff50a3c6122..6609e5dee3af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala @@ -30,30 +30,26 @@ class DefaultSource extends RelationProvider with DataSourceRegister { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) - val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) - val partitionColumn = parameters.getOrElse("partitionColumn", null) - val lowerBound = parameters.getOrElse("lowerBound", null) - val upperBound = parameters.getOrElse("upperBound", null) - val numPartitions = parameters.getOrElse("numPartitions", null) - - if (partitionColumn != null - && (lowerBound == null || upperBound == null || numPartitions == null)) { + val jdbcOptions = new JDBCOptions(parameters) + if (jdbcOptions.partitionColumn != null + && (jdbcOptions.lowerBound == null + || jdbcOptions.upperBound == null + || jdbcOptions.numPartitions == null)) { sys.error("Partitioning incompletely specified") } - val partitionInfo = if (partitionColumn == null) { + val partitionInfo = if (jdbcOptions.partitionColumn == null) { null } else { JDBCPartitioningInfo( - partitionColumn, - lowerBound.toLong, - upperBound.toLong, - numPartitions.toInt) + jdbcOptions.partitionColumn, + jdbcOptions.lowerBound.toLong, + jdbcOptions.upperBound.toLong, + jdbcOptions.numPartitions.toInt) } val parts = JDBCRelation.columnPartition(partitionInfo) val properties = new Properties() // Additional properties that we will pass to getConnection parameters.foreach(kv => properties.setProperty(kv._1, kv._2)) - JDBCRelation(url, table, parts, properties)(sqlContext.sparkSession) + JDBCRelation(jdbcOptions.url, jdbcOptions.table, parts, properties)(sqlContext.sparkSession) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala new file mode 100644 index 000000000000..6c6ec89746ee --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc + +/** + * Options for the JDBC data source. + */ +private[jdbc] class JDBCOptions( + @transient private val parameters: Map[String, String]) + extends Serializable { + + // a JDBC URL + val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) + // name of table + val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) + // the column used to partition + val partitionColumn = parameters.getOrElse("partitionColumn", null) + // the lower bound of partition column + val lowerBound = parameters.getOrElse("lowerBound", null) + // the upper bound of the partition column + val upperBound = parameters.getOrElse("upperBound", null) + // the number of partitions + val numPartitions = parameters.getOrElse("numPartitions", null) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 00352f23ae66..1ff217cbf0d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -19,16 +19,15 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.parquet.hadoop.metadata.CompressionCodecName -import org.apache.spark.internal.Logging import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -class ParquetOptions( +private[parquet] class ParquetOptions( @transient private val parameters: Map[String, String], @transient private val sqlConf: SQLConf) - extends Logging with Serializable { + extends Serializable { import ParquetOptions._ @@ -48,7 +47,7 @@ class ParquetOptions( } -object ParquetOptions { +private[parquet] object ParquetOptions { // The parquet compression short names private val shortParquetCompressionCodecNames = Map( "none" -> CompressionCodecName.UNCOMPRESSED, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala new file mode 100644 index 000000000000..91cf0dc960d5 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.orc + +/** + * Options for the ORC data source. + */ +private[orc] class OrcOptions( + @transient private val parameters: Map[String, String]) + extends Serializable { + + import OrcOptions._ + + /** + * Compression codec to use. By default snappy compression. + * Acceptable values are defined in [[shortOrcCompressionCodecNames]]. + */ + val compressionCodec: String = { + val codecName = parameters.getOrElse("compression", "snappy").toLowerCase + if (!shortOrcCompressionCodecNames.contains(codecName)) { + val availableCodecs = shortOrcCompressionCodecNames.keys.map(_.toLowerCase) + throw new IllegalArgumentException(s"Codec [$codecName] " + + s"is not available. Available codecs are ${availableCodecs.mkString(", ")}.") + } + shortOrcCompressionCodecNames(codecName) + } +} + +private[orc] object OrcOptions { + // The ORC compression short names + private val shortOrcCompressionCodecNames = Map( + "none" -> "NONE", + "uncompressed" -> "NONE", + "snappy" -> "SNAPPY", + "zlib" -> "ZLIB", + "lzo" -> "LZO") +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index fed31503043e..6e55137dd78e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.io.orc._ -import org.apache.hadoop.hive.ql.io.orc.OrcFile.OrcTableProperties import org.apache.hadoop.hive.serde2.objectinspector.{SettableStructObjectInspector, StructObjectInspector} import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils} import org.apache.hadoop.io.{NullWritable, Writable} @@ -37,7 +36,6 @@ import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.{HiveInspectors, HiveShim} import org.apache.spark.sql.sources.{Filter, _} @@ -66,28 +64,12 @@ private[sql] class DefaultSource job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { - val compressionCodec: Option[String] = options - .get("compression") - .map { codecName => - // Validate if given compression codec is supported or not. - val shortOrcCompressionCodecNames = OrcRelation.shortOrcCompressionCodecNames - if (!shortOrcCompressionCodecNames.contains(codecName.toLowerCase)) { - val availableCodecs = shortOrcCompressionCodecNames.keys.map(_.toLowerCase) - throw new IllegalArgumentException(s"Codec [$codecName] " + - s"is not available. Available codecs are ${availableCodecs.mkString(", ")}.") - } - codecName.toLowerCase - } + val orcOptions = new OrcOptions(options) - compressionCodec.foreach { codecName => - job.getConfiguration.set( - OrcTableProperties.COMPRESSION.getPropName, - OrcRelation - .shortOrcCompressionCodecNames - .getOrElse(codecName, CompressionKind.NONE).name()) - } + val configuration = job.getConfiguration - job.getConfiguration match { + configuration.set(OrcRelation.ORC_COMPRESSION, orcOptions.compressionCodec) + configuration match { case conf: JobConf => conf.setOutputFormat(classOf[OrcOutputFormat]) case conf => @@ -205,7 +187,7 @@ private[orc] class OrcOutputWriter( val partition = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") val compressionExtension = { - val name = conf.get(OrcTableProperties.COMPRESSION.getPropName) + val name = conf.get(OrcRelation.ORC_COMPRESSION) OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") } // It has the `.orc` extension at the end because (de)compression tools @@ -329,21 +311,15 @@ private[orc] object OrcTableScan { } private[orc] object OrcRelation extends HiveInspectors { - // The ORC compression short names - val shortOrcCompressionCodecNames = Map( - "none" -> CompressionKind.NONE, - "uncompressed" -> CompressionKind.NONE, - "snappy" -> CompressionKind.SNAPPY, - "zlib" -> CompressionKind.ZLIB, - "lzo" -> CompressionKind.LZO) + // The references of Hive's classes will be minimized. + val ORC_COMPRESSION = "orc.compress" // The extensions for ORC compression codecs val extensionsForCompressionCodecNames = Map( - CompressionKind.NONE.name -> "", - CompressionKind.SNAPPY.name -> ".snappy", - CompressionKind.ZLIB.name -> ".zlib", - CompressionKind.LZO.name -> ".lzo" - ) + "NONE" -> "", + "SNAPPY" -> ".snappy", + "ZLIB" -> ".zlib", + "LZO" -> ".lzo") def unwrapOrcStructs( conf: Configuration, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala index 965680ff0dda..0207b4e8c977 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive.orc import java.io.File import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.hive.ql.io.orc.{CompressionKind, OrcFile} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.Row @@ -98,9 +97,10 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { val fs = FileSystem.getLocal(conf) val maybeOrcFile = new File(path).listFiles().find(_.getName.endsWith(".zlib.orc")) assert(maybeOrcFile.isDefined) - val orcFilePath = new Path(maybeOrcFile.get.toPath.toString) - val orcReader = OrcFile.createReader(orcFilePath, OrcFile.readerOptions(conf)) - assert(orcReader.getCompression == CompressionKind.ZLIB) + val orcFilePath = maybeOrcFile.get.toPath.toString + val expectedCompressionKind = + OrcFileOperator.getFileReader(orcFilePath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) val copyDf = spark .read @@ -108,4 +108,14 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { checkAnswer(df, copyDf) } } + + test("Default compression codec is snappy for ORC compression") { + withTempPath { file => + spark.range(0, 10).write + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("SNAPPY" === expectedCompressionKind.name()) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 084546f99d49..9a0885822b8d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -171,7 +171,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "ZLIB") + .option("compression", "ZLIB") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression @@ -180,7 +180,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "SNAPPY") + .option("compression", "SNAPPY") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression @@ -189,7 +189,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "NONE") + .option("compression", "NONE") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression @@ -201,7 +201,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { ignore("LZO compression options for writing to an ORC file not supported in Hive 1.2.1") { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "LZO") + .option("compression", "LZO") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression From 78bf9a1aad657ca9434700fdd251b1d3b4723e41 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Fri, 13 May 2016 18:10:22 -0700 Subject: [PATCH 221/313] [TRIVIAL] Add () to SparkSession's builder function Was trying out `SparkSession` for the first time and the given class doc (when copied as is) did not work over Spark shell: ``` scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate() :27: error: org.apache.spark.sql.SparkSession.Builder does not take parameters SparkSession.builder().master("local").appName("Word Count").getOrCreate() ``` Adding () to the builder method in SparkSession. ``` scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate() res0: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38 scala> SparkSession.builder.master("local").appName("Word Count").getOrCreate() res1: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38 ``` Author: Tejas Patil Closes #13086 from tejasapatil/doc_correction. (cherry picked from commit 4210e2a6b77f922f03eedf781a4230ab03a189cd) Signed-off-by: Reynold Xin --- sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 02c9dc03ae82..100b43f6b58f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -755,7 +755,7 @@ object SparkSession { * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. * @since 2.0.0 */ - def builder: Builder = new Builder + def builder(): Builder = new Builder private val HIVE_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" private val HIVE_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" From 2d6f3bb4d56a0355625004806d0ed82fa407bce0 Mon Sep 17 00:00:00 2001 From: Nicholas Tietz Date: Sat, 14 May 2016 09:44:20 +0100 Subject: [PATCH 222/313] [SPARK-15197][DOCS] Added Scaladoc for countApprox and countByValueApprox parameters This pull request simply adds Scaladoc documentation of the parameters for countApprox and countByValueApprox. This is an important documentation change, as it clarifies what should be passed in for the timeout. Without units, this was previously unclear. I did not open a JIRA ticket per my understanding of the project contribution guidelines; as they state, the description in the ticket would be essentially just what is in the PR. If I should open one, let me know and I will do so. Author: Nicholas Tietz Closes #12955 from ntietz/rdd-countapprox-docs. (cherry picked from commit 0f1f31d3a6669fbac474518cf2a871485e202bdc) Signed-off-by: Sean Owen --- .../apache/spark/api/java/JavaRDDLike.scala | 29 +++++++++++++++++-- .../apache/spark/rdd/PairRDDFunctions.scala | 10 +++++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 16 ++++++++++ 3 files changed, 53 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index c17ca1237982..e4ccd9f11bd1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -445,6 +445,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) @@ -452,6 +462,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * @param timeout maximum time to wait for the job, in milliseconds */ def countApprox(timeout: Long): PartialResult[BoundedDouble] = rdd.countApprox(timeout) @@ -464,7 +476,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { mapAsSerializableJavaMap(rdd.countByValue()).asInstanceOf[JMap[T, jl.Long]] /** - * (Experimental) Approximate version of countByValue(). + * Approximate version of countByValue(). + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByValueApprox( timeout: Long, @@ -473,7 +495,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { rdd.countByValueApprox(timeout, confidence).map(mapAsSerializableJavaMap) /** - * (Experimental) Approximate version of countByValue(). + * Approximate version of countByValue(). + * + * @param timeout maximum time to wait for the job, in milliseconds + * @return a potentially incomplete result, with error bounds */ def countByValueApprox(timeout: Long): PartialResult[JMap[T, BoundedDouble]] = rdd.countByValueApprox(timeout).map(mapAsSerializableJavaMap) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7936d8e1d45a..3b12448d6393 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -375,6 +375,16 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = self.withScope { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index d85d0fff46be..e6db9b3eec8e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1107,10 +1107,21 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countApprox( timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L while (iter.hasNext) { @@ -1137,10 +1148,15 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of countByValue(). + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByValueApprox(timeout: Long, confidence: Double = 0.95) (implicit ord: Ordering[T] = null) : PartialResult[Map[T, BoundedDouble]] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } From d305f72275255f8d21ebbe62b545ac663d617f3b Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Sat, 14 May 2016 09:45:56 +0100 Subject: [PATCH 223/313] [SPARK-15096][ML] LogisticRegression MultiClassSummarizer numClasses can fail if no valid labels are found ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Throw better exception when numClasses is empty and empty.max is thrown. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Add a new unit test, which calls histogram with empty numClasses. Author: wm624@hotmail.com Closes #12969 from wangmiao1981/logisticR. (cherry picked from commit 354f8f11bd4b20fa99bd67a98da3525fd3d75c81) Signed-off-by: Sean Owen --- .../apache/spark/ml/classification/LogisticRegression.scala | 2 +- .../spark/ml/classification/LogisticRegressionSuite.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) 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 d2d4e249b420..62d68973680a 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 @@ -744,7 +744,7 @@ private[classification] class MultiClassSummarizer extends Serializable { def countInvalid: Long = totalInvalidCnt /** @return The number of distinct labels in the input dataset. */ - def numClasses: Int = distinctMap.keySet.max + 1 + def numClasses: Int = if (distinctMap.isEmpty) 0 else distinctMap.keySet.max + 1 /** @return The weightSum of each label in the input dataset. */ def histogram: Array[Double] = { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index f127aa217c94..69650ebb3690 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -256,6 +256,10 @@ class LogisticRegressionSuite assert(summarizer4.countInvalid === 2) assert(summarizer4.numClasses === 4) + val summarizer5 = new MultiClassSummarizer + assert(summarizer5.histogram.isEmpty) + assert(summarizer5.numClasses === 0) + // small map merges large one val summarizerA = summarizer1.merge(summarizer2) assert(summarizerA.hashCode() === summarizer2.hashCode()) From 4f2f96f5e31004045d2ac2cd9f76467325ee9aa1 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Mon, 16 May 2016 10:41:20 +0800 Subject: [PATCH 224/313] [SPARK-15253][SQL] Support old table schema config key "spark.sql.sources.schema" for DESCRIBE TABLE ## What changes were proposed in this pull request? "DESCRIBE table" is broken when table schema is stored at key "spark.sql.sources.schema". Originally, we used spark.sql.sources.schema to store the schema of a data source table. After SPARK-6024, we removed this flag. Although we are not using spark.sql.sources.schema any more, we need to still support it. ## How was this patch tested? Unit test. When using spark2.0 to load a table generated by spark 1.2. Before change: `DESCRIBE table` => Schema of this table is inferred at runtime,, After change: `DESCRIBE table` => correct output. Author: Sean Zhong Closes #13073 from clockfly/spark-15253. (cherry picked from commit 4a5ee1954a6fb77231abb492355fe70313f0b35b) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/command/ddl.scala | 29 ++++++++++++------- .../sql/hive/MetastoreDataSourcesSuite.scala | 3 ++ 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 1c1716f05052..49d7fe956f81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -497,20 +497,27 @@ private[sql] object DDLUtils { // will be inferred at runtime when the table is referenced. def getSchemaFromTableProperties(metadata: CatalogTable): Option[StructType] = { require(isDatasourceTable(metadata)) + val props = metadata.properties + if (props.isDefinedAt("spark.sql.sources.schema")) { + // Originally, we used spark.sql.sources.schema to store the schema of a data source table. + // After SPARK-6024, we removed this flag. + // Although we are not using spark.sql.sources.schema any more, we need to still support. + props.get("spark.sql.sources.schema").map(DataType.fromJson(_).asInstanceOf[StructType]) + } else { + metadata.properties.get("spark.sql.sources.schema.numParts").map { numParts => + val parts = (0 until numParts.toInt).map { index => + val part = metadata.properties.get(s"spark.sql.sources.schema.part.$index").orNull + if (part == null) { + throw new AnalysisException( + "Could not read schema from the metastore because it is corrupted " + + s"(missing part $index of the schema, $numParts parts are expected).") + } - metadata.properties.get("spark.sql.sources.schema.numParts").map { numParts => - val parts = (0 until numParts.toInt).map { index => - val part = metadata.properties.get(s"spark.sql.sources.schema.part.$index").orNull - if (part == null) { - throw new AnalysisException( - "Could not read schema from the metastore because it is corrupted " + - s"(missing part $index of the schema, $numParts parts are expected).") + part } - - part + // Stick all parts back to a single schema string. + DataType.fromJson(parts.mkString).asInstanceOf[StructType] } - // Stick all parts back to a single schema string. - DataType.fromJson(parts.mkString).asInstanceOf[StructType] } } 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 676fbd0a39b6..b507018e58d1 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 @@ -746,6 +746,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sessionState.invalidateTable(tableName) val actualSchema = table(tableName).schema assert(schema === actualSchema) + + // Checks the DESCRIBE output. + checkAnswer(sql("DESCRIBE spark6655"), Row("int", "int", "") :: Nil) } } From 5afde26a0b2e707f84f1be813970d436086ceb5c Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Mon, 16 May 2016 08:22:16 +0200 Subject: [PATCH 225/313] [SPARK-15305][ML][DOC] spark.ml document Bisectiong k-means has the incorrect format ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) The generated document has the incorrect format for biseckmeans. ![bug](https://cloud.githubusercontent.com/assets/5033592/15233120/d910098a-185a-11e6-901d-44aeafc8a011.jpg) ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Fix the formatting. ![fix](https://cloud.githubusercontent.com/assets/5033592/15233136/fce2ccd0-185a-11e6-9ded-14d71da4bdab.jpg) Author: wm624@hotmail.com Closes #13083 from wangmiao1981/doc. (cherry picked from commit c1836d66bdc93f80ff9e8852efe8f2d2bc1ca941) Signed-off-by: Nick Pentreath --- docs/ml-clustering.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 0d69bf67dfe4..a0955a3855ce 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -86,7 +86,6 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. - ## Latent Dirichlet allocation (LDA) `LDA` is implemented as an `Estimator` that supports both `EMLDAOptimizer` and `OnlineLDAOptimizer`, @@ -116,8 +115,8 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. {% include_example python/ml/lda_example.py %} -## Bisecting k-means +## Bisecting k-means Bisecting k-means is a kind of [hierarchical clustering](https://en.wikipedia.org/wiki/Hierarchical_clustering) using a divisive (or "top-down") approach: all observations start in one cluster, and splits are performed recursively as one @@ -148,5 +147,4 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. {% include_example python/ml/bisecting_k_means_example.py %} - From f937ce7663a894f819882ea2df7c52dadd4b3681 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 16 May 2016 09:55:35 +0200 Subject: [PATCH 226/313] [SPARK-14979][ML][PYSPARK] Add examples for GeneralizedLinearRegression ## What changes were proposed in this pull request? Add Scala/Java/Python examples for ```GeneralizedLinearRegression```. ## How was this patch tested? They are examples and have been tested offline. Author: Yanbo Liang Closes #12754 from yanboliang/spark-14979. (cherry picked from commit f116a84ef8bf8a201c1a67154fda6990e4222074) Signed-off-by: Nick Pentreath --- ...avaGeneralizedLinearRegressionExample.java | 83 +++++++++++++++++++ .../generalized_linear_regression_example.py | 66 +++++++++++++++ .../GeneralizedLinearRegressionExample.scala | 78 +++++++++++++++++ 3 files changed, 227 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java create mode 100644 examples/src/main/python/ml/generalized_linear_regression_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java new file mode 100644 index 000000000000..3f072d1e50eb --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java @@ -0,0 +1,83 @@ +/* + * 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.examples.ml; + +// $example on$ +import java.util.Arrays; + +import org.apache.spark.ml.regression.GeneralizedLinearRegression; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionModel; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionTrainingSummary; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * An example demonstrating generalized linear regression. + * Run with + *
    + * bin/run-example ml.JavaGeneralizedLinearRegressionExample
    + * 
    + */ + +public class JavaGeneralizedLinearRegressionExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaGeneralizedLinearRegressionExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset dataset = spark.read().format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt"); + + GeneralizedLinearRegression glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3); + + // Fit the model + GeneralizedLinearRegressionModel model = glr.fit(dataset); + + // Print the coefficients and intercept for generalized linear regression model + System.out.println("Coefficients: " + model.coefficients()); + System.out.println("Intercept: " + model.intercept()); + + // Summarize the model over the training set and print out some metrics + GeneralizedLinearRegressionTrainingSummary summary = model.summary(); + System.out.println("Coefficient Standard Errors: " + + Arrays.toString(summary.coefficientStandardErrors())); + System.out.println("T Values: " + Arrays.toString(summary.tValues())); + System.out.println("P Values: " + Arrays.toString(summary.pValues())); + System.out.println("Dispersion: " + summary.dispersion()); + System.out.println("Null Deviance: " + summary.nullDeviance()); + System.out.println("Residual Degree Of Freedom Null: " + summary.residualDegreeOfFreedomNull()); + System.out.println("Deviance: " + summary.deviance()); + System.out.println("Residual Degree Of Freedom: " + summary.residualDegreeOfFreedom()); + System.out.println("AIC: " + summary.aic()); + System.out.println("Deviance Residuals: "); + summary.residuals().show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/python/ml/generalized_linear_regression_example.py b/examples/src/main/python/ml/generalized_linear_regression_example.py new file mode 100644 index 000000000000..796752a60f3a --- /dev/null +++ b/examples/src/main/python/ml/generalized_linear_regression_example.py @@ -0,0 +1,66 @@ +# +# 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. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.regression import GeneralizedLinearRegression +# $example off$ + +""" +An example demonstrating generalized linear regression. +Run with: + bin/spark-submit examples/src/main/python/ml/generalized_linear_regression_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("GeneralizedLinearRegressionExample")\ + .getOrCreate() + + # $example on$ + # Load training data + dataset = spark.read.format("libsvm")\ + .load("data/mllib/sample_linear_regression_data.txt") + + glr = GeneralizedLinearRegression(family="gaussian", link="identity", maxIter=10, regParam=0.3) + + # Fit the model + model = glr.fit(dataset) + + # Print the coefficients and intercept for generalized linear regression model + print("Coefficients: " + str(model.coefficients)) + print("Intercept: " + str(model.intercept)) + + # Summarize the model over the training set and print out some metrics + summary = model.summary + print("Coefficient Standard Errors: " + str(summary.coefficientStandardErrors)) + print("T Values: " + str(summary.tValues)) + print("P Values: " + str(summary.pValues)) + print("Dispersion: " + str(summary.dispersion)) + print("Null Deviance: " + str(summary.nullDeviance)) + print("Residual Degree Of Freedom Null: " + str(summary.residualDegreeOfFreedomNull)) + print("Deviance: " + str(summary.deviance)) + print("Residual Degree Of Freedom: " + str(summary.residualDegreeOfFreedom)) + print("AIC: " + str(summary.aic)) + print("Deviance Residuals: ") + summary.residuals().show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala new file mode 100644 index 000000000000..1b86d7cad0b3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala @@ -0,0 +1,78 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.regression.GeneralizedLinearRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating generalized linear regression. + * Run with + * {{{ + * bin/run-example ml.GeneralizedLinearRegressionExample + * }}} + */ + +object GeneralizedLinearRegressionExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("GeneralizedLinearRegressionExample") + .getOrCreate() + + // $example on$ + // Load training data + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt") + + val glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3) + + // Fit the model + val model = glr.fit(dataset) + + // Print the coefficients and intercept for generalized linear regression model + println(s"Coefficients: ${model.coefficients}") + println(s"Intercept: ${model.intercept}") + + // Summarize the model over the training set and print out some metrics + val summary = model.summary + println(s"Coefficient Standard Errors: ${summary.coefficientStandardErrors.mkString(",")}") + println(s"T Values: ${summary.tValues.mkString(",")}") + println(s"P Values: ${summary.pValues.mkString(",")}") + println(s"Dispersion: ${summary.dispersion}") + println(s"Null Deviance: ${summary.nullDeviance}") + println(s"Residual Degree Of Freedom Null: ${summary.residualDegreeOfFreedomNull}") + println(s"Deviance: ${summary.deviance}") + println(s"Residual Degree Of Freedom: ${summary.residualDegreeOfFreedom}") + println(s"AIC: ${summary.aic}") + println("Deviance Residuals: ") + summary.residuals().show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println From 0dd1f872082f9e5d7712a1f7c1035dd2786b2a39 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Mon, 16 May 2016 12:59:55 -0700 Subject: [PATCH 227/313] [SPARK-14942][SQL][STREAMING] Reduce delay between batch construction and execution ## Problem Currently in `StreamExecution`, [we first run the batch, then construct the next](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L165): ```scala if (dataAvailable) runBatch() constructNextBatch() ``` This is good when we run batches ASAP, where data would get processed in the **very next batch**: ![1](https://cloud.githubusercontent.com/assets/15843379/14779964/2786e698-0b0d-11e6-9d2c-bb41513488b2.png) However, when we run batches at trigger like `ProcessTime("1 minute")`, data - such as _y_ below - may not get processed in the very next batch i.e. _batch 1_, but in _batch 2_: ![2](https://cloud.githubusercontent.com/assets/15843379/14779818/6f3bb064-0b0c-11e6-9f16-c1ce4897186b.png) ## What changes were proposed in this pull request? This patch reverses the order of `constructNextBatch()` and `runBatch()`. After this patch, data would get processed in the **very next batch**, i.e. _batch 1_: ![3](https://cloud.githubusercontent.com/assets/15843379/14779816/6f36ee62-0b0c-11e6-9e53-bc8397fade18.png) In addition, this patch alters when we do `currentBatchId += 1`: let's do that when the processing of the current batch's data is completed, so we won't bother passing `currentBatchId + 1` or `currentBatchId - 1` to states or sinks. ## How was this patch tested? New added test case. Also this should be covered by existing test suits, e.g. stress tests and others. Author: Liwei Lin Closes #12725 from lw-lin/construct-before-run-3. (cherry picked from commit 95f4fbae52d26ede94c3ba8248394749f3d95dcc) Signed-off-by: Shixiong Zhu --- .../streaming/IncrementalExecution.scala | 6 +- .../execution/streaming/StreamExecution.scala | 24 ++++-- .../sql/execution/streaming/memory.scala | 4 + .../spark/sql/streaming/StreamSuite.scala | 84 +++++++++++++++++-- 4 files changed, 99 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index e9052a309595..8b96f65bc31a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -27,12 +27,12 @@ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, * A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]] * plan incrementally. Possibly preserving state in between each execution. */ -class IncrementalExecution( +class IncrementalExecution private[sql]( sparkSession: SparkSession, logicalPlan: LogicalPlan, outputMode: OutputMode, checkpointLocation: String, - currentBatchId: Long) + val currentBatchId: Long) extends QueryExecution(sparkSession, logicalPlan) { // TODO: make this always part of planning. @@ -57,7 +57,7 @@ class IncrementalExecution( case StateStoreSaveExec(keys, None, UnaryExecNode(agg, StateStoreRestoreExec(keys2, None, child))) => - val stateId = OperatorStateId(checkpointLocation, operatorId, currentBatchId - 1) + val stateId = OperatorStateId(checkpointLocation, operatorId, currentBatchId) operatorId += 1 StateStoreSaveExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ea367b699f56..df6304d85fe7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -122,7 +122,7 @@ class StreamExecution( * processing is done. Thus, the Nth record in this log indicated data that is currently being * processed and the N-1th entry indicates which offsets have been durably committed to the sink. */ - private val offsetLog = + private[sql] val offsetLog = new HDFSMetadataLog[CompositeOffset](sparkSession, checkpointFile("offsets")) /** Whether the query is currently active or not */ @@ -174,12 +174,21 @@ class StreamExecution( // While active, repeatedly attempt to run batches. SQLContext.setActive(sparkSession.wrapped) - populateStartOffsets() - logDebug(s"Stream running from $committedOffsets to $availableOffsets") + triggerExecutor.execute(() => { if (isActive) { - if (dataAvailable) runBatch() - constructNextBatch() + if (currentBatchId < 0) { + // We'll do this initialization only once + populateStartOffsets() + logDebug(s"Stream running from $committedOffsets to $availableOffsets") + } else { + constructNextBatch() + } + if (dataAvailable) { + runBatch() + // We'll increase currentBatchId after we complete processing current batch's data + currentBatchId += 1 + } true } else { false @@ -214,7 +223,7 @@ class StreamExecution( offsetLog.getLatest() match { case Some((batchId, nextOffsets)) => logInfo(s"Resuming continuous query, starting with batch $batchId") - currentBatchId = batchId + 1 + currentBatchId = batchId availableOffsets = nextOffsets.toStreamProgress(sources) logDebug(s"Found possibly uncommitted offsets $availableOffsets") @@ -285,7 +294,6 @@ class StreamExecution( offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") } - currentBatchId += 1 logInfo(s"Committed offsets for batch $currentBatchId.") } else { awaitBatchLock.lock() @@ -352,7 +360,7 @@ class StreamExecution( val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) - sink.addBatch(currentBatchId - 1, nextBatch) + sink.addBatch(currentBatchId, nextBatch) awaitBatchLock.lock() try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index a34927ff994a..bcc33ae8c88b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -124,6 +124,10 @@ class MemorySink(val schema: StructType) extends Sink with Logging { batches.flatten } + def latestBatchId: Option[Int] = synchronized { + if (batches.size == 0) None else Some(batches.size - 1) + } + def lastBatch: Seq[Row] = synchronized { batches.last } def toDebugString: String = synchronized { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 6a8b28017467..013b7316938e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -137,20 +137,88 @@ class StreamSuite extends StreamTest with SharedSQLContext { } } - // This would fail for now -- error is "Timed out waiting for stream" - // Root cause is that data generated in batch 0 may not get processed in batch 1 - // Let's enable this after SPARK-14942: Reduce delay between batch construction and execution - ignore("minimize delay between batch construction and execution") { + test("minimize delay between batch construction and execution") { + + // For each batch, we would retrieve new data's offsets and log them before we run the execution + // This checks whether the key of the offset log is the expected batch id + def CheckOffsetLogLatestBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.offsetLog.getLatest().get._1 == expectedId, + s"offsetLog's latest should be $expectedId") + + // For each batch, we would log the state change during the execution + // This checks whether the key of the state change log is the expected batch id + def CheckIncrementalExecutionCurrentBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.lastExecution.asInstanceOf[IncrementalExecution].currentBatchId == expectedId, + s"lastExecution's currentBatchId should be $expectedId") + + // For each batch, we would log the sink change after the execution + // This checks whether the key of the sink change log is the expected batch id + def CheckSinkLatestBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.sink.asInstanceOf[MemorySink].latestBatchId.get == expectedId, + s"sink's lastBatchId should be $expectedId") + val inputData = MemoryStream[Int] testStream(inputData.toDS())( StartStream(ProcessingTime("10 seconds"), new ManualClock), + /* -- batch 0 ----------------------- */ - AddData(inputData, 1), - AddData(inputData, 2), - AddData(inputData, 3), + // Add some data in batch 0 + AddData(inputData, 1, 2, 3), AdvanceManualClock(10 * 1000), // 10 seconds + /* -- batch 1 ----------------------- */ - CheckAnswer(1, 2, 3)) + // Check the results of batch 0 + CheckAnswer(1, 2, 3), + CheckIncrementalExecutionCurrentBatchId(0), + CheckOffsetLogLatestBatchId(0), + CheckSinkLatestBatchId(0), + // Add some data in batch 1 + AddData(inputData, 4, 5, 6), + AdvanceManualClock(10 * 1000), + + /* -- batch _ ----------------------- */ + // Check the results of batch 1 + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + + AdvanceManualClock(10 * 1000), + AdvanceManualClock(10 * 1000), + AdvanceManualClock(10 * 1000), + + /* -- batch __ ---------------------- */ + // Check the results of batch 1 again; this is to make sure that, when there's no new data, + // the currentId does not get logged (e.g. as 2) even if the clock has advanced many times + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + + /* Stop then restart the Stream */ + StopStream, + StartStream(ProcessingTime("10 seconds"), new ManualClock), + + /* -- batch 1 rerun ----------------- */ + // this batch 1 would re-run because the latest batch id logged in offset log is 1 + AdvanceManualClock(10 * 1000), + + /* -- batch 2 ----------------------- */ + // Check the results of batch 1 + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + // Add some data in batch 2 + AddData(inputData, 7, 8, 9), + AdvanceManualClock(10 * 1000), + + /* -- batch 3 ----------------------- */ + // Check the results of batch 2 + CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8, 9), + CheckIncrementalExecutionCurrentBatchId(2), + CheckOffsetLogLatestBatchId(2), + CheckSinkLatestBatchId(2)) } } From 8e3ee683bb7ecc857480bc347e7a814e5a63ff28 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 17 May 2016 00:08:02 -0700 Subject: [PATCH 228/313] [SPARK-14906][ML] Copy linalg in PySpark to new ML package ## What changes were proposed in this pull request? Copy the linalg (Vector/Matrix and VectorUDT/MatrixUDT) in PySpark to new ML package. ## How was this patch tested? Existing tests. Author: Xiangrui Meng Author: Liang-Chi Hsieh Author: Liang-Chi Hsieh Closes #13099 from viirya/move-pyspark-vector-matrix-udt4. (cherry picked from commit 8ad9f08c94e98317a9095dd53d737c1b8df6e29c) Signed-off-by: Xiangrui Meng --- python/docs/pyspark.ml.rst | 8 + python/pyspark/ml/linalg/__init__.py | 1145 ++++++++++++++++++++++++++ python/pyspark/ml/tests.py | 456 +++++++++- 3 files changed, 1564 insertions(+), 45 deletions(-) create mode 100644 python/pyspark/ml/linalg/__init__.py diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst index 86d4186a2c79..26f7415e1a42 100644 --- a/python/docs/pyspark.ml.rst +++ b/python/docs/pyspark.ml.rst @@ -41,6 +41,14 @@ pyspark.ml.clustering module :undoc-members: :inherited-members: +pyspark.ml.linalg module +---------------------------- + +.. automodule:: pyspark.ml.linalg + :members: + :undoc-members: + :inherited-members: + pyspark.ml.recommendation module -------------------------------- diff --git a/python/pyspark/ml/linalg/__init__.py b/python/pyspark/ml/linalg/__init__.py new file mode 100644 index 000000000000..f42c589b9225 --- /dev/null +++ b/python/pyspark/ml/linalg/__init__.py @@ -0,0 +1,1145 @@ +# +# 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. +# + +""" +MLlib utilities for linear algebra. For dense vectors, MLlib +uses the NumPy C{array} type, so you can simply pass NumPy arrays +around. For sparse vectors, users can construct a L{SparseVector} +object from MLlib or pass SciPy C{scipy.sparse} column vectors if +SciPy is available in their environment. +""" + +import sys +import array +import struct + +if sys.version >= '3': + basestring = str + xrange = range + import copyreg as copy_reg + long = int +else: + from itertools import izip as zip + import copy_reg + +import numpy as np + +from pyspark import since +from pyspark.sql.types import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ + IntegerType, ByteType, BooleanType + + +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', + 'Matrix', 'DenseMatrix', 'SparseMatrix', 'Matrices'] + + +if sys.version_info[:2] == (2, 7): + # speed up pickling array in Python 2.7 + def fast_pickle_array(ar): + return array.array, (ar.typecode, ar.tostring()) + copy_reg.pickle(array.array, fast_pickle_array) + + +# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, +# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. + +try: + import scipy.sparse + _have_scipy = True +except: + # No SciPy in environment, but that's okay + _have_scipy = False + + +def _convert_to_vector(l): + if isinstance(l, Vector): + return l + elif type(l) in (array.array, np.array, np.ndarray, list, tuple, xrange): + return DenseVector(l) + elif _have_scipy and scipy.sparse.issparse(l): + assert l.shape[1] == 1, "Expected column vector" + csc = l.tocsc() + return SparseVector(l.shape[0], csc.indices, csc.data) + else: + raise TypeError("Cannot convert type %s into Vector" % type(l)) + + +def _vector_size(v): + """ + Returns the size of the vector. + + >>> _vector_size([1., 2., 3.]) + 3 + >>> _vector_size((1., 2., 3.)) + 3 + >>> _vector_size(array.array('d', [1., 2., 3.])) + 3 + >>> _vector_size(np.zeros(3)) + 3 + >>> _vector_size(np.zeros((3, 1))) + 3 + >>> _vector_size(np.zeros((1, 3))) + Traceback (most recent call last): + ... + ValueError: Cannot treat an ndarray of shape (1, 3) as a vector + """ + if isinstance(v, Vector): + return len(v) + elif type(v) in (array.array, list, tuple, xrange): + return len(v) + elif type(v) == np.ndarray: + if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): + return len(v) + else: + raise ValueError("Cannot treat an ndarray of shape %s as a vector" % str(v.shape)) + elif _have_scipy and scipy.sparse.issparse(v): + assert v.shape[1] == 1, "Expected column vector" + return v.shape[0] + else: + raise TypeError("Cannot treat type %s as a vector" % type(v)) + + +def _format_float(f, digits=4): + s = str(round(f, digits)) + if '.' in s: + s = s[:s.index('.') + 1 + digits] + return s + + +def _format_float_list(l): + return [_format_float(x) for x in l] + + +def _double_to_long_bits(value): + if np.isnan(value): + value = float('nan') + # pack double into 64 bits, then unpack as long int + return struct.unpack('Q', struct.pack('d', value))[0] + + +class VectorUDT(UserDefinedType): + """ + SQL user-defined type (UDT) for Vector. + """ + + @classmethod + def sqlType(cls): + return StructType([ + StructField("type", ByteType(), False), + StructField("size", IntegerType(), True), + StructField("indices", ArrayType(IntegerType(), False), True), + StructField("values", ArrayType(DoubleType(), False), True)]) + + @classmethod + def module(cls): + return "pyspark.ml.linalg" + + @classmethod + def scalaUDT(cls): + return "org.apache.spark.ml.linalg.VectorUDT" + + def serialize(self, obj): + if isinstance(obj, SparseVector): + indices = [int(i) for i in obj.indices] + values = [float(v) for v in obj.values] + return (0, obj.size, indices, values) + elif isinstance(obj, DenseVector): + values = [float(v) for v in obj] + return (1, None, None, values) + else: + raise TypeError("cannot serialize %r of type %r" % (obj, type(obj))) + + def deserialize(self, datum): + assert len(datum) == 4, \ + "VectorUDT.deserialize given row with length %d but requires 4" % len(datum) + tpe = datum[0] + if tpe == 0: + return SparseVector(datum[1], datum[2], datum[3]) + elif tpe == 1: + return DenseVector(datum[3]) + else: + raise ValueError("do not recognize type %r" % tpe) + + def simpleString(self): + return "vector" + + +class MatrixUDT(UserDefinedType): + """ + SQL user-defined type (UDT) for Matrix. + """ + + @classmethod + def sqlType(cls): + return StructType([ + StructField("type", ByteType(), False), + StructField("numRows", IntegerType(), False), + StructField("numCols", IntegerType(), False), + StructField("colPtrs", ArrayType(IntegerType(), False), True), + StructField("rowIndices", ArrayType(IntegerType(), False), True), + StructField("values", ArrayType(DoubleType(), False), True), + StructField("isTransposed", BooleanType(), False)]) + + @classmethod + def module(cls): + return "pyspark.ml.linalg" + + @classmethod + def scalaUDT(cls): + return "org.apache.spark.ml.linalg.MatrixUDT" + + def serialize(self, obj): + if isinstance(obj, SparseMatrix): + colPtrs = [int(i) for i in obj.colPtrs] + rowIndices = [int(i) for i in obj.rowIndices] + values = [float(v) for v in obj.values] + return (0, obj.numRows, obj.numCols, colPtrs, + rowIndices, values, bool(obj.isTransposed)) + elif isinstance(obj, DenseMatrix): + values = [float(v) for v in obj.values] + return (1, obj.numRows, obj.numCols, None, None, values, + bool(obj.isTransposed)) + else: + raise TypeError("cannot serialize type %r" % (type(obj))) + + def deserialize(self, datum): + assert len(datum) == 7, \ + "MatrixUDT.deserialize given row with length %d but requires 7" % len(datum) + tpe = datum[0] + if tpe == 0: + return SparseMatrix(*datum[1:]) + elif tpe == 1: + return DenseMatrix(datum[1], datum[2], datum[5], datum[6]) + else: + raise ValueError("do not recognize type %r" % tpe) + + def simpleString(self): + return "matrix" + + +class Vector(object): + + __UDT__ = VectorUDT() + + """ + Abstract class for DenseVector and SparseVector + """ + def toArray(self): + """ + Convert the vector into an numpy.ndarray + + :return: numpy.ndarray + """ + raise NotImplementedError + + +class DenseVector(Vector): + """ + A dense vector represented by a value array. We use numpy array for + storage and arithmetics will be delegated to the underlying numpy + array. + + >>> v = Vectors.dense([1.0, 2.0]) + >>> u = Vectors.dense([3.0, 4.0]) + >>> v + u + DenseVector([4.0, 6.0]) + >>> 2 - v + DenseVector([1.0, 0.0]) + >>> v / 2 + DenseVector([0.5, 1.0]) + >>> v * u + DenseVector([3.0, 8.0]) + >>> u / v + DenseVector([3.0, 2.0]) + >>> u % 2 + DenseVector([1.0, 0.0]) + """ + def __init__(self, ar): + if isinstance(ar, bytes): + ar = np.frombuffer(ar, dtype=np.float64) + elif not isinstance(ar, np.ndarray): + ar = np.array(ar, dtype=np.float64) + if ar.dtype != np.float64: + ar = ar.astype(np.float64) + self.array = ar + + def __reduce__(self): + return DenseVector, (self.array.tostring(),) + + def numNonzeros(self): + """ + Number of nonzero elements. This scans all active values and count non zeros + """ + return np.count_nonzero(self.array) + + def norm(self, p): + """ + Calculates the norm of a DenseVector. + + >>> a = DenseVector([0, -1, 2, -3]) + >>> a.norm(2) + 3.7... + >>> a.norm(1) + 6.0 + """ + return np.linalg.norm(self.array, p) + + def dot(self, other): + """ + Compute the dot product of two Vectors. We support + (Numpy array, list, SparseVector, or SciPy sparse) + and a target NumPy array that is either 1- or 2-dimensional. + Equivalent to calling numpy.dot of the two vectors. + + >>> dense = DenseVector(array.array('d', [1., 2.])) + >>> dense.dot(dense) + 5.0 + >>> dense.dot(SparseVector(2, [0, 1], [2., 1.])) + 4.0 + >>> dense.dot(range(1, 3)) + 5.0 + >>> dense.dot(np.array(range(1, 3))) + 5.0 + >>> dense.dot([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense.dot(np.reshape([1., 2., 3., 4.], (2, 2), order='F')) + array([ 5., 11.]) + >>> dense.dot(np.reshape([1., 2., 3.], (3, 1), order='F')) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + if type(other) == np.ndarray: + if other.ndim > 1: + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.array, other) + elif _have_scipy and scipy.sparse.issparse(other): + assert len(self) == other.shape[0], "dimension mismatch" + return other.transpose().dot(self.toArray()) + else: + assert len(self) == _vector_size(other), "dimension mismatch" + if isinstance(other, SparseVector): + return other.dot(self) + elif isinstance(other, Vector): + return np.dot(self.toArray(), other.toArray()) + else: + return np.dot(self.toArray(), other) + + def squared_distance(self, other): + """ + Squared distance of two Vectors. + + >>> dense1 = DenseVector(array.array('d', [1., 2.])) + >>> dense1.squared_distance(dense1) + 0.0 + >>> dense2 = np.array([2., 1.]) + >>> dense1.squared_distance(dense2) + 2.0 + >>> dense3 = [2., 1.] + >>> dense1.squared_distance(dense3) + 2.0 + >>> sparse1 = SparseVector(2, [0, 1], [2., 1.]) + >>> dense1.squared_distance(sparse1) + 2.0 + >>> dense1.squared_distance([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense1.squared_distance(SparseVector(1, [0,], [1.,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + assert len(self) == _vector_size(other), "dimension mismatch" + if isinstance(other, SparseVector): + return other.squared_distance(self) + elif _have_scipy and scipy.sparse.issparse(other): + return _convert_to_vector(other).squared_distance(self) + + if isinstance(other, Vector): + other = other.toArray() + elif not isinstance(other, np.ndarray): + other = np.array(other) + diff = self.toArray() - other + return np.dot(diff, diff) + + def toArray(self): + """ + Returns an numpy.ndarray + """ + return self.array + + @property + def values(self): + """ + Returns a list of values + """ + return self.array + + def __getitem__(self, item): + return self.array[item] + + def __len__(self): + return len(self.array) + + def __str__(self): + return "[" + ",".join([str(v) for v in self.array]) + "]" + + def __repr__(self): + return "DenseVector([%s])" % (', '.join(_format_float(i) for i in self.array)) + + def __eq__(self, other): + if isinstance(other, DenseVector): + return np.array_equal(self.array, other.array) + elif isinstance(other, SparseVector): + if len(self) != other.size: + return False + return Vectors._equals(list(xrange(len(self))), self.array, other.indices, other.values) + return False + + def __ne__(self, other): + return not self == other + + def __hash__(self): + size = len(self) + result = 31 + size + nnz = 0 + i = 0 + while i < size and nnz < 128: + if self.array[i] != 0: + result = 31 * result + i + bits = _double_to_long_bits(self.array[i]) + result = 31 * result + (bits ^ (bits >> 32)) + nnz += 1 + i += 1 + return result + + def __getattr__(self, item): + return getattr(self.array, item) + + def _delegate(op): + def func(self, other): + if isinstance(other, DenseVector): + other = other.array + return DenseVector(getattr(self.array, op)(other)) + return func + + __neg__ = _delegate("__neg__") + __add__ = _delegate("__add__") + __sub__ = _delegate("__sub__") + __mul__ = _delegate("__mul__") + __div__ = _delegate("__div__") + __truediv__ = _delegate("__truediv__") + __mod__ = _delegate("__mod__") + __radd__ = _delegate("__radd__") + __rsub__ = _delegate("__rsub__") + __rmul__ = _delegate("__rmul__") + __rdiv__ = _delegate("__rdiv__") + __rtruediv__ = _delegate("__rtruediv__") + __rmod__ = _delegate("__rmod__") + + +class SparseVector(Vector): + """ + A simple sparse vector class for passing data to MLlib. Users may + alternatively pass SciPy's {scipy.sparse} data types. + """ + def __init__(self, size, *args): + """ + Create a sparse vector, using either a dictionary, a list of + (index, value) pairs, or two separate arrays of indices and + values (sorted by index). + + :param size: Size of the vector. + :param args: Active entries, as a dictionary {index: value, ...}, + a list of tuples [(index, value), ...], or a list of strictly + increasing indices and a list of corresponding values [index, ...], + [value, ...]. Inactive entries are treated as zeros. + + >>> SparseVector(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) + """ + self.size = int(size) + """ Size of the vector. """ + assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" + if len(args) == 1: + pairs = args[0] + if type(pairs) == dict: + pairs = pairs.items() + pairs = sorted(pairs) + self.indices = np.array([p[0] for p in pairs], dtype=np.int32) + """ A list of indices corresponding to active entries. """ + self.values = np.array([p[1] for p in pairs], dtype=np.float64) + """ A list of values corresponding to active entries. """ + else: + if isinstance(args[0], bytes): + assert isinstance(args[1], bytes), "values should be string too" + if args[0]: + self.indices = np.frombuffer(args[0], np.int32) + self.values = np.frombuffer(args[1], np.float64) + else: + # np.frombuffer() doesn't work well with empty string in older version + self.indices = np.array([], dtype=np.int32) + self.values = np.array([], dtype=np.float64) + else: + self.indices = np.array(args[0], dtype=np.int32) + self.values = np.array(args[1], dtype=np.float64) + assert len(self.indices) == len(self.values), "index and value arrays not same length" + for i in xrange(len(self.indices) - 1): + if self.indices[i] >= self.indices[i + 1]: + raise TypeError( + "Indices %s and %s are not strictly increasing" + % (self.indices[i], self.indices[i + 1])) + + def numNonzeros(self): + """ + Number of nonzero elements. This scans all active values and count non zeros. + """ + return np.count_nonzero(self.values) + + def norm(self, p): + """ + Calculates the norm of a SparseVector. + + >>> a = SparseVector(4, [0, 1], [3., -4.]) + >>> a.norm(1) + 7.0 + >>> a.norm(2) + 5.0 + """ + return np.linalg.norm(self.values, p) + + def __reduce__(self): + return ( + SparseVector, + (self.size, self.indices.tostring(), self.values.tostring())) + + def dot(self, other): + """ + Dot product with a SparseVector or 1- or 2-dimensional Numpy array. + + >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) + >>> a.dot(a) + 25.0 + >>> a.dot(array.array('d', [1., 2., 3., 4.])) + 22.0 + >>> b = SparseVector(4, [2], [1.0]) + >>> a.dot(b) + 0.0 + >>> a.dot(np.array([[1, 1], [2, 2], [3, 3], [4, 4]])) + array([ 22., 22.]) + >>> a.dot([1., 2., 3.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.array([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(DenseVector([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.zeros((3, 2))) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + + if isinstance(other, np.ndarray): + if other.ndim not in [2, 1]: + raise ValueError("Cannot call dot with %d-dimensional array" % other.ndim) + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.values, other[self.indices]) + + assert len(self) == _vector_size(other), "dimension mismatch" + + if isinstance(other, DenseVector): + return np.dot(other.array[self.indices], self.values) + + elif isinstance(other, SparseVector): + # Find out common indices. + self_cmind = np.in1d(self.indices, other.indices, assume_unique=True) + self_values = self.values[self_cmind] + if self_values.size == 0: + return 0.0 + else: + other_cmind = np.in1d(other.indices, self.indices, assume_unique=True) + return np.dot(self_values, other.values[other_cmind]) + + else: + return self.dot(_convert_to_vector(other)) + + def squared_distance(self, other): + """ + Squared distance from a SparseVector or 1-dimensional NumPy array. + + >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) + >>> a.squared_distance(a) + 0.0 + >>> a.squared_distance(array.array('d', [1., 2., 3., 4.])) + 11.0 + >>> a.squared_distance(np.array([1., 2., 3., 4.])) + 11.0 + >>> b = SparseVector(4, [2], [1.0]) + >>> a.squared_distance(b) + 26.0 + >>> b.squared_distance(a) + 26.0 + >>> b.squared_distance([1., 2.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> b.squared_distance(SparseVector(3, [1,], [1.0,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + assert len(self) == _vector_size(other), "dimension mismatch" + + if isinstance(other, np.ndarray) or isinstance(other, DenseVector): + if isinstance(other, np.ndarray) and other.ndim != 1: + raise Exception("Cannot call squared_distance with %d-dimensional array" % + other.ndim) + if isinstance(other, DenseVector): + other = other.array + sparse_ind = np.zeros(other.size, dtype=bool) + sparse_ind[self.indices] = True + dist = other[sparse_ind] - self.values + result = np.dot(dist, dist) + + other_ind = other[~sparse_ind] + result += np.dot(other_ind, other_ind) + return result + + elif isinstance(other, SparseVector): + result = 0.0 + i, j = 0, 0 + while i < len(self.indices) and j < len(other.indices): + if self.indices[i] == other.indices[j]: + diff = self.values[i] - other.values[j] + result += diff * diff + i += 1 + j += 1 + elif self.indices[i] < other.indices[j]: + result += self.values[i] * self.values[i] + i += 1 + else: + result += other.values[j] * other.values[j] + j += 1 + while i < len(self.indices): + result += self.values[i] * self.values[i] + i += 1 + while j < len(other.indices): + result += other.values[j] * other.values[j] + j += 1 + return result + else: + return self.squared_distance(_convert_to_vector(other)) + + def toArray(self): + """ + Returns a copy of this SparseVector as a 1-dimensional NumPy array. + """ + arr = np.zeros((self.size,), dtype=np.float64) + arr[self.indices] = self.values + return arr + + def __len__(self): + return self.size + + def __str__(self): + inds = "[" + ",".join([str(i) for i in self.indices]) + "]" + vals = "[" + ",".join([str(v) for v in self.values]) + "]" + return "(" + ",".join((str(self.size), inds, vals)) + ")" + + def __repr__(self): + inds = self.indices + vals = self.values + entries = ", ".join(["{0}: {1}".format(inds[i], _format_float(vals[i])) + for i in xrange(len(inds))]) + return "SparseVector({0}, {{{1}}})".format(self.size, entries) + + def __eq__(self, other): + if isinstance(other, SparseVector): + return other.size == self.size and np.array_equal(other.indices, self.indices) \ + and np.array_equal(other.values, self.values) + elif isinstance(other, DenseVector): + if self.size != len(other): + return False + return Vectors._equals(self.indices, self.values, list(xrange(len(other))), other.array) + return False + + def __getitem__(self, index): + inds = self.indices + vals = self.values + if not isinstance(index, int): + raise TypeError( + "Indices must be of type integer, got type %s" % type(index)) + + if index >= self.size or index < -self.size: + raise ValueError("Index %d out of bounds." % index) + if index < 0: + index += self.size + + if (inds.size == 0) or (index > inds.item(-1)): + return 0. + + insert_index = np.searchsorted(inds, index) + row_ind = inds[insert_index] + if row_ind == index: + return vals[insert_index] + return 0. + + def __ne__(self, other): + return not self.__eq__(other) + + def __hash__(self): + result = 31 + self.size + nnz = 0 + i = 0 + while i < len(self.values) and nnz < 128: + if self.values[i] != 0: + result = 31 * result + int(self.indices[i]) + bits = _double_to_long_bits(self.values[i]) + result = 31 * result + (bits ^ (bits >> 32)) + nnz += 1 + i += 1 + return result + + +class Vectors(object): + + """ + Factory methods for working with vectors. Note that dense vectors + are simply represented as NumPy array objects, so there is no need + to covert them for use in MLlib. For sparse vectors, the factory + methods in this class create an MLlib-compatible type, or users + can pass in SciPy's C{scipy.sparse} column vectors. + """ + + @staticmethod + def sparse(size, *args): + """ + Create a sparse vector, using either a dictionary, a list of + (index, value) pairs, or two separate arrays of indices and + values (sorted by index). + + :param size: Size of the vector. + :param args: Non-zero entries, as a dictionary, list of tuples, + or two sorted lists containing indices and values. + + >>> Vectors.sparse(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) + """ + return SparseVector(size, *args) + + @staticmethod + def dense(*elements): + """ + Create a dense vector of 64-bit floats from a Python list or numbers. + + >>> Vectors.dense([1, 2, 3]) + DenseVector([1.0, 2.0, 3.0]) + >>> Vectors.dense(1.0, 2.0) + DenseVector([1.0, 2.0]) + """ + if len(elements) == 1 and not isinstance(elements[0], (float, int, long)): + # it's list, numpy.array or other iterable object. + elements = elements[0] + return DenseVector(elements) + + @staticmethod + def squared_distance(v1, v2): + """ + Squared distance between two vectors. + a and b can be of type SparseVector, DenseVector, np.ndarray + or array.array. + + >>> a = Vectors.sparse(4, [(0, 1), (3, 4)]) + >>> b = Vectors.dense([2, 5, 4, 1]) + >>> a.squared_distance(b) + 51.0 + """ + v1, v2 = _convert_to_vector(v1), _convert_to_vector(v2) + return v1.squared_distance(v2) + + @staticmethod + def norm(vector, p): + """ + Find norm of the given vector. + """ + return _convert_to_vector(vector).norm(p) + + @staticmethod + def zeros(size): + return DenseVector(np.zeros(size)) + + @staticmethod + def _equals(v1_indices, v1_values, v2_indices, v2_values): + """ + Check equality between sparse/dense vectors, + v1_indices and v2_indices assume to be strictly increasing. + """ + v1_size = len(v1_values) + v2_size = len(v2_values) + k1 = 0 + k2 = 0 + all_equal = True + while all_equal: + while k1 < v1_size and v1_values[k1] == 0: + k1 += 1 + while k2 < v2_size and v2_values[k2] == 0: + k2 += 1 + + if k1 >= v1_size or k2 >= v2_size: + return k1 >= v1_size and k2 >= v2_size + + all_equal = v1_indices[k1] == v2_indices[k2] and v1_values[k1] == v2_values[k2] + k1 += 1 + k2 += 1 + return all_equal + + +class Matrix(object): + + __UDT__ = MatrixUDT() + + """ + Represents a local matrix. + """ + def __init__(self, numRows, numCols, isTransposed=False): + self.numRows = numRows + self.numCols = numCols + self.isTransposed = isTransposed + + def toArray(self): + """ + Returns its elements in a NumPy ndarray. + """ + raise NotImplementedError + + @staticmethod + def _convert_to_array(array_like, dtype): + """ + Convert Matrix attributes which are array-like or buffer to array. + """ + if isinstance(array_like, bytes): + return np.frombuffer(array_like, dtype=dtype) + return np.asarray(array_like, dtype=dtype) + + +class DenseMatrix(Matrix): + """ + Column-major dense matrix. + """ + def __init__(self, numRows, numCols, values, isTransposed=False): + Matrix.__init__(self, numRows, numCols, isTransposed) + values = self._convert_to_array(values, np.float64) + assert len(values) == numRows * numCols + self.values = values + + def __reduce__(self): + return DenseMatrix, ( + self.numRows, self.numCols, self.values.tostring(), + int(self.isTransposed)) + + def __str__(self): + """ + Pretty printing of a DenseMatrix + + >>> dm = DenseMatrix(2, 2, range(4)) + >>> print(dm) + DenseMatrix([[ 0., 2.], + [ 1., 3.]]) + >>> dm = DenseMatrix(2, 2, range(4), isTransposed=True) + >>> print(dm) + DenseMatrix([[ 0., 1.], + [ 2., 3.]]) + """ + # Inspired by __repr__ in scipy matrices. + array_lines = repr(self.toArray()).splitlines() + + # We need to adjust six spaces which is the difference in number + # of letters between "DenseMatrix" and "array" + x = '\n'.join([(" " * 6 + line) for line in array_lines[1:]]) + return array_lines[0].replace("array", "DenseMatrix") + "\n" + x + + def __repr__(self): + """ + Representation of a DenseMatrix + + >>> dm = DenseMatrix(2, 2, range(4)) + >>> dm + DenseMatrix(2, 2, [0.0, 1.0, 2.0, 3.0], False) + """ + # If the number of values are less than seventeen then return as it is. + # Else return first eight values and last eight values. + if len(self.values) < 17: + entries = _format_float_list(self.values) + else: + entries = ( + _format_float_list(self.values[:8]) + + ["..."] + + _format_float_list(self.values[-8:]) + ) + + entries = ", ".join(entries) + return "DenseMatrix({0}, {1}, [{2}], {3})".format( + self.numRows, self.numCols, entries, self.isTransposed) + + def toArray(self): + """ + Return an numpy.ndarray + + >>> m = DenseMatrix(2, 2, range(4)) + >>> m.toArray() + array([[ 0., 2.], + [ 1., 3.]]) + """ + if self.isTransposed: + return np.asfortranarray( + self.values.reshape((self.numRows, self.numCols))) + else: + return self.values.reshape((self.numRows, self.numCols), order='F') + + def toSparse(self): + """Convert to SparseMatrix""" + if self.isTransposed: + values = np.ravel(self.toArray(), order='F') + else: + values = self.values + indices = np.nonzero(values)[0] + colCounts = np.bincount(indices // self.numRows) + colPtrs = np.cumsum(np.hstack( + (0, colCounts, np.zeros(self.numCols - colCounts.size)))) + values = values[indices] + rowIndices = indices % self.numRows + + return SparseMatrix(self.numRows, self.numCols, colPtrs, rowIndices, values) + + def __getitem__(self, indices): + i, j = indices + if i < 0 or i >= self.numRows: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j >= self.numCols or j < 0: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + + if self.isTransposed: + return self.values[i * self.numCols + j] + else: + return self.values[i + j * self.numRows] + + def __eq__(self, other): + if (not isinstance(other, DenseMatrix) or + self.numRows != other.numRows or + self.numCols != other.numCols): + return False + + self_values = np.ravel(self.toArray(), order='F') + other_values = np.ravel(other.toArray(), order='F') + return all(self_values == other_values) + + +class SparseMatrix(Matrix): + """Sparse Matrix stored in CSC format.""" + def __init__(self, numRows, numCols, colPtrs, rowIndices, values, + isTransposed=False): + Matrix.__init__(self, numRows, numCols, isTransposed) + self.colPtrs = self._convert_to_array(colPtrs, np.int32) + self.rowIndices = self._convert_to_array(rowIndices, np.int32) + self.values = self._convert_to_array(values, np.float64) + + if self.isTransposed: + if self.colPtrs.size != numRows + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numRows + 1, self.colPtrs.size)) + else: + if self.colPtrs.size != numCols + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numCols + 1, self.colPtrs.size)) + if self.rowIndices.size != self.values.size: + raise ValueError("Expected rowIndices of length %d, got %d." + % (self.rowIndices.size, self.values.size)) + + def __str__(self): + """ + Pretty printing of a SparseMatrix + + >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + >>> print(sm1) + 2 X 2 CSCMatrix + (0,0) 2.0 + (1,0) 3.0 + (1,1) 4.0 + >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4], True) + >>> print(sm1) + 2 X 2 CSRMatrix + (0,0) 2.0 + (0,1) 3.0 + (1,1) 4.0 + """ + spstr = "{0} X {1} ".format(self.numRows, self.numCols) + if self.isTransposed: + spstr += "CSRMatrix\n" + else: + spstr += "CSCMatrix\n" + + cur_col = 0 + smlist = [] + + # Display first 16 values. + if len(self.values) <= 16: + zipindval = zip(self.rowIndices, self.values) + else: + zipindval = zip(self.rowIndices[:16], self.values[:16]) + for i, (rowInd, value) in enumerate(zipindval): + if self.colPtrs[cur_col + 1] <= i: + cur_col += 1 + if self.isTransposed: + smlist.append('({0},{1}) {2}'.format( + cur_col, rowInd, _format_float(value))) + else: + smlist.append('({0},{1}) {2}'.format( + rowInd, cur_col, _format_float(value))) + spstr += "\n".join(smlist) + + if len(self.values) > 16: + spstr += "\n.." * 2 + return spstr + + def __repr__(self): + """ + Representation of a SparseMatrix + + >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + >>> sm1 + SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2.0, 3.0, 4.0], False) + """ + rowIndices = list(self.rowIndices) + colPtrs = list(self.colPtrs) + + if len(self.values) <= 16: + values = _format_float_list(self.values) + + else: + values = ( + _format_float_list(self.values[:8]) + + ["..."] + + _format_float_list(self.values[-8:]) + ) + rowIndices = rowIndices[:8] + ["..."] + rowIndices[-8:] + + if len(self.colPtrs) > 16: + colPtrs = colPtrs[:8] + ["..."] + colPtrs[-8:] + + values = ", ".join(values) + rowIndices = ", ".join([str(ind) for ind in rowIndices]) + colPtrs = ", ".join([str(ptr) for ptr in colPtrs]) + return "SparseMatrix({0}, {1}, [{2}], [{3}], [{4}], {5})".format( + self.numRows, self.numCols, colPtrs, rowIndices, + values, self.isTransposed) + + def __reduce__(self): + return SparseMatrix, ( + self.numRows, self.numCols, self.colPtrs.tostring(), + self.rowIndices.tostring(), self.values.tostring(), + int(self.isTransposed)) + + def __getitem__(self, indices): + i, j = indices + if i < 0 or i >= self.numRows: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j < 0 or j >= self.numCols: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + + # If a CSR matrix is given, then the row index should be searched + # for in ColPtrs, and the column index should be searched for in the + # corresponding slice obtained from rowIndices. + if self.isTransposed: + j, i = i, j + + colStart = self.colPtrs[j] + colEnd = self.colPtrs[j + 1] + nz = self.rowIndices[colStart: colEnd] + ind = np.searchsorted(nz, i) + colStart + if ind < colEnd and self.rowIndices[ind] == i: + return self.values[ind] + else: + return 0.0 + + def toArray(self): + """ + Return an numpy.ndarray + """ + A = np.zeros((self.numRows, self.numCols), dtype=np.float64, order='F') + for k in xrange(self.colPtrs.size - 1): + startptr = self.colPtrs[k] + endptr = self.colPtrs[k + 1] + if self.isTransposed: + A[k, self.rowIndices[startptr:endptr]] = self.values[startptr:endptr] + else: + A[self.rowIndices[startptr:endptr], k] = self.values[startptr:endptr] + return A + + def toDense(self): + densevals = np.ravel(self.toArray(), order='F') + return DenseMatrix(self.numRows, self.numCols, densevals) + + # TODO: More efficient implementation: + def __eq__(self, other): + return np.all(self.toArray() == other.toArray()) + + +class Matrices(object): + @staticmethod + def dense(numRows, numCols, values): + """ + Create a DenseMatrix + """ + return DenseMatrix(numRows, numCols, values) + + @staticmethod + def sparse(numRows, numCols, colPtrs, rowIndices, values): + """ + Create a SparseMatrix + """ + return SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + + +def _test(): + import doctest + (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS) + if failure_count: + exit(-1) + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 8e56b0d6fff0..c5679057597e 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -18,7 +18,6 @@ """ Unit tests for Spark ML Python APIs. """ -import array import sys if sys.version > '3': xrange = range @@ -40,15 +39,21 @@ from shutil import rmtree import tempfile +import array as pyarray import numpy as np +from numpy import ( + array, array_equal, zeros, inf, random, exp, dot, all, mean, abs, arange, tile, ones) +from numpy import sum as array_sum import inspect -from pyspark import keyword_only +from pyspark import keyword_only, SparkContext from pyspark.ml import Estimator, Model, Pipeline, PipelineModel, Transformer from pyspark.ml.classification import * from pyspark.ml.clustering import * from pyspark.ml.evaluation import BinaryClassificationEvaluator, RegressionEvaluator from pyspark.ml.feature import * +from pyspark.ml.linalg import Vector, SparseVector, DenseVector, VectorUDT,\ + DenseMatrix, SparseMatrix, Vectors, Matrices, MatrixUDT, _convert_to_vector from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed from pyspark.ml.recommendation import ALS @@ -57,13 +62,28 @@ from pyspark.ml.tuning import * from pyspark.ml.wrapper import JavaParams from pyspark.mllib.common import _java2py -from pyspark.mllib.linalg import Vectors, DenseVector, SparseVector +from pyspark.mllib.linalg import SparseVector as OldSparseVector, DenseVector as OldDenseVector,\ + DenseMatrix as OldDenseMatrix, MatrixUDT as OldMatrixUDT, SparseMatrix as OldSparseMatrix,\ + Vectors as OldVectors, VectorUDT as OldVectorUDT +from pyspark.mllib.regression import LabeledPoint +from pyspark.serializers import PickleSerializer from pyspark.sql import DataFrame, Row, SparkSession from pyspark.sql.functions import rand from pyspark.sql.utils import IllegalArgumentException from pyspark.storagelevel import * from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase +ser = PickleSerializer() + + +class MLlibTestCase(unittest.TestCase): + def setUp(self): + self.sc = SparkContext('local[4]', "MLlib tests") + self.spark = SparkSession(self.sc) + + def tearDown(self): + self.spark.stop() + class SparkSessionTestCase(PySparkTestCase): @classmethod @@ -142,23 +162,23 @@ def test_float(self): def test_vector(self): ewp = ElementwiseProduct(scalingVec=[1, 3]) - self.assertEqual(ewp.getScalingVec(), DenseVector([1.0, 3.0])) + self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.0, 3.0])) ewp = ElementwiseProduct(scalingVec=np.array([1.2, 3.4])) - self.assertEqual(ewp.getScalingVec(), DenseVector([1.2, 3.4])) + self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.2, 3.4])) self.assertRaises(TypeError, lambda: ElementwiseProduct(scalingVec=["a", "b"])) def test_list(self): l = [0, 1] - for lst_like in [l, np.array(l), DenseVector(l), SparseVector(len(l), range(len(l)), l), - array.array('l', l), xrange(2), tuple(l)]: + for lst_like in [l, np.array(l), OldDenseVector(l), OldSparseVector(len(l), + range(len(l)), l), pyarray.array('l', l), xrange(2), tuple(l)]: converted = TypeConverters.toList(lst_like) self.assertEqual(type(converted), list) self.assertListEqual(converted, l) def test_list_int(self): - for indices in [[1.0, 2.0], np.array([1.0, 2.0]), DenseVector([1.0, 2.0]), - SparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), - array.array('d', [1.0, 2.0])]: + for indices in [[1.0, 2.0], np.array([1.0, 2.0]), OldDenseVector([1.0, 2.0]), + OldSparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), + pyarray.array('d', [1.0, 2.0])]: vs = VectorSlicer(indices=indices) self.assertListEqual(vs.getIndices(), [1, 2]) self.assertTrue(all([type(v) == int for v in vs.getIndices()])) @@ -390,9 +410,9 @@ def test_binarizer(self): def test_idf(self): dataset = self.spark.createDataFrame([ - (DenseVector([1.0, 2.0]),), - (DenseVector([0.0, 1.0]),), - (DenseVector([3.0, 0.2]),)], ["tf"]) + (OldDenseVector([1.0, 2.0]),), + (OldDenseVector([0.0, 1.0]),), + (OldDenseVector([3.0, 0.2]),)], ["tf"]) idf0 = IDF(inputCol="tf") self.assertListEqual(idf0.params, [idf0.inputCol, idf0.minDocFreq, idf0.outputCol]) idf0m = idf0.fit(dataset, {idf0.outputCol: "idf"}) @@ -437,10 +457,10 @@ def test_stopwordsremover(self): def test_count_vectorizer_with_binary(self): dataset = self.spark.createDataFrame([ - (0, "a a a b b c".split(' '), SparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), - (1, "a a".split(' '), SparseVector(3, {0: 1.0}),), - (2, "a b".split(' '), SparseVector(3, {0: 1.0, 1: 1.0}),), - (3, "c".split(' '), SparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) + (0, "a a a b b c".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), + (1, "a a".split(' '), OldSparseVector(3, {0: 1.0}),), + (2, "a b".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0}),), + (3, "c".split(' '), OldSparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) cv = CountVectorizer(binary=True, inputCol="words", outputCol="features") model = cv.fit(dataset) @@ -561,11 +581,11 @@ def test_save_load(self): # Save/load for CrossValidator will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(Vectors.dense([0.0]), 0.0), - (Vectors.dense([0.4]), 1.0), - (Vectors.dense([0.5]), 0.0), - (Vectors.dense([0.6]), 1.0), - (Vectors.dense([1.0]), 1.0)] * 10, + [(OldVectors.dense([0.0]), 0.0), + (OldVectors.dense([0.4]), 1.0), + (OldVectors.dense([0.5]), 0.0), + (OldVectors.dense([0.6]), 1.0), + (OldVectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -634,11 +654,11 @@ def test_save_load(self): # Save/load for TrainValidationSplit will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(Vectors.dense([0.0]), 0.0), - (Vectors.dense([0.4]), 1.0), - (Vectors.dense([0.5]), 0.0), - (Vectors.dense([0.6]), 1.0), - (Vectors.dense([1.0]), 1.0)] * 10, + [(OldVectors.dense([0.0]), 0.0), + (OldVectors.dense([0.4]), 1.0), + (OldVectors.dense([0.5]), 0.0), + (OldVectors.dense([0.6]), 1.0), + (OldVectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -837,8 +857,8 @@ def _compare(self, m1, m2): def test_persistence(self): # Test save/load for LDA, LocalLDAModel, DistributedLDAModel. df = self.spark.createDataFrame([ - [1, Vectors.dense([0.0, 1.0])], - [2, Vectors.sparse(2, {0: 1.0})], + [1, OldVectors.dense([0.0, 1.0])], + [2, OldVectors.sparse(2, {0: 1.0})], ], ["id", "features"]) # Fit model lda = LDA(k=2, seed=1, optimizer="em") @@ -873,9 +893,8 @@ def test_persistence(self): class TrainingSummaryTest(SparkSessionTestCase): def test_linear_regression_summary(self): - from pyspark.mllib.linalg import Vectors - df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), - (0.0, 2.0, Vectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), + (0.0, 2.0, OldVectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight", fitIntercept=False) @@ -947,9 +966,8 @@ def test_glr_summary(self): self.assertAlmostEqual(sameSummary.deviance, s.deviance) def test_logistic_regression_summary(self): - from pyspark.mllib.linalg import Vectors - df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), - (0.0, 2.0, Vectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), + (0.0, 2.0, OldVectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) model = lr.fit(df) @@ -978,9 +996,9 @@ def test_logistic_regression_summary(self): class OneVsRestTests(SparkSessionTestCase): def test_copy(self): - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), + (1.0, OldVectors.sparse(2, [], [])), + (2.0, OldVectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -992,9 +1010,9 @@ def test_copy(self): self.assertEqual(model1.getPredictionCol(), "indexed") def test_output_columns(self): - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), + (1.0, OldVectors.sparse(2, [], [])), + (2.0, OldVectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1004,9 +1022,9 @@ def test_output_columns(self): def test_save_load(self): temp_path = tempfile.mkdtemp() - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), + (1.0, OldVectors.sparse(2, [], [])), + (2.0, OldVectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1034,7 +1052,7 @@ def test_apply_binary_term_freqs(self): hashingTF.setInputCol("words").setOutputCol("features").setNumFeatures(n).setBinary(True) output = hashingTF.transform(df) features = output.select("features").first().features.toArray() - expected = Vectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() + expected = OldVectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() for i in range(0, n): self.assertAlmostEqual(features[i], expected[i], 14, "Error at " + str(i) + ": expected " + str(expected[i]) + ", got " + str(features[i])) @@ -1109,6 +1127,354 @@ def test_java_params(self): self.check_params(cls()) +def _squared_distance(a, b): + if isinstance(a, Vector): + return a.squared_distance(b) + else: + return b.squared_distance(a) + + +class VectorTests(MLlibTestCase): + + def _test_serialize(self, v): + self.assertEqual(v, ser.loads(ser.dumps(v))) + jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) + nv = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvec))) + self.assertEqual(v, nv) + vs = [v] * 100 + jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) + nvs = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvecs))) + self.assertEqual(vs, nvs) + + def test_serialize(self): + # Because pickle path still uses old vector/matrix + # TODO: Change this to new vector/matrix when pickle for new vector/matrix is ready. + self._test_serialize(OldDenseVector(range(10))) + self._test_serialize(OldDenseVector(array([1., 2., 3., 4.]))) + self._test_serialize(OldDenseVector(pyarray.array('d', range(10)))) + self._test_serialize(OldSparseVector(4, {1: 1, 3: 2})) + self._test_serialize(OldSparseVector(3, {})) + self._test_serialize(OldDenseMatrix(2, 3, range(6))) + sm1 = OldSparseMatrix( + 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) + self._test_serialize(sm1) + + def test_dot(self): + sv = SparseVector(4, {1: 1, 3: 2}) + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([1, 2, 3, 4]) + mat = array([[1., 2., 3., 4.], + [1., 2., 3., 4.], + [1., 2., 3., 4.], + [1., 2., 3., 4.]]) + arr = pyarray.array('d', [0, 1, 2, 3]) + self.assertEqual(10.0, sv.dot(dv)) + self.assertTrue(array_equal(array([3., 6., 9., 12.]), sv.dot(mat))) + self.assertEqual(30.0, dv.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), dv.dot(mat))) + self.assertEqual(30.0, lst.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), lst.dot(mat))) + self.assertEqual(7.0, sv.dot(arr)) + + def test_squared_distance(self): + sv = SparseVector(4, {1: 1, 3: 2}) + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([4, 3, 2, 1]) + lst1 = [4, 3, 2, 1] + arr = pyarray.array('d', [0, 2, 1, 3]) + narr = array([0, 2, 1, 3]) + self.assertEqual(15.0, _squared_distance(sv, dv)) + self.assertEqual(25.0, _squared_distance(sv, lst)) + self.assertEqual(20.0, _squared_distance(dv, lst)) + self.assertEqual(15.0, _squared_distance(dv, sv)) + self.assertEqual(25.0, _squared_distance(lst, sv)) + self.assertEqual(20.0, _squared_distance(lst, dv)) + self.assertEqual(0.0, _squared_distance(sv, sv)) + self.assertEqual(0.0, _squared_distance(dv, dv)) + self.assertEqual(0.0, _squared_distance(lst, lst)) + self.assertEqual(25.0, _squared_distance(sv, lst1)) + self.assertEqual(3.0, _squared_distance(sv, arr)) + self.assertEqual(3.0, _squared_distance(sv, narr)) + + def test_hash(self): + v1 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v2 = SparseVector(4, [(1, 1.0), (3, 5.5)]) + v3 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v4 = SparseVector(4, [(1, 1.0), (3, 2.5)]) + self.assertEqual(hash(v1), hash(v2)) + self.assertEqual(hash(v1), hash(v3)) + self.assertEqual(hash(v2), hash(v3)) + self.assertFalse(hash(v1) == hash(v4)) + self.assertFalse(hash(v2) == hash(v4)) + + def test_eq(self): + v1 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v2 = SparseVector(4, [(1, 1.0), (3, 5.5)]) + v3 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v4 = SparseVector(6, [(1, 1.0), (3, 5.5)]) + v5 = DenseVector([0.0, 1.0, 0.0, 2.5]) + v6 = SparseVector(4, [(1, 1.0), (3, 2.5)]) + self.assertEqual(v1, v2) + self.assertEqual(v1, v3) + self.assertFalse(v2 == v4) + self.assertFalse(v1 == v5) + self.assertFalse(v1 == v6) + + def test_equals(self): + indices = [1, 2, 4] + values = [1., 3., 2.] + self.assertTrue(Vectors._equals(indices, values, list(range(5)), [0., 1., 3., 0., 2.])) + self.assertFalse(Vectors._equals(indices, values, list(range(5)), [0., 3., 1., 0., 2.])) + self.assertFalse(Vectors._equals(indices, values, list(range(5)), [0., 3., 0., 2.])) + self.assertFalse(Vectors._equals(indices, values, list(range(5)), [0., 1., 3., 2., 2.])) + + def test_conversion(self): + # numpy arrays should be automatically upcast to float64 + # tests for fix of [SPARK-5089] + v = array([1, 2, 3, 4], dtype='float64') + dv = DenseVector(v) + self.assertTrue(dv.array.dtype == 'float64') + v = array([1, 2, 3, 4], dtype='float32') + dv = DenseVector(v) + self.assertTrue(dv.array.dtype == 'float64') + + def test_sparse_vector_indexing(self): + sv = SparseVector(5, {1: 1, 3: 2}) + self.assertEqual(sv[0], 0.) + self.assertEqual(sv[3], 2.) + self.assertEqual(sv[1], 1.) + self.assertEqual(sv[2], 0.) + self.assertEqual(sv[4], 0.) + self.assertEqual(sv[-1], 0.) + self.assertEqual(sv[-2], 2.) + self.assertEqual(sv[-3], 0.) + self.assertEqual(sv[-5], 0.) + for ind in [5, -6]: + self.assertRaises(ValueError, sv.__getitem__, ind) + for ind in [7.8, '1']: + self.assertRaises(TypeError, sv.__getitem__, ind) + + zeros = SparseVector(4, {}) + self.assertEqual(zeros[0], 0.0) + self.assertEqual(zeros[3], 0.0) + for ind in [4, -5]: + self.assertRaises(ValueError, zeros.__getitem__, ind) + + empty = SparseVector(0, {}) + for ind in [-1, 0, 1]: + self.assertRaises(ValueError, empty.__getitem__, ind) + + def test_matrix_indexing(self): + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) + expected = [[0, 6], [1, 8], [4, 10]] + for i in range(3): + for j in range(2): + self.assertEqual(mat[i, j], expected[i][j]) + + def test_repr_dense_matrix(self): + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) + self.assertTrue( + repr(mat), + 'DenseMatrix(3, 2, [0.0, 1.0, 4.0, 6.0, 8.0, 10.0], False)') + + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10], True) + self.assertTrue( + repr(mat), + 'DenseMatrix(3, 2, [0.0, 1.0, 4.0, 6.0, 8.0, 10.0], False)') + + mat = DenseMatrix(6, 3, zeros(18)) + self.assertTrue( + repr(mat), + 'DenseMatrix(6, 3, [0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, ..., \ + 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0], False)') + + def test_repr_sparse_matrix(self): + sm1t = SparseMatrix( + 3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], + isTransposed=True) + self.assertTrue( + repr(sm1t), + 'SparseMatrix(3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], True)') + + indices = tile(arange(6), 3) + values = ones(18) + sm = SparseMatrix(6, 3, [0, 6, 12, 18], indices, values) + self.assertTrue( + repr(sm), "SparseMatrix(6, 3, [0, 6, 12, 18], \ + [0, 1, 2, 3, 4, 5, 0, 1, ..., 4, 5, 0, 1, 2, 3, 4, 5], \ + [1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, ..., \ + 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0], False)") + + self.assertTrue( + str(sm), + "6 X 3 CSCMatrix\n\ + (0,0) 1.0\n(1,0) 1.0\n(2,0) 1.0\n(3,0) 1.0\n(4,0) 1.0\n(5,0) 1.0\n\ + (0,1) 1.0\n(1,1) 1.0\n(2,1) 1.0\n(3,1) 1.0\n(4,1) 1.0\n(5,1) 1.0\n\ + (0,2) 1.0\n(1,2) 1.0\n(2,2) 1.0\n(3,2) 1.0\n..\n..") + + sm = SparseMatrix(1, 18, zeros(19), [], []) + self.assertTrue( + repr(sm), + 'SparseMatrix(1, 18, \ + [0, 0, 0, 0, 0, 0, 0, 0, ..., 0, 0, 0, 0, 0, 0, 0, 0], [], [], False)') + + def test_sparse_matrix(self): + # Test sparse matrix creation. + sm1 = SparseMatrix( + 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) + self.assertEqual(sm1.numRows, 3) + self.assertEqual(sm1.numCols, 4) + self.assertEqual(sm1.colPtrs.tolist(), [0, 2, 2, 4, 4]) + self.assertEqual(sm1.rowIndices.tolist(), [1, 2, 1, 2]) + self.assertEqual(sm1.values.tolist(), [1.0, 2.0, 4.0, 5.0]) + self.assertTrue( + repr(sm1), + 'SparseMatrix(3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0], False)') + + # Test indexing + expected = [ + [0, 0, 0, 0], + [1, 0, 4, 0], + [2, 0, 5, 0]] + + for i in range(3): + for j in range(4): + self.assertEqual(expected[i][j], sm1[i, j]) + self.assertTrue(array_equal(sm1.toArray(), expected)) + + # Test conversion to dense and sparse. + smnew = sm1.toDense().toSparse() + self.assertEqual(sm1.numRows, smnew.numRows) + self.assertEqual(sm1.numCols, smnew.numCols) + self.assertTrue(array_equal(sm1.colPtrs, smnew.colPtrs)) + self.assertTrue(array_equal(sm1.rowIndices, smnew.rowIndices)) + self.assertTrue(array_equal(sm1.values, smnew.values)) + + sm1t = SparseMatrix( + 3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], + isTransposed=True) + self.assertEqual(sm1t.numRows, 3) + self.assertEqual(sm1t.numCols, 4) + self.assertEqual(sm1t.colPtrs.tolist(), [0, 2, 3, 5]) + self.assertEqual(sm1t.rowIndices.tolist(), [0, 1, 2, 0, 2]) + self.assertEqual(sm1t.values.tolist(), [3.0, 2.0, 4.0, 9.0, 8.0]) + + expected = [ + [3, 2, 0, 0], + [0, 0, 4, 0], + [9, 0, 8, 0]] + + for i in range(3): + for j in range(4): + self.assertEqual(expected[i][j], sm1t[i, j]) + self.assertTrue(array_equal(sm1t.toArray(), expected)) + + def test_dense_matrix_is_transposed(self): + mat1 = DenseMatrix(3, 2, [0, 4, 1, 6, 3, 9], isTransposed=True) + mat = DenseMatrix(3, 2, [0, 1, 3, 4, 6, 9]) + self.assertEqual(mat1, mat) + + expected = [[0, 4], [1, 6], [3, 9]] + for i in range(3): + for j in range(2): + self.assertEqual(mat1[i, j], expected[i][j]) + self.assertTrue(array_equal(mat1.toArray(), expected)) + + sm = mat1.toSparse() + self.assertTrue(array_equal(sm.rowIndices, [1, 2, 0, 1, 2])) + self.assertTrue(array_equal(sm.colPtrs, [0, 2, 5])) + self.assertTrue(array_equal(sm.values, [1, 3, 4, 6, 9])) + + def test_norms(self): + a = DenseVector([0, 2, 3, -1]) + self.assertAlmostEqual(a.norm(2), 3.742, 3) + self.assertTrue(a.norm(1), 6) + self.assertTrue(a.norm(inf), 3) + a = SparseVector(4, [0, 2], [3, -4]) + self.assertAlmostEqual(a.norm(2), 5) + self.assertTrue(a.norm(1), 7) + self.assertTrue(a.norm(inf), 4) + + tmp = SparseVector(4, [0, 2], [3, 0]) + self.assertEqual(tmp.numNonzeros(), 1) + + +class VectorUDTTests(MLlibTestCase): + + dv0 = DenseVector([]) + dv1 = DenseVector([1.0, 2.0]) + sv0 = SparseVector(2, [], []) + sv1 = SparseVector(2, [1], [2.0]) + udt = VectorUDT() + + old_dv0 = OldDenseVector([]) + old_dv1 = OldDenseVector([1.0, 2.0]) + old_sv0 = OldSparseVector(2, [], []) + old_sv1 = OldSparseVector(2, [1], [2.0]) + old_udt = OldVectorUDT() + + def test_json_schema(self): + self.assertEqual(VectorUDT.fromJson(self.udt.jsonValue()), self.udt) + + def test_serialization(self): + for v in [self.dv0, self.dv1, self.sv0, self.sv1]: + self.assertEqual(v, self.udt.deserialize(self.udt.serialize(v))) + + def test_infer_schema(self): + rdd = self.sc.parallelize([LabeledPoint(1.0, self.old_dv1), + LabeledPoint(0.0, self.old_sv1)]) + df = rdd.toDF() + schema = df.schema + field = [f for f in schema.fields if f.name == "features"][0] + self.assertEqual(field.dataType, self.old_udt) + vectors = df.rdd.map(lambda p: p.features).collect() + self.assertEqual(len(vectors), 2) + for v in vectors: + if isinstance(v, OldSparseVector): + self.assertEqual(v, self.old_sv1) + elif isinstance(v, OldDenseVector): + self.assertEqual(v, self.old_dv1) + else: + raise TypeError("expecting a vector but got %r of type %r" % (v, type(v))) + + +class MatrixUDTTests(MLlibTestCase): + + dm1 = DenseMatrix(3, 2, [0, 1, 4, 5, 9, 10]) + dm2 = DenseMatrix(3, 2, [0, 1, 4, 5, 9, 10], isTransposed=True) + sm1 = SparseMatrix(1, 1, [0, 1], [0], [2.0]) + sm2 = SparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) + udt = MatrixUDT() + + old_dm1 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10]) + old_dm2 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10], isTransposed=True) + old_sm1 = OldSparseMatrix(1, 1, [0, 1], [0], [2.0]) + old_sm2 = OldSparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) + old_udt = OldMatrixUDT() + + def test_json_schema(self): + self.assertEqual(MatrixUDT.fromJson(self.udt.jsonValue()), self.udt) + + def test_serialization(self): + for m in [self.dm1, self.dm2, self.sm1, self.sm2]: + self.assertEqual(m, self.udt.deserialize(self.udt.serialize(m))) + + def test_infer_schema(self): + rdd = self.sc.parallelize([("dense", self.old_dm1), ("sparse", self.old_sm1)]) + df = rdd.toDF() + schema = df.schema + self.assertTrue(schema.fields[1].dataType, self.old_udt) + matrices = df.rdd.map(lambda x: x._2).collect() + self.assertEqual(len(matrices), 2) + for m in matrices: + if isinstance(m, OldDenseMatrix): + self.assertTrue(m, self.old_dm1) + elif isinstance(m, OldSparseMatrix): + self.assertTrue(m, self.old_sm1) + else: + raise ValueError("Expected a matrix but got type %r" % type(m)) + + if __name__ == "__main__": from pyspark.ml.tests import * if xmlrunner: From 0d5e29655f9c3758393794367c0b5d3fd395d1f6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 15 May 2016 15:56:46 +0100 Subject: [PATCH 229/313] [SPARK-12972][CORE] Update org.apache.httpcomponents.httpclient ## What changes were proposed in this pull request? (Retry of https://github.com/apache/spark/pull/13049) - update to httpclient 4.5 / httpcore 4.4 - remove some defunct exclusions - manage httpmime version to match - update selenium / httpunit to support 4.5 (possible now that Jetty 9 is used) ## How was this patch tested? Jenkins tests. Also, locally running the same test command of one Jenkins profile that failed: `mvn -Phadoop-2.6 -Pyarn -Phive -Phive-thriftserver -Pkinesis-asl ...` Author: Sean Owen Closes #13117 from srowen/SPARK-12972.2. (cherry picked from commit f5576a052da0bb59343bc2a6b6ce06c6abaac75b) Signed-off-by: Sean Owen --- core/pom.xml | 11 +++-- dev/deps/spark-deps-hadoop-2.2 | 4 +- dev/deps/spark-deps-hadoop-2.3 | 4 +- dev/deps/spark-deps-hadoop-2.4 | 4 +- dev/deps/spark-deps-hadoop-2.6 | 4 +- dev/deps/spark-deps-hadoop-2.7 | 4 +- external/docker-integration-tests/pom.xml | 2 - pom.xml | 54 +++++++---------------- sql/hive-thriftserver/pom.xml | 11 +++-- streaming/pom.xml | 5 +++ 10 files changed, 40 insertions(+), 63 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index c9853528b4e8..4b8fb4e5fb0f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -275,12 +275,11 @@ org.seleniumhq.selenium selenium-java - - - com.google.guava - guava - - + test + + + org.seleniumhq.selenium + selenium-htmlunit-driver test diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 2477312d74bf..c3be6b2fee99 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -69,8 +69,8 @@ hadoop-yarn-server-web-proxy-2.2.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 0181a47a79d3..61ed4c0889b8 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.3.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index f7ff23472b9b..fb014921765f 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.4.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 92db55d674a4..0baf4e84fff0 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 44b9b049344a..8be218cd68d9 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 53a24f3e06e0..3169a0ebaeb9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -85,13 +85,11 @@ org.apache.httpcomponents httpclient - 4.5 test org.apache.httpcomponents httpcore - 4.4.1 test diff --git a/pom.xml b/pom.xml index 9e9aad223a5e..40d9bf5ccccb 100644 --- a/pom.xml +++ b/pom.xml @@ -149,8 +149,8 @@ 0.10.2 - 4.3.2 - 4.3.2 + 4.5.2 + 4.4.4 3.1 3.4.1 @@ -179,6 +179,7 @@ 0.9.2 4.5.2-1 1.1 + 2.52.0 ${java.home} @@ -410,6 +411,11 @@ httpclient ${commons.httpclient.version}
    + + org.apache.httpcomponents + httpmime + ${commons.httpclient.version} + org.apache.httpcomponents httpcore @@ -418,7 +424,7 @@ org.seleniumhq.selenium selenium-java - 2.45.0 + ${selenium.version} test @@ -431,6 +437,12 @@ + + org.seleniumhq.selenium + selenium-htmlunit-driver + ${selenium.version} + test + xml-apis @@ -731,18 +743,6 @@ guava com.google.guava - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - commons-logging - httpclient - commons-logging commons-logging @@ -1448,14 +1448,6 @@ ${hive.group} hive-shims - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.apache.curator curator-framework @@ -1807,14 +1799,6 @@ libthrift ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api @@ -1826,14 +1810,6 @@ libfb303 ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a8b7aaff62a1..5cf16e42b9f8 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -70,6 +70,11 @@ selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + org.apache.spark spark-sql_${scala.binary.version} @@ -106,12 +111,6 @@ - - - - org.codehaus.mojo - build-helper-maven-plugin - add-source generate-sources diff --git a/streaming/pom.xml b/streaming/pom.xml index e7415863e356..2055cffe751c 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -97,6 +97,11 @@ selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + org.mockito mockito-core From 6d10b28261e8f1c989d4cab9f59f5f082fd267de Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 16 May 2016 16:27:04 +0100 Subject: [PATCH 230/313] [SPARK-12972][CORE][TEST-MAVEN][TEST-HADOOP2.2] Update org.apache.httpcomponents.httpclient, commons-io ## What changes were proposed in this pull request? This is sort of a hot-fix for https://github.com/apache/spark/pull/13117, but, the problem is limited to Hadoop 2.2. The change is to manage `commons-io` to 2.4 for all Hadoop builds, which is only a net change for Hadoop 2.2, which was using 2.1. ## How was this patch tested? Jenkins tests -- normal PR builder, then the `[test-hadoop2.2] [test-maven]` if successful. Author: Sean Owen Closes #13132 from srowen/SPARK-12972.3. (cherry picked from commit fabc8e5b128849a08d820d8c0b3425e39258e02e) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 2 +- pom.xml | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index c3be6b2fee99..91b333fcae4f 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -30,7 +30,7 @@ commons-configuration-1.6.jar commons-dbcp-1.4.jar commons-digester-1.8.jar commons-httpclient-3.1.jar -commons-io-2.1.jar +commons-io-2.4.jar commons-lang-2.6.jar commons-lang3-3.3.2.jar commons-logging-1.1.3.jar diff --git a/pom.xml b/pom.xml index 40d9bf5ccccb..864824d86c3c 100644 --- a/pom.xml +++ b/pom.xml @@ -166,6 +166,7 @@ 1.1.2 1.2.0-incubating 1.10 + 2.4 2.6 @@ -376,6 +377,11 @@ commons-lang ${commons-lang2.version} + + commons-io + commons-io + ${commons-io.version} + commons-codec commons-codec From 1426235bfb1ecfa55859930913ae45d085912bf7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 17 May 2016 09:55:53 +0100 Subject: [PATCH 231/313] [SPARK-15290][BUILD] Move annotations, like @Since / @DeveloperApi, into spark-tags ## What changes were proposed in this pull request? (See https://github.com/apache/spark/pull/12416 where most of this was already reviewed and committed; this is just the module structure and move part. This change does not move the annotations into test scope, which was the apparently problem last time.) Rename `spark-test-tags` -> `spark-tags`; move common annotations like `Since` to `spark-tags` ## How was this patch tested? Jenkins tests. Author: Sean Owen Closes #13074 from srowen/SPARK-15290. (cherry picked from commit 122302cbf5cbf1133067a5acdffd6ab96765dafe) Signed-off-by: Sean Owen --- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 6 +++--- .../spark/annotation/AlphaComponent.java | 0 .../apache/spark/annotation/DeveloperApi.java | 0 .../apache/spark/annotation/Experimental.java | 0 .../org/apache/spark/annotation/Private.java | 0 .../org/apache/spark/annotation/Since.scala | 0 .../apache/spark/annotation/package-info.java | 0 .../org/apache/spark/annotation/package.scala | 0 common/unsafe/pom.xml | 8 ++++---- core/pom.xml | 2 +- dev/sparktestsupport/modules.py | 19 +++++++++++++------ external/docker-integration-tests/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 4 ++++ mllib/pom.xml | 2 +- pom.xml | 3 +-- project/MimaExcludes.scala | 8 ++++++++ project/SparkBuild.scala | 10 +++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- yarn/pom.xml | 2 +- 35 files changed, 59 insertions(+), 41 deletions(-) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/AlphaComponent.java (100%) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/DeveloperApi.java (100%) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/Experimental.java (100%) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/Private.java (100%) rename {core => common/tags}/src/main/scala/org/apache/spark/annotation/Since.scala (100%) rename {core => common/tags}/src/main/scala/org/apache/spark/annotation/package-info.java (100%) rename {core => common/tags}/src/main/scala/org/apache/spark/annotation/package.scala (100%) diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bd507c2cb6c4..5444ae6d70a9 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -66,7 +66,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.mockito diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 810ec10ca05b..e736436aec4c 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -80,7 +80,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} log4j diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index bc83ef24c30e..1fd3af2e6e62 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -48,7 +48,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 8bc1f5279894..bbbb0bd5aa05 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -38,7 +38,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 8e702b4fefe8..14e94eca93b2 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -27,12 +27,12 @@ org.apache.spark - spark-test-tags_2.11 + spark-tags_2.11 jar - Spark Project Test Tags + Spark Project Tags http://spark.apache.org/ - test-tags + tags diff --git a/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java b/common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/AlphaComponent.java rename to common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java diff --git a/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java b/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/DeveloperApi.java rename to common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java diff --git a/core/src/main/java/org/apache/spark/annotation/Experimental.java b/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Experimental.java rename to common/tags/src/main/java/org/apache/spark/annotation/Experimental.java diff --git a/core/src/main/java/org/apache/spark/annotation/Private.java b/common/tags/src/main/java/org/apache/spark/annotation/Private.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Private.java rename to common/tags/src/main/java/org/apache/spark/annotation/Private.java diff --git a/core/src/main/scala/org/apache/spark/annotation/Since.scala b/common/tags/src/main/scala/org/apache/spark/annotation/Since.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/Since.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/Since.scala diff --git a/core/src/main/scala/org/apache/spark/annotation/package-info.java b/common/tags/src/main/scala/org/apache/spark/annotation/package-info.java similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/package-info.java rename to common/tags/src/main/scala/org/apache/spark/annotation/package-info.java diff --git a/core/src/main/scala/org/apache/spark/annotation/package.scala b/common/tags/src/main/scala/org/apache/spark/annotation/package.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/package.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/package.scala diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 93b9580f26b8..d0d1da69ea80 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -36,6 +36,10 @@ + + org.apache.spark + spark-tags_${scala.binary.version} + com.twitter chill_${scala.binary.version} @@ -59,10 +63,6 @@ - - org.apache.spark - spark-test-tags_${scala.binary.version} - org.mockito mockito-core diff --git a/core/pom.xml b/core/pom.xml index 4b8fb4e5fb0f..45f8bfcc05aa 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -331,7 +331,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index f81db8e44a1d..d7eb44d7a589 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -92,10 +92,17 @@ def __ne__(self, other): def __hash__(self): return hash(self.name) +tags = Module( + name="tags", + dependencies=[], + source_file_regexes=[ + "common/tags/", + ] +) catalyst = Module( name="catalyst", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "sql/catalyst/", ], @@ -165,7 +172,7 @@ def __hash__(self): sketch = Module( name="sketch", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "common/sketch/", ], @@ -177,7 +184,7 @@ def __hash__(self): graphx = Module( name="graphx", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "graphx/", ], @@ -189,7 +196,7 @@ def __hash__(self): streaming = Module( name="streaming", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "streaming", ], @@ -205,7 +212,7 @@ def __hash__(self): # fail other PRs. streaming_kinesis_asl = Module( name="streaming-kinesis-asl", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "external/kinesis-asl/", "external/kinesis-asl-assembly/", @@ -270,7 +277,7 @@ def __hash__(self): mllib_local = Module( name="mllib-local", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "mllib-local", ], diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 3169a0ebaeb9..21d40863b77f 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -126,7 +126,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} ${project.version} test diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index e4effe158c82..016af934bb89 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -92,7 +92,7 @@
    org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/flume/pom.xml b/external/flume/pom.xml index d650dd034d63..f51d334de087 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -68,7 +68,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1ea9196e9dfe..60e3ff60df06 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -72,7 +72,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index cccfda3c61a4..4a20b78917ef 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -88,7 +88,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 935155eb5d36..b5f5ff2854cf 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -77,7 +77,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/graphx/pom.xml b/graphx/pom.xml index 1813f383cdcb..fc6c700dd1ec 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -72,7 +72,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/launcher/pom.xml b/launcher/pom.xml index ef731948826e..e7303853e656 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -65,7 +65,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 68f15dd90502..1c6ab2b62d8f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -53,6 +53,10 @@ mockito-core test + + org.apache.spark + spark-tags_${scala.binary.version} + diff --git a/mllib/pom.xml b/mllib/pom.xml index 24d8274e2222..c324afb2a2d1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -116,7 +116,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/pom.xml b/pom.xml index 864824d86c3c..9c13af17e4ed 100644 --- a/pom.xml +++ b/pom.xml @@ -286,9 +286,8 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} ${project.version} - test com.twitter diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 69161e0d61f4..1a02f660fddf 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -716,6 +716,14 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") + ) ++ Seq( + // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Private"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.AlphaComponent"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Experimental"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.DeveloperApi") ) case v if v.startsWith("1.6") => Seq( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d83afa03f5bb..3ad9873f4367 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -50,10 +50,10 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) val allProjects@Seq( - core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, testTags, sketch, _* + core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, _* ) = Seq( "core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe", - "test-tags", "sketch" + "tags", "sketch" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(yarn, java8Tests, sparkGangliaLgpl, @@ -340,7 +340,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, hiveCompatibility, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, testTags, sketch, mllibLocal + unsafe, tags, sketch, mllibLocal ).contains(x) } @@ -685,9 +685,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags), // Skip actual catalyst, but include the subproject. // Catalyst is not public API and contains quasiquotes which break scaladoc. diff --git a/repl/pom.xml b/repl/pom.xml index 0f396c9b809b..c12d121c6115 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -87,7 +87,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.xbean diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 1748fa2778d6..1923199f4b86 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -55,7 +55,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.spark diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 184fa2f6bd31..2ea980bf20df 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -73,7 +73,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.parquet diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5cf16e42b9f8..809d36dc69b9 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -84,7 +84,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} net.sf.jpam diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 177b6884fa13..c8b20f0afc4e 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -60,7 +60,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/yarn/pom.xml b/yarn/pom.xml index 11df2b3f4f76..e07b93ab9545 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -54,7 +54,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.hadoop From c0bcecf914a0e0f6669a62a50e6198af38d4aac6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 17 May 2016 17:02:52 +0800 Subject: [PATCH 232/313] [SPARK-15351][SQL] RowEncoder should support array as the external type for ArrayType ## What changes were proposed in this pull request? This PR improves `RowEncoder` and `MapObjects`, to support array as the external type for `ArrayType`. The idea is straightforward, we use `Object` as the external input type for `ArrayType`, and determine its type at runtime in `MapObjects`. ## How was this patch tested? new test in `RowEncoderSuite` Author: Wenchen Fan Closes #13138 from cloud-fan/map-object. (cherry picked from commit c36ca651f9177f8e7a3f6a0098cba5a810ee9deb) Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/sql/Row.scala | 4 +- .../sql/catalyst/encoders/RowEncoder.scala | 22 +++++ .../expressions/objects/objects.scala | 99 +++++++++---------- .../sql/catalyst/util/GenericArrayData.scala | 5 + .../catalyst/encoders/RowEncoderSuite.scala | 17 ++++ 5 files changed, 92 insertions(+), 55 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 726291b96c29..a257b831dd1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -151,7 +151,7 @@ trait Row extends Serializable { * BinaryType -> byte array * ArrayType -> scala.collection.Seq (use getList for java.util.List) * MapType -> scala.collection.Map (use getJavaMap for java.util.Map) - * StructType -> org.apache.spark.sql.Row (or Product) + * StructType -> org.apache.spark.sql.Row * }}} */ def apply(i: Int): Any = get(i) @@ -176,7 +176,7 @@ trait Row extends Serializable { * BinaryType -> byte array * ArrayType -> scala.collection.Seq (use getList for java.util.List) * MapType -> scala.collection.Map (use getJavaMap for java.util.Map) - * StructType -> org.apache.spark.sql.Row (or Product) + * StructType -> org.apache.spark.sql.Row * }}} */ def get(i: Int): Any diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index ae842a9f8779..a5f39aaa2314 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -32,6 +32,26 @@ import org.apache.spark.unsafe.types.UTF8String /** * A factory for constructing encoders that convert external row to/from the Spark SQL * internal binary representation. + * + * The following is a mapping between Spark SQL types and its allowed external types: + * {{{ + * BooleanType -> java.lang.Boolean + * ByteType -> java.lang.Byte + * ShortType -> java.lang.Short + * IntegerType -> java.lang.Integer + * FloatType -> java.lang.Float + * DoubleType -> java.lang.Double + * StringType -> String + * DecimalType -> java.math.BigDecimal or scala.math.BigDecimal or Decimal + * + * DateType -> java.sql.Date + * TimestampType -> java.sql.Timestamp + * + * BinaryType -> byte array + * ArrayType -> scala.collection.Seq or Array + * MapType -> scala.collection.Map + * StructType -> org.apache.spark.sql.Row or Product + * }}} */ object RowEncoder { def apply(schema: StructType): ExpressionEncoder[Row] = { @@ -166,6 +186,8 @@ object RowEncoder { // In order to support both Decimal and java/scala BigDecimal in external row, we make this // as java.lang.Object. case _: DecimalType => ObjectType(classOf[java.lang.Object]) + // In order to support both Array and Seq in external row, we make this as java.lang.Object. + case _: ArrayType => ObjectType(classOf[java.lang.Object]) case _ => externalDataTypeFor(dt) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index e8a6c742bfe1..7df6e06805a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -376,45 +376,6 @@ case class MapObjects private( lambdaFunction: Expression, inputData: Expression) extends Expression with NonSQLExpression { - @tailrec - private def itemAccessorMethod(dataType: DataType): String => String = dataType match { - case NullType => - val nullTypeClassName = NullType.getClass.getName + ".MODULE$" - (i: String) => s".get($i, $nullTypeClassName)" - case IntegerType => (i: String) => s".getInt($i)" - case LongType => (i: String) => s".getLong($i)" - case FloatType => (i: String) => s".getFloat($i)" - case DoubleType => (i: String) => s".getDouble($i)" - case ByteType => (i: String) => s".getByte($i)" - case ShortType => (i: String) => s".getShort($i)" - case BooleanType => (i: String) => s".getBoolean($i)" - case StringType => (i: String) => s".getUTF8String($i)" - case s: StructType => (i: String) => s".getStruct($i, ${s.size})" - case a: ArrayType => (i: String) => s".getArray($i)" - case _: MapType => (i: String) => s".getMap($i)" - case udt: UserDefinedType[_] => itemAccessorMethod(udt.sqlType) - case DecimalType.Fixed(p, s) => (i: String) => s".getDecimal($i, $p, $s)" - case DateType => (i: String) => s".getInt($i)" - } - - private lazy val (lengthFunction, itemAccessor, primitiveElement) = inputData.dataType match { - case ObjectType(cls) if classOf[Seq[_]].isAssignableFrom(cls) => - (".size()", (i: String) => s".apply($i)", false) - case ObjectType(cls) if cls.isArray => - (".length", (i: String) => s"[$i]", false) - case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) => - (".size()", (i: String) => s".get($i)", false) - case ArrayType(t, _) => - val (sqlType, primitiveElement) = t match { - case m: MapType => (m, false) - case s: StructType => (s, false) - case s: StringType => (s, false) - case udt: UserDefinedType[_] => (udt.sqlType, false) - case o => (o, true) - } - (".numElements()", itemAccessorMethod(sqlType), primitiveElement) - } - override def nullable: Boolean = true override def children: Seq[Expression] = lambdaFunction :: inputData :: Nil @@ -425,7 +386,6 @@ case class MapObjects private( override def dataType: DataType = ArrayType(lambdaFunction.dataType) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val javaType = ctx.javaType(dataType) val elementJavaType = ctx.javaType(loopVar.dataType) ctx.addMutableState("boolean", loopVar.isNull, "") ctx.addMutableState(elementJavaType, loopVar.value, "") @@ -448,27 +408,61 @@ case class MapObjects private( s"new $convertedType[$dataLength]" } - val loopNullCheck = if (primitiveElement) { - s"${loopVar.isNull} = ${genInputData.value}.isNullAt($loopIndex);" - } else { - s"${loopVar.isNull} = ${genInputData.isNull} || ${loopVar.value} == null;" + // In RowEncoder, we use `Object` to represent Array or Seq, so we need to determine the type + // of input collection at runtime for this case. + val seq = ctx.freshName("seq") + val array = ctx.freshName("array") + val determineCollectionType = inputData.dataType match { + case ObjectType(cls) if cls == classOf[Object] => + val seqClass = classOf[Seq[_]].getName + s""" + $seqClass $seq = null; + $elementJavaType[] $array = null; + if (${genInputData.value}.getClass().isArray()) { + $array = ($elementJavaType[]) ${genInputData.value}; + } else { + $seq = ($seqClass) ${genInputData.value}; + } + """ + case _ => "" + } + + + val (getLength, getLoopVar) = inputData.dataType match { + case ObjectType(cls) if classOf[Seq[_]].isAssignableFrom(cls) => + s"${genInputData.value}.size()" -> s"${genInputData.value}.apply($loopIndex)" + case ObjectType(cls) if cls.isArray => + s"${genInputData.value}.length" -> s"${genInputData.value}[$loopIndex]" + case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) => + s"${genInputData.value}.size()" -> s"${genInputData.value}.get($loopIndex)" + case ArrayType(et, _) => + s"${genInputData.value}.numElements()" -> ctx.getValue(genInputData.value, et, loopIndex) + case ObjectType(cls) if cls == classOf[Object] => + s"$seq == null ? $array.length : $seq.size()" -> + s"$seq == null ? $array[$loopIndex] : $seq.apply($loopIndex)" + } + + val loopNullCheck = inputData.dataType match { + case _: ArrayType => s"${loopVar.isNull} = ${genInputData.value}.isNullAt($loopIndex);" + // The element of primitive array will never be null. + case ObjectType(cls) if cls.isArray && cls.getComponentType.isPrimitive => + s"${loopVar.isNull} = false" + case _ => s"${loopVar.isNull} = ${loopVar.value} == null;" } val code = s""" ${genInputData.code} + ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - boolean ${ev.isNull} = ${genInputData.value} == null; - $javaType ${ev.value} = ${ctx.defaultValue(dataType)}; - - if (!${ev.isNull}) { + if (!${genInputData.isNull}) { + $determineCollectionType $convertedType[] $convertedArray = null; - int $dataLength = ${genInputData.value}$lengthFunction; + int $dataLength = $getLength; $convertedArray = $arrayConstructor; int $loopIndex = 0; while ($loopIndex < $dataLength) { - ${loopVar.value} = - ($elementJavaType)${genInputData.value}${itemAccessor(loopIndex)}; + ${loopVar.value} = ($elementJavaType) ($getLoopVar); $loopNullCheck ${genFunction.code} @@ -481,11 +475,10 @@ case class MapObjects private( $loopIndex += 1; } - ${ev.isNull} = false; ${ev.value} = new ${classOf[GenericArrayData].getName}($convertedArray); } """ - ev.copy(code = code) + ev.copy(code = code, isNull = genInputData.isNull) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index 2b8cdc1e23ab..3a665d370830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -37,6 +37,11 @@ class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(primitiveArray: Array[Byte]) = this(primitiveArray.toSeq) def this(primitiveArray: Array[Boolean]) = this(primitiveArray.toSeq) + def this(seqOrArray: Any) = this(seqOrArray match { + case seq: Seq[Any] => seq + case array: Array[_] => array.toSeq + }) + override def copy(): ArrayData = new GenericArrayData(array.clone()) override def numElements(): Int = array.length diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 4800e2e26eb5..7bb006c173b0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -185,6 +185,23 @@ class RowEncoderSuite extends SparkFunSuite { assert(encoder.serializer.head.nullable == false) } + test("RowEncoder should support array as the external type for ArrayType") { + val schema = new StructType() + .add("array", ArrayType(IntegerType)) + .add("nestedArray", ArrayType(ArrayType(StringType))) + .add("deepNestedArray", ArrayType(ArrayType(ArrayType(LongType)))) + val encoder = RowEncoder(schema) + val input = Row( + Array(1, 2, null), + Array(Array("abc", null), null), + Array(Seq(Array(0L, null), null), null)) + val row = encoder.toRow(input) + val convertedBack = encoder.fromRow(row) + assert(convertedBack.getSeq(0) == Seq(1, 2, null)) + assert(convertedBack.getSeq(1) == Seq(Seq("abc", null), null)) + assert(convertedBack.getSeq(2) == Seq(Seq(Seq(0L, null), null), null)) + } + private def encodeDecodeTest(schema: StructType): Unit = { test(s"encode/decode: ${schema.simpleString}") { val encoder = RowEncoder(schema) From b031ea7dc29b3e55dfaf8e8466b6d8f33cb81a3e Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Tue, 17 May 2016 15:20:47 +0200 Subject: [PATCH 233/313] [SPARK-14434][ML] User guide doc and examples for GaussianMixture in spark.ml ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Add guide doc and examples for GaussianMixture in Spark.ml in Java, Scala and Python. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manual compile and test all examples Author: wm624@hotmail.com Closes #12788 from wangmiao1981/example. (cherry picked from commit 4134ff0c657efcbf0f61eff0423215afd6132837) Signed-off-by: Nick Pentreath --- docs/ml-clustering.md | 82 +++++++++++++++++++ .../ml/JavaGaussianMixtureExample.java | 64 +++++++++++++++ .../python/ml/gaussian_mixture_example.py | 48 +++++++++++ .../examples/ml/GaussianMixtureExample.scala | 58 +++++++++++++ 4 files changed, 252 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java create mode 100644 examples/src/main/python/ml/gaussian_mixture_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index a0955a3855ce..33e4b7b0d2cc 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -148,3 +148,85 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. {% include_example python/ml/bisecting_k_means_example.py %} + +## Gaussian Mixture Model (GMM) + +A [Gaussian Mixture Model](http://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) +represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions, +each with its own probability. The `spark.ml` implementation uses the +[expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) +algorithm to induce the maximum-likelihood model given a set of samples. + +`GaussianMixture` is implemented as an `Estimator` and generates a `GaussianMixtureModel` as the base +model. + +### Input Columns + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    featuresColVector"features"Feature vector
    + +### Output Columns + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    predictionColInt"prediction"Predicted cluster center
    probabilityColVector"probability"Probability of each cluster
    + +### Example + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.GaussianMixture) for more details. + +{% include_example scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/GaussianMixture.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.GaussianMixture) for more details. + +{% include_example python/ml/gaussian_mixture_example.py %} +
    +
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java new file mode 100644 index 000000000000..79b99095815a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java @@ -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.examples.ml; + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture; +import org.apache.spark.ml.clustering.GaussianMixtureModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + + +/** + * An example demonstrating Gaussian Mixture Model. + * Run with + *
    + * bin/run-example ml.JavaGaussianMixtureExample
    + * 
    + */ +public class JavaGaussianMixtureExample { + + public static void main(String[] args) { + + // Creates a SparkSession + SparkSession spark = SparkSession + .builder() + .appName("JavaGaussianMixtureExample") + .getOrCreate(); + + // $example on$ + // Loads data + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); + + // Trains a GaussianMixture model + GaussianMixture gmm = new GaussianMixture() + .setK(2); + GaussianMixtureModel model = gmm.fit(dataset); + + // Output the parameters of the mixture model + for (int i = 0; i < model.getK(); i++) { + System.out.printf("weight=%f\nmu=%s\nsigma=\n%s\n", + model.weights()[i], model.gaussians()[i].mean(), model.gaussians()[i].cov()); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/python/ml/gaussian_mixture_example.py b/examples/src/main/python/ml/gaussian_mixture_example.py new file mode 100644 index 000000000000..2ca13d68f689 --- /dev/null +++ b/examples/src/main/python/ml/gaussian_mixture_example.py @@ -0,0 +1,48 @@ +# +# 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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.clustering import GaussianMixture +# $example off$ +from pyspark.sql import SparkSession + +""" +A simple example demonstrating Gaussian Mixture Model (GMM). +Run with: + bin/spark-submit examples/src/main/python/ml/gaussian_mixture_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PythonGuassianMixtureExample")\ + .getOrCreate() + + # $example on$ + # loads data + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + gmm = GaussianMixture().setK(2) + model = gmm.fit(dataset) + + print("Gaussians: ") + model.gaussiansDF.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala new file mode 100644 index 000000000000..c484ee55569b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala @@ -0,0 +1,58 @@ +/* + * 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.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture +import org.apache.spark.sql.SparkSession +// $example off$ + +/** + * An example demonstrating Gaussian Mixture Model (GMM). + * Run with + * {{{ + * bin/run-example ml.GaussianMixtureExample + * }}} + */ +object GaussianMixtureExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() + + // $example on$ + // Loads data + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + // Trains Gaussian Mixture Model + val gmm = new GaussianMixture() + .setK(2) + val model = gmm.fit(dataset) + + // output parameters of mixture model model + for (i <- 0 until model.getK) { + println("weight=%f\nmu=%s\nsigma=\n%s\n" format + (model.weights(i), model.gaussians(i).mean, model.gaussians(i).cov)) + } + // $example off$ + + spark.stop() + } +} +// scalastyle:on println From 273f3d05294f8fcd8f3f4e116afcd96bd4b50920 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 17 May 2016 16:40:38 +0100 Subject: [PATCH 234/313] [SPARK-15333][DOCS] Reorganize building-spark.md; rationalize vs wiki ## What changes were proposed in this pull request? See JIRA for the motivation. The changes are almost entirely movement of text and edits to sections. Minor changes to text include: - Copying in / merging text from the "Useful Developer Tools" wiki, in areas of - Docker - R - Running one test - standardizing on ./build/mvn not mvn, and likewise for ./build/sbt - correcting some typos - standardizing code block formatting No text has been removed from this doc; text has been imported from the https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools wiki ## How was this patch tested? Jekyll doc build and inspection of resulting HTML in browser. Author: Sean Owen Closes #13124 from srowen/SPARK-15333. (cherry picked from commit 932d8002931d352dd2ec87184e6c84ec5fa859cd) Signed-off-by: Sean Owen --- docs/building-spark.md | 295 ++++++++++++++++++++++------------------- 1 file changed, 156 insertions(+), 139 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 63532c742e39..2c987cf8346e 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -7,48 +7,18 @@ redirect_from: "building-with-maven.html" * This will become a table of contents (this text will be scraped). {:toc} -Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+. -The Spark build can supply a suitable Maven binary; see below. - -# Building with `build/mvn` - -Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: - -{% highlight bash %} -build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package -{% endhighlight %} - -Other build examples can be found below. - -**Note:** When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: - - -Xmax-classfile-name - 128 - -and in `project/SparkBuild.scala` add: - - scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), - -to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. - -# Building a Runnable Distribution +# Building Apache Spark -To create a Spark distribution like those distributed by the -[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as -to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured -with Maven profile settings and so on like the direct Maven build. Example: +## Apache Maven - ./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn - -For more information on usage, run `./dev/make-distribution.sh --help` +The Maven-based build is the build of reference for Apache Spark. +Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+. -# Setting up Maven's Memory Usage +### Setting up Maven's Memory Usage You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: -{% highlight bash %} -export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" -{% endhighlight %} + export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" If you don't run this, you may see errors like the following: @@ -65,7 +35,26 @@ You can fix this by setting the `MAVEN_OPTS` variable as discussed before. * For Java 8 and above this step is not required. * If using `build/mvn` with no `MAVEN_OPTS` set, the script will automate this for you. -# Specifying the Hadoop Version +### build/mvn + +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: + + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package + +Other build examples can be found below. + +## Building a Runnable Distribution + +To create a Spark distribution like those distributed by the +[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as +to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured +with Maven profile settings and so on like the direct Maven build. Example: + + ./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn + +For more information on usage, run `./dev/make-distribution.sh --help` + +## Specifying the Hadoop Version Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the `hadoop.version` property. If unset, Spark will build against Hadoop 2.2.0 by default. Note that certain build profiles are required for particular Hadoop versions: @@ -87,87 +76,63 @@ You can enable the `yarn` profile and optionally set the `yarn.version` property Examples: -{% highlight bash %} + # Apache Hadoop 2.2.X + ./build/mvn -Pyarn -Phadoop-2.2 -DskipTests clean package -# Apache Hadoop 2.2.X -mvn -Pyarn -Phadoop-2.2 -DskipTests clean package + # Apache Hadoop 2.3.X + ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package -# Apache Hadoop 2.3.X -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package + # Apache Hadoop 2.4.X or 2.5.X + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package -# Apache Hadoop 2.4.X or 2.5.X -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + # Apache Hadoop 2.6.X + ./build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package -# Apache Hadoop 2.6.X -mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package + # Apache Hadoop 2.7.X and later + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package -# Apache Hadoop 2.7.X and later -mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package + # Different versions of HDFS and YARN. + ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package -# Different versions of HDFS and YARN. -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package -{% endhighlight %} +## Building With Hive and JDBC Support -# Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. By default Spark will build with Hive 1.2.1 bindings. -{% highlight bash %} -# Apache Hadoop 2.4.X with Hive 1.2.1 support -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package -{% endhighlight %} - -# Building for Scala 2.10 -To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: - - ./dev/change-scala-version.sh 2.10 - mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package - -# PySpark Tests with Maven -If you are building PySpark and wish to run the PySpark tests you will need to build Spark with hive support. + # Apache Hadoop 2.4.X with Hive 1.2.1 support + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package -{% highlight bash %} -build/mvn -DskipTests clean package -Phive -./python/run-tests -{% endhighlight %} +## Packaging without Hadoop Dependencies for YARN -The run-tests script also can be limited to a specific Python version or a specific module - - ./python/run-tests --python-executables=python --modules=pyspark-sql - -**Note:** You can also run Python tests with an sbt build, provided you build Spark with hive support. - -# Spark Tests in Maven - -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). +The assembly directory produced by `mvn package` will, by default, include all of Spark's +dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this +causes multiple versions of these to appear on executor classpaths: the version packaged in +the Spark assembly and the version on each node, included with `yarn.application.classpath`. +The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, +like ZooKeeper and Hadoop itself. -Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: - - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package - mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test - -The ScalaTest plugin also supports running only a specific test suite as follows: +## Building for Scala 2.10 +To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: - mvn -Dhadoop.version=... -DwildcardSuites=org.apache.spark.repl.ReplSuite test + ./dev/change-scala-version.sh 2.10 + ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package -# Building submodules individually +## Building submodules individually It's possible to build Spark sub-modules using the `mvn -pl` option. For instance, you can build the Spark Streaming module using: -{% highlight bash %} -mvn -pl :spark-streaming_2.11 clean install -{% endhighlight %} + ./build/mvn -pl :spark-streaming_2.11 clean install where `spark-streaming_2.11` is the `artifactId` as defined in `streaming/pom.xml` file. -# Continuous Compilation +## Continuous Compilation We use the scala-maven-plugin which supports incremental and continuous compilation. E.g. - mvn scala:cc + ./build/mvn scala:cc should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: @@ -182,86 +147,138 @@ the `spark-parent` module). Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: - $ mvn install + $ ./build/mvn install $ cd core - $ mvn scala:cc + $ ../build/mvn scala:cc -# Building Spark with IntelliJ IDEA or Eclipse +## Speeding up Compilation with Zinc -For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the -[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-IDESetup). +[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental +compiler. When run locally as a background process, it speeds up builds of Scala-based projects +like Spark. Developers who regularly recompile Spark with Maven will be the most interested in +Zinc. The project site gives instructions for building and running `zinc`; OS X users can +install it using `brew install zinc`. -# Running Java 8 Test Suites +If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all +builds. This process will auto-start after the first time `build/mvn` is called and bind to port +3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be +shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically +restart whenever `build/mvn` is called. -Running only Java 8 tests and nothing else. +## Building with SBT - mvn install -DskipTests - mvn -pl :java8-tests_2.11 test +Maven is the official build tool recommended for packaging Spark, and is the *build of reference*. +But SBT is supported for day-to-day development since it can provide much faster iterative +compilation. More advanced developers may wish to use SBT. -or +The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables +can be set to control the SBT build. For example: - sbt java8-tests/test + ./build/sbt -Pyarn -Phadoop-2.3 package -Java 8 tests are automatically enabled when a Java 8 JDK is detected. -If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +To avoid the overhead of launching sbt each time you need to re-compile, you can launch sbt +in interactive mode by running `build/sbt`, and then run all build commands at the command +prompt. For more recommendations on reducing build time, refer to the +[wiki page](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-ReducingBuildTimes). -# Running Docker based Integration Test Suites +## Encrypted Filesystems -Running only docker based integration tests and nothing else. +When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: - mvn install -DskipTests - mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + -Xmax-classfile-name + 128 -or +and in `project/SparkBuild.scala` add: - sbt docker-integration-tests/test + scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), +to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. -# Packaging without Hadoop Dependencies for YARN +## IntelliJ IDEA or Eclipse -The assembly directory produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with `yarn.application.classpath`. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. +For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the +[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-IDESetup). -# Building with SBT -Maven is the official build tool recommended for packaging Spark, and is the *build of reference*. -But SBT is supported for day-to-day development since it can provide much faster iterative -compilation. More advanced developers may wish to use SBT. +# Running Tests -The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables -can be set to control the SBT build. For example: +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). - build/sbt -Pyarn -Phadoop-2.3 package +Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: -To avoid the overhead of launching sbt each time you need to re-compile, you can launch sbt -in interactive mode by running `build/sbt`, and then run all build commands at the command -prompt. For more recommendations on reducing build time, refer to the -[wiki page](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-ReducingBuildTimes). + ./build/mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package + ./build/mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + +The ScalaTest plugin also supports running only a specific Scala test suite as follows: + + ./build/mvn -P... -Dtest=none -DwildcardSuites=org.apache.spark.repl.ReplSuite test + ./build/mvn -P... -Dtest=none -DwildcardSuites=org.apache.spark.repl.* test + +or a Java test: -# Testing with SBT + ./build/mvn test -P... -DwildcardSuites=none -Dtest=org.apache.spark.streaming.JavaAPISuite + +## Testing with SBT Some of the tests require Spark to be packaged first, so always run `build/sbt package` the first time. The following is an example of a correct (build, test) sequence: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver package - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver package + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.*" To run test suites of a specific sub project as follows: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test -# Speeding up Compilation with Zinc +## Running Java 8 Test Suites -[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental -compiler. When run locally as a background process, it speeds up builds of Scala-based projects -like Spark. Developers who regularly recompile Spark with Maven will be the most interested in -Zinc. The project site gives instructions for building and running `zinc`; OS X users can -install it using `brew install zinc`. +Running only Java 8 tests and nothing else. -If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all -builds. This process will auto-start after the first time `build/mvn` is called and bind to port -3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be -shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically -restart whenever `build/mvn` is called. + ./build/mvn install -DskipTests + ./build/mvn -pl :java8-tests_2.11 test + +or + + ./build/sbt java8-tests/test + +Java 8 tests are automatically enabled when a Java 8 JDK is detected. +If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. + +## PySpark Tests with Maven + +If you are building PySpark and wish to run the PySpark tests you will need to build Spark with Hive support. + + ./build/mvn -DskipTests clean package -Phive + ./python/run-tests + +The run-tests script also can be limited to a specific Python version or a specific module + + ./python/run-tests --python-executables=python --modules=pyspark-sql + +**Note:** You can also run Python tests with an sbt build, provided you build Spark with Hive support. + +## Running R Tests + +To run the SparkR tests you will need to install the R package `testthat` +(run `install.packages(testthat)` from R shell). You can run just the SparkR tests using +the command: + + ./R/run-tests.sh + +## Running Docker-based Integration Test Suites + +In order to run Docker integration tests, you have to install the `docker` engine on your box. +The instructions for installation can be found at [the Docker site](https://docs.docker.com/engine/installation/). +Once installed, the `docker` service needs to be started, if not already running. +On Linux, this can be done by `sudo service docker start`. + + ./build/mvn install -DskipTests + ./build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + +or + + ./build/sbt docker-integration-tests/test From 670f482225e20d512c2c1c1fccee5b9a7d3745b0 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Tue, 17 May 2016 16:51:01 +0100 Subject: [PATCH 235/313] [SPARK-15318][ML][EXAMPLE] spark.ml Collaborative Filtering example does not work in spark-shell ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) copy & paste example in ml-collaborative-filtering.html into spark-shell, we see the following errors. scala> case class Rating(userId: Int, movieId: Int, rating: Float, timestamp: Long) defined class Rating scala> object Rating { def parseRating(str: String): Rating = { | val fields = str.split("::") | assert(fields.size == 4) | Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) | } } :29: error: Rating.type does not take parameters Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) ^ In standard scala repl, it has the same error. Scala/spark-shell repl has some quirks (e.g. packages are also not well supported). The reason of errors is that scala/spark-shell repl discards previous definitions when we define the Object with the same class name. Solution: We can rename the Object Rating. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually test it: 1). ./bin/run-example ALSExample 2). copy & paste example in the generated document. It works fine. Author: wm624@hotmail.com Closes #13110 from wangmiao1981/repl. (cherry picked from commit bebe5f9811f968db92c2d33e2b30c35cfb808a4a) Signed-off-by: Sean Owen --- .../apache/spark/examples/ml/ALSExample.scala | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index 6b151a622e26..da19ea9f10ec 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -24,16 +24,21 @@ import org.apache.spark.ml.recommendation.ALS // $example off$ import org.apache.spark.sql.SparkSession +/** + * An example demonstrating ALS. + * Run with + * {{{ + * bin/run-example ml.ALSExample + * }}} + */ object ALSExample { // $example on$ case class Rating(userId: Int, movieId: Int, rating: Float, timestamp: Long) - object Rating { - def parseRating(str: String): Rating = { - val fields = str.split("::") - assert(fields.size == 4) - Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) - } + def parseRating(str: String): Rating = { + val fields = str.split("::") + assert(fields.size == 4) + Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) } // $example off$ @@ -46,7 +51,7 @@ object ALSExample { // $example on$ val ratings = spark.read.text("data/mllib/als/sample_movielens_ratings.txt") - .map(Rating.parseRating) + .map(parseRating) .toDF() val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2)) From 110876b9afe5e4205062fd8e8979e096e585737d Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 17 May 2016 10:07:01 -0700 Subject: [PATCH 236/313] [SPARK-15165] [SQL] Codegen can break because toCommentSafeString is not actually safe ## What changes were proposed in this pull request? toCommentSafeString method replaces "\u" with "\\\\u" to avoid codegen breaking. But if the even number of "\" is put before "u", like "\\\\u", in the string literal in the query, codegen can break. Following code causes compilation error. ``` val df = Seq(...).toDF df.select("'\\\\\\\\u002A/'").show ``` The reason of the compilation error is because "\\\\\\\\\\\\\\\\u002A/" is translated into "*/" (the end of comment). Due to this unsafety, arbitrary code can be injected like as follows. ``` val df = Seq(...).toDF // Inject "System.exit(1)" df.select("'\\\\\\\\u002A/{System.exit(1);}/*'").show ``` ## How was this patch tested? Added new test cases. Author: Kousuke Saruta Author: sarutak Closes #12939 from sarutak/SPARK-15165. (cherry picked from commit c0c3ec35476c756e569a1f34c4b258eb0490585c) Signed-off-by: Davies Liu --- .../spark/sql/catalyst/util/package.scala | 13 +- .../expressions/CodeGenerationSuite.scala | 44 +++ .../org/apache/spark/sql/SQLQuerySuite.scala | 264 ++++++++++++++++++ 3 files changed, 320 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 3d2a624ba3b3..f1d6cab9a5a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -162,7 +162,18 @@ package object util { def toCommentSafeString(str: String): String = { val len = math.min(str.length, 128) val suffix = if (str.length > len) "..." else "" - str.substring(0, len).replace("*/", "\\*\\/").replace("\\u", "\\\\u") + suffix + + // Unicode literals, like \u0022, should be escaped before + // they are put in code comment to avoid codegen breaking. + // To escape them, single "\" should be prepended to a series of "\" just before "u" + // only when the number of "\" is odd. + // For example, \u0022 should become to \\u0022 + // but \\u0022 should not become to \\\u0022 because the first backslash escapes the second one, + // and \u0022 will remain, means not escaped. + // Otherwise, the runtime Java compiler will fail to compile or code injection can be allowed. + // For details, see SPARK-15165. + str.substring(0, len).replace("*/", "*\\/") + .replaceAll("(^|[^\\\\])(\\\\(\\\\\\\\)*u)", "$1\\\\$2") + suffix } /* FIX ME diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 2082cea0f60f..db34d12e286f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -194,4 +194,48 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { true, InternalRow(UTF8String.fromString("\\u"))) } + + test("check compilation error doesn't occur caused by specific literal") { + // The end of comment (*/) should be escaped. + GenerateUnsafeProjection.generate( + Literal.create("*/Compilation error occurs/*", StringType) :: Nil) + + // `\u002A` is `*` and `\u002F` is `/` + // so if the end of comment consists of those characters in queries, we need to escape them. + GenerateUnsafeProjection.generate( + Literal.create("\\u002A/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u002A/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\u002a/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u002a/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\u002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\\\u002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\002fCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\\\002fCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\002A\\002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\002A\\002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\002A\\\\002FCompilation error occurs/*", StringType) :: Nil) + + // \ u002X is an invalid unicode literal so it should be escaped. + GenerateUnsafeProjection.generate( + Literal.create("\\u002X/Compilation error occurs", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u002X/Compilation error occurs", StringType) :: Nil) + + // \ u001 is an invalid unicode literal so it should be escaped. + GenerateUnsafeProjection.generate( + Literal.create("\\u001/Compilation error occurs", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u001/Compilation error occurs", StringType) :: Nil) + + } } 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 7020841d3171..b67e2bdeb366 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 @@ -2496,4 +2496,268 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("check code injection is prevented") { + // The end of comment (*/) should be escaped. + var literal = + """|*/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + var expected = + """|*/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + // `\u002A` is `*` and `\u002F` is `/` + // so if the end of comment consists of those characters in queries, we need to escape them. + literal = + """|\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A/"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002a/"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"*\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|*\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"*\\u002f"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|*\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\\\u002A\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002A\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A\\\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + } } From adc1c2685ea0cfbf23716a4199b85c65021d15c6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 17 May 2016 10:12:51 -0700 Subject: [PATCH 237/313] [SPARK-14346][SQL][FOLLOW-UP] add tests for CREAT TABLE USING with partition and bucket ## What changes were proposed in this pull request? https://github.com/apache/spark/pull/12781 introduced PARTITIONED BY, CLUSTERED BY, and SORTED BY keywords to CREATE TABLE USING. This PR adds tests to make sure those keywords are handled correctly. This PR also fixes a mistake that we should create non-hive-compatible table if partition or bucket info exists. ## How was this patch tested? N/A Author: Wenchen Fan Closes #13144 from cloud-fan/add-test. (cherry picked from commit 20a89478e168cb6901ef89f4cb6aa79193ed244a) Signed-off-by: Yin Huai --- .../command/createDataSourceTables.scala | 11 +++- .../execution/command/DDLCommandSuite.scala | 53 +++++++++++++++++++ .../sql/execution/command/DDLSuite.scala | 44 +++++++++++++++ 3 files changed, 106 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 7d3c52570f34..70e5108d938b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -399,8 +399,8 @@ object CreateDataSourceTableUtils extends Logging { "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive." (None, message) - case (Some(serde), relation: HadoopFsRelation) - if relation.location.paths.length == 1 && relation.partitionSchema.isEmpty => + case (Some(serde), relation: HadoopFsRelation) if relation.location.paths.length == 1 && + relation.partitionSchema.isEmpty && relation.bucketSpec.isEmpty => val hiveTable = newHiveCompatibleMetastoreTable(relation, serde) val message = s"Persisting data source relation $qualifiedTableName with a single input path " + @@ -415,6 +415,13 @@ object CreateDataSourceTableUtils extends Logging { "Input path(s): " + relation.location.paths.mkString("\n", "\n", "") (None, message) + case (Some(serde), relation: HadoopFsRelation) if relation.bucketSpec.nonEmpty => + val message = + s"Persisting bucketed data source relation $qualifiedTableName into " + + "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " + + "Input path(s): " + relation.location.paths.mkString("\n", "\n", "") + (None, message) + case (Some(serde), relation: HadoopFsRelation) => val message = s"Persisting data source relation $qualifiedTableName with multiple input paths into " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 13df4493e24d..897170ea57cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -24,7 +24,9 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser +import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsing} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { @@ -238,6 +240,57 @@ class DDLCommandSuite extends PlanTest { } } + test("create table using - with partitioned by") { + val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (a)" + val expected = CreateTableUsing( + TableIdentifier("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + "parquet", + false, + Map.empty, + null, + None, + false, + true) + + parser.parsePlan(query) match { + case ct: CreateTableUsing => + // We can't compare array in `CreateTableUsing` directly, so here we compare + // `partitionColumns` ahead, and make `partitionColumns` null before plan comparison. + assert(Seq("a") == ct.partitionColumns.toSeq) + comparePlans(ct.copy(partitionColumns = null), expected) + case other => + fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table using - with bucket") { + val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" + val expected = CreateTableUsing( + TableIdentifier("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + "parquet", + false, + Map.empty, + null, + Some(BucketSpec(5, Seq("a"), Seq("b"))), + false, + true) + + parser.parsePlan(query) match { + case ct: CreateTableUsing => + // `Array.empty == Array.empty` returns false, here we set `partitionColumns` to null before + // plan comparison. + assert(ct.partitionColumns.isEmpty) + comparePlans(ct.copy(partitionColumns = null), expected) + case other => + fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + // ALTER TABLE table_name RENAME TO new_table_name; // ALTER VIEW view_name RENAME TO new_view_name; test("alter table/view: rename table/view") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 82123bec882e..d72dc092e277 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -29,8 +29,10 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFor import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StructType} class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { private val escapedIdentifier = "`(.+)`".r @@ -350,6 +352,48 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(catalog.getTableMetadata(tableIdent1) === expectedTable) } + test("create table using") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.schema == Seq(CatalogColumn("a", "int"), CatalogColumn("b", "int"))) + assert(table.properties("spark.sql.sources.provider") == "parquet") + } + } + + test("create table using - with partitioned by") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet PARTITIONED BY (a)") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.schema.isEmpty) // partitioned datasource table is not hive-compatible + assert(table.properties("spark.sql.sources.provider") == "parquet") + assert(DDLUtils.getSchemaFromTableProperties(table) == + Some(new StructType().add("a", IntegerType).add("b", IntegerType))) + assert(DDLUtils.getPartitionColumnsFromTableProperties(table) == + Seq("a")) + } + } + + test("create table using - with bucket") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.schema.isEmpty) // partitioned datasource table is not hive-compatible + assert(table.properties("spark.sql.sources.provider") == "parquet") + assert(DDLUtils.getSchemaFromTableProperties(table) == + Some(new StructType().add("a", IntegerType).add("b", IntegerType))) + assert(DDLUtils.getBucketSpecFromTableProperties(table) == + Some(BucketSpec(5, Seq("a"), Seq("b")))) + } + } + test("alter table: rename") { val catalog = spark.sessionState.catalog val tableIdent1 = TableIdentifier("tab1", Some("dbx")) From af37bdd3a7cee5206f98b3a2ba9113e71b53a2f4 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 17 May 2016 11:18:51 -0700 Subject: [PATCH 238/313] [SPARK-10216][SQL] Avoid creating empty files during overwriting with group by query ## What changes were proposed in this pull request? Currently, `INSERT INTO` with `GROUP BY` query tries to make at least 200 files (default value of `spark.sql.shuffle.partition`), which results in lots of empty files. This PR makes it avoid creating empty files during overwriting into Hive table and in internal data sources with group by query. This checks whether the given partition has data in it or not and creates/writes file only when it actually has data. ## How was this patch tested? Unittests in `InsertIntoHiveTableSuite` and `HadoopFsRelationTest`. Closes #8411 Author: hyukjinkwon Author: Keuntae Park Closes #12855 from HyukjinKwon/pr/8411. (cherry picked from commit 8d05a7a98bdbd3ce7c81d273e05a375877ebe68f) Signed-off-by: Michael Armbrust --- .../datasources/WriterContainer.scala | 221 +++++++++--------- .../spark/sql/hive/hiveWriterContainers.scala | 24 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 41 +++- .../sql/sources/HadoopFsRelationTest.scala | 22 +- 4 files changed, 182 insertions(+), 126 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 3b064a5bc489..7e12bbb2128b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -239,48 +239,50 @@ private[sql] class DefaultWriterContainer( extends BaseWriterContainer(relation, job, isAppend) { def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - executorSideSetup(taskContext) - val configuration = taskAttemptContext.getConfiguration - configuration.set("spark.sql.sources.output.path", outputPath) - var writer = newOutputWriter(getWorkPath) - writer.initConverter(dataSchema) - - // If anything below fails, we should abort the task. - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - while (iterator.hasNext) { - val internalRow = iterator.next() - writer.writeInternal(internalRow) - } - commitTask() - }(catchBlock = abortTask()) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } + if (iterator.hasNext) { + executorSideSetup(taskContext) + val configuration = taskAttemptContext.getConfiguration + configuration.set("spark.sql.sources.output.path", outputPath) + var writer = newOutputWriter(getWorkPath) + writer.initConverter(dataSchema) - def commitTask(): Unit = { + // If anything below fails, we should abort the task. try { - if (writer != null) { - writer.close() - writer = null - } - super.commitTask() + Utils.tryWithSafeFinallyAndFailureCallbacks { + while (iterator.hasNext) { + val internalRow = iterator.next() + writer.writeInternal(internalRow) + } + commitTask() + }(catchBlock = abortTask()) } catch { - case cause: Throwable => - // This exception will be handled in `InsertIntoHadoopFsRelation.insert$writeRows`, and - // will cause `abortTask()` to be invoked. - throw new RuntimeException("Failed to commit task", cause) + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) } - } - def abortTask(): Unit = { - try { - if (writer != null) { - writer.close() + def commitTask(): Unit = { + try { + if (writer != null) { + writer.close() + writer = null + } + super.commitTask() + } catch { + case cause: Throwable => + // This exception will be handled in `InsertIntoHadoopFsRelation.insert$writeRows`, and + // will cause `abortTask()` to be invoked. + throw new RuntimeException("Failed to commit task", cause) + } + } + + def abortTask(): Unit = { + try { + if (writer != null) { + writer.close() + } + } finally { + super.abortTask() } - } finally { - super.abortTask() } } } @@ -363,84 +365,87 @@ private[sql] class DynamicPartitionWriterContainer( } def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - executorSideSetup(taskContext) - - // We should first sort by partition columns, then bucket id, and finally sorting columns. - val sortingExpressions: Seq[Expression] = partitionColumns ++ bucketIdExpression ++ sortColumns - val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) - - val sortingKeySchema = StructType(sortingExpressions.map { - case a: Attribute => StructField(a.name, a.dataType, a.nullable) - // The sorting expressions are all `Attribute` except bucket id. - case _ => StructField("bucketId", IntegerType, nullable = false) - }) - - // Returns the data columns to be written given an input row - val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) - - // Returns the partition path given a partition key. - val getPartitionString = - UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) - - // Sorts the data before write, so that we only need one writer at the same time. - // TODO: inject a local sort operator in planning. - val sorter = new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) - - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) - } - logInfo(s"Sorting complete. Writing out partition files one at a time.") - - val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { - identity - } else { - UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { - case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + if (iterator.hasNext) { + executorSideSetup(taskContext) + + // We should first sort by partition columns, then bucket id, and finally sorting columns. + val sortingExpressions: Seq[Expression] = + partitionColumns ++ bucketIdExpression ++ sortColumns + val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) + + val sortingKeySchema = StructType(sortingExpressions.map { + case a: Attribute => StructField(a.name, a.dataType, a.nullable) + // The sorting expressions are all `Attribute` except bucket id. + case _ => StructField("bucketId", IntegerType, nullable = false) }) - } - val sortedIterator = sorter.sortedIterator() + // Returns the data columns to be written given an input row + val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) + + // Returns the partition path given a partition key. + val getPartitionString = + UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) + + // Sorts the data before write, so that we only need one writer at the same time. + // TODO: inject a local sort operator in planning. + val sorter = new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(dataColumns), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + logInfo(s"Sorting complete. Writing out partition files one at a time.") + + val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { + identity + } else { + UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { + case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + }) + } - // If anything below fails, we should abort the task. - var currentWriter: OutputWriter = null - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - var currentKey: UnsafeRow = null - while (sortedIterator.next()) { - val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] - if (currentKey != nextKey) { - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - currentKey = nextKey.copy() - logDebug(s"Writing partition: $currentKey") + val sortedIterator = sorter.sortedIterator() - currentWriter = newOutputWriter(currentKey, getPartitionString) + // If anything below fails, we should abort the task. + var currentWriter: OutputWriter = null + try { + Utils.tryWithSafeFinallyAndFailureCallbacks { + var currentKey: UnsafeRow = null + while (sortedIterator.next()) { + val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + currentKey = nextKey.copy() + logDebug(s"Writing partition: $currentKey") + + currentWriter = newOutputWriter(currentKey, getPartitionString) + } + currentWriter.writeInternal(sortedIterator.getValue) + } + if (currentWriter != null) { + currentWriter.close() + currentWriter = null } - currentWriter.writeInternal(sortedIterator.getValue) - } - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - commitTask() - }(catchBlock = { - if (currentWriter != null) { - currentWriter.close() - } - abortTask() - }) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) + commitTask() + }(catchBlock = { + if (currentWriter != null) { + currentWriter.close() + } + abortTask() + }) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 794fe264ead5..706fdbc2604f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -178,19 +178,21 @@ private[hive] class SparkHiveWriterContainer( // this function is executed on executor side def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() - executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - - iterator.foreach { row => - var i = 0 - while (i < fieldOIs.length) { - outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) - i += 1 + if (iterator.hasNext) { + val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() + executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) + + iterator.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) + i += 1 + } + writer.write(serializer.serialize(outputData, standardOI)) } - writer.write(serializer.serialize(outputData, standardOI)) - } - close() + close() + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 82d3e49f929d..883cdac110e0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -19,13 +19,13 @@ package org.apache.spark.sql.hive import java.io.File -import org.apache.hadoop.hive.conf.HiveConf import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException -import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -118,10 +118,10 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef sql( s""" - |CREATE TABLE table_with_partition(c1 string) - |PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) - |location '${tmpDir.toURI.toString}' - """.stripMargin) + |CREATE TABLE table_with_partition(c1 string) + |PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) sql( """ |INSERT OVERWRITE TABLE table_with_partition @@ -216,6 +216,35 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef sql("DROP TABLE hiveTableWithStructValue") } + test("SPARK-10216: Avoid empty files during overwrite into Hive table with group by query") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + val testDataset = hiveContext.sparkContext.parallelize( + (1 to 2).map(i => TestData(i, i.toString))).toDF() + testDataset.createOrReplaceTempView("testDataset") + + val tmpDir = Utils.createTempDir() + sql( + s""" + |CREATE TABLE table1(key int,value string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table1 + |SELECT count(key), value FROM testDataset GROUP BY value + """.stripMargin) + + val overwrittenFiles = tmpDir.listFiles() + .filter(f => f.isFile && !f.getName.endsWith(".crc")) + .sortBy(_.getName) + val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0) + + assert(overwrittenFiles === overwrittenFilesWithoutEmpty) + + sql("DROP TABLE table1") + } + } + test("Reject partitioning that does not match table") { withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index f4d63334b657..78d2dc28d6b5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.execution.DataSourceScanExec -import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{FileScanRDD, LocalityTestFileSystem} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -879,6 +879,26 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } } } + + test("SPARK-10216: Avoid empty files during overwriting with group by query") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + withTempPath { path => + val df = spark.range(0, 5) + val groupedDF = df.groupBy("id").count() + groupedDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .save(path.getCanonicalPath) + + val overwrittenFiles = path.listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + .sortBy(_.getName) + val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0) + + assert(overwrittenFiles === overwrittenFilesWithoutEmpty) + } + } + } } // This class is used to test SPARK-8578. We should not use any custom output committer when From 025b3e9f17d511b1768282d9635145fa87378b5b Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Tue, 17 May 2016 20:44:19 +0200 Subject: [PATCH 239/313] [SPARK-15182][ML] Copy MLlib doc to ML: ml.feature.tf, idf ## What changes were proposed in this pull request? We should now begin copying algorithm details from the spark.mllib guide to spark.ml as needed, rather than just linking back to the corresponding algorithms in the spark.mllib user guide. ## How was this patch tested? manual review for doc. Author: Yuhao Yang Author: Yuhao Yang Closes #12957 from hhbyyh/tfidfdoc. (cherry picked from commit 3308a862ba0983268c9d5acf9e2a7d2b62d3ec27) Signed-off-by: Nick Pentreath --- docs/ml-features.md | 51 ++++++++++++++++++++++++++------ docs/mllib-feature-extraction.md | 3 ++ 2 files changed, 45 insertions(+), 9 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index c79bcac46179..c44ace91f23f 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -18,27 +18,60 @@ This section covers algorithms for working with features, roughly divided into t # Feature Extractors -## TF-IDF (HashingTF and IDF) - -[Term Frequency-Inverse Document Frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a common text pre-processing step. In Spark ML, TF-IDF is separate into two parts: TF (+hashing) and IDF. +## TF-IDF + +[Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) +is a feature vectorization method widely used in text mining to reflect the importance of a term +to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. +Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, while +document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. If we only use +term frequency to measure the importance, it is very easy to over-emphasize terms that appear very +often but carry little information about the document, e.g., "a", "the", and "of". If a term appears +very often across the corpus, it means it doesn't carry special information about a particular document. +Inverse document frequency is a numerical measure of how much information a term provides: +`\[ +IDF(t, D) = \log \frac{|D| + 1}{DF(t, D) + 1}, +\]` +where `$|D|$` is the total number of documents in the corpus. Since logarithm is used, if a term +appears in all documents, its IDF value becomes 0. Note that a smoothing term is applied to avoid +dividing by zero for terms outside the corpus. The TF-IDF measure is simply the product of TF and IDF: +`\[ +TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). +\]` +There are several variants on the definition of term frequency and document frequency. +In MLlib, we separate TF and IDF to make them flexible. **TF**: Both `HashingTF` and `CountVectorizer` can be used to generate the term frequency vectors. `HashingTF` is a `Transformer` which takes sets of terms and converts those sets into fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. -The algorithm combines Term Frequency (TF) counts with the -[hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) for dimensionality reduction. +`HashingTF` utilizes the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). +A raw feature is mapped into an index (term) by applying a hash function. Then term frequencies +are calculated based on the mapped indices. This approach avoids the need to compute a global +term-to-index map, which can be expensive for a large corpus, but it suffers from potential hash +collisions, where different raw features may become the same term after hashing. To reduce the +chance of collision, we can increase the target feature dimension, i.e., the number of buckets +of the hash table. Since a simple modulo is used to transform the hash function to a column index, +it is advisable to use a power of two as the feature dimension, otherwise the features will +not be mapped evenly to the columns. The default feature dimension is `$2^{18} = 262,144$`. `CountVectorizer` converts text documents to vectors of term counts. Refer to [CountVectorizer ](ml-features.html#countvectorizer) for more details. **IDF**: `IDF` is an `Estimator` which is fit on a dataset and produces an `IDFModel`. The -`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and scales each column. -Intuitively, it down-weights columns which appear frequently in a corpus. +`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and +scales each column. Intuitively, it down-weights columns which appear frequently in a corpus. -Please refer to the [MLlib user guide on TF-IDF](mllib-feature-extraction.html#tf-idf) for more details on Term Frequency and Inverse Document Frequency. +**Note:** `spark.ml` doesn't provide tools for text segmentation. +We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and +[scalanlp/chalk](https://github.com/scalanlp/chalk). + +**Examples** -In the following code segment, we start with a set of sentences. We split each sentence into words using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance when using text as features. Our feature vectors could then be passed to a learning algorithm. +In the following code segment, we start with a set of sentences. We split each sentence into words +using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into +a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance +when using text as features. Our feature vectors could then be passed to a learning algorithm.
    diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 7a9728503265..4c027c84ec90 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -10,6 +10,9 @@ displayTitle: Feature Extraction and Transformation - spark.mllib ## TF-IDF +**Note** We recommend using the DataFrame-based API, which is detailed in the [ML user guide on +TF-IDF](ml-features.html#tf-idf). + [Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a feature vectorization method widely used in text mining to reflect the importance of a term to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. From 1ad3bbd0a4c7d4ab4aad0806f345d3904d7cd441 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 May 2016 20:50:22 +0200 Subject: [PATCH 240/313] [MINOR][DOCS] Replace remaining 'sqlContext' in ScalaDoc/JavaDoc. ## What changes were proposed in this pull request? According to the recent change, this PR replaces all the remaining `sqlContext` usage with `spark` in ScalaDoc/JavaDoc (.scala/.java files) except `SQLContext.scala`, `SparkPlan.scala', and `DatasetHolder.scala`. ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13125 from dongjoon-hyun/minor_doc_sparksession. (cherry picked from commit 9f176dd3918129a72282a6b7a12e2899cbb6dac9) Signed-off-by: Nick Pentreath --- .../scala/org/apache/spark/ml/feature/package.scala | 2 +- .../scala/org/apache/spark/sql/DataFrameReader.scala | 4 ++-- .../org/apache/spark/sql/DataFrameStatFunctions.scala | 10 +++++----- .../org/apache/spark/sql/ExperimentalMethods.scala | 2 +- .../datasources/PartitioningAwareFileCatalog.scala | 8 ++++---- .../main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala index 4571ab26800c..b94187ae787c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.DataFrame * import org.apache.spark.ml.Pipeline * * // a DataFrame with three columns: id (integer), text (string), and rating (double). - * val df = sqlContext.createDataFrame(Seq( + * val df = spark.createDataFrame(Seq( * (0, "Hi I heard about Spark", 3.0), * (1, "I wish Java could use case classes", 4.0), * (2, "Logistic regression models are neat", 4.0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index e1a64dfc5e7b..011aff4ff6c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -446,10 +446,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * Each line in the text file is a new row in the resulting Dataset. For example: * {{{ * // Scala: - * sqlContext.read.text("/path/to/spark/README.md") + * spark.read.text("/path/to/spark/README.md") * * // Java: - * sqlContext.read().text("/path/to/spark/README.md") + * spark.read().text("/path/to/spark/README.md") * }}} * * @param paths input path diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index 3eb1f0f0d58f..1855eab96eaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -160,8 +160,8 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @return A DataFrame containing for the contingency table. * * {{{ - * val df = sqlContext.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), - * (3, 3))).toDF("key", "value") + * val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3))) + * .toDF("key", "value") * val ct = df.stat.crosstab("key", "value") * ct.show() * +---------+---+---+---+ @@ -197,7 +197,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * val rows = Seq.tabulate(100) { i => * if (i % 2 == 0) (1, -1.0) else (i, i * -1.0) * } - * val df = sqlContext.createDataFrame(rows).toDF("a", "b") + * val df = spark.createDataFrame(rows).toDF("a", "b") * // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns * // "a" and "b" * val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4) @@ -258,7 +258,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * val rows = Seq.tabulate(100) { i => * if (i % 2 == 0) (1, -1.0) else (i, i * -1.0) * } - * val df = sqlContext.createDataFrame(rows).toDF("a", "b") + * val df = spark.createDataFrame(rows).toDF("a", "b") * // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns * // "a" and "b" * val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4) @@ -314,7 +314,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @return a new [[DataFrame]] that represents the stratified sample * * {{{ - * val df = sqlContext.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), + * val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), * (3, 3))).toDF("key", "value") * val fractions = Map(1 -> 1.0, 3 -> 0.5) * df.stat.sampleBy("key", fractions, 36L).show() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index a49da6dc2b8b..a435734b0cae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.Rule * regarding binary compatibility and source compatibility of methods here. * * {{{ - * sqlContext.experimental.extraStrategies += ... + * spark.experimental.extraStrategies += ... * }}} * * @since 1.3.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index e0e4ddc30b0f..406d2e8e81f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -168,17 +168,17 @@ abstract class PartitioningAwareFileCatalog( * * By default, the paths of the dataset provided by users will be base paths. * Below are three typical examples, - * Case 1) `sqlContext.read.parquet("/path/something=true/")`: the base path will be + * Case 1) `spark.read.parquet("/path/something=true/")`: the base path will be * `/path/something=true/`, and the returned DataFrame will not contain a column of `something`. - * Case 2) `sqlContext.read.parquet("/path/something=true/a.parquet")`: the base path will be + * Case 2) `spark.read.parquet("/path/something=true/a.parquet")`: the base path will be * still `/path/something=true/`, and the returned DataFrame will also not contain a column of * `something`. - * Case 3) `sqlContext.read.parquet("/path/")`: the base path will be `/path/`, and the returned + * Case 3) `spark.read.parquet("/path/")`: the base path will be `/path/`, and the returned * DataFrame will have the column of `something`. * * Users also can override the basePath by setting `basePath` in the options to pass the new base * path to the data source. - * For example, `sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")`, + * For example, `spark.read.option("basePath", "/path/").parquet("/path/something=true/")`, * and the returned DataFrame will have the column of `something`. */ private def basePaths: Set[Path] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 07f55042eeb4..65bc04307675 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2952,8 +2952,8 @@ object functions { * import org.apache.spark.sql._ * * val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") - * val sqlContext = df.sqlContext - * sqlContext.udf.register("simpleUDF", (v: Int) => v * v) + * val spark = df.sparkSession + * spark.udf.register("simpleUDF", (v: Int) => v * v) * df.select($"id", callUDF("simpleUDF", $"value")) * }}} * From ff1cfce18829ccb176f27d4fcc242cbb341a2277 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 17 May 2016 12:51:07 -0700 Subject: [PATCH 241/313] [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms ## What changes were proposed in this pull request? Once SPARK-14487 and SPARK-14549 are merged, we will migrate to use the new vector and matrix type in the new ml pipeline based apis. ## How was this patch tested? Unit tests Author: DB Tsai Author: Liang-Chi Hsieh Author: Xiangrui Meng Closes #12627 from dbtsai/SPARK-14615-NewML. (cherry picked from commit e2efe0529acd748f26dbaa41331d1733ed256237) Signed-off-by: Xiangrui Meng --- dev/sparktestsupport/modules.py | 1 + .../ml/JavaBisectingKMeansExample.java | 2 +- .../examples/ml/JavaDeveloperApiExample.java | 8 +- .../ml/JavaElementwiseProductExample.java | 6 +- .../spark/examples/ml/JavaKMeansExample.java | 2 +- .../ml/AFTSurvivalRegressionExample.scala | 2 +- .../examples/ml/ChiSqSelectorExample.scala | 2 +- .../apache/spark/examples/ml/DCTExample.scala | 2 +- .../spark/examples/ml/DataFrameExample.scala | 3 +- .../examples/ml/DecisionTreeExample.scala | 2 +- .../examples/ml/DeveloperApiExample.scala | 4 +- .../ml/ElementwiseProductExample.scala | 2 +- .../ml/EstimatorTransformerParamExample.scala | 2 +- ...elSelectionViaCrossValidationExample.scala | 2 +- .../apache/spark/examples/ml/PCAExample.scala | 2 +- .../spark/examples/ml/PipelineExample.scala | 2 +- .../ml/PolynomialExpansionExample.scala | 2 +- .../examples/ml/SimpleParamsExample.scala | 4 +- .../ml/SimpleTextClassificationPipeline.scala | 2 +- .../examples/ml/VectorAssemblerExample.scala | 2 +- .../examples/ml/VectorSlicerExample.scala | 2 +- .../scala/org/apache/spark/ml/Predictor.scala | 4 +- .../scala/org/apache/spark/ml/ann/Layer.scala | 22 +- .../spark/ml/attribute/AttributeGroup.scala | 2 +- .../spark/ml/classification/Classifier.scala | 4 +- .../DecisionTreeClassifier.scala | 4 +- .../ml/classification/GBTClassifier.scala | 4 +- .../classification/LogisticRegression.scala | 5 +- .../MultilayerPerceptronClassifier.scala | 4 +- .../spark/ml/classification/NaiveBayes.scala | 9 +- .../spark/ml/classification/OneVsRest.scala | 2 +- .../ProbabilisticClassifier.scala | 2 +- .../RandomForestClassifier.scala | 4 +- .../spark/ml/clustering/BisectingKMeans.scala | 16 +- .../spark/ml/clustering/GaussianMixture.scala | 9 +- .../apache/spark/ml/clustering/KMeans.scala | 18 +- .../org/apache/spark/ml/clustering/LDA.scala | 18 +- .../BinaryClassificationEvaluator.scala | 2 +- .../apache/spark/ml/feature/Binarizer.scala | 2 +- .../spark/ml/feature/ChiSqSelector.scala | 21 +- .../spark/ml/feature/CountVectorizer.scala | 2 +- .../org/apache/spark/ml/feature/DCT.scala | 2 +- .../spark/ml/feature/ElementwiseProduct.scala | 6 +- .../apache/spark/ml/feature/HashingTF.scala | 3 +- .../org/apache/spark/ml/feature/IDF.scala | 15 +- .../apache/spark/ml/feature/Instance.scala | 2 +- .../apache/spark/ml/feature/Interaction.scala | 2 +- .../spark/ml/feature/LabeledPoint.scala | 38 ++++ .../spark/ml/feature/MaxAbsScaler.scala | 8 +- .../spark/ml/feature/MinMaxScaler.scala | 9 +- .../apache/spark/ml/feature/Normalizer.scala | 5 +- .../spark/ml/feature/OneHotEncoder.scala | 2 +- .../org/apache/spark/ml/feature/PCA.scala | 21 +- .../ml/feature/PolynomialExpansion.scala | 2 +- .../apache/spark/ml/feature/RFormula.scala | 2 +- .../spark/ml/feature/RFormulaParser.scala | 2 +- .../spark/ml/feature/StandardScaler.scala | 15 +- .../spark/ml/feature/VectorAssembler.scala | 2 +- .../spark/ml/feature/VectorIndexer.scala | 2 +- .../spark/ml/feature/VectorSlicer.scala | 2 +- .../apache/spark/ml/feature/Word2Vec.scala | 3 +- .../apache/spark/ml/linalg/VectorUDT.scala | 2 +- .../IterativelyReweightedLeastSquares.scala | 2 +- .../spark/ml/optim/WeightedLeastSquares.scala | 3 +- .../org/apache/spark/ml/param/params.scala | 7 +- .../ml/regression/AFTSurvivalRegression.scala | 3 +- .../ml/regression/DecisionTreeRegressor.scala | 4 +- .../spark/ml/regression/GBTRegressor.scala | 4 +- .../GeneralizedLinearRegression.scala | 2 +- .../ml/regression/IsotonicRegression.scala | 2 +- .../ml/regression/LinearRegression.scala | 6 +- .../ml/regression/RandomForestRegressor.scala | 4 +- .../ml/source/libsvm/LibSVMRelation.scala | 4 +- .../scala/org/apache/spark/ml/tree/Node.scala | 2 +- .../org/apache/spark/ml/tree/Split.scala | 2 +- .../ml/tree/impl/DecisionTreeMetadata.scala | 2 +- .../ml/tree/impl/GradientBoostedTrees.scala | 4 +- .../spark/ml/tree/impl/RandomForest.scala | 2 +- .../apache/spark/ml/tree/impl/TreePoint.scala | 2 +- .../org/apache/spark/ml/tree/treeModels.scala | 2 +- .../apache/spark/ml/util/MetadataUtils.scala | 2 +- .../mllib/api/python/PythonMLLibAPI.scala | 195 +++++++++++++++++- .../classification/LogisticRegression.scala | 4 +- .../apache/spark/mllib/linalg/Matrices.scala | 22 ++ .../apache/spark/mllib/linalg/Vectors.scala | 22 ++ .../spark/mllib/regression/LabeledPoint.scala | 9 + .../mllib/tree/GradientBoostedTrees.scala | 12 +- .../spark/mllib/tree/RandomForest.scala | 4 +- .../apache/spark/ml/JavaPipelineSuite.java | 4 +- .../JavaDecisionTreeClassifierSuite.java | 4 +- .../JavaGBTClassifierSuite.java | 4 +- .../JavaLogisticRegressionSuite.java | 6 +- ...vaMultilayerPerceptronClassifierSuite.java | 4 +- .../classification/JavaNaiveBayesSuite.java | 4 +- .../ml/classification/JavaOneVsRestSuite.java | 4 +- .../JavaRandomForestClassifierSuite.java | 6 +- .../spark/ml/clustering/JavaKMeansSuite.java | 2 +- .../apache/spark/ml/feature/JavaDCTSuite.java | 6 +- .../spark/ml/feature/JavaHashingTFSuite.java | 2 +- .../spark/ml/feature/JavaNormalizerSuite.java | 2 +- .../apache/spark/ml/feature/JavaPCASuite.java | 24 ++- .../feature/JavaPolynomialExpansionSuite.java | 6 +- .../ml/feature/JavaStandardScalerSuite.java | 2 +- .../ml/feature/JavaVectorAssemblerSuite.java | 6 +- .../ml/feature/JavaVectorIndexerSuite.java | 2 +- .../ml/feature/JavaVectorSlicerSuite.java | 4 +- .../spark/ml/feature/JavaWord2VecSuite.java | 2 +- .../JavaDecisionTreeRegressorSuite.java | 4 +- .../ml/regression/JavaGBTRegressorSuite.java | 4 +- .../regression/JavaLinearRegressionSuite.java | 4 +- .../JavaRandomForestRegressorSuite.java | 6 +- .../libsvm/JavaLibSVMRelationSuite.java | 4 +- .../ml/tuning/JavaCrossValidatorSuite.java | 4 +- .../org/apache/spark/ml/PipelineSuite.scala | 4 +- .../org/apache/spark/ml/ann/ANNSuite.scala | 5 +- .../apache/spark/ml/ann/GradientSuite.scala | 2 +- .../ml/classification/ClassifierSuite.scala | 4 +- .../DecisionTreeClassifierSuite.scala | 18 +- .../classification/GBTClassifierSuite.scala | 10 +- .../LogisticRegressionSuite.scala | 129 +++++++++++- .../MultilayerPerceptronClassifierSuite.scala | 15 +- .../ml/classification/NaiveBayesSuite.scala | 56 ++++- .../ml/classification/OneVsRestSuite.scala | 12 +- .../ProbabilisticClassifierSuite.scala | 2 +- .../RandomForestClassifierSuite.scala | 10 +- .../spark/ml/clustering/KMeansSuite.scala | 2 +- .../apache/spark/ml/clustering/LDASuite.scala | 4 +- .../BinaryClassificationEvaluatorSuite.scala | 2 +- .../evaluation/RegressionEvaluatorSuite.scala | 2 +- .../spark/ml/feature/BinarizerSuite.scala | 2 +- .../spark/ml/feature/BucketizerSuite.scala | 4 +- .../spark/ml/feature/ChiSqSelectorSuite.scala | 5 +- .../ml/feature/CountVectorizerSuite.scala | 4 +- .../apache/spark/ml/feature/DCTSuite.scala | 2 +- .../ml/feature/ElementwiseProductSuite.scala | 2 +- .../spark/ml/feature/HashingTFSuite.scala | 4 +- .../apache/spark/ml/feature/IDFSuite.scala | 5 +- .../spark/ml/feature/InteractionSuite.scala | 2 +- .../spark/ml/feature/MaxAbsScalerSuite.scala | 2 +- .../spark/ml/feature/MinMaxScalerSuite.scala | 2 +- .../spark/ml/feature/NormalizerSuite.scala | 4 +- .../spark/ml/feature/OneHotEncoderSuite.scala | 2 +- .../apache/spark/ml/feature/PCASuite.scala | 9 +- .../ml/feature/PolynomialExpansionSuite.scala | 4 +- .../spark/ml/feature/RFormulaSuite.scala | 2 +- .../ml/feature/StandardScalerSuite.scala | 4 +- .../ml/feature/VectorAssemblerSuite.scala | 2 +- .../spark/ml/feature/VectorIndexerSuite.scala | 2 +- .../spark/ml/feature/VectorSlicerSuite.scala | 2 +- .../spark/ml/feature/Word2VecSuite.scala | 4 +- .../spark/ml/linalg/VectorUDTSuite.scala | 10 +- ...erativelyReweightedLeastSquaresSuite.scala | 4 +- .../ml/optim/WeightedLeastSquaresSuite.scala | 4 +- .../apache/spark/ml/param/ParamsSuite.scala | 4 +- .../spark/ml/recommendation/ALSSuite.scala | 4 +- .../AFTSurvivalRegressionSuite.scala | 4 +- .../DecisionTreeRegressorSuite.scala | 9 +- .../ml/regression/GBTRegressorSuite.scala | 10 +- .../GeneralizedLinearRegressionSuite.scala | 8 +- .../regression/IsotonicRegressionSuite.scala | 2 +- .../ml/regression/LinearRegressionSuite.scala | 16 +- .../RandomForestRegressorSuite.scala | 10 +- .../source/libsvm/LibSVMRelationSuite.scala | 2 +- .../tree/impl/GradientBoostedTreesSuite.scala | 6 +- .../ml/tree/impl/RandomForestSuite.scala | 20 +- .../apache/spark/ml/tree/impl/TreeTests.scala | 4 +- .../spark/ml/tuning/CrossValidatorSuite.scala | 10 +- .../ml/tuning/TrainValidationSplitSuite.scala | 8 +- .../apache/spark/ml/util/MLTestingUtils.scala | 2 +- .../spark/mllib/linalg/MatricesSuite.scala | 51 +++++ .../spark/mllib/linalg/VectorsSuite.scala | 39 ++++ .../mllib/regression/LabeledPointSuite.scala | 13 ++ .../spark/mllib/tree/DecisionTreeSuite.scala | 20 +- project/MimaExcludes.scala | 46 +++++ python/pyspark/ml/classification.py | 14 +- python/pyspark/ml/clustering.py | 8 +- python/pyspark/ml/evaluation.py | 2 +- python/pyspark/ml/feature.py | 26 +-- python/pyspark/ml/param/__init__.py | 2 +- python/pyspark/ml/regression.py | 14 +- python/pyspark/ml/tests.py | 136 ++++++------ python/pyspark/ml/tuning.py | 4 +- 182 files changed, 1176 insertions(+), 484 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index d7eb44d7a589..0d6aa7422a53 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -413,6 +413,7 @@ def __hash__(self): "pyspark.ml.feature", "pyspark.ml.classification", "pyspark.ml.clustering", + "pyspark.ml.linalg.__init__", "pyspark.ml.recommendation", "pyspark.ml.regression", "pyspark.ml.tuning", diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 62871448e36f..8c82aaaacca3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -20,7 +20,7 @@ // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 3265c4d7ec1f..8971c0ac002b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -24,13 +24,13 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.classification.Classifier; import org.apache.spark.ml.classification.ClassificationModel; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.BLAS; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.ml.param.IntParam; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.util.Identifiable$; -import org.apache.spark.mllib.linalg.BLAS; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java index 9126242f9eb7..d2e70c23babc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -26,9 +26,9 @@ import java.util.List; import org.apache.spark.ml.feature.ElementwiseProduct; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.DataTypes; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index 2489a9b80b07..d8f948ae38cb 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -20,7 +20,7 @@ // $example on$ import org.apache.spark.ml.clustering.KMeansModel; import org.apache.spark.ml.clustering.KMeans; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala index b44304d8104a..b6d7b369162d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -19,8 +19,8 @@ package org.apache.spark.examples.ml // $example on$ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.regression.AFTSurvivalRegression -import org.apache.spark.mllib.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 80f50cd3556a..c9394dd9c64b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.ChiSqSelector -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala index 5a888b15eb1c..ddc671752872 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.DCT -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 6cb81cde6f79..8ed636778720 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -24,7 +24,8 @@ import com.google.common.io.Files import scopt.OptionParser import org.apache.spark.examples.mllib.AbstractParams -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.sql.{DataFrame, Row, SparkSession} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index 310418008c21..5e51dbad760f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -27,10 +27,10 @@ import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage, Transformer} import org.apache.spark.ml.classification.{DecisionTreeClassificationModel, DecisionTreeClassifier} import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.evaluation.{MulticlassMetrics, RegressionMetrics} -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, SparkSession} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index b8f47bf12b87..d94d837d10e9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -19,10 +19,10 @@ package org.apache.spark.examples.ml import org.apache.spark.ml.classification.{ClassificationModel, Classifier, ClassifierParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.{Dataset, Row, SparkSession} /** diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala index b99b76e58cef..c0ffc01934b6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.ElementwiseProduct -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala index a2918d66ea67..f18d86e1a692 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -20,8 +20,8 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.sql.Row // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala index c29d36210ab1..c1ff9ef52170 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala @@ -23,8 +23,8 @@ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tuning.{CrossValidator, ParamGridBuilder} -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.Row // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala index 7927323b4285..dca96eea2ba4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.PCA -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala index e5e916ac166f..b16692b1fa36 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -22,7 +22,7 @@ package org.apache.spark.examples.ml import org.apache.spark.ml.{Pipeline, PipelineModel} import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.Row // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala index 94b17a3cd706..54d2e6b36d14 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.PolynomialExpansion -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 83bab5c55758..3355c8ffa295 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -19,9 +19,9 @@ package org.apache.spark.examples.ml import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.{Row, SparkSession} /** diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index c78ff2378bc1..0b2a058bb61a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -23,7 +23,7 @@ import scala.beans.BeanInfo import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.{Row, SparkSession} @BeanInfo diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala index 8e382ccc4844..8910470c1cf7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.VectorAssembler -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala index b1a3997f48c9..85dd5c27766c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -23,7 +23,7 @@ import java.util.Arrays import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} import org.apache.spark.ml.feature.VectorSlicer -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.sql.Row import org.apache.spark.sql.types.StructType // $example off$ diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index 81140d1f7b21..569a5fb99376 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} 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, Dataset, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 3588ac1e95be..889b7f9f719c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -21,7 +21,9 @@ import java.util.Random import breeze.linalg.{*, axpy => Baxpy, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.optimization._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom @@ -578,10 +580,10 @@ private[ann] object FeedForwardModel { */ private[ann] class ANNGradient(topology: Topology, dataStacker: DataStacker) extends Gradient { override def compute( - data: Vector, + data: OldVector, label: Double, - weights: Vector, - cumGradient: Vector): Double = { + weights: OldVector, + cumGradient: OldVector): Double = { val (input, target, realBatchSize) = dataStacker.unstack(data) val model = topology.model(weights) model.computeGradient(input, target, cumGradient, realBatchSize) @@ -655,15 +657,15 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) private[ann] class ANNUpdater extends Updater { override def compute( - weightsOld: Vector, - gradient: Vector, + weightsOld: OldVector, + gradient: OldVector, stepSize: Double, iter: Int, - regParam: Double): (Vector, Double) = { + regParam: Double): (OldVector, Double) = { val thisIterStepSize = stepSize val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector Baxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) - (Vectors.fromBreeze(brzWeights), 0) + (OldVectors.fromBreeze(brzWeights), 0) } } @@ -806,7 +808,9 @@ private[ml] class FeedForwardTrainer( getWeights } // TODO: deprecate standard optimizer because it needs Vector - val newWeights = optimizer.optimize(dataStacker.stack(data), w) + val newWeights = optimizer.optimize(dataStacker.stack(data).map { v => + (v._1, OldVectors.fromML(v._2)) + }, w) topology.model(newWeights) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index 2c29eeb01a92..12b9732a4c3d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.attribute import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} /** 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 bc5fe35ad4a5..e35b04a1cf42 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 @@ -20,10 +20,10 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.{MetadataUtils, 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, Dataset, Row} import org.apache.spark.sql.functions._ 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 31a69d49a078..881dcefb79be 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 @@ -22,13 +22,13 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD 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 acc04582b869..f843df449c61 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 @@ -24,14 +24,14 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD 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 62d68973680a..40cff8a86f36 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 @@ -27,12 +27,13 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 72cf55f6bb99..683ae4aaf407 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -24,11 +24,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.ann.{FeedForwardTopology, FeedForwardTrainer} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed, HasStepSize, HasTol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.Dataset /** Params for Multilayer Perceptron. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 267d63b51eb6..a98bdeca6b72 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -22,14 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.util._ import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes} import org.apache.spark.mllib.classification.{NaiveBayesModel => OldNaiveBayesModel} -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset /** * Params for Naive Bayes Classifiers. @@ -102,7 +102,8 @@ class NaiveBayes @Since("1.5.0") ( setDefault(modelType -> OldNaiveBayes.Multinomial) override protected def train(dataset: Dataset[_]): NaiveBayesModel = { - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val oldDataset: RDD[OldLabeledPoint] = + extractLabeledPoints(dataset).map(OldLabeledPoint.fromML) val oldModel = OldNaiveBayes.train(oldDataset, $(smoothing), $(modelType)) NaiveBayesModel.fromOld(oldModel, this) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index f10c60a78df1..047a378b79aa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -32,9 +32,9 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ 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 d00fee12b08c..59277d0f42b3 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 @@ -18,9 +18,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.linalg.{DenseVector, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} 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 1d33ae83c211..b3c074f83925 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 @@ -21,13 +21,13 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 6cc9117da3fe..138e059f9493 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -21,12 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.clustering. - {BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.clustering.{BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -105,7 +107,7 @@ class BisectingKMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) @Since("2.0.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** * Computes the sum of squared distances between the input points and their corresponding cluster @@ -115,7 +117,7 @@ class BisectingKMeansModel private[ml] ( def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) val data = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } - parentModel.computeCost(data) + parentModel.computeCost(data.map(OldVectors.fromML)) } @Since("2.0.0") @@ -216,7 +218,9 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): BisectingKMeansModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val bkm = new MLlibBisectingKMeans() .setK($(k)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index ac86e4ce25e8..63ca812609b7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -32,6 +32,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{GaussianMixture => MLlibGM} import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors, VectorUDT => OldVectorUDT} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -94,8 +95,8 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - val predUDF = udf((vector: OldVector) => predict(vector.asML)) - val probUDF = udf((vector: OldVector) => OldVectors.fromML(predictProbability(vector.asML))) + val predUDF = udf((vector: Vector) => predict(vector)) + val probUDF = udf((vector: Vector) => predictProbability(vector)) dataset.withColumn($(predictionCol), predUDF(col($(featuresCol)))) .withColumn($(probabilityCol), probUDF(col($(featuresCol)))) } @@ -296,7 +297,9 @@ class GaussianMixture @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): GaussianMixtureModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: OldVector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val algo = new MLlibGM() .setK($(k)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 42a25396adb4..41c0aec0ecf9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -22,11 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -127,7 +130,7 @@ class KMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) @Since("1.5.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this @@ -137,7 +140,9 @@ class KMeansModel private[ml] ( @Since("2.0.0") def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) - val data = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val data: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } parentModel.computeCost(data) } @@ -210,7 +215,8 @@ object KMeansModel extends MLReadable[KMeansModel] { val dataPath = new Path(path, "data").toString val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] val clusterCenters = data.collect().sortBy(_.clusterIdx).map(_.clusterCenter) - val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) + val model = new KMeansModel(metadata.uid, + new MLlibKMeansModel(clusterCenters.map(OldVectors.fromML))) DefaultParamsReader.getAndSetParams(model, metadata) model @@ -277,7 +283,9 @@ class KMeans @Since("1.5.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): KMeansModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val instr = Instrumentation.create(this, rdd) instr.logParams(featuresCol, predictionCol, k, initMode, initSteps, maxIter, seed, tol) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 38ecc5a102c1..5a83b2870005 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed} import org.apache.spark.ml.util._ @@ -30,7 +31,10 @@ import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedL LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel, OnlineLDAOptimizer => OldOnlineLDAOptimizer} import org.apache.spark.mllib.impl.PeriodicCheckpointer -import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Vector => OldVector, + Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, monotonicallyIncreasingId, udf} @@ -405,7 +409,11 @@ sealed abstract class LDAModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { if ($(topicDistributionCol).nonEmpty) { - val t = udf(oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext)) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer = oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext) + + val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML } dataset.withColumn($(topicDistributionCol), t(col($(featuresCol)))).toDF } else { logWarning("LDAModel.transform was called without any output columns. Set an output column" + @@ -437,7 +445,7 @@ sealed abstract class LDAModel private[ml] ( * collecting a large amount of data to the driver (on the order of vocabSize x k). */ @Since("1.6.0") - def topicsMatrix: Matrix = oldLocalModel.topicsMatrix + def topicsMatrix: Matrix = oldLocalModel.topicsMatrix.asML /** Indicates whether this instance is of type [[DistributedLDAModel]] */ @Since("1.6.0") @@ -872,13 +880,13 @@ class LDA @Since("1.6.0") ( private[clustering] object LDA extends DefaultParamsReadable[LDA] { /** Get dataset for spark.mllib LDA */ - def getOldDataset(dataset: Dataset[_], featuresCol: String): RDD[(Long, Vector)] = { + def getOldDataset(dataset: Dataset[_], featuresCol: String): RDD[(Long, OldVector)] = { dataset .withColumn("docId", monotonicallyIncreasingId()) .select("docId", featuresCol) .rdd .map { case Row(docId: Long, features: Vector) => - (docId, features) + (docId, OldVectors.fromML(features)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 0cbc391d96f8..bff72b20e1c3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 35bc46a5f343..318c8b8b2f7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -22,10 +22,10 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.BinaryAttribute +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 29f55a7f715c..e73a8f5d6608 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -22,12 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ import org.apache.spark.ml.attribute.{AttributeGroup, _} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -80,10 +82,11 @@ final class ChiSqSelector(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): ChiSqSelectorModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - } + val input: RDD[OldLabeledPoint] = + dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + OldLabeledPoint(label, OldVectors.fromML(features)) + } val chiSqSelector = new feature.ChiSqSelector($(numTopFeatures)).fit(input) copyValues(new ChiSqSelectorModel(uid, chiSqSelector).setParent(this)) } @@ -132,7 +135,11 @@ final class ChiSqSelectorModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { val transformedSchema = transformSchema(dataset.schema, logging = true) val newField = transformedSchema.last - val selector = udf { chiSqSelector.transform _ } + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => chiSqSelector.transform(OldVectors.fromML(v)).asML + + val selector = udf(transformer) dataset.withColumn($(outputCol), selector(col($(featuresCol))), newField.metadata) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 3fbfce9d48dd..fc4885bf4ba8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -21,10 +21,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.broadcast.Broadcast import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vectors, VectorUDT} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index a6f878151de7..301358ef1226 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -21,9 +21,9 @@ import edu.emory.mathcs.jtransforms.dct._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.BooleanParam import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.types.DataType /** 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 index 1b0a9a12e83b..91989c3d2f5d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -19,10 +19,12 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.Param import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.types.DataType /** @@ -52,7 +54,7 @@ class ElementwiseProduct(override val uid: String) override protected def createTransformFunc: Vector => Vector = { require(params.contains(scalingVec), s"transformation requires a weight vector") val elemScaler = new feature.ElementwiseProduct($(scalingVec)) - elemScaler.transform + v => elemScaler.transform(v) } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index 66ae91cfc097..94e1825ba61e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -86,7 +86,8 @@ class HashingTF(override val uid: String) override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) val hashingTF = new feature.HashingTF($(numFeatures)).setBinary($(binary)) - val t = udf { terms: Seq[_] => hashingTF.transform(terms) } + // TODO: Make the hashingTF.transform natively in ml framework to avoid extra conversion. + val t = udf { terms: Seq[_] => hashingTF.transform(terms).asML } val metadata = outputSchema($(outputCol)).metadata dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 5075b78c9856..f85f4c65af7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -21,11 +21,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType @@ -79,7 +81,9 @@ final class IDF(override val uid: String) extends Estimator[IDFModel] with IDFBa @Since("2.0.0") override def fit(dataset: Dataset[_]): IDFModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val idf = new feature.IDF($(minDocFreq)).fit(input) copyValues(new IDFModel(uid, idf).setParent(this)) } @@ -119,7 +123,8 @@ class IDFModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val idf = udf { vec: Vector => idfModel.transform(vec) } + // TODO: Make the idfModel.transform natively in ml framework to avoid extra conversion. + val idf = udf { vec: Vector => idfModel.transform(OldVectors.fromML(vec)).asML } dataset.withColumn($(outputCol), idf(col($(inputCol)))) } @@ -134,7 +139,7 @@ class IDFModel private[ml] ( /** Returns the IDF vector. */ @Since("1.6.0") - def idf: Vector = idfModel.idf + def idf: Vector = idfModel.idf.asML @Since("1.6.0") override def write: MLWriter = new IDFModelWriter(this) @@ -166,7 +171,7 @@ object IDFModel extends MLReadable[IDFModel] { .select("idf") .head() val idf = data.getAs[Vector](0) - val model = new IDFModel(metadata.uid, new feature.IDFModel(idf)) + val model = new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf))) DefaultParamsReader.getAndSetParams(model, metadata) model } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index 12176757aee3..cce3ca45ccd8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector /** * Class that represents an instance of weighted data point with label and features. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index 9ca34e9ae22f..fa65ff987917 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -26,7 +26,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.ml.Transformer -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala new file mode 100644 index 000000000000..f7f1d4203959 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import scala.beans.BeanInfo + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.Vector + +/** + * Class that represents the features and labels of a data point. + * + * @param label Label for this data point. + * @param features List of features for this data point. + */ +@Since("2.0.0") +@Experimental +@BeanInfo +case class LabeledPoint(@Since("2.0.0") label: Double, @Since("2.0.0") features: Vector) { + override def toString: String = { + s"($label,$features)" + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index e9df600c8a99..0dffba93ac57 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -21,11 +21,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.{ParamMap, Params} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -69,7 +71,9 @@ class MaxAbsScaler @Since("2.0.0") (override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): MaxAbsScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val summary = Statistics.colStats(input) val minVals = summary.min.toArray val maxVals = summary.max.toArray diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 125becbb8a5b..c6ff639f2962 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -21,11 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamMap, Params} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -106,7 +109,9 @@ class MinMaxScaler(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): MinMaxScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val summary = Statistics.colStats(input) copyValues(new MinMaxScalerModel(uid, summary.min, summary.max).setParent(this)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index a603b3f83320..942ac7ebdb3b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -19,10 +19,11 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamValidators} import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.sql.types.DataType /** @@ -52,7 +53,7 @@ class Normalizer(override val uid: String) override protected def createTransformFunc: Vector => Vector = { val normalizer = new feature.Normalizer($(p)) - normalizer.transform + vector => normalizer.transform(OldVectors.fromML(vector)).asML } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 99357793dbae..3d1e6dd81882 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 9cf722e12169..141d3b924b4f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -21,11 +21,16 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, DenseVector => OldDenseVector, + Matrices => OldMatrices, Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -71,7 +76,9 @@ class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams @Since("2.0.0") override def fit(dataset: Dataset[_]): PCAModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v} + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val pca = new feature.PCA(k = $(k)) val pcaModel = pca.fit(input) copyValues(new PCAModel(uid, pcaModel.pc, pcaModel.explainedVariance).setParent(this)) @@ -128,8 +135,14 @@ class PCAModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val pcaModel = new feature.PCAModel($(k), pc, explainedVariance) - val pcaOp = udf { pcaModel.transform _ } + val pcaModel = new feature.PCAModel($(k), + OldMatrices.fromML(pc).asInstanceOf[OldDenseMatrix], + OldVectors.fromML(explainedVariance).asInstanceOf[OldDenseVector]) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => pcaModel.transform(OldVectors.fromML(v)).asML + + val pcaOp = udf(transformer) dataset.withColumn($(outputCol), pcaOp(col($(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 0a9b9719c15d..a01867701bd8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -21,9 +21,9 @@ import scala.collection.mutable import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.types.DataType /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index a2f3d44132d1..c0feaa01fc86 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -25,10 +25,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model, Pipeline, PipelineModel, PipelineStage, Transformer} import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.VectorUDT import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala index cf52710ab8cb..19aecff03885 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable import scala.util.parsing.combinator.RegexParsers -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types._ /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 626e97efb47c..9d084b520c48 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -21,11 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -93,7 +96,9 @@ class StandardScaler(override val uid: String) extends Estimator[StandardScalerM @Since("2.0.0") override def fit(dataset: Dataset[_]): StandardScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val scaler = new feature.StandardScaler(withMean = $(withMean), withStd = $(withStd)) val scalerModel = scaler.fit(input) copyValues(new StandardScalerModel(uid, scalerModel.std, scalerModel.mean).setParent(this)) @@ -145,7 +150,11 @@ class StandardScalerModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) val scaler = new feature.StandardScalerModel(std, mean, $(withStd), $(withMean)) - val scale = udf { scaler.transform _ } + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => scaler.transform(OldVectors.fromML(v)).asML + + val scale = udf(transformer) dataset.withColumn($(outputCol), scale(col($(inputCol)))) } 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 4d3e46e488c6..1bc24202b761 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 @@ -23,10 +23,10 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute, UnresolvedAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 68b699d569c7..2bc9d225ac2f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -27,10 +27,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{StructField, StructType} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index 7a9468b87b73..103738cd91c0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntArrayParam, ParamMap, StringArrayParam} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index c49e263df0a6..1469bfd5e857 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -22,11 +22,12 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala index c29f7f86e9f2..0b9b2ff5c5e2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._ * User-defined type for [[Vector]] in [[mllib-local]] which allows easy interaction with SQL * via [[org.apache.spark.sql.Dataset]]. */ -private[ml] class VectorUDT extends UserDefinedType[Vector] { +private[spark] class VectorUDT extends UserDefinedType[Vector] { override def sqlType: StructType = { // type: 0 = sparse, 1 = dense diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index a2b52835e177..6ed193cf57d4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 7d21302f962b..8f5f4427e1f4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -19,7 +19,8 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ +import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index c368aadd2366..82f2de7ccdfd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -29,8 +29,9 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.JsonVectorConverter import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: DeveloperApi :: @@ -92,7 +93,7 @@ class Param[T](val parent: String, val name: String, val doc: String, val isVali case x: String => compact(render(JString(x))) case v: Vector => - v.toJson + JsonVectorConverter.toJson(v) case _ => throw new NotImplementedError( "The default jsonEncode only supports string and vector. " + @@ -128,7 +129,7 @@ private[ml] object Param { val keys = v.map(_._1) assert(keys.contains("type") && keys.contains("values"), s"Expect a JSON serialized vector but cannot find fields 'type' and 'values' in $json.") - Vectors.fromJson(json).asInstanceOf[T] + JsonVectorConverter.fromJson(json).asInstanceOf[T] case _ => throw new NotImplementedError( "The default jsonDecode only supports string and vector. " + diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 89ba6ab5d277..cc16c2f038be 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -27,10 +27,11 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} 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 874d2a81db21..c4df9d11127f 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 @@ -23,13 +23,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD 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 c41fb4b0629b..81f2139f0b42 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 @@ -24,13 +24,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 05fffa0d97e6..4aa7c2cc0b9b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -25,11 +25,11 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector} import org.apache.spark.ml.optim._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 7a78ecbdf16d..ba0f59e89bb2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -22,11 +22,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.regression.IsotonicRegressionModel.IsotonicRegressionModelWriter import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.regression.{IsotonicRegression => MLlibIsotonicRegression} import org.apache.spark.mllib.regression.{IsotonicRegressionModel => MLlibIsotonicRegressionModel} import org.apache.spark.rdd.RDD 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 d13b15fd82f0..3e9a3f9db5f4 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 @@ -28,14 +28,16 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.optim.WeightedLeastSquares import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} 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 9605de72020f..a6dbf21d55e2 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 @@ -22,13 +22,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 39bdd1afadcc..5ba768d55184 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -26,8 +26,8 @@ import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, DataFrameReader, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index f71d28cf5953..d5e5c454605b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.tree import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{ImpurityStats, InformationGainStats => OldInformationGainStats, Node => OldNode, Predict => OldPredict} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index a4287483d18e..9704e15cd838 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.tree import java.util.Objects import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} import org.apache.spark.mllib.tree.model.{Split => OldSplit} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index 5f7c40f6071f..442f52bf0231 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -21,8 +21,8 @@ import scala.collection.mutable import scala.util.Try import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.RandomForestParams -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ import org.apache.spark.mllib.tree.configuration.Strategy diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index b6334762c7a7..a0faff236e9d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.configuration.{BoostingStrategy => OldBoostingStrategy} import org.apache.spark.mllib.tree.impurity.{Variance => OldVariance} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 2038a6873db7..be3792eb7732 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -24,10 +24,10 @@ import scala.util.Random import org.apache.spark.internal.Logging import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.util.Instrumentation -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.ImpurityStats diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala index 3a2bf3c72573..a6ac64a0463c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.tree.impl +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.{ContinuousSplit, Split} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index f38e1ec7c09a..56c85c9b53e1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -23,11 +23,11 @@ import org.apache.hadoop.fs.Path import org.json4s._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.{Param, Params} import org.apache.spark.ml.tree.DecisionTreeModelReadWrite.NodeData import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter} import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala index 96a38a3bde96..f34a8310ddf1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.util import scala.collection.immutable.HashMap import org.apache.spark.ml.attribute._ -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.StructField diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 8daee7b3aa1e..90d382753131 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -30,6 +30,8 @@ import net.razorvine.pickle._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.ml.feature.{LabeledPoint => MLLabeledPoint} +import org.apache.spark.ml.linalg.{DenseMatrix => NewDenseMatrix, DenseVector => NewDenseVector, SparseMatrix => NewSparseMatrix, SparseVector => NewSparseVector, Vector => NewVector, Vectors => NewVectors} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.evaluation.RankingMetrics @@ -1207,6 +1209,7 @@ private[python] class PythonMLLibAPI extends Serializable { private[spark] object SerDe extends Serializable { val PYSPARK_PACKAGE = "pyspark.mllib" + val PYSPARK_ML_PACKAGE = "pyspark.ml" /** * Base class used for pickle @@ -1214,8 +1217,10 @@ private[spark] object SerDe extends Serializable { private[python] abstract class BasePickler[T: ClassTag] extends IObjectPickler with IObjectConstructor { + protected def packageName: String = PYSPARK_PACKAGE + private val cls = implicitly[ClassTag[T]].runtimeClass - private val module = PYSPARK_PACKAGE + "." + cls.getName.split('.')(4) + private val module = packageName + "." + cls.getName.split('.')(4) private val name = cls.getSimpleName // register this to Pickler and Unpickler @@ -1262,7 +1267,7 @@ private[spark] object SerDe extends Serializable { private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } - // Pickler for DenseVector + // Pickler for (mllib) DenseVector private[python] class DenseVectorPickler extends BasePickler[DenseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1294,7 +1299,41 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for DenseMatrix + // Pickler for (new) DenseVector + private[python] class NewDenseVectorPickler extends BasePickler[NewDenseVector] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val vector: NewDenseVector = obj.asInstanceOf[NewDenseVector] + val bytes = new Array[Byte](8 * vector.size) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + db.put(vector.values) + + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE1) + } + + def construct(args: Array[Object]): Object = { + require(args.length == 1) + if (args.length != 1) { + throw new PickleException("should be 1") + } + val bytes = getBytes(args(0)) + val bb = ByteBuffer.wrap(bytes, 0, bytes.length) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](bytes.length / 8) + db.get(ans) + NewVectors.dense(ans) + } + } + + // Pickler for (mllib) DenseMatrix private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1331,7 +1370,46 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for SparseMatrix + // Pickler for (new) DenseMatrix + private[python] class NewDenseMatrixPickler extends BasePickler[NewDenseMatrix] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val m: NewDenseMatrix = obj.asInstanceOf[NewDenseMatrix] + val bytes = new Array[Byte](8 * m.values.length) + val order = ByteOrder.nativeOrder() + val isTransposed = if (m.isTransposed) 1 else 0 + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 4) { + throw new PickleException("should be 4") + } + val bytes = getBytes(args(2)) + val n = bytes.length / 8 + val values = new Array[Double](n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(3).asInstanceOf[Int] == 1 + new NewDenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) + } + } + + // Pickler for (mllib) SparseMatrix private[python] class SparseMatrixPickler extends BasePickler[SparseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1386,7 +1464,64 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for SparseVector + // Pickler for (new) SparseMatrix + private[python] class NewSparseMatrixPickler extends BasePickler[NewSparseMatrix] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val s = obj.asInstanceOf[NewSparseMatrix] + val order = ByteOrder.nativeOrder() + + val colPtrsBytes = new Array[Byte](4 * s.colPtrs.length) + val indicesBytes = new Array[Byte](4 * s.rowIndices.length) + val valuesBytes = new Array[Byte](8 * s.values.length) + val isTransposed = if (s.isTransposed) 1 else 0 + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().put(s.colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().put(s.rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().put(s.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(colPtrsBytes.length)) + out.write(colPtrsBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indicesBytes.length)) + out.write(indicesBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valuesBytes.length)) + out.write(valuesBytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 6) { + throw new PickleException("should be 6") + } + val order = ByteOrder.nativeOrder() + val colPtrsBytes = getBytes(args(2)) + val indicesBytes = getBytes(args(3)) + val valuesBytes = getBytes(args(4)) + val colPtrs = new Array[Int](colPtrsBytes.length / 4) + val rowIndices = new Array[Int](indicesBytes.length / 4) + val values = new Array[Double](valuesBytes.length / 8) + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().get(colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().get(rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(5).asInstanceOf[Int] == 1 + new NewSparseMatrix( + args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], colPtrs, rowIndices, values, + isTransposed) + } + } + + // Pickler for (mllib) SparseVector private[python] class SparseVectorPickler extends BasePickler[SparseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1428,7 +1563,51 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for LabeledPoint + // Pickler for (new) SparseVector + private[python] class NewSparseVectorPickler extends BasePickler[NewSparseVector] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val v: NewSparseVector = obj.asInstanceOf[NewSparseVector] + val n = v.indices.length + val indiceBytes = new Array[Byte](4 * n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) + val valueBytes = new Array[Byte](8 * n) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(v.size)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) + out.write(indiceBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valueBytes.length)) + out.write(valueBytes) + out.write(Opcodes.TUPLE3) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + val size = args(0).asInstanceOf[Int] + val indiceBytes = getBytes(args(1)) + val valueBytes = getBytes(args(2)) + val n = indiceBytes.length / 4 + val indices = new Array[Int](n) + val values = new Array[Double](n) + if (n > 0) { + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) + } + new NewSparseVector(size, indices, values) + } + } + + // Pickler for MLlib LabeledPoint private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1482,6 +1661,10 @@ private[spark] object SerDe extends Serializable { new DenseMatrixPickler().register() new SparseMatrixPickler().register() new SparseVectorPickler().register() + new NewDenseVectorPickler().register() + new NewDenseMatrixPickler().register() + new NewSparseMatrixPickler().register() + new NewSparseVectorPickler().register() new LabeledPointPickler().register() new RatingPickler().register() initialized = true diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 1d25a58e0f2e..f3c52f61a3bb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -431,7 +431,7 @@ class LogisticRegressionWithLBFGS if (userSuppliedWeights) { val uid = Identifiable.randomUID("logreg-static") lr.setInitialModel(new org.apache.spark.ml.classification.LogisticRegressionModel( - uid, initialWeights, 1.0)) + uid, initialWeights.asML, 1.0)) } lr.setFitIntercept(addIntercept) lr.setMaxIter(optimizer.getNumIterations()) @@ -439,7 +439,7 @@ class LogisticRegressionWithLBFGS // Convert our input into a DataFrame val sqlContext = new SQLContext(input.context) import sqlContext.implicits._ - val df = input.toDF() + val df = input.map(_.asML).toDF() // Determine if we should cache the DF val handlePersistence = input.getStorageLevel == StorageLevel.NONE // Train our model diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 076cca6016ec..5c9a112ca69c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.linalg import java.util.{Arrays, Random} import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, HashSet => MHashSet} +import scala.language.implicitConversions import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -1212,3 +1213,24 @@ object Matrices { SparseMatrix.fromML(sm) } } + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Matrix]] to + * [[org.apache.spark.ml.linalg.Matrix]] and vice versa. + */ +private[spark] object MatrixImplicits { + + implicit def mllibMatrixToMLMatrix(m: Matrix): newlinalg.Matrix = m.asML + + implicit def mllibDenseMatrixToMLDenseMatrix(m: DenseMatrix): newlinalg.DenseMatrix = m.asML + + implicit def mllibSparseMatrixToMLSparseMatrix(m: SparseMatrix): newlinalg.SparseMatrix = m.asML + + implicit def mlMatrixToMLlibMatrix(m: newlinalg.Matrix): Matrix = Matrices.fromML(m) + + implicit def mlDenseMatrixToMLlibDenseMatrix(m: newlinalg.DenseMatrix): DenseMatrix = + Matrices.fromML(m).asInstanceOf[DenseMatrix] + + implicit def mlSparseMatrixToMLlibSparseMatrix(m: newlinalg.SparseMatrix): SparseMatrix = + Matrices.fromML(m).asInstanceOf[SparseMatrix] +} 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 132e54a8c3de..1f1cfa0cb28b 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 @@ -22,6 +22,7 @@ import java.util import scala.annotation.varargs import scala.collection.JavaConverters._ +import scala.language.implicitConversions import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.json4s.DefaultFormats @@ -926,3 +927,24 @@ object SparseVector { new SparseVector(v.size, v.indices, v.values) } } + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Vector]] to + * [[org.apache.spark.ml.linalg.Vector]] and vice versa. + */ +private[spark] object VectorImplicits { + + implicit def mllibVectorToMLVector(v: Vector): newlinalg.Vector = v.asML + + implicit def mllibDenseVectorToMLDenseVector(v: DenseVector): newlinalg.DenseVector = v.asML + + implicit def mllibSparseVectorToMLSparseVector(v: SparseVector): newlinalg.SparseVector = v.asML + + implicit def mlVectorToMLlibVector(v: newlinalg.Vector): Vector = Vectors.fromML(v) + + implicit def mlDenseVectorToMLlibDenseVector(v: newlinalg.DenseVector): DenseVector = + Vectors.fromML(v).asInstanceOf[DenseVector] + + implicit def mlSparseVectorToMLlibSparseVector(v: newlinalg.SparseVector): SparseVector = + Vectors.fromML(v).asInstanceOf[SparseVector] +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 45540f0c5c4c..f082b16b95e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import scala.beans.BeanInfo import org.apache.spark.annotation.Since +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.NumericParser import org.apache.spark.SparkException @@ -38,6 +39,10 @@ case class LabeledPoint @Since("1.0.0") ( override def toString: String = { s"($label,$features)" } + + private[spark] def asML: NewLabeledPoint = { + NewLabeledPoint(label, features.asML) + } } /** @@ -67,4 +72,8 @@ object LabeledPoint { LabeledPoint(label, features) } } + + private[spark] def fromML(point: NewLabeledPoint): LabeledPoint = { + LabeledPoint(point.label, Vectors.fromML(point.features)) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 7fe60e2d99e4..ece1e41d986d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.tree import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.ml.tree.impl.{GradientBoostedTrees => NewGBT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.BoostingStrategy @@ -66,7 +67,9 @@ class GradientBoostedTrees private[spark] ( @Since("1.2.0") def run(input: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.run(input, boostingStrategy, seed.toLong) + val (trees, treeWeights) = NewGBT.run(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } @@ -94,8 +97,11 @@ class GradientBoostedTrees private[spark] ( input: RDD[LabeledPoint], validationInput: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.runWithValidation(input, validationInput, boostingStrategy, - seed.toLong) + val (trees, treeWeights) = NewGBT.runWithValidation(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, validationInput.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index c31ed9c1ce0b..14f11ce51b87 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -90,8 +90,8 @@ private class RandomForest ( * @return RandomForestModel that can be used for prediction. */ def run(input: RDD[LabeledPoint]): RandomForestModel = { - val trees: Array[NewDTModel] = - NewRandomForest.run(input, strategy, numTrees, featureSubsetStrategy, seed.toLong, None) + val trees: Array[NewDTModel] = NewRandomForest.run(input.map(_.asML), strategy, numTrees, + featureSubsetStrategy, seed.toLong, None) new RandomForestModel(strategy.algo, trees.map(_.toOld)) } diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index e0c4363597da..46c26e8b929e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -24,12 +24,12 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegression; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.feature.StandardScaler; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; /** * Test Pipeline construction and fitting in Java. diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java index 8b899913277f..c76a1947c64f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java index 682371eb9e4d..4648926c3421 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index e3ff68364e69..98abca221c88 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -27,12 +27,12 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; public class JavaLogisticRegressionSuite implements Serializable { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java index b0624cea3ecd..48edbc838c81 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java @@ -26,8 +26,8 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java index 3fc364862715..787909821be9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java @@ -26,8 +26,8 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index 486fbbd58c17..58bc5a448aed 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -29,11 +29,11 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; public class JavaOneVsRestSuite implements Serializable { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java index e3855662fb6d..1ed20b1bfa17 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java @@ -28,10 +28,10 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java index 3ab09ac27d30..9d07170fa1d5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java @@ -28,7 +28,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index 06482d8f0dcd..9d8c09b30c01 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -27,9 +27,9 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java index 0e21d4a94f24..3c37441a77be 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java @@ -25,7 +25,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java index 04b2897b1811..b3e213a49700 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java @@ -25,7 +25,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index 32f6b4375ee7..a4bce2283b86 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -31,9 +31,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -88,9 +88,25 @@ public void testPCA() { ); JavaRDD dataRDD = jsc.parallelize(points, 2); - RowMatrix mat = new RowMatrix(dataRDD.rdd()); + RowMatrix mat = new RowMatrix(dataRDD.map( + new Function() { + public org.apache.spark.mllib.linalg.Vector call(Vector vector) { + return new org.apache.spark.mllib.linalg.DenseVector(vector.toArray()); + } + } + ).rdd()); + Matrix pc = mat.computePrincipalComponents(3); - JavaRDD expected = mat.multiply(pc).rows().toJavaRDD(); + + mat.multiply(pc).rows().toJavaRDD(); + + JavaRDD expected = mat.multiply(pc).rows().toJavaRDD().map( + new Function() { + public Vector call(org.apache.spark.mllib.linalg.Vector vector) { + return vector.asML(); + } + } + ); JavaRDD featuresExpected = dataRDD.zip(expected).map( new Function, VectorPair>() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java index 8f726077a249..a28f73f10a0b 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java @@ -26,9 +26,9 @@ import org.junit.Test; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java index c7397bdd6878..8415fdb84f21 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java @@ -25,7 +25,7 @@ import org.junit.Test; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java index 8774cd0c69f1..fedaa7717658 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java @@ -25,9 +25,9 @@ import org.junit.Test; import org.apache.spark.SparkConf; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java index c386c9a45b09..a8dd44608db3 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -29,7 +29,7 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index 59ad3c2f61e8..a565c77af4ab 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -28,8 +28,8 @@ import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java index 392aabc96d37..bef7eb0f995f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java @@ -24,7 +24,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java index bbd59a04ec86..4ea3f2255efa 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java index 5370b58e8fda..3b5edf1e15ae 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index 00c59f08b679..d3ef5f6fca07 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -27,11 +27,11 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; public class JavaLinearRegressionSuite implements Serializable { diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java index fdb41ffc1038..d601e7c540e1 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java @@ -28,10 +28,10 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java index 058f2ddafd86..022dcf94bd00 100644 --- a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java @@ -28,8 +28,8 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.DenseVector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.DenseVector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java index 8b4d034ffea0..b874ccd48b3b 100644 --- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java @@ -27,13 +27,13 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegression; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; public class JavaCrossValidatorSuite implements Serializable { diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala index 55448325e44d..3b490cdf5601 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -27,9 +27,9 @@ import org.scalatest.mock.MockitoSugar.mock import org.apache.spark.SparkFunSuite import org.apache.spark.ml.Pipeline.SharedReadWrite import org.apache.spark.ml.feature.{HashingTF, MinMaxScaler} -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamPair} +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala index dc91fc5f9e45..35586320cb82 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.ann import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ - class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala index 04cc426c40b5..f0c0183323c9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.ann import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class GradientSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala index 98116656ba86..4db5f03fb00b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.ml.classification import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.classification.ClassifierSuite.MockClassifier +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index 91a947f44bc3..089d30abb5ef 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.{CategoricalSplit, InternalNode, LeafNode} import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD @@ -44,17 +45,18 @@ class DecisionTreeClassifierSuite override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()).map(_.asML) orderedLabeledPointsWithLabel0RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()).map(_.asML) orderedLabeledPointsWithLabel1RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()).map(_.asML) categoricalDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()).map(_.asML) continuousDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()).map(_.asML) categoricalDataPointsForMulticlassForOrderedFeaturesRDD = sc.parallelize( OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures()) + .map(_.asML) } test("params") { @@ -395,7 +397,7 @@ private[ml] object DecisionTreeClassifierSuite extends SparkFunSuite { numClasses: Int): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures, numClasses) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index 5a5e5c15fc59..8d588ccfd354 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.classification import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -51,10 +52,13 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } test("params") { @@ -229,7 +233,7 @@ private object GBTClassifierSuite extends SparkFunSuite { val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) - val oldModel = oldGBT.run(data) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 2) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 69650ebb3690..a1b48539c46e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.ml.classification +import scala.collection.JavaConverters._ import scala.language.existentials import scala.util.Random +import scala.util.control.Breaks._ import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.{Instance, LabeledPoint} +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.lit @@ -967,4 +968,122 @@ object LogisticRegressionSuite { "standardization" -> false, "threshold" -> 0.6 ) + + def generateLogisticInputAsList( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + generateLogisticInput(offset, scale, nPoints, seed).asJava + } + + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val y = (0 until nPoints).map { i => + val p = 1.0 / (1.0 + math.exp(-(offset + scale * x1(i)))) + if (rnd.nextDouble() < p) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) + testData + } + + /** + * Generates `k` classes multinomial synthetic logistic input in `n` dimensional space given the + * model weights and mean/variance of the features. The synthetic data will be drawn from + * the probability distribution constructed by weights using the following formula. + * + * P(y = 0 | x) = 1 / norm + * P(y = 1 | x) = exp(x * w_1) / norm + * P(y = 2 | x) = exp(x * w_2) / norm + * ... + * P(y = k-1 | x) = exp(x * w_{k-1}) / norm + * where norm = 1 + exp(x * w_1) + exp(x * w_2) + ... + exp(x * w_{k-1}) + * + * @param weights matrix is flatten into a vector; as a result, the dimension of weights vector + * will be (k - 1) * (n + 1) if `addIntercept == true`, and + * if `addIntercept != true`, the dimension will be (k - 1) * n. + * @param xMean the mean of the generated features. Lots of time, if the features are not properly + * standardized, the algorithm with poor implementation will have difficulty + * to converge. + * @param xVariance the variance of the generated features. + * @param addIntercept whether to add intercept. + * @param nPoints the number of instance of generated data. + * @param seed the seed for random generator. For consistent testing result, it will be fixed. + */ + def generateMultinomialLogisticInput( + weights: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + addIntercept: Boolean, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + + val xDim = xMean.length + val xWithInterceptsDim = if (addIntercept) xDim + 1 else xDim + val nClasses = weights.length / xWithInterceptsDim + 1 + + val x = Array.fill[Vector](nPoints)(Vectors.dense(Array.fill[Double](xDim)(rnd.nextGaussian()))) + + x.foreach { vector => + // This doesn't work if `vector` is a sparse vector. + val vectorArray = vector.toArray + var i = 0 + val len = vectorArray.length + while (i < len) { + vectorArray(i) = vectorArray(i) * math.sqrt(xVariance(i)) + xMean(i) + i += 1 + } + } + + val y = (0 until nPoints).map { idx => + val xArray = x(idx).toArray + val margins = Array.ofDim[Double](nClasses) + val probs = Array.ofDim[Double](nClasses) + + for (i <- 0 until nClasses - 1) { + for (j <- 0 until xDim) margins(i + 1) += weights(i * xWithInterceptsDim + j) * xArray(j) + if (addIntercept) margins(i + 1) += weights((i + 1) * xWithInterceptsDim - 1) + } + // Preventing the overflow when we compute the probability + val maxMargin = margins.max + if (maxMargin > 0) for (i <- 0 until nClasses) margins(i) -= maxMargin + + // Computing the probabilities for each class from the margins. + val norm = { + var temp = 0.0 + for (i <- 0 until nClasses) { + probs(i) = math.exp(margins(i)) + temp += probs(i) + } + temp + } + for (i <- 0 until nClasses) probs(i) /= norm + + // Compute the cumulative probability so we can generate a random number and assign a label. + for (i <- 1 until nClasses) probs(i) += probs(i - 1) + val p = rnd.nextDouble() + var y = 0 + breakable { + for (i <- 0 until nClasses) { + if (p < probs(i)) { + y = i + break + } + } + } + y + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), x(i))) + testData + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index d5282e07d65c..85f325f0765a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -18,14 +18,16 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{Dataset, Row} class MultilayerPerceptronClassifierSuite @@ -134,12 +136,13 @@ class MultilayerPerceptronClassifierSuite .setNumClasses(numClasses) lr.optimizer.setRegParam(0.0) .setNumIterations(numIterations) - val lrModel = lr.run(rdd) - val lrPredictionAndLabels = lrModel.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val lrModel = lr.run(rdd.map(OldLabeledPoint.fromML)) + val lrPredictionAndLabels = + lrModel.predict(rdd.map(p => OldVectors.fromML(p.features))).zip(rdd.map(_.label)) // MLP's predictions should not differ a lot from LR's. val lrMetrics = new MulticlassMetrics(lrPredictionAndLabels) val mlpMetrics = new MulticlassMetrics(mlpPredictionAndLabels) - assert(mlpMetrics.confusionMatrix ~== lrMetrics.confusionMatrix absTol 100) + assert(mlpMetrics.confusionMatrix.asML ~== lrMetrics.confusionMatrix.asML absTol 100) } test("read/write: MultilayerPerceptronClassifier") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 2a05c446e516..ff52115ec0ec 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.ml.classification -import breeze.linalg.{Vector => BV} +import scala.util.Random + +import breeze.linalg.{DenseVector => BDV, Vector => BV} +import breeze.stats.distributions.{Multinomial => BrzMultinomial} import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.NaiveBayesSuite._ +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.NaiveBayes.{Bernoulli, Multinomial} -import org.apache.spark.mllib.classification.NaiveBayesSuite._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Dataset, Row} class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -206,4 +210,48 @@ object NaiveBayesSuite { "predictionCol" -> "myPrediction", "smoothing" -> 0.1 ) + + private def calcLabel(p: Double, pi: Array[Double]): Int = { + var sum = 0.0 + for (j <- 0 until pi.length) { + sum += pi(j) + if (p < sum) return j + } + -1 + } + + // Generate input of the form Y = (theta * x).argmax() + def generateNaiveBayesInput( + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int, + modelType: String = Multinomial, + sample: Int = 10): Seq[LabeledPoint] = { + val D = theta(0).length + val rnd = new Random(seed) + val _pi = pi.map(math.pow(math.E, _)) + val _theta = theta.map(row => row.map(math.pow(math.E, _))) + + for (i <- 0 until nPoints) yield { + val y = calcLabel(rnd.nextDouble(), _pi) + val xi = modelType match { + case Bernoulli => Array.tabulate[Double] (D) { j => + if (rnd.nextDouble () < _theta(y)(j) ) 1 else 0 + } + case Multinomial => + val mult = BrzMultinomial(BDV(_theta(y))) + val emptyMap = (0 until D).map(x => (x, 0.0)).toMap + val counts = emptyMap ++ mult.sample(sample).groupBy(x => x).map { + case (index, reps) => (index, reps.size.toDouble) + } + counts.toArray.sortBy(_._1).map(_._2) + case _ => + // This should never happen. + throw new UnknownError(s"Invalid modelType: $modelType.") + } + + LabeledPoint(y, Vectors.dense(xi)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 5044d40998d6..361dd74cb082 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -19,14 +19,16 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.feature.StringIndexer +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{ParamMap, ParamsSuite} import org.apache.spark.ml.util.{DefaultReadWriteTest, MetadataUtils, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD @@ -88,8 +90,8 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with Defau val lr = new LogisticRegressionWithLBFGS().setIntercept(true).setNumClasses(numClasses) lr.optimizer.setRegParam(0.1).setNumIterations(100) - val model = lr.run(rdd) - val results = model.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val model = lr.run(rdd.map(OldLabeledPoint.fromML)) + val results = model.predict(rdd.map(p => OldVectors.fromML(p.features))).zip(rdd.map(_.label)) // determine the #confusion matrix in each class. // bound how much error we allow compared to multinomial logistic regression. val expectedMetrics = new MulticlassMetrics(results) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala index cfa75ecf387c..b3bd2b3e57b3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} final class TestProbabilisticClassificationModel( override val uid: String, diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index 8002a2f4f29e..2e99ee157ae9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -46,8 +47,10 @@ class RandomForestClassifierSuite super.beforeAll() orderedLabeledPoints50_1000 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + .map(_.asML) orderedLabeledPoints5_20 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 5, 20)) + .map(_.asML) } ///////////////////////////////////////////////////////////////////////////// @@ -233,7 +236,8 @@ private object RandomForestClassifierSuite extends SparkFunSuite { val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, rf.getOldImpurity) val oldModel = OldRandomForest.trainClassifier( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + data.map(OldLabeledPoint.fromML), oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, + rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 2832db2f9915..88f31a1cd26f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.clustering import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index 34e896428603..ddfa87555427 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.clustering import org.apache.hadoop.fs.Path import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql._ diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala index a8766f903524..9ee3df5eb5e3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class BinaryClassificationEvaluatorSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala index dcc004358db1..42ff8adf6bd6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala @@ -44,7 +44,7 @@ class RegressionEvaluatorSuite */ val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) /** * Using the following R code to load the data, train the model and evaluate metrics. diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala index e91f758112cf..9cb84a6ee9b8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala index 98b2316d78db..cd10c78311e1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.ml.feature import scala.util.Random import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index 4fcc9745b738..40d5b4881f83 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -18,12 +18,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{Row, SparkSession} class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala index b82e3e90b4f7..a59203c33d81 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index dbd5ae834536..c02e9610418b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -22,8 +22,8 @@ import scala.beans.BeanInfo import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala index fc1c05de233e..a4cca27be781 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class ElementwiseProductSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala index 89d67d8e6f93..99b800776bb6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala @@ -19,12 +19,12 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{HashingTF => MLlibHashingTF} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala index 208ea849136c..09dc8b9b932f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{IDFModel => OldIDFModel} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class IDFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala index 340992800729..3429172a8c90 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala @@ -21,9 +21,9 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.functions.col diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala index 73d69ebfeed0..d6400ee02f95 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala index e495c8e57134..5da84711758c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala index 241a1e9fb5c9..b69283171446 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 06ffbc386feb..7841b4fbc77a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions.col diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala index 4befa84dbb75..ddb51fb1706a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class PCASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -45,9 +46,9 @@ class PCASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead val dataRDD = sc.parallelize(data, 2) - val mat = new RowMatrix(dataRDD) + val mat = new RowMatrix(dataRDD.map(OldVectors.fromML)) val pc = mat.computePrincipalComponents(3) - val expected = mat.multiply(pc).rows + val expected = mat.multiply(pc).rows.map(_.asML) val df = spark.createDataFrame(dataRDD.zip(expected)).toDF("features", "expected") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala index e3adbba9d5df..8e1f9ddb36cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.ml.feature import org.scalatest.exceptions.TestFailedException import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class PolynomialExpansionSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index c623a6210bda..c12ab8fe9efe 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.types.DoubleType diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala index d62301be141b..2243a0f972d3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext 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 250011c859bc..14973e79bf34 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 @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row import org.apache.spark.sql.functions.col diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index d1c0270a02c0..707142332349 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -22,9 +22,9 @@ import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala index 88a077f9a1a5..1746ce53107c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.{StructField, StructType} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index 8cbe0f3defc4..280a36f56e93 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{Word2VecModel => OldWord2VecModel} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 7b50876d3319..6ddb12cb76aa 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -17,19 +17,11 @@ package org.apache.spark.ml.linalg -import scala.beans.BeanInfo - import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.sql.catalyst.JavaTypeInference import org.apache.spark.sql.types._ -@BeanInfo -case class LabeledPoint(label: Double, features: Vector) { - override def toString: String = { - s"($label,$features)" - } -} - class VectorUDTSuite extends SparkFunSuite { test("preloaded VectorUDT") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala index 604021220a13..b30d995794d4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala index 0b58a9821f57..c8de796b2de8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala index a3366c0e5934..aa9c53ca30ee 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.ml.param -import java.io.{ByteArrayOutputStream, NotSerializableException, ObjectOutputStream} +import java.io.{ByteArrayOutputStream, ObjectOutputStream} import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.MyParams -import org.apache.spark.mllib.linalg.{Vector, Vectors} class ParamsSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 9da0c32deede..bbfc415cbb9b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -31,11 +31,11 @@ import org.apache.commons.io.filefilter.TrueFileFilter import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.recommendation.ALS._ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} import org.apache.spark.sql.{DataFrame, Row, SparkSession} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index e4772df622d1..05aae80c660e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.ml.regression import scala.util.Random import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random.{ExponentialGenerator, WeibullGenerator} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class AFTSurvivalRegressionSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index 2d30cbf36766..9afb742406ec 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -38,7 +39,7 @@ class DecisionTreeRegressorSuite override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints().map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -170,7 +171,7 @@ private[ml] object DecisionTreeRegressorSuite extends SparkFunSuite { categoricalFeatures: Map[Int, Int]): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index ac833b833d7d..7b5df8f31bb3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -48,10 +49,13 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } test("Regression with continuous features") { @@ -197,7 +201,7 @@ private object GBTRegressorSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) - val oldModel = oldGBT.run(data) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index 3d9aeb8c0a2d..a4568e83faca 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -20,15 +20,15 @@ package org.apache.spark.ml.regression import scala.util.Random import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LogisticRegressionSuite._ import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ -import org.apache.spark.mllib.linalg.{BLAS, DenseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random._ -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala index bed4978b25b3..14d8a4e4e334 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index a98227d2c14f..332d331a4773 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -21,12 +21,12 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class LinearRegressionSuite @@ -45,7 +45,7 @@ class LinearRegressionSuite datasetWithDenseFeature = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML)) /* datasetWithDenseFeatureWithoutIntercept is not needed for correctness testing but is useful for illustrating training model without intercept @@ -53,7 +53,7 @@ class LinearRegressionSuite datasetWithDenseFeatureWithoutIntercept = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( intercept = 0.0, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML)) val r = new Random(seed) // When feature size is larger than 4096, normal optimizer is choosed @@ -64,7 +64,7 @@ class LinearRegressionSuite intercept = 0.0, weights = Seq.fill(featureSize)(r.nextDouble()).toArray, xMean = Seq.fill(featureSize)(r.nextDouble()).toArray, xVariance = Seq.fill(featureSize)(r.nextDouble()).toArray, nPoints = 200, - seed, eps = 0.1, sparsity = 0.7), 2)) + seed, eps = 0.1, sparsity = 0.7), 2).map(_.asML)) /* R code: @@ -795,7 +795,7 @@ class LinearRegressionSuite Seq("auto", "l-bfgs", "normal").foreach { solver => val (data, weightedData) = { val activeData = LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1).map(_.asML) val rnd = new Random(8392) val signedData = activeData.map { case p: LabeledPoint => @@ -822,7 +822,7 @@ class LinearRegressionSuite } val noiseData = LinearDataGenerator.generateLinearInput( - 2, Array(1, 3), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) + 2, Array(1, 3), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1).map(_.asML) val weightedNoiseData = noiseData.map { case LabeledPoint(label, features) => Instance(label, weight = 0, features) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index 7a3a3698f950..c08335f9f84a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -40,7 +41,8 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex override def beforeAll() { super.beforeAll() orderedLabeledPoints50_1000 = - sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) + .map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -139,8 +141,8 @@ private object RandomForestRegressorSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, rf.getOldImpurity) - val oldModel = OldRandomForest.trainRegressor( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + val oldModel = OldRandomForest.trainRegressor(data.map(OldLabeledPoint.fromML), oldStrategy, + rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala index 7d0e01fd8fb9..2517de59fed6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets import com.google.common.io.Files import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{Row, SaveMode} import org.apache.spark.util.Utils diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala index de92b51eb020..5c50a88c8314 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.mllib.tree.{GradientBoostedTreesSuite => OldGBTSuite} import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ @@ -35,8 +35,8 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext test("runWithValidation stops early and performs better on a validation dataset") { // Set numIterations large enough so that it stops early. val numIterations = 20 - val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2) - val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2) + val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2).map(_.asML) + val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2).map(_.asML) val trainDF = spark.createDataFrame(trainRdd) val validateDF = spark.createDataFrame(validateRdd) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 9739e6c05dcb..dcc2f305df75 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -21,14 +21,14 @@ import scala.collection.mutable import org.apache.spark.SparkFunSuite import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.tree.{DecisionTreeSuite => OldDTSuite, EnsembleTestHelper} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, QuantileStrategy, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, GiniCalculator} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.collection.OpenHashMap /** @@ -43,7 +43,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { ///////////////////////////////////////////////////////////////////////////// test("Binary classification with continuous features: split calculation") { - val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1() + val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, 3, 2, 100) @@ -55,7 +55,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Binary classification with binary (ordered) categorical features: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -72,7 +72,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { test("Binary classification with 3-ary (ordered) categorical features," + " with no samples for one category: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -148,7 +148,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Multiclass classification with unordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy( @@ -189,7 +189,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Multiclass classification with ordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures().map(_.asML) assert(arr.length === 3000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 100, @@ -334,7 +334,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Second level node building with vs. without groups") { - val arr = OldDTSuite.generateOrderedLabeledPoints() + val arr = OldDTSuite.generateOrderedLabeledPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) // For tree with 1 group @@ -378,7 +378,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: OldStrategy) { val numFeatures = 50 val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) - val rdd = sc.parallelize(arr) + val rdd = sc.parallelize(arr).map(_.asML) // Select feature subset for top nodes. Return true if OK. def checkFeatureSubsetStrategy( diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala index 12ade4c92f85..8cbd652bacf3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala @@ -22,9 +22,9 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SparkSession} diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index 85df6da7a16a..30bd390381e9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -20,17 +20,17 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model, Pipeline} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} import org.apache.spark.ml.feature.HashingTF +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{ParamMap, ParamPair} import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.types.StructType class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -69,7 +69,7 @@ class CrossValidatorSuite test("cross validation with linear regression") { val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index f8d3de19b0f8..c1e9c2fc1dc1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -20,15 +20,15 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.StructType class TrainValidationSplitSuite @@ -60,7 +60,7 @@ class TrainValidationSplitSuite test("train validation with linear regression") { val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index ad7d2c9b8d40..6aae625fc83f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -20,9 +20,9 @@ package org.apache.spark.ml.util import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree.impl.TreeTests -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index b7df02e6c098..8c5b4bda2518 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -560,4 +560,55 @@ class MatricesSuite extends SparkFunSuite { compare(oldSM0, newSM0) compare(oldDM0, newDM0) } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibMatrixToTriple(m: Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibDenseMatrixToTriple(m: DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibSparseMatrixToTriple(m: SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlMatrixToTriple(m: newlinalg.Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlDenseMatrixToTriple(m: newlinalg.DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlSparseMatrixToTriple(m: newlinalg.SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def compare(m1: (Array[Double], Int, Int), m2: (Array[Double], Int, Int)): Unit = { + assert(m1._1 === m2._1) + assert(m1._2 === m2._2) + assert(m1._3 === m2._3) + } + + val dm: DenseMatrix = new DenseMatrix(3, 2, Array(0.0, 0.0, 1.0, 0.0, 2.0, 3.5)) + val sm: SparseMatrix = dm.toSparse + val sm0: Matrix = sm.asInstanceOf[Matrix] + val dm0: Matrix = dm.asInstanceOf[Matrix] + + val newSM: newlinalg.SparseMatrix = sm.asML + val newDM: newlinalg.DenseMatrix = dm.asML + val newSM0: newlinalg.Matrix = sm0.asML + val newDM0: newlinalg.Matrix = dm0.asML + + import org.apache.spark.mllib.linalg.MatrixImplicits._ + + compare(mllibMatrixToTriple(dm0), mllibMatrixToTriple(newDM0)) + compare(mllibMatrixToTriple(sm0), mllibMatrixToTriple(newSM0)) + + compare(mllibDenseMatrixToTriple(dm), mllibDenseMatrixToTriple(newDM)) + compare(mllibSparseMatrixToTriple(sm), mllibSparseMatrixToTriple(newSM)) + + compare(mlMatrixToTriple(dm0), mlMatrixToTriple(newDM)) + compare(mlMatrixToTriple(sm0), mlMatrixToTriple(newSM0)) + + compare(mlDenseMatrixToTriple(dm), mlDenseMatrixToTriple(newDM)) + compare(mlSparseMatrixToTriple(sm), mlSparseMatrixToTriple(newSM)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index a7c1a076044e..2e9c40ab88ed 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -422,4 +422,43 @@ class VectorsSuite extends SparkFunSuite with Logging { assert(oldSV0.toArray === newSV0.toArray) assert(oldDV0.toArray === newDV0.toArray) } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibVectorToArray(v: Vector): Array[Double] = v.toArray + + def mllibDenseVectorToArray(v: DenseVector): Array[Double] = v.toArray + + def mllibSparseVectorToArray(v: SparseVector): Array[Double] = v.toArray + + def mlVectorToArray(v: newlinalg.Vector): Array[Double] = v.toArray + + def mlDenseVectorToArray(v: newlinalg.DenseVector): Array[Double] = v.toArray + + def mlSparseVectorToArray(v: newlinalg.SparseVector): Array[Double] = v.toArray + + val dv: DenseVector = new DenseVector(Array(1.0, 2.0, 3.5)) + val sv: SparseVector = new SparseVector(5, Array(1, 2, 4), Array(1.1, 2.2, 4.4)) + val sv0: Vector = sv.asInstanceOf[Vector] + val dv0: Vector = dv.asInstanceOf[Vector] + + val newSV: newlinalg.SparseVector = sv.asML + val newDV: newlinalg.DenseVector = dv.asML + val newSV0: newlinalg.Vector = sv0.asML + val newDV0: newlinalg.Vector = dv0.asML + + import org.apache.spark.mllib.linalg.VectorImplicits._ + + assert(mllibVectorToArray(dv0) === mllibVectorToArray(newDV0)) + assert(mllibVectorToArray(sv0) === mllibVectorToArray(newSV0)) + + assert(mllibDenseVectorToArray(dv) === mllibDenseVectorToArray(newDV)) + assert(mllibSparseVectorToArray(sv) === mllibSparseVectorToArray(newSV)) + + assert(mlVectorToArray(dv0) === mlVectorToArray(newDV0)) + assert(mlVectorToArray(sv0) === mlVectorToArray(newSV0)) + + assert(mlDenseVectorToArray(dv) === mlDenseVectorToArray(newDV)) + assert(mlSparseVectorToArray(sv) === mlSparseVectorToArray(newSV)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index f8d0af8820e6..252a068dcd72 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors class LabeledPointSuite extends SparkFunSuite { @@ -40,4 +41,16 @@ class LabeledPointSuite extends SparkFunSuite { val point = LabeledPoint.parse("1.0,1.0 0.0 -2.0") assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.0))) } + + test("conversions between new ml LabeledPoint and mllib LabeledPoint") { + val points: Seq[LabeledPoint] = Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) + + val newPoints: Seq[NewLabeledPoint] = points.map(_.asML) + + points.zip(newPoints).foreach { case (p1, p2) => + assert(p1 === LabeledPoint.fromML(p2)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 49cb7e1f24e3..441d0f7614bf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -73,7 +73,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -100,7 +100,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1 -> 2)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -116,7 +116,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -133,7 +133,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -150,7 +150,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -167,7 +167,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -183,7 +183,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(strategy.isMulticlassClassification) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -240,7 +240,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = maxBins, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -288,7 +288,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) @@ -310,7 +310,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1a02f660fddf..45f729704838 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -716,6 +716,52 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") + ) ++ Seq( + // [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.getOldDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.estimatedDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.topicsMatrix"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.clusterCenters"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.decodeLabel"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.encodeLabeledPoint"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.weights"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.predict"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.theta"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.pi"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.probability2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.getScalingVec"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.setScalingVec"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.PCAModel.pc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMax"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMin"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.IDFModel.idf"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.mean"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.std"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predictQuantiles"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.boundaries"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.this") ) ++ Seq( // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 5c11aa71b459..a1c3f7298409 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -53,7 +53,7 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti Currently, this class only supports binary classification. >>> from pyspark.sql import Row - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sc.parallelize([ ... Row(label=1.0, weight=2.0, features=Vectors.dense(1.0)), ... Row(label=0.0, weight=2.0, features=Vectors.sparse(1, [], []))]).toDF() @@ -496,7 +496,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred It supports both binary and multiclass labels, as well as both continuous and categorical features. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), @@ -625,7 +625,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> import numpy >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), @@ -752,7 +752,7 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol `SPARK-4240 `_ >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), @@ -884,7 +884,7 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H The input feature values must be nonnegative. >>> from pyspark.sql import Row - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... Row(label=0.0, features=Vectors.dense([0.0, 0.0])), ... Row(label=0.0, features=Vectors.dense([0.0, 1.0])), @@ -1028,7 +1028,7 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, Number of inputs has to be equal to the size of feature vectors. Number of outputs has to be equal to the total number of labels. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (0.0, Vectors.dense([0.0, 0.0])), ... (1.0, Vectors.dense([0.0, 1.0])), @@ -1193,7 +1193,7 @@ class OneVsRest(Estimator, OneVsRestParams, MLReadable, MLWritable): is picked to label the example. >>> from pyspark.sql import Row - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sc.parallelize([ ... Row(label=0.0, features=Vectors.dense(1.0, 0.8)), ... Row(label=1.0, features=Vectors.sparse(2, [], [])), diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 921633164bac..ac7183d2efff 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -65,7 +65,7 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte GaussianMixture clustering. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([-0.1, -0.05 ]),), ... (Vectors.dense([-0.01, -0.1]),), @@ -194,7 +194,7 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol K-means clustering with a k-means++ like initialization mode (the k-means|| algorithm by Bahmani et al). - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] >>> df = sqlContext.createDataFrame(data, ["features"]) @@ -347,7 +347,7 @@ class BisectingKMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte If bisecting all divisible clusters on the bottom level would result more than `k` leaf clusters, larger clusters get higher priority. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] >>> df = sqlContext.createDataFrame(data, ["features"]) @@ -625,7 +625,7 @@ class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInter :py:class:`pyspark.ml.feature.Tokenizer` and :py:class:`pyspark.ml.feature.CountVectorizer` can be useful for converting text to word count vectors. - >>> from pyspark.mllib.linalg import Vectors, SparseVector + >>> from pyspark.ml.linalg import Vectors, SparseVector >>> from pyspark.ml.clustering import LDA >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], ... [2, SparseVector(2, {0: 1.0})],], ["id", "features"]) diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 719c0c7d79fc..fc9099b7ec17 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -111,7 +111,7 @@ class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPrediction The rawPrediction column can be of type double (binary 0/1 prediction, or probability of label 1) or of type vector (length-2 vector of raw predictions, scores, or label probabilities). - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> scoreAndLabels = map(lambda x: (Vectors.dense([1.0 - x[0], x[0]]), x[1]), ... [(0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)]) >>> dataset = sqlContext.createDataFrame(scoreAndLabels, ["raw", "label"]) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 606a6e7c22b4..983b6a5301ae 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -23,11 +23,11 @@ from pyspark import since, keyword_only from pyspark.rdd import ignore_unicode_prefix +from pyspark.ml.linalg import _convert_to_vector from pyspark.ml.param.shared import * from pyspark.ml.util import JavaMLReadable, JavaMLWritable from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer, _jvm from pyspark.mllib.common import inherit_doc -from pyspark.mllib.linalg import _convert_to_vector __all__ = ['Binarizer', 'Bucketizer', @@ -380,7 +380,7 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit .. seealso:: `More information on Wikipedia \ `_. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df1 = sqlContext.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) >>> dct = DCT(inverse=False, inputCol="vec", outputCol="resultVec") >>> df2 = dct.transform(df1) @@ -447,7 +447,7 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada with a provided "weight" vector. In other words, it scales each column of the dataset by a scalar multiplier. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([2.0, 1.0, 3.0]),)], ["values"]) >>> ep = ElementwiseProduct(scalingVec=Vectors.dense([1.0, 2.0, 3.0]), ... inputCol="values", outputCol="eprod") @@ -582,7 +582,7 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab Compute the Inverse Document Frequency (IDF) given a collection of documents. - >>> from pyspark.mllib.linalg import DenseVector + >>> from pyspark.ml.linalg import DenseVector >>> df = sqlContext.createDataFrame([(DenseVector([1.0, 2.0]),), ... (DenseVector([0.0, 1.0]),), (DenseVector([3.0, 0.2]),)], ["tf"]) >>> idf = IDF(minDocFreq=3, inputCol="tf", outputCol="idf") @@ -670,7 +670,7 @@ class MaxAbsScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav absolute value in each feature. It does not shift/center the data, and thus does not destroy any sparsity. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> maScaler = MaxAbsScaler(inputCol="a", outputCol="scaled") >>> model = maScaler.fit(df) @@ -757,7 +757,7 @@ class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav Note that since zero values will probably be transformed to non-zero values, output of the transformer will be DenseVector even for sparse input. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> mmScaler = MinMaxScaler(inputCol="a", outputCol="scaled") >>> model = mmScaler.fit(df) @@ -961,7 +961,7 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav Normalize a vector to have unit norm using the given p-norm. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> svec = Vectors.sparse(4, {1: 4.0, 3: 3.0}) >>> df = sqlContext.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) >>> normalizer = Normalizer(p=2.0, inputCol="dense", outputCol="features") @@ -1114,7 +1114,7 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead multiplication distributes over addition". Take a 2-variable feature vector as an example: `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) >>> px = PolynomialExpansion(degree=2, inputCol="dense", outputCol="expanded") >>> px.transform(df).head().expanded @@ -1459,7 +1459,7 @@ class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, J Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the training set. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") >>> model = standardScaler.fit(df) @@ -1942,7 +1942,7 @@ class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Ja - Add warning if a categorical feature has only 1 category. - Add option for allowing unknown categories. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([-1.0, 0.0]),), ... (Vectors.dense([0.0, 1.0]),), (Vectors.dense([0.0, 2.0]),)], ["a"]) >>> indexer = VectorIndexer(maxCategories=2, inputCol="a", outputCol="indexed") @@ -2062,7 +2062,7 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J The output vector will order features with the selected indices first (in the order given), followed by the selected names (in the order given). - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (Vectors.dense([-2.0, 2.3, 0.0, 0.0, 1.0]),), ... (Vectors.dense([0.0, 0.0, 0.0, 0.0, 0.0]),), @@ -2329,7 +2329,7 @@ class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab PCA trains a model to project vectors to a low-dimensional space using PCA. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), ... (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), ... (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] @@ -2547,7 +2547,7 @@ class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, Ja Chi-Squared feature selection, which selects categorical features to use for predicting a categorical label. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame( ... [(Vectors.dense([0.0, 0.0, 18.0, 1.0]), 1.0), ... (Vectors.dense([0.0, 1.0, 12.0, 0.0]), 0.0), diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index d9513ca5b273..ade4864e1d78 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -29,8 +29,8 @@ from py4j.java_gateway import JavaObject from pyspark import since +from pyspark.ml.linalg import DenseVector, Vector from pyspark.ml.util import Identifiable -from pyspark.mllib.linalg import DenseVector, Vector __all__ = ['Param', 'Params', 'TypeConverters'] diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index fcdc29e69b1d..cfcbbfc98e82 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -54,7 +54,7 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction - L1 (Lasso) - L2 + L1 (elastic net) - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, 2.0, Vectors.dense(1.0)), ... (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) @@ -412,7 +412,7 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti Currently implemented using parallelized pool adjacent violators algorithm. Only univariate (single feature) algorithm supported. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -642,7 +642,7 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi learning algorithm for regression. It supports both continuous and categorical features. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -808,7 +808,7 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi It supports both continuous and categorical features. >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -920,7 +920,7 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, It supports both continuous and categorical features. >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -1055,7 +1055,7 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi .. seealso:: `AFT Model `_ - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0), 1.0), ... (0.0, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) @@ -1252,7 +1252,7 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha .. seealso:: `GLM `_ - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(0.0, 0.0)), ... (1.0, Vectors.dense(1.0, 2.0)), diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index c5679057597e..e3511120bdec 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -62,10 +62,6 @@ from pyspark.ml.tuning import * from pyspark.ml.wrapper import JavaParams from pyspark.mllib.common import _java2py -from pyspark.mllib.linalg import SparseVector as OldSparseVector, DenseVector as OldDenseVector,\ - DenseMatrix as OldDenseMatrix, MatrixUDT as OldMatrixUDT, SparseMatrix as OldSparseMatrix,\ - Vectors as OldVectors, VectorUDT as OldVectorUDT -from pyspark.mllib.regression import LabeledPoint from pyspark.serializers import PickleSerializer from pyspark.sql import DataFrame, Row, SparkSession from pyspark.sql.functions import rand @@ -162,22 +158,22 @@ def test_float(self): def test_vector(self): ewp = ElementwiseProduct(scalingVec=[1, 3]) - self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.0, 3.0])) + self.assertEqual(ewp.getScalingVec(), DenseVector([1.0, 3.0])) ewp = ElementwiseProduct(scalingVec=np.array([1.2, 3.4])) - self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.2, 3.4])) + self.assertEqual(ewp.getScalingVec(), DenseVector([1.2, 3.4])) self.assertRaises(TypeError, lambda: ElementwiseProduct(scalingVec=["a", "b"])) def test_list(self): l = [0, 1] - for lst_like in [l, np.array(l), OldDenseVector(l), OldSparseVector(len(l), + for lst_like in [l, np.array(l), DenseVector(l), SparseVector(len(l), range(len(l)), l), pyarray.array('l', l), xrange(2), tuple(l)]: converted = TypeConverters.toList(lst_like) self.assertEqual(type(converted), list) self.assertListEqual(converted, l) def test_list_int(self): - for indices in [[1.0, 2.0], np.array([1.0, 2.0]), OldDenseVector([1.0, 2.0]), - OldSparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), + for indices in [[1.0, 2.0], np.array([1.0, 2.0]), DenseVector([1.0, 2.0]), + SparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), pyarray.array('d', [1.0, 2.0])]: vs = VectorSlicer(indices=indices) self.assertListEqual(vs.getIndices(), [1, 2]) @@ -410,9 +406,9 @@ def test_binarizer(self): def test_idf(self): dataset = self.spark.createDataFrame([ - (OldDenseVector([1.0, 2.0]),), - (OldDenseVector([0.0, 1.0]),), - (OldDenseVector([3.0, 0.2]),)], ["tf"]) + (DenseVector([1.0, 2.0]),), + (DenseVector([0.0, 1.0]),), + (DenseVector([3.0, 0.2]),)], ["tf"]) idf0 = IDF(inputCol="tf") self.assertListEqual(idf0.params, [idf0.inputCol, idf0.minDocFreq, idf0.outputCol]) idf0m = idf0.fit(dataset, {idf0.outputCol: "idf"}) @@ -457,10 +453,10 @@ def test_stopwordsremover(self): def test_count_vectorizer_with_binary(self): dataset = self.spark.createDataFrame([ - (0, "a a a b b c".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), - (1, "a a".split(' '), OldSparseVector(3, {0: 1.0}),), - (2, "a b".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0}),), - (3, "c".split(' '), OldSparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) + (0, "a a a b b c".split(' '), SparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), + (1, "a a".split(' '), SparseVector(3, {0: 1.0}),), + (2, "a b".split(' '), SparseVector(3, {0: 1.0, 1: 1.0}),), + (3, "c".split(' '), SparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) cv = CountVectorizer(binary=True, inputCol="words", outputCol="features") model = cv.fit(dataset) @@ -581,11 +577,11 @@ def test_save_load(self): # Save/load for CrossValidator will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(OldVectors.dense([0.0]), 0.0), - (OldVectors.dense([0.4]), 1.0), - (OldVectors.dense([0.5]), 0.0), - (OldVectors.dense([0.6]), 1.0), - (OldVectors.dense([1.0]), 1.0)] * 10, + [(Vectors.dense([0.0]), 0.0), + (Vectors.dense([0.4]), 1.0), + (Vectors.dense([0.5]), 0.0), + (Vectors.dense([0.6]), 1.0), + (Vectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -654,11 +650,11 @@ def test_save_load(self): # Save/load for TrainValidationSplit will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(OldVectors.dense([0.0]), 0.0), - (OldVectors.dense([0.4]), 1.0), - (OldVectors.dense([0.5]), 0.0), - (OldVectors.dense([0.6]), 1.0), - (OldVectors.dense([1.0]), 1.0)] * 10, + [(Vectors.dense([0.0]), 0.0), + (Vectors.dense([0.4]), 1.0), + (Vectors.dense([0.5]), 0.0), + (Vectors.dense([0.6]), 1.0), + (Vectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -857,8 +853,8 @@ def _compare(self, m1, m2): def test_persistence(self): # Test save/load for LDA, LocalLDAModel, DistributedLDAModel. df = self.spark.createDataFrame([ - [1, OldVectors.dense([0.0, 1.0])], - [2, OldVectors.sparse(2, {0: 1.0})], + [1, Vectors.dense([0.0, 1.0])], + [2, Vectors.sparse(2, {0: 1.0})], ], ["id", "features"]) # Fit model lda = LDA(k=2, seed=1, optimizer="em") @@ -893,8 +889,8 @@ def test_persistence(self): class TrainingSummaryTest(SparkSessionTestCase): def test_linear_regression_summary(self): - df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), - (0.0, 2.0, OldVectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight", fitIntercept=False) @@ -930,7 +926,7 @@ def test_linear_regression_summary(self): self.assertAlmostEqual(sameSummary.explainedVariance, s.explainedVariance) def test_glr_summary(self): - from pyspark.mllib.linalg import Vectors + from pyspark.ml.linalg import Vectors df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) @@ -966,8 +962,8 @@ def test_glr_summary(self): self.assertAlmostEqual(sameSummary.deviance, s.deviance) def test_logistic_regression_summary(self): - df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), - (0.0, 2.0, OldVectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) model = lr.fit(df) @@ -996,9 +992,9 @@ def test_logistic_regression_summary(self): class OneVsRestTests(SparkSessionTestCase): def test_copy(self): - df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), - (1.0, OldVectors.sparse(2, [], [])), - (2.0, OldVectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1010,9 +1006,9 @@ def test_copy(self): self.assertEqual(model1.getPredictionCol(), "indexed") def test_output_columns(self): - df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), - (1.0, OldVectors.sparse(2, [], [])), - (2.0, OldVectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1022,9 +1018,9 @@ def test_output_columns(self): def test_save_load(self): temp_path = tempfile.mkdtemp() - df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), - (1.0, OldVectors.sparse(2, [], [])), - (2.0, OldVectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1052,7 +1048,7 @@ def test_apply_binary_term_freqs(self): hashingTF.setInputCol("words").setOutputCol("features").setNumFeatures(n).setBinary(True) output = hashingTF.transform(df) features = output.select("features").first().features.toArray() - expected = OldVectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() + expected = Vectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() for i in range(0, n): self.assertAlmostEqual(features[i], expected[i], 14, "Error at " + str(i) + ": expected " + str(expected[i]) + ", got " + str(features[i])) @@ -1147,15 +1143,13 @@ def _test_serialize(self, v): self.assertEqual(vs, nvs) def test_serialize(self): - # Because pickle path still uses old vector/matrix - # TODO: Change this to new vector/matrix when pickle for new vector/matrix is ready. - self._test_serialize(OldDenseVector(range(10))) - self._test_serialize(OldDenseVector(array([1., 2., 3., 4.]))) - self._test_serialize(OldDenseVector(pyarray.array('d', range(10)))) - self._test_serialize(OldSparseVector(4, {1: 1, 3: 2})) - self._test_serialize(OldSparseVector(3, {})) - self._test_serialize(OldDenseMatrix(2, 3, range(6))) - sm1 = OldSparseMatrix( + self._test_serialize(DenseVector(range(10))) + self._test_serialize(DenseVector(array([1., 2., 3., 4.]))) + self._test_serialize(DenseVector(pyarray.array('d', range(10)))) + self._test_serialize(SparseVector(4, {1: 1, 3: 2})) + self._test_serialize(SparseVector(3, {})) + self._test_serialize(DenseMatrix(2, 3, range(6))) + sm1 = SparseMatrix( 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) self._test_serialize(sm1) @@ -1407,12 +1401,6 @@ class VectorUDTTests(MLlibTestCase): sv1 = SparseVector(2, [1], [2.0]) udt = VectorUDT() - old_dv0 = OldDenseVector([]) - old_dv1 = OldDenseVector([1.0, 2.0]) - old_sv0 = OldSparseVector(2, [], []) - old_sv1 = OldSparseVector(2, [1], [2.0]) - old_udt = OldVectorUDT() - def test_json_schema(self): self.assertEqual(VectorUDT.fromJson(self.udt.jsonValue()), self.udt) @@ -1421,19 +1409,19 @@ def test_serialization(self): self.assertEqual(v, self.udt.deserialize(self.udt.serialize(v))) def test_infer_schema(self): - rdd = self.sc.parallelize([LabeledPoint(1.0, self.old_dv1), - LabeledPoint(0.0, self.old_sv1)]) + rdd = self.sc.parallelize([Row(label=1.0, features=self.dv1), + Row(label=0.0, features=self.sv1)]) df = rdd.toDF() schema = df.schema field = [f for f in schema.fields if f.name == "features"][0] - self.assertEqual(field.dataType, self.old_udt) + self.assertEqual(field.dataType, self.udt) vectors = df.rdd.map(lambda p: p.features).collect() self.assertEqual(len(vectors), 2) for v in vectors: - if isinstance(v, OldSparseVector): - self.assertEqual(v, self.old_sv1) - elif isinstance(v, OldDenseVector): - self.assertEqual(v, self.old_dv1) + if isinstance(v, SparseVector): + self.assertEqual(v, self.sv1) + elif isinstance(v, DenseVector): + self.assertEqual(v, self.dv1) else: raise TypeError("expecting a vector but got %r of type %r" % (v, type(v))) @@ -1446,12 +1434,6 @@ class MatrixUDTTests(MLlibTestCase): sm2 = SparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) udt = MatrixUDT() - old_dm1 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10]) - old_dm2 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10], isTransposed=True) - old_sm1 = OldSparseMatrix(1, 1, [0, 1], [0], [2.0]) - old_sm2 = OldSparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) - old_udt = OldMatrixUDT() - def test_json_schema(self): self.assertEqual(MatrixUDT.fromJson(self.udt.jsonValue()), self.udt) @@ -1460,17 +1442,17 @@ def test_serialization(self): self.assertEqual(m, self.udt.deserialize(self.udt.serialize(m))) def test_infer_schema(self): - rdd = self.sc.parallelize([("dense", self.old_dm1), ("sparse", self.old_sm1)]) + rdd = self.sc.parallelize([("dense", self.dm1), ("sparse", self.sm1)]) df = rdd.toDF() schema = df.schema - self.assertTrue(schema.fields[1].dataType, self.old_udt) + self.assertTrue(schema.fields[1].dataType, self.udt) matrices = df.rdd.map(lambda x: x._2).collect() self.assertEqual(len(matrices), 2) for m in matrices: - if isinstance(m, OldDenseMatrix): - self.assertTrue(m, self.old_dm1) - elif isinstance(m, OldSparseMatrix): - self.assertTrue(m, self.old_sm1) + if isinstance(m, DenseMatrix): + self.assertTrue(m, self.dm1) + elif isinstance(m, SparseMatrix): + self.assertTrue(m, self.sm1) else: raise ValueError("Expected a matrix but got type %r" % type(m)) diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 0920ae6ea144..75789c4d0918 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -151,7 +151,7 @@ class CrossValidator(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> dataset = sqlContext.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), @@ -310,7 +310,7 @@ class TrainValidationSplit(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> dataset = sqlContext.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), From c0bb77132b9acac951074fd623892abafeb02512 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 May 2016 13:05:07 -0700 Subject: [PATCH 242/313] [SPARK-15244] [PYTHON] Type of column name created with createDataFrame is not consistent. ## What changes were proposed in this pull request? **createDataFrame** returns inconsistent types for column names. ```python >>> from pyspark.sql.types import StructType, StructField, StringType >>> schema = StructType([StructField(u"col", StringType())]) >>> df1 = spark.createDataFrame([("a",)], schema) >>> df1.columns # "col" is str ['col'] >>> df2 = spark.createDataFrame([("a",)], [u"col"]) >>> df2.columns # "col" is unicode [u'col'] ``` The reason is only **StructField** has the following code. ``` if not isinstance(name, str): name = name.encode('utf-8') ``` This PR adds the same logic into **createDataFrame** for consistency. ``` if isinstance(schema, list): schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] ``` ## How was this patch tested? Pass the Jenkins test (with new python doctest) Author: Dongjoon Hyun Closes #13097 from dongjoon-hyun/SPARK-15244. (cherry picked from commit 0f576a5748244f7e874b925f8d841f1ca238f087) Signed-off-by: Davies Liu --- python/pyspark/sql/session.py | 2 ++ python/pyspark/sql/tests.py | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index ae314359d512..0781b442cbcd 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -465,6 +465,8 @@ def prepare(obj): return (obj, ) schema = StructType().add("value", datatype) else: + if isinstance(schema, list): + schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] prepare = lambda obj: obj if isinstance(data, RDD): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 0c73f58c3b24..0977c43a398d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -228,6 +228,13 @@ def test_duplicated_column_names(self): self.assertRaises(AnalysisException, lambda: df.select(df.c).first()) self.assertRaises(AnalysisException, lambda: df.select(df["c"]).first()) + def test_column_name_encoding(self): + """Ensure that created columns has `str` type consistently.""" + columns = self.spark.createDataFrame([('Alice', 1)], ['name', u'age']).columns + self.assertEqual(columns, ['name', 'age']) + self.assertTrue(isinstance(columns[0], str)) + self.assertTrue(isinstance(columns[1], str)) + def test_explode(self): from pyspark.sql.functions import explode d = [Row(a=1, intlist=[1, 2, 3], mapfield={"a": "b"})] From 7b62b7c1180dc3414b7d7e07561d0e6b89ff2b37 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 17 May 2016 14:57:21 -0700 Subject: [PATCH 243/313] [SPARK-11735][CORE][SQL] Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped ## What changes were proposed in this pull request? Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13154 from zsxwing/check-spark-context-stop. (cherry picked from commit 8e8bc9f957de6c0aefbc6ef4b18c421b486477a6) Signed-off-by: Yin Huai --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 ++ sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e39159933607..e6cdd0d298f3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -94,7 +94,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] val stopped: AtomicBoolean = new AtomicBoolean(false) - private def assertNotStopped(): Unit = { + private[spark] def assertNotStopped(): Unit = { if (stopped.get()) { val activeContext = SparkContext.activeContext.get() val activeCreationSite = 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 c64e284635ec..44511885a7db 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 @@ -67,6 +67,8 @@ class SQLContext private[sql]( self => + sparkSession.sparkContext.assertNotStopped() + // Note: Since Spark 2.0 this class has become a wrapper of SparkSession, where the // real functionality resides. This class remains mainly for backward compatibility. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 100b43f6b58f..aa974f29cc34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -68,6 +68,7 @@ class SparkSession private( this(sc, None) } + sparkContext.assertNotStopped() /* ----------------------- * | Session-related state | From 2dddec40d6562d1d16bb242bf7dc730431ee1e3e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 17 May 2016 15:56:44 -0700 Subject: [PATCH 244/313] [SPARK-14346][SQL] Native SHOW CREATE TABLE for Hive tables/views ## What changes were proposed in this pull request? This is a follow-up of #12781. It adds native `SHOW CREATE TABLE` support for Hive tables and views. A new field `hasUnsupportedFeatures` is added to `CatalogTable` to indicate whether all table metadata retrieved from the concrete underlying external catalog (i.e. Hive metastore in this case) can be mapped to fields in `CatalogTable`. This flag is useful when the target Hive table contains structures that can't be handled by Spark SQL, e.g., skewed columns and storage handler, etc.. ## How was this patch tested? New test cases are added in `ShowCreateTableSuite` to do round-trip tests. Author: Cheng Lian Closes #13079 from liancheng/spark-14346-show-create-table-for-hive-tables. (cherry picked from commit b674e67c22bf663334e537e35787c00533adbb04) Signed-off-by: Yin Huai --- .../sql/catalyst/catalog/interface.scala | 12 +- .../spark/sql/execution/command/tables.scala | 184 ++++++++++++----- .../sql/hive/client/HiveClientImpl.scala | 10 +- .../spark/sql/hive/ShowCreateTableSuite.scala | 185 +++++++++++++++++- 4 files changed, 333 insertions(+), 58 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index d21565526ec0..d4f5cbb62585 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -79,6 +79,12 @@ case class CatalogTablePartition( * * Note that Hive's metastore also tracks skewed columns. We should consider adding that in the * future once we have a better understanding of how we want to handle skewed columns. + * + * @param hasUnsupportedFeatures is used to indicate whether all table metadata entries retrieved + * from the concrete underlying external catalog (e.g. Hive metastore) are supported by + * Spark SQL. For example, if the underlying Hive table has skewed columns, this information + * can't be mapped to [[CatalogTable]] since Spark SQL doesn't handle skewed columns for now. + * In this case `hasUnsupportedFeatures` is set to true. By default, it is false. */ case class CatalogTable( identifier: TableIdentifier, @@ -95,7 +101,8 @@ case class CatalogTable( properties: Map[String, String] = Map.empty, viewOriginalText: Option[String] = None, viewText: Option[String] = None, - comment: Option[String] = None) { + comment: Option[String] = None, + hasUnsupportedFeatures: Boolean = false) { // Verify that the provided columns are part of the schema private val colNames = schema.map(_.name).toSet @@ -200,6 +207,7 @@ case class SimpleCatalogRelation( } } - require(metadata.identifier.database == Some(databaseName), + require( + metadata.identifier.database.contains(databaseName), "provided database does not match the one specified in the table definition") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index bb4f1ff4f75f..1fc02d1d4b46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -626,40 +626,149 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { showCreateDataSourceTable(tableMetadata) } else { - throw new UnsupportedOperationException( - "SHOW CREATE TABLE only supports Spark SQL data source tables.") + showCreateHiveTable(tableMetadata) } Seq(Row(stmt)) } + private def showCreateHiveTable(metadata: CatalogTable): String = { + def reportUnsupportedError(): Unit = { + throw new UnsupportedOperationException( + s"Failed to execute SHOW CREATE TABLE against table ${metadata.identifier.quotedString}, " + + "because it contains table structure(s) (e.g. skewed columns) that Spark SQL doesn't " + + "support yet." + ) + } + + if (metadata.hasUnsupportedFeatures) { + reportUnsupportedError() + } + + val builder = StringBuilder.newBuilder + + val tableTypeString = metadata.tableType match { + case EXTERNAL => " EXTERNAL TABLE" + case VIEW => " VIEW" + case MANAGED => " TABLE" + case INDEX => reportUnsupportedError() + } + + builder ++= s"CREATE$tableTypeString ${table.quotedString}" + + if (metadata.tableType == VIEW) { + if (metadata.schema.nonEmpty) { + builder ++= metadata.schema.map(_.name).mkString("(", ", ", ")") + } + builder ++= metadata.viewText.mkString(" AS\n", "", "\n") + } else { + showHiveTableHeader(metadata, builder) + showHiveTableNonDataColumns(metadata, builder) + showHiveTableStorageInfo(metadata, builder) + showHiveTableProperties(metadata, builder) + } + + builder.toString() + } + + private def showHiveTableHeader(metadata: CatalogTable, builder: StringBuilder): Unit = { + val columns = metadata.schema.filterNot { column => + metadata.partitionColumnNames.contains(column.name) + }.map(columnToDDLFragment) + + if (columns.nonEmpty) { + builder ++= columns.mkString("(", ", ", ")\n") + } + + metadata + .comment + .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") + .foreach(builder.append) + } + + private def columnToDDLFragment(column: CatalogColumn): String = { + val comment = column.comment.map(escapeSingleQuotedString).map(" COMMENT '" + _ + "'") + s"${quoteIdentifier(column.name)} ${column.dataType}${comment.getOrElse("")}" + } + + private def showHiveTableNonDataColumns(metadata: CatalogTable, builder: StringBuilder): Unit = { + if (metadata.partitionColumns.nonEmpty) { + val partCols = metadata.partitionColumns.map(columnToDDLFragment) + builder ++= partCols.mkString("PARTITIONED BY (", ", ", ")\n") + } + + if (metadata.bucketColumnNames.nonEmpty) { + throw new UnsupportedOperationException( + "Creating Hive table with bucket spec is not supported yet.") + } + } + + private def showHiveTableStorageInfo(metadata: CatalogTable, builder: StringBuilder): Unit = { + val storage = metadata.storage + + storage.serde.foreach { serde => + builder ++= s"ROW FORMAT SERDE '$serde'\n" + + val serdeProps = metadata.storage.serdeProperties.map { + case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + + builder ++= serdeProps.mkString("WITH SERDEPROPERTIES (", ",\n ", "\n)\n") + } + + if (storage.inputFormat.isDefined || storage.outputFormat.isDefined) { + builder ++= "STORED AS\n" + + storage.inputFormat.foreach { format => + builder ++= s" INPUTFORMAT '${escapeSingleQuotedString(format)}'\n" + } + + storage.outputFormat.foreach { format => + builder ++= s" OUTPUTFORMAT '${escapeSingleQuotedString(format)}'\n" + } + } + + if (metadata.tableType == EXTERNAL) { + storage.locationUri.foreach { uri => + builder ++= s"LOCATION '$uri'\n" + } + } + } + + private def showHiveTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { + if (metadata.properties.nonEmpty) { + val filteredProps = metadata.properties.filterNot { + // Skips "EXTERNAL" property for external tables + case (key, _) => key == "EXTERNAL" && metadata.tableType == EXTERNAL + } + + val props = filteredProps.map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + + if (props.nonEmpty) { + builder ++= props.mkString("TBLPROPERTIES (", ",\n ", ")\n") + } + } + } + private def showCreateDataSourceTable(metadata: CatalogTable): String = { val builder = StringBuilder.newBuilder builder ++= s"CREATE TABLE ${table.quotedString} " - showDataSourceTableDataCols(metadata, builder) + showDataSourceTableDataColumns(metadata, builder) showDataSourceTableOptions(metadata, builder) showDataSourceTableNonDataColumns(metadata, builder) builder.toString() } - private def showDataSourceTableDataCols(metadata: CatalogTable, builder: StringBuilder): Unit = { - val props = metadata.properties - val schemaParts = for { - numParts <- props.get("spark.sql.sources.schema.numParts").toSeq - index <- 0 until numParts.toInt - } yield props.getOrElse( - s"spark.sql.sources.schema.part.$index", - throw new AnalysisException( - s"Corrupted schema in catalog: $numParts parts expected, but part $index is missing." - ) - ) - - if (schemaParts.nonEmpty) { - val fields = DataType.fromJson(schemaParts.mkString).asInstanceOf[StructType].fields - val colTypeList = fields.map(f => s"${quoteIdentifier(f.name)} ${f.dataType.sql}") - builder ++= colTypeList.mkString("(", ", ", ")") + private def showDataSourceTableDataColumns( + metadata: CatalogTable, builder: StringBuilder): Unit = { + DDLUtils.getSchemaFromTableProperties(metadata).foreach { schema => + val columns = schema.fields.map(f => s"${quoteIdentifier(f.name)} ${f.dataType.sql}") + builder ++= columns.mkString("(", ", ", ")") } builder ++= "\n" @@ -688,40 +797,21 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman private def showDataSourceTableNonDataColumns( metadata: CatalogTable, builder: StringBuilder): Unit = { - val props = metadata.properties - - def getColumnNamesByType(colType: String, typeName: String): Seq[String] = { - (for { - numCols <- props.get(s"spark.sql.sources.schema.num${colType.capitalize}Cols").toSeq - index <- 0 until numCols.toInt - } yield props.getOrElse( - s"spark.sql.sources.schema.${colType}Col.$index", - throw new AnalysisException( - s"Corrupted $typeName in catalog: $numCols parts expected, but part $index is missing." - ) - )).map(quoteIdentifier) - } - - val partCols = getColumnNamesByType("part", "partitioning columns") + val partCols = DDLUtils.getPartitionColumnsFromTableProperties(metadata) if (partCols.nonEmpty) { builder ++= s"PARTITIONED BY ${partCols.mkString("(", ", ", ")")}\n" } - val bucketCols = getColumnNamesByType("bucket", "bucketing columns") - if (bucketCols.nonEmpty) { - builder ++= s"CLUSTERED BY ${bucketCols.mkString("(", ", ", ")")}\n" - - val sortCols = getColumnNamesByType("sort", "sorting columns") - if (sortCols.nonEmpty) { - builder ++= s"SORTED BY ${sortCols.mkString("(", ", ", ")")}\n" - } + DDLUtils.getBucketSpecFromTableProperties(metadata).foreach { spec => + if (spec.bucketColumnNames.nonEmpty) { + builder ++= s"CLUSTERED BY ${spec.bucketColumnNames.mkString("(", ", ", ")")}\n" - val numBuckets = props.getOrElse( - "spark.sql.sources.schema.numBuckets", - throw new AnalysisException("Corrupted bucket spec in catalog: missing bucket number") - ) + if (spec.sortColumnNames.nonEmpty) { + builder ++= s"SORTED BY ${spec.sortColumnNames.mkString("(", ", ", ")")}\n" + } - builder ++= s"INTO $numBuckets BUCKETS\n" + builder ++= s"INTO ${spec.numBuckets} BUCKETS\n" + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 78c457b6c2af..a4e9f03b4334 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -339,6 +339,13 @@ private[hive] class HiveClientImpl( // partition columns are part of the schema val partCols = h.getPartCols.asScala.map(fromHiveColumn) val schema = h.getCols.asScala.map(fromHiveColumn) ++ partCols + + // Skew spec, storage handler, and bucketing info can't be mapped to CatalogTable (yet) + val hasUnsupportedFeatures = + !h.getSkewedColNames.isEmpty || + h.getStorageHandler != null || + !h.getBucketCols.isEmpty + CatalogTable( identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), tableType = h.getTableType match { @@ -365,7 +372,8 @@ private[hive] class HiveClientImpl( ), properties = h.getParameters.asScala.toMap, viewOriginalText = Option(h.getViewOriginalText), - viewText = Option(h.getViewExpandedText)) + viewText = Option(h.getViewExpandedText), + hasUnsupportedFeatures = hasUnsupportedFeatures) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala index 12a1ad8987c1..3b8068d3bc47 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -116,24 +116,177 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing .bucketBy(2, "c", "d") .saveAsTable("ddl_test5") - checkCreateTable(TableIdentifier("ddl_test5", Some("default"))) + checkCreateTable("ddl_test5") } } + test("simple hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("simple external hive table") { + withTempDir { dir => + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |LOCATION '$dir' + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + } + + test("partitioned hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |COMMENT 'bla' + |PARTITIONED BY ( + | p1 BIGINT COMMENT 'bla', + | p2 STRING + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with explicit storage info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |COLLECTION ITEMS TERMINATED BY '@' + |MAP KEYS TERMINATED BY '#' + |NULL DEFINED AS 'NaN' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with STORED AS clause") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |STORED AS PARQUET + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with serde info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |WITH SERDEPROPERTIES ( + | 'mapkey.delim' = ',', + | 'field.delim' = ',' + |) + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive view") { + withView("v1") { + sql("CREATE VIEW v1 AS SELECT 1 AS a") + checkCreateView("v1") + } + } + + test("hive view with output columns") { + withView("v1") { + sql("CREATE VIEW v1 (b) AS SELECT 1 AS a") + checkCreateView("v1") + } + } + + test("hive bucketing not supported") { + withTable("t1") { + createRawHiveTable( + s"""CREATE TABLE t1 (a INT, b STRING) + |CLUSTERED BY (a) + |SORTED BY (b) + |INTO 2 BUCKETS + """.stripMargin + ) + + intercept[UnsupportedOperationException] { + sql("SHOW CREATE TABLE t1") + } + } + } + + private def createRawHiveTable(ddl: String): Unit = { + hiveContext.sharedState.metadataHive.runSqlHive(ddl) + } + private def checkCreateTable(table: String): Unit = { - checkCreateTable(TableIdentifier(table, Some("default"))) + checkCreateTableOrView(TableIdentifier(table, Some("default")), "TABLE") + } + + private def checkCreateView(table: String): Unit = { + checkCreateTableOrView(TableIdentifier(table, Some("default")), "VIEW") } - private def checkCreateTable(table: TableIdentifier): Unit = { + private def checkCreateTableOrView(table: TableIdentifier, checkType: String): Unit = { val db = table.database.getOrElse("default") val expected = spark.externalCatalog.getTable(db, table.table) val shownDDL = sql(s"SHOW CREATE TABLE ${table.quotedString}").head().getString(0) - sql(s"DROP TABLE ${table.quotedString}") + sql(s"DROP $checkType ${table.quotedString}") - withTable(table.table) { + try { sql(shownDDL) val actual = spark.externalCatalog.getTable(db, table.table) checkCatalogTables(expected, actual) + } finally { + sql(s"DROP $checkType IF EXISTS ${table.table}") } } @@ -155,15 +308,31 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing "totalSize", "totalNumberFiles", "maxFileSize", - "minFileSize" + "minFileSize", + // EXTERNAL is not non-deterministic, but it is filtered out for external tables. + "EXTERNAL" ) table.copy( createTime = 0L, lastAccessTime = 0L, - properties = table.properties.filterKeys(!nondeterministicProps.contains(_))) + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + // View texts are checked separately + viewOriginalText = None, + viewText = None + ) + } + + // Normalizes attributes auto-generated by Spark SQL for views + def normalizeGeneratedAttributes(str: String): String = { + str.replaceAll("gen_attr_[0-9]+", "gen_attr_0") + } + + // We use expanded canonical view text as original view text of the new table + assertResult(expected.viewText.map(normalizeGeneratedAttributes)) { + actual.viewOriginalText.map(normalizeGeneratedAttributes) } - assert(normalize(expected) == normalize(actual)) + assert(normalize(actual) == normalize(expected)) } } From 1db37417c25429c0001c19d2f10f4a314fe4585c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 17 May 2016 18:02:31 -0700 Subject: [PATCH 245/313] [SPARK-14346] Fix scala-2.10 build ## What changes were proposed in this pull request? Scala 2.10 build was broken by #13079. I am reverting the change of that line. Author: Yin Huai Closes #13157 from yhuai/SPARK-14346-fix-scala2.10. (cherry picked from commit 2a5db9c140b9d60a5ec91018be19bec7b80850ee) Signed-off-by: Yin Huai --- .../scala/org/apache/spark/sql/catalyst/catalog/interface.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index d4f5cbb62585..3fdd411ac4cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -208,6 +208,6 @@ case class SimpleCatalogRelation( } require( - metadata.identifier.database.contains(databaseName), + metadata.identifier.database == Some(databaseName), "provided database does not match the one specified in the table definition") } From 5f5270ead86d5294af6c871e36112e2a833e9d64 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 18 May 2016 09:01:59 +0800 Subject: [PATCH 246/313] [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable ## What changes were proposed in this pull request? Update the unit test code, examples, and documents to remove calls to deprecated method `dataset.registerTempTable`. ## How was this patch tested? This PR only changes the unit test code, examples, and comments. It should be safe. This is a follow up of PR https://github.com/apache/spark/pull/12945 which was merged. Author: Sean Zhong Closes #13098 from clockfly/spark-15171-remove-deprecation. (cherry picked from commit 25b315e6cad7c27b62dcaa2c194293c1115fdfb3) Signed-off-by: Cheng Lian --- docs/sql-programming-guide.md | 48 ++++---- docs/streaming-programming-guide.md | 12 +- .../spark/examples/sql/JavaSparkSQL.java | 8 +- .../streaming/JavaSqlNetworkWordCount.java | 2 +- examples/src/main/python/sql.py | 2 +- .../python/streaming/sql_network_wordcount.py | 2 +- .../spark/examples/sql/RDDRelation.scala | 6 +- .../examples/sql/hive/HiveFromSpark.scala | 4 +- .../streaming/SqlNetworkWordCount.scala | 2 +- .../apache/spark/ml/JavaPipelineSuite.java | 2 +- .../JavaLogisticRegressionSuite.java | 10 +- .../regression/JavaLinearRegressionSuite.java | 4 +- python/pyspark/sql/context.py | 4 +- python/pyspark/sql/readwriter.py | 2 +- python/pyspark/sql/session.py | 2 +- python/pyspark/sql/tests.py | 25 +++-- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/JavaApplySchemaSuite.java | 8 +- .../spark/sql/sources/JavaSaveLoadSuite.java | 2 +- .../apache/spark/sql/CachedTableSuite.scala | 60 +++++----- .../spark/sql/ColumnExpressionSuite.scala | 2 +- .../sql/DataFrameTimeWindowingSuite.scala | 2 +- .../spark/sql/DataFrameWindowSuite.scala | 22 ++-- .../org/apache/spark/sql/JoinSuite.scala | 4 +- .../apache/spark/sql/ListTablesSuite.scala | 4 +- .../apache/spark/sql/SQLContextSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 103 +++++++++--------- .../sql/ScalaReflectionRelationSuite.scala | 10 +- .../org/apache/spark/sql/SubquerySuite.scala | 8 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 12 +- .../spark/sql/UserDefinedTypeSuite.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 10 +- .../benchmark/AggregateBenchmark.scala | 3 +- .../columnar/InMemoryColumnarQuerySuite.scala | 8 +- .../columnar/PartitionBatchPruningSuite.scala | 2 +- .../datasources/json/JsonSuite.scala | 58 +++++----- .../ParquetPartitionDiscoverySuite.scala | 10 +- .../parquet/ParquetQuerySuite.scala | 4 +- .../parquet/ParquetReadBenchmark.scala | 20 ++-- .../datasources/parquet/TPCDSBenchmark.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 8 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../sources/CreateTableAsSelectSuite.scala | 2 +- .../spark/sql/sources/InsertSuite.scala | 6 +- .../spark/sql/sources/SaveLoadSuite.scala | 4 +- .../spark/sql/streaming/StreamSuite.scala | 2 +- .../apache/spark/sql/test/SQLTestData.scala | 46 ++++---- .../spark/sql/hive/JavaDataFrameSuite.java | 2 +- .../hive/JavaMetastoreDataSourcesSuite.java | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 4 +- .../spark/sql/hive/HiveParquetSuite.scala | 4 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 8 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 12 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 8 +- .../hive/ParquetHiveCompatibilitySuite.scala | 2 +- .../spark/sql/hive/QueryPartitionSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../org/apache/spark/sql/hive/UDFSuite.scala | 2 +- .../execution/AggregationQuerySuite.scala | 8 +- .../sql/hive/execution/HiveExplainSuite.scala | 2 +- .../HiveOperatorQueryableSuite.scala | 4 +- .../sql/hive/execution/HivePlanTest.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 18 +-- .../hive/execution/HiveResolutionSuite.scala | 10 +- .../hive/execution/HiveTableScanSuite.scala | 2 +- .../sql/hive/execution/HiveUDFSuite.scala | 22 ++-- .../sql/hive/execution/SQLQuerySuite.scala | 66 +++++------ .../execution/SQLWindowFunctionSuite.scala | 16 +-- .../hive/orc/OrcPartitionDiscoverySuite.scala | 8 +- .../spark/sql/hive/orc/OrcQuerySuite.scala | 12 +- .../spark/sql/hive/orc/OrcSourceSuite.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 9 +- .../sql/sources/HadoopFsRelationTest.scala | 10 +- .../hive/HiveContextCompatibilitySuite.scala | 4 +- 74 files changed, 407 insertions(+), 401 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a16a6bb1d93e..a9e1f9d5ce46 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -529,7 +529,7 @@ case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF() -people.registerTempTable("people") +people.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19") @@ -605,7 +605,7 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m // Apply a schema to an RDD of JavaBeans and register it as a table. DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); -schemaPeople.registerTempTable("people"); +schemaPeople.createOrReplaceTempView("people"); // SQL can be run over RDDs that have been registered as tables. DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -643,7 +643,7 @@ people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) # Infer the schema, and register the DataFrame as a table. schemaPeople = sqlContext.createDataFrame(people) -schemaPeople.registerTempTable("people") +schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -703,8 +703,8 @@ val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) // Apply the schema to the RDD. val peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema) -// Register the DataFrames as a table. -peopleDataFrame.registerTempTable("people") +// Creates a temporary view using the DataFrame. +peopleDataFrame.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val results = sqlContext.sql("SELECT name FROM people") @@ -771,10 +771,10 @@ JavaRDD rowRDD = people.map( // Apply the schema to the RDD. DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema); -// Register the DataFrame as a table. -peopleDataFrame.registerTempTable("people"); +// Creates a temporary view using the DataFrame. +peopleDataFrame.createOrReplaceTempView("people"); -// SQL can be run over RDDs that have been registered as tables. +// SQL can be run over a temporary view created using DataFrames. DataFrame results = sqlContext.sql("SELECT name FROM people"); // The results of SQL queries are DataFrames and support all the normal RDD operations. @@ -824,8 +824,8 @@ schema = StructType(fields) # Apply the schema to the RDD. schemaPeople = sqlContext.createDataFrame(people, schema) -# Register the DataFrame as a table. -schemaPeople.registerTempTable("people") +# Creates a temporary view using the DataFrame +schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. results = sqlContext.sql("SELECT name FROM people") @@ -844,7 +844,7 @@ for name in names.collect(): # Data Sources Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. -A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. +A DataFrame can be operated on as normal RDDs and can also be used to create a temporary view. Registering a DataFrame as a table allows you to run SQL queries over its data. This section describes the general methods for loading and saving data using the Spark Data Sources and then goes into specific options that are available for the built-in data sources. @@ -1072,8 +1072,8 @@ people.write.parquet("people.parquet") // The result of loading a Parquet file is also a DataFrame. val parquetFile = sqlContext.read.parquet("people.parquet") -//Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile") +// Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile") val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -1094,8 +1094,8 @@ schemaPeople.write().parquet("people.parquet"); // The result of loading a parquet file is also a DataFrame. DataFrame parquetFile = sqlContext.read().parquet("people.parquet"); -// Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); +// Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile"); DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { @@ -1120,8 +1120,8 @@ schemaPeople.write.parquet("people.parquet") # The result of loading a parquet file is also a DataFrame. parquetFile = sqlContext.read.parquet("people.parquet") -# Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); +# Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile"); teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) for teenName in teenNames.collect(): @@ -1144,7 +1144,7 @@ write.parquet(schemaPeople, "people.parquet") # The result of loading a parquet file is also a DataFrame. parquetFile <- read.parquet(sqlContext, "people.parquet") -# Parquet files can also be registered as tables and then used in SQL statements. +# Parquet files can also be used to create a temporary view and then used in SQL statements. registerTempTable(parquetFile, "parquetFile") teenagers <- sql(sqlContext, "SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") schema <- structType(structField("name", "string")) @@ -1506,8 +1506,8 @@ people.printSchema() // |-- age: long (nullable = true) // |-- name: string (nullable = true) -// Register this DataFrame as a table. -people.registerTempTable("people") +// Creates a temporary view using the DataFrame +people.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -1544,8 +1544,8 @@ people.printSchema(); // |-- age: long (nullable = true) // |-- name: string (nullable = true) -// Register this DataFrame as a table. -people.registerTempTable("people"); +// Creates a temporary view using the DataFrame +people.createOrReplaceTempView("people"); // SQL statements can be run by using the sql methods provided by sqlContext. DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -1582,8 +1582,8 @@ people.printSchema() # |-- age: long (nullable = true) # |-- name: string (nullable = true) -# Register this DataFrame as a table. -people.registerTempTable("people") +# Creates a temporary view using the DataFrame. +people.createOrReplaceTempView("people") # SQL statements can be run by using the sql methods provided by `sqlContext`. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 9ca9b1844c45..4d0a1122dcd1 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1553,8 +1553,8 @@ words.foreachRDD { rdd => // Convert RDD[String] to DataFrame val wordsDataFrame = rdd.toDF("word") - // Register as table - wordsDataFrame.registerTempTable("words") + // Create a temporary view + wordsDataFrame.createOrReplaceTempView("words") // Do word count on DataFrame using SQL and print it val wordCountsDataFrame = @@ -1606,8 +1606,8 @@ words.foreachRDD( }); DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class); - // Register as table - wordsDataFrame.registerTempTable("words"); + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words"); // Do word count on table using SQL and print it DataFrame wordCountsDataFrame = @@ -1646,8 +1646,8 @@ def process(time, rdd): rowRdd = rdd.map(lambda w: Row(word=w)) wordsDataFrame = sqlContext.createDataFrame(rowRdd) - # Register as table - wordsDataFrame.registerTempTable("words") + # Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index cf0167f13a6f..55e591d0ce16 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -73,11 +73,11 @@ public Person call(String line) { } }); - // Apply a schema to an RDD of Java Beans and register it as a table. + // Apply a schema to an RDD of Java Beans and create a temporary view Dataset schemaPeople = spark.createDataFrame(people, Person.class); schemaPeople.createOrReplaceTempView("people"); - // SQL can be run over RDDs that have been registered as tables. + // SQL can be run over RDDs which backs a temporary view. Dataset teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrames and support all the normal RDD operations. @@ -101,7 +101,7 @@ public String call(Row row) { // The result of loading a parquet file is also a DataFrame. Dataset parquetFile = spark.read().parquet("people.parquet"); - //Parquet files can also be registered as tables and then used in SQL statements. + // A temporary view can be created by using Parquet files and then used in SQL statements. parquetFile.createOrReplaceTempView("parquetFile"); Dataset teenagers2 = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); @@ -130,7 +130,7 @@ public String call(Row row) { // |-- age: IntegerType // |-- name: StringType - // Register this DataFrame as a table. + // Creates a temporary view using the DataFrame peopleFromJsonFile.createOrReplaceTempView("people"); // SQL statements can be run by using the sql methods provided by `spark` diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 51305227708e..b8e9e125ba59 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -94,7 +94,7 @@ public JavaRecord call(String word) { }); Dataset wordsDataFrame = spark.createDataFrame(rowRDD, JavaRecord.class); - // Register as table + // Creates a temporary view using the DataFrame wordsDataFrame.createOrReplaceTempView("words"); // Do word count on table using SQL and print it diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 234024063f74..ac7246938d3b 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -66,7 +66,7 @@ # |-- age: long (nullable = true) # |-- name: string (nullable = true) - # Register this DataFrame as a temporary table. + # Creates a temporary view using the DataFrame. people.createOrReplaceTempView("people") # SQL statements can be run by using the sql methods provided by `spark` diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index 25e821534e19..398ac8d2d8f5 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -70,7 +70,7 @@ def process(time, rdd): rowRdd = rdd.map(lambda w: Row(word=w)) wordsDataFrame = spark.createDataFrame(rowRdd) - # Register as table + # Creates a temporary view using the DataFrame. wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index d1bda0ff841d..1b019fbb5177 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -35,8 +35,8 @@ object RDDRelation { import spark.implicits._ val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) - // Any RDD containing case classes can be registered as a table. The schema of the table is - // automatically inferred using scala reflection. + // Any RDD containing case classes can be used to create a temporary view. The schema of the + // view is automatically inferred using scala reflection. df.createOrReplaceTempView("records") // Once tables have been registered, you can run SQL queries over them. @@ -66,7 +66,7 @@ object RDDRelation { // Queries can be run using the DSL on parquet files just like the original RDD. parquetFile.where($"key" === 1).select($"value".as("a")).collect().foreach(println) - // These files can also be registered as tables. + // These files can also be used to create a temporary view. parquetFile.createOrReplaceTempView("parquetFile") spark.sql("SELECT * FROM parquetFile").collect().foreach(println) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index a15cf5ded0e7..7293cb51b261 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -70,9 +70,9 @@ object HiveFromSpark { case Row(key: Int, value: String) => s"Key: $key, Value: $value" } - // You can also register RDDs as temporary tables within a HiveContext. + // You can also use RDDs to create temporary views within a HiveContext. val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.toDF().registerTempTable("records") + rdd.toDF().createOrReplaceTempView("records") // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 688c5b23c288..787bbec73b28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -66,7 +66,7 @@ object SqlNetworkWordCount { // Convert RDD[String] to RDD[case class] to DataFrame val wordsDataFrame = rdd.map(w => Record(w)).toDF() - // Register as table + // Creates a temporary view using the DataFrame wordsDataFrame.createOrReplaceTempView("words") // Do word count on table using SQL and print it diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index 46c26e8b929e..a81a36d1b1c4 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -68,7 +68,7 @@ public void pipeline() { Pipeline pipeline = new Pipeline() .setStages(new PipelineStage[]{scaler, lr}); PipelineModel model = pipeline.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index 98abca221c88..b8da04c26a49 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -54,7 +54,7 @@ public void setUp() { List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); + dataset.createOrReplaceTempView("dataset"); } @After @@ -68,7 +68,7 @@ public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); Assert.assertEquals(lr.getLabelCol(), "label"); LogisticRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); // Check defaults @@ -97,14 +97,14 @@ public void logisticRegressionWithSetters() { // Modify model params, and check that the params worked. model.setThreshold(1.0); - model.transform(dataset).registerTempTable("predAllZero"); + model.transform(dataset).createOrReplaceTempView("predAllZero"); Dataset predAllZero = spark.sql("SELECT prediction, myProbability FROM predAllZero"); for (Row r : predAllZero.collectAsList()) { Assert.assertEquals(0.0, r.getDouble(0), eps); } // Call transform with params, and check that the params worked. model.transform(dataset, model.threshold().w(0.0), model.probabilityCol().w("myProb")) - .registerTempTable("predNotAllZero"); + .createOrReplaceTempView("predNotAllZero"); Dataset predNotAllZero = spark.sql("SELECT prediction, myProb FROM predNotAllZero"); boolean foundNonZero = false; for (Row r : predNotAllZero.collectAsList()) { @@ -130,7 +130,7 @@ public void logisticRegressionPredictorClassifierMethods() { LogisticRegressionModel model = lr.fit(dataset); Assert.assertEquals(2, model.numClasses()); - model.transform(dataset).registerTempTable("transformed"); + model.transform(dataset).createOrReplaceTempView("transformed"); Dataset trans1 = spark.sql("SELECT rawPrediction, probability FROM transformed"); for (Row row : trans1.collectAsList()) { Vector raw = (Vector) row.get(0); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index d3ef5f6fca07..126aa6298f3d 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -50,7 +50,7 @@ public void setUp() { List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); + dataset.createOrReplaceTempView("dataset"); } @After @@ -65,7 +65,7 @@ public void linearRegressionDefaultParams() { assertEquals("label", lr.getLabelCol()); assertEquals("auto", lr.getSolver()); LinearRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); Dataset predictions = spark.sql("SELECT label, prediction FROM prediction"); predictions.collect(); // Check defaults diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index ca111ae9bb7d..e8e60c64121b 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -57,7 +57,7 @@ def __init__(self, sparkContext, sparkSession=None, jsqlContext=None): ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() - >>> df.registerTempTable("allTypes") + >>> df.createOrReplaceTempView("allTypes") >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() [Row((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \ @@ -106,7 +106,7 @@ def getOrCreate(cls, sc): def newSession(self): """ Returns a new SQLContext as new session, that has separate SQLConf, - registered temporary tables and UDFs, but shared SparkContext and + registered temporary views and UDFs, but shared SparkContext and table cache. """ return self.__class__(self._sc, self.sparkSession.newSession()) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index c98aef1a0e69..8e6bce90010e 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -266,7 +266,7 @@ def table(self, tableName): :param tableName: string, name of the table. >>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned') - >>> df.registerTempTable('tmpTable') + >>> df.createOrReplaceTempView('tmpTable') >>> spark.read.table('tmpTable').dtypes [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] """ diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 0781b442cbcd..257a239c8d7b 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -186,7 +186,7 @@ def __init__(self, sparkContext, jsparkSession=None): def newSession(self): """ Returns a new SparkSession as new session, that has separate SQLConf, - registered temporary tables and UDFs, but shared SparkContext and + registered temporary views and UDFs, but shared SparkContext and table cache. """ return self.__class__(self._sc, self._jsparkSession.newSession()) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 0977c43a398d..e86f44281d49 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -294,7 +294,8 @@ def test_udf(self): def test_udf2(self): self.spark.catalog.registerFunction("strlen", lambda string: len(string), IntegerType()) - self.spark.createDataFrame(self.sc.parallelize([Row(a="test")])).registerTempTable("test") + self.spark.createDataFrame(self.sc.parallelize([Row(a="test")]))\ + .createOrReplaceTempView("test") [res] = self.spark.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() self.assertEqual(4, res[0]) @@ -320,7 +321,7 @@ def test_multiple_udfs(self): def test_udf_with_array_type(self): d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) - self.spark.createDataFrame(rdd).registerTempTable("test") + self.spark.createDataFrame(rdd).createOrReplaceTempView("test") self.spark.catalog.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) self.spark.catalog.registerFunction("maplen", lambda d: len(d), IntegerType()) [(l1, l2)] = self.spark.sql("select copylist(l), maplen(d) from test").collect() @@ -360,7 +361,7 @@ def test_basic_functions(self): self.assertTrue(df.is_cached) self.assertEqual(2, df.count()) - df.registerTempTable("temp") + df.createOrReplaceTempView("temp") df = self.spark.sql("select foo from temp") df.count() df.collect() @@ -420,7 +421,7 @@ def test_infer_schema(self): df = self.spark.createDataFrame(rdd) self.assertEqual([], df.rdd.map(lambda r: r.l).first()) self.assertEqual([None, ""], df.rdd.map(lambda r: r.s).collect()) - df.registerTempTable("test") + df.createOrReplaceTempView("test") result = self.spark.sql("SELECT l[0].a from test where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) @@ -428,7 +429,7 @@ def test_infer_schema(self): self.assertEqual(df.schema, df2.schema) self.assertEqual({}, df2.rdd.map(lambda r: r.d).first()) self.assertEqual([None, ""], df2.rdd.map(lambda r: r.s).collect()) - df2.registerTempTable("test2") + df2.createOrReplaceTempView("test2") result = self.spark.sql("SELECT l[0].a from test2 where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) @@ -487,7 +488,7 @@ def test_apply_schema(self): datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) self.assertEqual(r, results.first()) - df.registerTempTable("table2") + df.createOrReplaceTempView("table2") r = self.spark.sql("SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + "short1 + 1 AS short1, short2 - 1 AS short2, int1 - 1 AS int1, " + "float1 + 1.5 as float1 FROM table2").first() @@ -515,7 +516,7 @@ def test_convert_row_to_dict(self): row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) self.assertEqual(1, row.asDict()['l'][0].a) df = self.sc.parallelize([row]).toDF() - df.registerTempTable("test") + df.createOrReplaceTempView("test") row = self.spark.sql("select l, d from test").head() self.assertEqual(1, row.asDict()["l"][0].a) self.assertEqual(1.0, row.asDict()['d']['key'].c) @@ -556,7 +557,7 @@ def test_infer_schema_with_udt(self): schema = df.schema field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), ExamplePointUDT) - df.registerTempTable("labeled_point") + df.createOrReplaceTempView("labeled_point") point = self.spark.sql("SELECT point FROM labeled_point").head().point self.assertEqual(point, ExamplePoint(1.0, 2.0)) @@ -565,7 +566,7 @@ def test_infer_schema_with_udt(self): schema = df.schema field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), PythonOnlyUDT) - df.registerTempTable("labeled_point") + df.createOrReplaceTempView("labeled_point") point = self.spark.sql("SELECT point FROM labeled_point").head().point self.assertEqual(point, PythonOnlyPoint(1.0, 2.0)) @@ -1427,7 +1428,7 @@ def test_list_tables(self): spark.sql("CREATE DATABASE some_db") self.assertEquals(spark.catalog.listTables(), []) self.assertEquals(spark.catalog.listTables("some_db"), []) - spark.createDataFrame([(1, 1)]).registerTempTable("temp_tab") + spark.createDataFrame([(1, 1)]).createOrReplaceTempView("temp_tab") spark.sql("CREATE TABLE tab1 (name STRING, age INT)") spark.sql("CREATE TABLE some_db.tab2 (name STRING, age INT)") tables = sorted(spark.catalog.listTables(), key=lambda t: t.name) @@ -1554,8 +1555,8 @@ def test_list_columns(self): def test_cache(self): spark = self.spark - spark.createDataFrame([(2, 2), (3, 3)]).registerTempTable("tab1") - spark.createDataFrame([(2, 2), (3, 3)]).registerTempTable("tab2") + spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab1") + spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab2") self.assertFalse(spark.catalog.isCached("tab1")) self.assertFalse(spark.catalog.isCached("tab2")) spark.catalog.cacheTable("tab1") 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 44511885a7db..a3e2b49556e4 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 @@ -376,7 +376,7 @@ class SQLContext private[sql]( * // |-- name: string (nullable = false) * // |-- age: integer (nullable = true) * - * dataFrame.registerTempTable("people") + * dataFrame.createOrReplaceTempView("people") * sqlContext.sql("select name from people").collect.foreach(println) * }}} * diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java index f2ae40e64424..573d0e359436 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java @@ -108,7 +108,7 @@ public Row call(Person person) throws Exception { StructType schema = DataTypes.createStructType(fields); Dataset df = spark.createDataFrame(rowRDD, schema); - df.registerTempTable("people"); + df.createOrReplaceTempView("people"); List actual = spark.sql("SELECT * FROM people").collectAsList(); List expected = new ArrayList<>(2); @@ -144,7 +144,7 @@ public Row call(Person person) { StructType schema = DataTypes.createStructType(fields); Dataset df = spark.createDataFrame(rowRDD, schema); - df.registerTempTable("people"); + df.createOrReplaceTempView("people"); List actual = spark.sql("SELECT * FROM people").toJavaRDD() .map(new Function() { @Override @@ -202,14 +202,14 @@ public void applySchemaToJSON() { Dataset df1 = spark.read().json(jsonRDD); StructType actualSchema1 = df1.schema(); Assert.assertEquals(expectedSchema, actualSchema1); - df1.registerTempTable("jsonTable1"); + df1.createOrReplaceTempView("jsonTable1"); List actual1 = spark.sql("select * from jsonTable1").collectAsList(); Assert.assertEquals(expectedResult, actual1); Dataset df2 = spark.read().schema(expectedSchema).json(jsonRDD); StructType actualSchema2 = df2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); - df2.registerTempTable("jsonTable2"); + df2.createOrReplaceTempView("jsonTable2"); List actual2 = spark.sql("select * from jsonTable2").collectAsList(); Assert.assertEquals(expectedResult, actual2); } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java index d0435e4d4395..9840bc46f95f 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java @@ -72,7 +72,7 @@ public void setUp() throws IOException { } JavaRDD rdd = jsc.parallelize(jsonObjects); df = spark.read().json(rdd); - df.registerTempTable("jsonTable"); + df.createOrReplaceTempView("jsonTable"); } @After diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 6d8de80a11fd..1c96bdc05cfc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -71,7 +71,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("cache temp table") { - testData.select('key).registerTempTable("tempTable") + testData.select('key).createOrReplaceTempView("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) spark.catalog.cacheTable("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable")) @@ -99,8 +99,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("uncaching temp table") { - testData.select('key).registerTempTable("tempTable1") - testData.select('key).registerTempTable("tempTable2") + testData.select('key).createOrReplaceTempView("tempTable1") + testData.select('key).createOrReplaceTempView("tempTable2") spark.catalog.cacheTable("tempTable1") assertCached(sql("SELECT COUNT(*) FROM tempTable1")) @@ -116,7 +116,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("too big for memory") { val data = "*" * 1000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF() - .registerTempTable("bigData") + .createOrReplaceTempView("bigData") spark.table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(spark.table("bigData").count() === 200000L) spark.table("bigData").unpersist(blocking = true) @@ -191,7 +191,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("SELECT star from cached table") { - sql("SELECT * FROM testData").registerTempTable("selectStar") + sql("SELECT * FROM testData").createOrReplaceTempView("selectStar") spark.catalog.cacheTable("selectStar") checkAnswer( sql("SELECT * FROM selectStar WHERE key = 1"), @@ -286,15 +286,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Drops temporary table") { - testData.select('key).registerTempTable("t1") + testData.select('key).createOrReplaceTempView("t1") spark.table("t1") spark.catalog.dropTempView("t1") intercept[AnalysisException](spark.table("t1")) } test("Drops cached temporary table") { - testData.select('key).registerTempTable("t1") - testData.select('key).registerTempTable("t2") + testData.select('key).createOrReplaceTempView("t1") + testData.select('key).createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") assert(spark.catalog.isCached("t1")) @@ -306,15 +306,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Clear all cache") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") spark.catalog.clearCache() assert(spark.cacheManager.isEmpty) - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") sql("Clear CACHE") @@ -322,8 +322,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Clear accumulators when uncacheTable to prevent memory leaking") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -350,7 +350,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("SPARK-10327 Cache Table is not working while subquery has alias in its project list") { sparkContext.parallelize((1, 1) :: (2, 2) :: Nil) - .toDF("key", "value").selectExpr("key", "value", "key+1").registerTempTable("abc") + .toDF("key", "value").selectExpr("key", "value", "key+1").createOrReplaceTempView("abc") spark.catalog.cacheTable("abc") val sparkPlan = sql( @@ -371,9 +371,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { val table3x = testData.union(testData).union(testData) - table3x.registerTempTable("testData3x") + table3x.createOrReplaceTempView("testData3x") - sql("SELECT key, value FROM testData3x ORDER BY key").registerTempTable("orderedTable") + sql("SELECT key, value FROM testData3x ORDER BY key").createOrReplaceTempView("orderedTable") spark.catalog.cacheTable("orderedTable") assertCached(spark.table("orderedTable")) // Should not have an exchange as the query is already sorted on the group by key. @@ -388,8 +388,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // different number of partitions. for (numPartitions <- 1 until 10 by 4) { withTempTable("t1", "t2") { - testData.repartition(numPartitions, $"key").registerTempTable("t1") - testData2.repartition(numPartitions, $"a").registerTempTable("t2") + testData.repartition(numPartitions, $"key").createOrReplaceTempView("t1") + testData2.repartition(numPartitions, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -410,8 +410,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // Distribute the tables into non-matching number of partitions. Need to shuffle one side. withTempTable("t1", "t2") { - testData.repartition(6, $"key").registerTempTable("t1") - testData2.repartition(3, $"a").registerTempTable("t2") + testData.repartition(6, $"key").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -427,8 +427,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // One side of join is not partitioned in the desired way. Need to shuffle one side. withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(6, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(6, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -443,8 +443,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(12, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(12, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -462,8 +462,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // the side that has already partitioned is smaller than the side that is not partitioned, // we shuffle both side. withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(3, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -479,7 +479,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // repartition's column ordering is different from group by column ordering. // But they use the same set of columns. withTempTable("t1") { - testData.repartition(6, $"value", $"key").registerTempTable("t1") + testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1") spark.catalog.cacheTable("t1") val query = sql("SELECT value, key from t1 group by key, value") @@ -496,9 +496,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // See PartitioningSuite for more details. withTempTable("t1", "t2") { val df1 = testData - df1.repartition(6, $"value", $"key").registerTempTable("t1") + df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1") val df2 = testData2.select($"a", $"b".cast("string")) - df2.repartition(6, $"a", $"b").registerTempTable("t2") + df2.repartition(6, $"a", $"b").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a5aecca13f61..e89fa32b1514 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -321,7 +321,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { nanvl($"b", $"e"), nanvl($"e", $"f")), Row(null, 3.0, 10.0, null, Double.PositiveInfinity, 3.0, 1.0) ) - testData.registerTempTable("t") + testData.createOrReplaceTempView("t") checkAnswer( sql( "select nanvl(a, 5), nanvl(b, 10), nanvl(10, b), nanvl(c, null), nanvl(d, 10), " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 4ee2006421e1..a15b4e1221d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -245,7 +245,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B Seq( ("2016-03-27 19:39:34", 1), ("2016-03-27 19:39:56", 2), - ("2016-03-27 19:39:27", 4)).toDF("time", "value").registerTempTable(tableName) + ("2016-03-27 19:39:27", 4)).toDF("time", "value").createOrReplaceTempView(tableName) try { f(tableName) } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala index 91095af0ddae..07aad3c406b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala @@ -49,7 +49,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lead") { val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( @@ -59,7 +59,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lag") { val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( @@ -70,7 +70,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lead with default value") { val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( lead("value", 2, "n/a").over(Window.partitionBy("key").orderBy("value"))), @@ -80,7 +80,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lag with default value") { val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( lag("value", 2, "n/a").over(Window.partitionBy($"key").orderBy($"value"))), @@ -89,7 +89,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("rank functions in unspecific window") { val df = Seq((1, "1"), (2, "2"), (1, "2"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -112,7 +112,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between") { val df = Seq((1, "1"), (2, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( avg("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 2))), @@ -121,7 +121,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between") { val df = Seq((1, "1"), (1, "1"), (3, "1"), (2, "2"), (2, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( avg("key").over(Window.partitionBy($"value").orderBy($"key").rangeBetween(-1, 1))), @@ -131,7 +131,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between with unbounded") { val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -146,7 +146,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between with unbounded") { val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -357,7 +357,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between with unbounded + predicate pushdown") { val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") val selectList = Seq($"key", $"value", last("key").over( Window.partitionBy($"value").orderBy($"key").rowsBetween(0, Long.MaxValue)), @@ -372,7 +372,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between with unbounded + predicate pushdown") { val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") val selectList = Seq($"key", $"value", last("value").over( Window.partitionBy($"value").orderBy($"key").rangeBetween(-2, -1)).equalTo("2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index da567db5eec6..a6b83b3d0750 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -344,8 +344,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { } test("full outer join") { - upperCaseData.where('N <= 4).registerTempTable("`left`") - upperCaseData.where('N >= 3).registerTempTable("`right`") + upperCaseData.where('N <= 4).createOrReplaceTempView("`left`") + upperCaseData.where('N >= 3).createOrReplaceTempView("`right`") val left = UnresolvedRelation(TableIdentifier("left"), None) val right = UnresolvedRelation(TableIdentifier("right"), None) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index 1c6e6cc15d77..65fe271b6917 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -29,7 +29,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex private lazy val df = (1 to 10).map(i => (i, s"str$i")).toDF("key", "value") before { - df.registerTempTable("listtablessuitetable") + df.createOrReplaceTempView("listtablessuitetable") } after { @@ -74,7 +74,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex case tableDF => assert(expectedSchema === tableDF.schema) - tableDF.registerTempTable("tables") + tableDF.createOrReplaceTempView("tables") checkAnswer( sql( "SELECT isTemporary, tableName from tables WHERE tableName = 'listtablessuitetable'"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 1d5fc570c65d..38d7b6e25b82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -60,7 +60,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { // temporary table should not be shared val df = session1.range(10) - df.registerTempTable("test1") + df.createOrReplaceTempView("test1") assert(session1.tableNames().contains("test1")) assert(!session2.tableNames().contains("test1")) 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 b67e2bdeb366..010dea5b3038 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 @@ -39,7 +39,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { setupTestData() test("having clause") { - Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v").registerTempTable("hav") + Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") + .createOrReplaceTempView("hav") checkAnswer( sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"), Row("one", 6) :: Row("three", 3) :: Nil) @@ -47,7 +48,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-8010: promote numeric to string") { val df = Seq((1, 1)).toDF("key", "value") - df.registerTempTable("src") + df.createOrReplaceTempView("src") val queryCaseWhen = sql("select case when true then 1.0 else '1' end from src ") val queryCoalesce = sql("select coalesce(null, 1, '1') from src ") @@ -100,7 +101,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (83, 0, 38), (26, 0, 79), (43, 81, 24) - ).toDF("a", "b", "c").registerTempTable("cachedData") + ).toDF("a", "b", "c").createOrReplaceTempView("cachedData") spark.catalog.cacheTable("cachedData") checkAnswer( @@ -109,7 +110,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("self join with aliases") { - Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").registerTempTable("df") + Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").createOrReplaceTempView("df") checkAnswer( sql( @@ -137,7 +138,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .toDF("int", "str") .groupBy("str") .agg($"str", count("str").as("strCount")) - .registerTempTable("df") + .createOrReplaceTempView("df") checkAnswer( sql( @@ -195,7 +196,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("grouping on nested fields") { spark.read.json(sparkContext.parallelize( """{"nested": {"attribute": 1}, "value": 2}""" :: Nil)) - .registerTempTable("rows") + .createOrReplaceTempView("rows") checkAnswer( sql( @@ -214,7 +215,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.read.json( sparkContext.parallelize( Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}"))) - .registerTempTable("d") + .createOrReplaceTempView("d") checkAnswer( sql("select * from d where d.a in (1,2)"), @@ -225,7 +226,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.read.json( sparkContext.parallelize( Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", ""))) - .registerTempTable("d") + .createOrReplaceTempView("d") checkAnswer( sql("select count(1) from d"), @@ -261,7 +262,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.table("testData") .union(spark.table("testData")) .union(spark.table("testData")) - .registerTempTable("testData3x") + .createOrReplaceTempView("testData3x") try { // Just to group rows. @@ -391,7 +392,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-3173 Timestamp support in the parser") { - (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").registerTempTable("timestamps") + (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").createOrReplaceTempView("timestamps") checkAnswer(sql( "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"), @@ -746,7 +747,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("count of empty table") { withTempTable("t") { - Seq.empty[(Int, Int)].toDF("a", "b").registerTempTable("t") + Seq.empty[(Int, Int)].toDF("a", "b").createOrReplaceTempView("t") checkAnswer( sql("select count(a) from t"), Row(0)) @@ -891,10 +892,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-3349 partitioning after limit") { sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) - .registerTempTable("subset1") + .createOrReplaceTempView("subset1") sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC") .limit(2) - .registerTempTable("subset2") + .createOrReplaceTempView("subset2") checkAnswer( sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"), Row(3, "c", 3) :: @@ -1111,7 +1112,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df1 = spark.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + df1.createOrReplaceTempView("applySchema1") checkAnswer( sql("SELECT * FROM applySchema1"), Row(1, "A1", true, null) :: @@ -1141,7 +1142,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df2 = spark.createDataFrame(rowRDD2, schema2) - df2.registerTempTable("applySchema2") + df2.createOrReplaceTempView("applySchema2") checkAnswer( sql("SELECT * FROM applySchema2"), Row(Row(1, true), Map("A1" -> null)) :: @@ -1166,7 +1167,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df3 = spark.createDataFrame(rowRDD3, schema2) - df3.registerTempTable("applySchema3") + df3.createOrReplaceTempView("applySchema3") checkAnswer( sql("SELECT f1.f11, f2['D4'] FROM applySchema3"), @@ -1214,7 +1215,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { def validateMetadata(rdd: DataFrame): Unit = { assert(rdd.schema("name").metadata.getString(docKey) == docValue) } - personWithMeta.registerTempTable("personWithMeta") + personWithMeta.createOrReplaceTempView("personWithMeta") validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"id", $"name")) @@ -1409,7 +1410,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-3483 Special chars in column names") { val data = sparkContext.parallelize( Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) - spark.read.json(data).registerTempTable("records") + spark.read.json(data).createOrReplaceTempView("records") sql("SELECT `key?number1`, `key.number2` FROM records") } @@ -1451,12 +1452,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-4322 Grouping field with struct field as sub expression") { spark.read.json(sparkContext.makeRDD("""{"a": {"b": [{"c": 1}]}}""" :: Nil)) - .registerTempTable("data") + .createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b[0].c FROM data GROUP BY a.b[0].c"), Row(1)) spark.catalog.dropTempView("data") spark.read.json( - sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).registerTempTable("data") + sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b + 1 FROM data GROUP BY a.b + 1"), Row(2)) spark.catalog.dropTempView("data") } @@ -1478,10 +1479,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1, "1") :: TestData(2, null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) - rdd1.toDF().registerTempTable("nulldata1") + rdd1.toDF().createOrReplaceTempView("nulldata1") val nullCheckData2 = TestData(1, "1") :: TestData(2, null) :: Nil val rdd2 = sparkContext.parallelize((0 to 1).map(i => nullCheckData2(i))) - rdd2.toDF().registerTempTable("nulldata2") + rdd2.toDF().createOrReplaceTempView("nulldata2") checkAnswer(sql("SELECT nulldata1.key FROM nulldata1 join " + "nulldata2 on nulldata1.value <=> nulldata2.value"), (1 to 2).map(i => Row(i))) @@ -1490,7 +1491,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Multi-column COUNT(DISTINCT ...)") { val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF().registerTempTable("distinctData") + rdd.toDF().createOrReplaceTempView("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2)) } @@ -1498,7 +1499,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF().registerTempTable("testTable1") + rdd.toDF().createOrReplaceTempView("testTable1") checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) } } @@ -1506,7 +1507,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-6145: ORDER BY test for nested fields") { spark.read.json(sparkContext.makeRDD( """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)) - .registerTempTable("nestedOrder") + .createOrReplaceTempView("nestedOrder") checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1)) checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1)) @@ -1517,8 +1518,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-6145: special cases") { - spark.read.json(sparkContext.makeRDD( - """{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)).registerTempTable("t") + spark.read + .json(sparkContext.makeRDD("""{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)) + .createOrReplaceTempView("t") + checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY _c0.a"), Row(1)) checkAnswer(sql("SELECT b[0].a FROM t ORDER BY _c0.a"), Row(1)) } @@ -1526,14 +1529,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-6898: complete support for special chars in column names") { spark.read.json(sparkContext.makeRDD( """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) - .registerTempTable("t") + .createOrReplaceTempView("t") checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } test("SPARK-6583 order by aggregated function") { Seq("1" -> 3, "1" -> 4, "2" -> 7, "2" -> 8, "3" -> 5, "3" -> 6, "4" -> 1, "4" -> 2) - .toDF("a", "b").registerTempTable("orderByData") + .toDF("a", "b").createOrReplaceTempView("orderByData") checkAnswer( sql( @@ -1619,7 +1622,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (0, null, null, false), (1, null, null, false), (null, null, null, true) - ).toDF("i", "b", "r1", "r2").registerTempTable("t") + ).toDF("i", "b", "r1", "r2").createOrReplaceTempView("t") checkAnswer(sql("select i = b from t"), sql("select r1 from t")) checkAnswer(sql("select i <=> b from t"), sql("select r2 from t")) @@ -1629,14 +1632,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-7067: order by queries for complex ExtractValue chain") { withTempTable("t") { spark.read.json(sparkContext.makeRDD( - """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).registerTempTable("t") + """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).createOrReplaceTempView("t") checkAnswer(sql("SELECT a.b FROM t ORDER BY b[0].d"), Row(Seq(Row(1)))) } } test("SPARK-8782: ORDER BY NULL") { withTempTable("t") { - Seq((1, 2), (1, 2)).toDF("a", "b").registerTempTable("t") + Seq((1, 2), (1, 2)).toDF("a", "b").createOrReplaceTempView("t") checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), Row(1, 2))) } } @@ -1645,7 +1648,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withTempTable("t") { val df = Seq(1 -> "a").toDF("count", "sort") checkAnswer(df.filter("count > 0"), Row(1, "a")) - df.registerTempTable("t") + df.createOrReplaceTempView("t") checkAnswer(sql("select count, sort from t"), Row(1, "a")) } } @@ -1759,7 +1762,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withTempTable("1one") { sparkContext.parallelize(1 to 10).map(i => (i, i.toString)) .toDF("num", "str") - .registerTempTable("1one") + .createOrReplaceTempView("1one") checkAnswer(sql("select count(num) from 1one"), Row(10)) } } @@ -1801,7 +1804,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-10130 type coercion for IF should have children resolved first") { withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer( sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) } @@ -1809,7 +1812,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-10389: order by non-attribute grouping expression on Aggregate") { withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1"), Seq(Row(1), Row(1))) checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY (key + 1) * 2"), @@ -1872,7 +1875,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-11032: resolve having correctly") { withTempTable("src") { - Seq(1 -> "a").toDF("i", "j").registerTempTable("src") + Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("src") checkAnswer( sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t HAVING(COUNT(1) > 0)"), Row(1)) @@ -1910,8 +1913,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1, 1, 1, 1) :: Row(1, 2, 2, 1) :: Row(2, 1, 1, 2) :: Row(2, 2, 2, 2) :: Row(3, 1, 1, 3) :: Row(3, 2, 2, 3) :: Nil) - // Try with a registered table. - sql("select struct(a, b) as record from testData2").registerTempTable("structTable") + // Try with a temporary view + sql("select struct(a, b) as record from testData2").createOrReplaceTempView("structTable") checkAnswer( sql("SELECT record.* FROM structTable"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) @@ -1975,9 +1978,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { nestedStructData.select($"record.r1.*"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) - // Try with a registered table + // Try with a temporary view withTempTable("nestedStructTable") { - nestedStructData.registerTempTable("nestedStructTable") + nestedStructData.createOrReplaceTempView("nestedStructTable") checkAnswer( sql("SELECT record.* FROM nestedStructTable"), nestedStructData.select($"record.*")) @@ -2000,7 +2003,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { | (SELECT struct(a, b) as `col$.a_`, struct(b, a) as `a.b.c.` FROM testData2) tmp """.stripMargin) withTempTable("specialCharacterTable") { - specialCharacterPath.registerTempTable("specialCharacterTable") + specialCharacterPath.createOrReplaceTempView("specialCharacterTable") checkAnswer( specialCharacterPath.select($"`r&&b.c`.*"), nestedStructData.select($"record.*")) @@ -2024,7 +2027,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { // Create a data set that contains a naming conflict val nameConflict = sql("SELECT struct(a, b) as nameConflict, a as a FROM testData2") withTempTable("nameConflict") { - nameConflict.registerTempTable("nameConflict") + nameConflict.createOrReplaceTempView("nameConflict") // Unqualified should resolve to table. checkAnswer(sql("SELECT nameConflict.* FROM nameConflict"), Row(Row(1, 1), 1) :: Row(Row(1, 2), 1) :: Row(Row(2, 1), 2) :: Row(Row(2, 2), 2) :: @@ -2328,7 +2331,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-13056: Null in map value causes NPE") { val df = Seq(1 -> Map("abc" -> "somestring", "cba" -> null)).toDF("key", "value") withTempTable("maptest") { - df.registerTempTable("maptest") + df.createOrReplaceTempView("maptest") // local optimization will by pass codegen code, so we should keep the filter `key=1` checkAnswer(sql("SELECT value['abc'] FROM maptest where key = 1"), Row("somestring")) checkAnswer(sql("SELECT value['cba'] FROM maptest where key = 1"), Row(null)) @@ -2338,7 +2341,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("hash function") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") withTempTable("tbl") { - df.registerTempTable("tbl") + df.createOrReplaceTempView("tbl") checkAnswer( df.select(hash($"i", $"j")), sql("SELECT hash(i, j) from tbl") @@ -2390,8 +2393,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df1 = Seq(("one", 1), ("two", 2), ("three", 3)).toDF("k", "v1") val df2 = Seq(("one", 1), ("two", 22), ("one", 5)).toDF("k", "v2") withTempTable("nt1", "nt2") { - df1.registerTempTable("nt1") - df2.registerTempTable("nt2") + df1.createOrReplaceTempView("nt1") + df2.createOrReplaceTempView("nt2") checkAnswer( sql("SELECT * FROM nt1 natural join nt2 where k = \"one\""), Row("one", 1, 1) :: Row("one", 1, 5) :: Nil) @@ -2418,9 +2421,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df3 = Seq((null, "r1c2", "t3r1c3"), ("r2c1", "r2c2", "t3r2c3"), ("r3c1y", "r3c2", "t3r3c3")).toDF("c1", "c2", "c3") withTempTable("t1", "t2", "t3") { - df1.registerTempTable("t1") - df2.registerTempTable("t2") - df3.registerTempTable("t3") + df1.createOrReplaceTempView("t1") + df2.createOrReplaceTempView("t2") + df3.createOrReplaceTempView("t3") // inner join with one using column checkAnswer( sql("SELECT * FROM t1 join t2 using (c1)"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 295f02f9a7b5..491bdb3ef9db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -78,7 +78,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3)) - Seq(data).toDF().registerTempTable("reflectData") + Seq(data).toDF().createOrReplaceTempView("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -88,7 +88,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) - Seq(data).toDF().registerTempTable("reflectNullData") + Seq(data).toDF().createOrReplaceTempView("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -96,7 +96,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) - Seq(data).toDF().registerTempTable("reflectOptionalData") + Seq(data).toDF().createOrReplaceTempView("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -104,7 +104,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { // Equality is broken for Arrays, so we test that separately. test("query binary data") { - Seq(ReflectBinary(Array[Byte](1))).toDF().registerTempTable("reflectBinary") + Seq(ReflectBinary(Array[Byte](1))).toDF().createOrReplaceTempView("reflectBinary") val result = sql("SELECT data FROM reflectBinary") .collect().head(0).asInstanceOf[Array[Byte]] @@ -124,7 +124,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), Nested(None, "abc"))) - Seq(data).toDF().registerTempTable("reflectComplexData") + Seq(data).toDF().createOrReplaceTempView("reflectComplexData") assert(sql("SELECT * FROM reflectComplexData").collect().head === Row( Seq(1, 2, 3), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 17ac0c8c6e49..48196927332f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -49,9 +49,9 @@ class SubquerySuite extends QueryTest with SharedSQLContext { protected override def beforeAll(): Unit = { super.beforeAll() - l.registerTempTable("l") - r.registerTempTable("r") - t.registerTempTable("t") + l.createOrReplaceTempView("l") + r.createOrReplaceTempView("r") + t.createOrReplaceTempView("t") } test("simple uncorrelated scalar subquery") { @@ -99,7 +99,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { test("uncorrelated scalar subquery on a DataFrame generated query") { val df = Seq((1, "one"), (2, "two"), (3, "three")).toDF("key", "value") - df.registerTempTable("subqueryData") + df.createOrReplaceTempView("subqueryData") checkAnswer( sql("select (select key from subqueryData where key > 2 order by key limit 1) + 1"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 922154320c39..547d3c1abe85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -53,7 +53,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { test("SPARK-8003 spark_partition_id") { val df = Seq((1, "Tearing down the walls that divide us")).toDF("id", "saying") - df.registerTempTable("tmp_table") + df.createOrReplaceTempView("tmp_table") checkAnswer(sql("select spark_partition_id() from tmp_table").toDF(), Row(0)) spark.catalog.dropTempView("tmp_table") } @@ -62,7 +62,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { withTempPath { dir => val data = sparkContext.parallelize(0 to 10, 2).toDF("id") data.write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("test_table") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("test_table") val answer = sql("select input_file_name() from test_table").head().getString(0) assert(answer.contains(dir.getCanonicalPath)) assert(sql("select input_file_name() from test_table").distinct().collect().length >= 2) @@ -107,7 +107,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("integerData") + df.createOrReplaceTempView("integerData") val result = sql("SELECT * FROM integerData WHERE oneArgFilter(key)") @@ -119,7 +119,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -138,7 +138,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -158,7 +158,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 3057e016c14c..7d7b486530de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -95,7 +95,7 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT test("UDTs and UDFs") { spark.udf.register("testType", (d: UDT.MyDenseVector) => d.isInstanceOf[UDT.MyDenseVector]) - pointsRDD.registerTempTable("points") + pointsRDD.createOrReplaceTempView("points") checkAnswer( sql("SELECT testType(features) from points"), Seq(Row(true), Row(true))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index d2e1ea12fd89..2a5295d0d223 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -78,7 +78,7 @@ class PlannerSuite extends SharedSQLContext { val schema = StructType(fields) val row = Row.fromSeq(Seq.fill(fields.size)(null)) val rowRDD = sparkContext.parallelize(row :: Nil) - spark.createDataFrame(rowRDD, schema).registerTempTable("testLimit") + spark.createDataFrame(rowRDD, schema).createOrReplaceTempView("testLimit") val planned = sql( """ @@ -132,7 +132,7 @@ class PlannerSuite extends SharedSQLContext { test("InMemoryRelation statistics propagation") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "81920") { withTempTable("tiny") { - testData.limit(3).registerTempTable("tiny") + testData.limit(3).createOrReplaceTempView("tiny") sql("CACHE TABLE tiny") val a = testData.as("a") @@ -199,9 +199,9 @@ class PlannerSuite extends SharedSQLContext { test("PartitioningCollection") { withTempTable("normal", "small", "tiny") { - testData.registerTempTable("normal") - testData.limit(10).registerTempTable("small") - testData.limit(3).registerTempTable("tiny") + testData.createOrReplaceTempView("normal") + testData.limit(10).createOrReplaceTempView("small") + testData.limit(3).createOrReplaceTempView("tiny") // Disable broadcast join withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index b31338e82768..bf3a39c84b3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -134,7 +134,8 @@ class AggregateBenchmark extends BenchmarkBase { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) - sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") + .createOrReplaceTempView("test") def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 2099d4e1b38f..e2fb91352d58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("default size avoids broadcast") { // TODO: Improve this test when we have better statistics sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)) - .toDF().registerTempTable("sizeTst") + .toDF().createOrReplaceTempView("sizeTst") spark.catalog.cacheTable("sizeTst") assert( spark.table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > @@ -92,7 +92,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-2729 regression: timestamp data type") { val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time") - timestamps.registerTempTable("timestamps") + timestamps.createOrReplaceTempView("timestamps") checkAnswer( sql("SELECT time FROM timestamps"), @@ -133,7 +133,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(df.schema.head.dataType === DecimalType(15, 10)) - df.cache().registerTempTable("test_fixed_decimal") + df.cache().createOrReplaceTempView("test_fixed_decimal") checkAnswer( sql("SELECT * FROM test_fixed_decimal"), (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) @@ -179,7 +179,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { (i to i + 10).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, Row((i - 0.25).toFloat, Seq(true, false, null))) } - spark.createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") + spark.createDataFrame(rdd, schema).createOrReplaceTempView("InMemoryCache_different_data_types") // Cache the table. sql("cache table InMemoryCache_different_data_types") // Make sure the table is indeed cached. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index 48c798986b27..a118cec0bbb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -63,7 +63,7 @@ class PartitionBatchPruningSuite val string = if (((key - 1) / 10) % 2 == 0) null else key.toString TestData(key, string) }, 5).toDF() - pruningData.registerTempTable("pruningData") + pruningData.createOrReplaceTempView("pruningData") spark.catalog.cacheTable("pruningData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 63fe4658d67d..46213a22ed3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -239,7 +239,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("nullstr", StringType, true):: Nil) assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select nullstr, headers.Host from jsonTable"), @@ -261,7 +261,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -302,7 +302,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -376,7 +376,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("GetField operation on complex data type") { val jsonDF = spark.read.json(complexFieldAndType1) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -403,7 +403,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -464,7 +464,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ignore("Type conflict in primitive field values (Ignored)") { val jsonDF = spark.read.json(primitiveFieldValueTypeConflict) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Right now, the analyzer does not promote strings in a boolean expression. // Number and Boolean conflict: resolve the type as boolean in this query. @@ -528,7 +528,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -550,7 +550,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -580,7 +580,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") } test("Loading a JSON dataset from a text file") { @@ -601,7 +601,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -633,7 +633,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -674,7 +674,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -759,7 +759,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -885,7 +885,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(schema === jsonDF1.schema) - jsonDF1.registerTempTable("jsonTable1") + jsonDF1.createOrReplaceTempView("jsonTable1") checkAnswer( sql("select * from jsonTable1"), @@ -902,7 +902,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(schema === jsonDF2.schema) - jsonDF2.registerTempTable("jsonTable2") + jsonDF2.createOrReplaceTempView("jsonTable2") checkAnswer( sql("select * from jsonTable2"), @@ -921,7 +921,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) - jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap") + jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") checkAnswer( sql("select `map` from jsonWithSimpleMap"), @@ -949,7 +949,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) - jsonWithComplexMap.registerTempTable("jsonWithComplexMap") + jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") checkAnswer( sql("select `map` from jsonWithComplexMap"), @@ -974,7 +974,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-2096 Correctly parse dot notations") { val jsonDF = spark.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -992,7 +992,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-3390 Complex arrays") { val jsonDF = spark.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -1015,7 +1015,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-3308 Read top level JSON arrays") { val jsonDF = spark.read.json(jsonArray) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -1084,7 +1084,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { withTempTable("jsonTable") { val jsonDF = spark.read.json(corruptRecords) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( StructField("_unparsed", StringType, true) :: StructField("a", StringType, true) :: @@ -1156,7 +1156,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-4068: nulls in arrays") { val jsonDF = spark.read.json(nullsInArrays) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( StructField("field1", @@ -1202,7 +1202,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val df1 = spark.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + df1.createOrReplaceTempView("applySchema1") val df2 = df1.toDF val result = df2.toJSON.collect() // scalastyle:off @@ -1225,7 +1225,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val df3 = spark.createDataFrame(rowRDD2, schema2) - df3.registerTempTable("applySchema2") + df3.createOrReplaceTempView("applySchema2") val df4 = df3.toDF val result2 = df4.toJSON.collect() @@ -1234,7 +1234,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val jsonDF = spark.read.json(primitiveFieldAndType) val primTable = spark.read.json(jsonDF.toJSON.rdd) - primTable.registerTempTable("primitiveTable") + primTable.createOrReplaceTempView("primitiveTable") checkAnswer( sql("select * from primitiveTable"), Row(new java.math.BigDecimal("92233720368547758070"), @@ -1247,7 +1247,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val complexJsonDF = spark.read.json(complexFieldAndType1) val compTable = spark.read.json(complexJsonDF.toJSON.rdd) - compTable.registerTempTable("complexTable") + compTable.createOrReplaceTempView("complexTable") // Access elements of a primitive array. checkAnswer( sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from complexTable"), @@ -1387,7 +1387,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { "col1", "abd") - spark.read.json(root.getAbsolutePath).registerTempTable("test_myjson_with_part") + spark.read.json(root.getAbsolutePath).createOrReplaceTempView("test_myjson_with_part") checkAnswer(sql( "SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abc'"), Row(4)) checkAnswer(sql( @@ -1531,7 +1531,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { { val jsonDF = spark.read.schema(schema).json(additionalCorruptRecords) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // In HiveContext, backticks should be used to access columns starting with a underscore. checkAnswer( @@ -1639,7 +1639,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val schema = (new StructType).add("ts", TimestampType) val jsonDF = spark.read.schema(schema).json(timestampAsLong) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select ts from jsonTable"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 8707e1346144..847ea6bd523d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -400,7 +400,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha // Introduce _temporary dir to the base dir the robustness of the schema discovery process. new File(base.getCanonicalPath, "_temporary").mkdir() - spark.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -484,7 +484,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - spark.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -533,7 +533,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + parquetRelation.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -573,7 +573,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + parquetRelation.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -609,7 +609,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha .option("mergeSchema", "true") .format("parquet") .load(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index f9f9f8035212..725e14c0fb2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -46,7 +46,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("appending") { val data = (0 until 10).map(i => (i, i.toString)) - spark.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") // Query appends, don't test with both read modes. withParquetTable(data, "t", false) { sql("INSERT INTO TABLE t SELECT * FROM tmp") @@ -58,7 +58,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) - spark.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withParquetTable(data, "t") { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(spark.table("t"), data.map(Row.fromTuple)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala index 69a600a55b12..487d7a7e5ac8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala @@ -75,10 +75,10 @@ object ParquetReadBenchmark { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast(id as INT) as id from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") sqlBenchmark.addCase("SQL Parquet Vectorized") { iter => spark.sql("select sum(id) from tempTable").collect() @@ -159,10 +159,10 @@ object ParquetReadBenchmark { def intStringScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast(id as INT) as c1, cast(id as STRING) as c2 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("Int and String Scan", values) @@ -193,10 +193,10 @@ object ParquetReadBenchmark { def stringDictionaryScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast((id % 200) + 10000 as STRING) as c1 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("String Dictionary", values) @@ -225,10 +225,10 @@ object ParquetReadBenchmark { def partitionTableScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select id % 2 as p, cast(id as INT) as id from t1") .write.partitionBy("p").parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("Partitioned Table", values) @@ -260,11 +260,11 @@ object ParquetReadBenchmark { def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql(s"select IF(rand(1) < $fractionOfNulls, NULL, cast(id as STRING)) as c1, " + s"IF(rand(2) < $fractionOfNulls, NULL, cast(id as STRING)) as c2 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("String with Nulls Scan", values) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala index 08b7eb3cf7b2..228ae6f840cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala @@ -1187,7 +1187,7 @@ object TPCDSBenchmark { def setupTables(dataLocation: String): Map[String, Long] = { tables.map { tableName => - spark.read.parquet(s"$dataLocation/$tableName").registerTempTable(tableName) + spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) tableName -> spark.table(tableName).count() }.toMap } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 1b827694283e..08f596f130aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -165,7 +165,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // Because SortMergeJoin may skip different rows if the number of partitions is different, this // test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -183,7 +183,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // Because SortMergeJoin may skip different rows if the number of partitions is different, // this test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -237,7 +237,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("BroadcastNestedLoopJoin metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -265,7 +265,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("CartesianProduct metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 44d1b9ddda66..9c9abfeb2a63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -441,7 +441,7 @@ class JDBCSuite extends SparkFunSuite test("test DATE types in cache") { val rows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties).collect() spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) - .cache().registerTempTable("mycached_date") + .cache().createOrReplaceTempView("mycached_date") val cachedRows = sql("select * from mycached_date").collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index c1dc9b983407..03c18ad009d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -34,7 +34,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") } override def afterAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 5ac39f54b91c..854fec5b22f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -31,7 +31,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") sql( s""" |CREATE TEMPORARY TABLE jsonTable (a int, b string) @@ -111,7 +111,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to less part files. val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 5) - caseInsensitiveContext.read.json(rdd1).registerTempTable("jt1") + caseInsensitiveContext.read.json(rdd1).createOrReplaceTempView("jt1") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt1 @@ -123,7 +123,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to more part files. val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 10) - caseInsensitiveContext.read.json(rdd2).registerTempTable("jt2") + caseInsensitiveContext.read.json(rdd2).createOrReplaceTempView("jt2") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index bb2c54aa6497..7738e4107df9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -42,7 +42,7 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) df = caseInsensitiveContext.read.json(rdd) - df.registerTempTable("jsonTable") + df.createOrReplaceTempView("jsonTable") } override def afterAll(): Unit = { @@ -123,7 +123,7 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA // verify the append mode df.write.mode(SaveMode.Append).json(path.toString) val df2 = df.union(df) - df2.registerTempTable("jsonTable2") + df2.createOrReplaceTempView("jsonTable2") checkLoad(df2, "jsonTable2") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 013b7316938e..b742206b5813 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -77,7 +77,7 @@ class StreamSuite extends StreamTest with SharedSQLContext { test("sql queries") { val inputData = MemoryStream[Int] - inputData.toDF().registerTempTable("stream") + inputData.toDF().createOrReplaceTempView("stream") val evens = sql("SELECT * FROM stream WHERE value % 2 = 0") testStream(evens)( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 03369c5a4827..421f6bca7f86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -41,14 +41,14 @@ private[sql] trait SQLTestData { self => protected lazy val emptyTestData: DataFrame = { val df = spark.sparkContext.parallelize( Seq.empty[Int].map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("emptyTestData") + df.createOrReplaceTempView("emptyTestData") df } protected lazy val testData: DataFrame = { val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("testData") + df.createOrReplaceTempView("testData") df } @@ -60,7 +60,7 @@ private[sql] trait SQLTestData { self => TestData2(2, 2) :: TestData2(3, 1) :: TestData2(3, 2) :: Nil, 2).toDF() - df.registerTempTable("testData2") + df.createOrReplaceTempView("testData2") df } @@ -68,14 +68,14 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( TestData3(1, None) :: TestData3(2, Some(2)) :: Nil).toDF() - df.registerTempTable("testData3") + df.createOrReplaceTempView("testData3") df } protected lazy val negativeData: DataFrame = { val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(-i, (-i).toString))).toDF() - df.registerTempTable("negativeData") + df.createOrReplaceTempView("negativeData") df } @@ -87,7 +87,7 @@ private[sql] trait SQLTestData { self => LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: LargeAndSmallInts(3, 2) :: Nil).toDF() - df.registerTempTable("largeAndSmallInts") + df.createOrReplaceTempView("largeAndSmallInts") df } @@ -99,7 +99,7 @@ private[sql] trait SQLTestData { self => DecimalData(2, 2) :: DecimalData(3, 1) :: DecimalData(3, 2) :: Nil).toDF() - df.registerTempTable("decimalData") + df.createOrReplaceTempView("decimalData") df } @@ -110,7 +110,7 @@ private[sql] trait SQLTestData { self => BinaryData("122".getBytes(StandardCharsets.UTF_8), 3) :: BinaryData("121".getBytes(StandardCharsets.UTF_8), 2) :: BinaryData("123".getBytes(StandardCharsets.UTF_8), 4) :: Nil).toDF() - df.registerTempTable("binaryData") + df.createOrReplaceTempView("binaryData") df } @@ -122,7 +122,7 @@ private[sql] trait SQLTestData { self => UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: UpperCaseData(6, "F") :: Nil).toDF() - df.registerTempTable("upperCaseData") + df.createOrReplaceTempView("upperCaseData") df } @@ -132,7 +132,7 @@ private[sql] trait SQLTestData { self => LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil).toDF() - df.registerTempTable("lowerCaseData") + df.createOrReplaceTempView("lowerCaseData") df } @@ -140,7 +140,7 @@ private[sql] trait SQLTestData { self => val rdd = spark.sparkContext.parallelize( ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: ArrayData(Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) - rdd.toDF().registerTempTable("arrayData") + rdd.toDF().createOrReplaceTempView("arrayData") rdd } @@ -151,13 +151,13 @@ private[sql] trait SQLTestData { self => MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - rdd.toDF().registerTempTable("mapData") + rdd.toDF().createOrReplaceTempView("mapData") rdd } protected lazy val repeatedData: RDD[StringData] = { val rdd = spark.sparkContext.parallelize(List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("repeatedData") + rdd.toDF().createOrReplaceTempView("repeatedData") rdd } @@ -165,7 +165,7 @@ private[sql] trait SQLTestData { self => val rdd = spark.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("nullableRepeatedData") + rdd.toDF().createOrReplaceTempView("nullableRepeatedData") rdd } @@ -175,7 +175,7 @@ private[sql] trait SQLTestData { self => NullInts(2) :: NullInts(3) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("nullInts") + df.createOrReplaceTempView("nullInts") df } @@ -185,7 +185,7 @@ private[sql] trait SQLTestData { self => NullInts(null) :: NullInts(null) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("allNulls") + df.createOrReplaceTempView("allNulls") df } @@ -194,13 +194,13 @@ private[sql] trait SQLTestData { self => NullStrings(1, "abc") :: NullStrings(2, "ABC") :: NullStrings(3, null) :: Nil).toDF() - df.registerTempTable("nullStrings") + df.createOrReplaceTempView("nullStrings") df } protected lazy val tableName: DataFrame = { val df = spark.sparkContext.parallelize(TableName("test") :: Nil).toDF() - df.registerTempTable("tableName") + df.createOrReplaceTempView("tableName") df } @@ -215,7 +215,7 @@ private[sql] trait SQLTestData { self => // An RDD with 4 elements and 8 partitions protected lazy val withEmptyParts: RDD[IntField] = { val rdd = spark.sparkContext.parallelize((1 to 4).map(IntField), 8) - rdd.toDF().registerTempTable("withEmptyParts") + rdd.toDF().createOrReplaceTempView("withEmptyParts") rdd } @@ -223,7 +223,7 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( Person(0, "mike", 30) :: Person(1, "jim", 20) :: Nil).toDF() - df.registerTempTable("person") + df.createOrReplaceTempView("person") df } @@ -231,7 +231,7 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( Salary(0, 2000.0) :: Salary(1, 1000.0) :: Nil).toDF() - df.registerTempTable("salary") + df.createOrReplaceTempView("salary") df } @@ -240,7 +240,7 @@ private[sql] trait SQLTestData { self => ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1, 1, 1), true) :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2, 2, 2), false) :: Nil).toDF() - df.registerTempTable("complexData") + df.createOrReplaceTempView("complexData") df } @@ -251,7 +251,7 @@ private[sql] trait SQLTestData { self => CourseSales("dotNET", 2012, 5000) :: CourseSales("dotNET", 2013, 48000) :: CourseSales("Java", 2013, 30000) :: Nil).toDF() - df.registerTempTable("courseSales") + df.createOrReplaceTempView("courseSales") df } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java index 64f2ded447a0..f664d5a4cdad 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -57,7 +57,7 @@ public void setUp() throws IOException { jsonObjects.add("{\"key\":" + i + ", \"value\":\"str" + i + "\"}"); } df = hc.read().json(sc.parallelize(jsonObjects)); - df.registerTempTable("window_table"); + df.createOrReplaceTempView("window_table"); } @After diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index f13c32db9d23..e73117c8144c 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -85,7 +85,7 @@ public void setUp() throws IOException { } JavaRDD rdd = sc.parallelize(jsonObjects); df = sqlContext.read().json(rdd); - df.registerTempTable("jsonTable"); + df.createOrReplaceTempView("jsonTable"); } @After 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 d96eb0169ee8..d2cb62c617d4 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 @@ -33,8 +33,8 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd if (spark.wrapped.tableNames().contains("src")) { spark.catalog.dropTempView("src") } - Seq((1, "")).toDF("key", "value").registerTempTable("src") - Seq((1, 1, 1)).toDF("a", "a", "b").registerTempTable("dupAttributes") + Seq((1, "")).toDF("key", "value").createOrReplaceTempView("src") + Seq((1, 1, 1)).toDF("a", "a", "b").createOrReplaceTempView("dupAttributes") } override protected def afterEach(): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index b5af758a65b1..e2304b5397ca 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -51,7 +51,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton test("Converting Hive to Parquet Table via saveAsParquetFile") { withTempPath { dir => sql("SELECT * FROM src").write.parquet(dir.getCanonicalPath) - hiveContext.read.parquet(dir.getCanonicalPath).registerTempTable("p") + hiveContext.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("p") withTempTable("p") { checkAnswer( sql("SELECT * FROM src ORDER BY key"), @@ -65,7 +65,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton withParquetTable((1 to 10).map(i => (i, s"val_$i")), "t", false) { withTempPath { file => sql("SELECT * FROM t LIMIT 1").write.parquet(file.getCanonicalPath) - hiveContext.read.parquet(file.getCanonicalPath).registerTempTable("p") + hiveContext.read.parquet(file.getCanonicalPath).createOrReplaceTempView("p") withTempTable("p") { // let's do three overwrites for good measure sql("INSERT OVERWRITE TABLE p SELECT * FROM t") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index d05a3623ae01..a4bbe96cf805 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -355,7 +355,7 @@ object TemporaryHiveUDFTest extends Logging { """.stripMargin) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Actually use the loaded UDF. logInfo("Using the UDF.") val result = hiveContext.sql( @@ -393,7 +393,7 @@ object PermanentHiveUDFTest1 extends Logging { """.stripMargin) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Actually use the loaded UDF. logInfo("Using the UDF.") val result = hiveContext.sql( @@ -429,7 +429,7 @@ object PermanentHiveUDFTest2 extends Logging { hiveContext.sessionState.catalog.createFunction(function, ignoreIfExists = false) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Actually use the loaded UDF. logInfo("Using the UDF.") val result = hiveContext.sql( @@ -491,7 +491,7 @@ object SparkSubmitClassLoaderTest extends Logging { """.stripMargin) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Load a Hive SerDe from the jar. logInfo("Creating a Hive table with a SerDe provided in a jar.") hiveContext.sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 883cdac110e0..b25684562075 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -45,8 +45,8 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef // Since every we are doing tests for DDL statements, // it is better to reset before every test. hiveContext.reset() - // Register the testData, which will be used in every test. - testData.registerTempTable("testData") + // Creates a temporary view with testData, which will be used in all tests. + testData.createOrReplaceTempView("testData") } test("insertInto() HiveTable") { @@ -98,7 +98,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef val rowRDD = hiveContext.sparkContext.parallelize( (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithMapValue") + df.createOrReplaceTempView("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m MAP )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -171,7 +171,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef StructField("a", ArrayType(StringType, containsNull = false)))) val rowRDD = hiveContext.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithArrayValue") + df.createOrReplaceTempView("tableWithArrayValue") sql("CREATE TABLE hiveTableWithArrayValue(a Array )") sql("INSERT OVERWRITE TABLE hiveTableWithArrayValue SELECT a FROM tableWithArrayValue") @@ -188,7 +188,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef val rowRDD = hiveContext.sparkContext.parallelize( (1 to 100).map(i => Row(Map(s"key$i" -> s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithMapValue") + df.createOrReplaceTempView("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m Map )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -205,7 +205,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef val rowRDD = hiveContext.sparkContext.parallelize( (1 to 100).map(i => Row(Row(s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithStructValue") + df.createOrReplaceTempView("tableWithStructValue") sql("CREATE TABLE hiveTableWithStructValue(s Struct )") sql("INSERT OVERWRITE TABLE hiveTableWithStructValue SELECT s FROM tableWithStructValue") 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 b507018e58d1..00adb9a44b14 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 @@ -80,7 +80,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM jsonTable"), sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM expectedJsonTable")) @@ -110,7 +110,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert(expectedSchema === table("jsonTable").schema) withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT b, ``.`=` FROM jsonTable"), sql("SELECT b, ``.`=` FROM expectedJsonTable")) @@ -248,7 +248,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT * FROM jsonTable"), @@ -554,7 +554,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("scan a parquet table created through a CTAS statement") { withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { withTempTable("jt") { - (1 to 10).map(i => i -> s"str$i").toDF("a", "b").registerTempTable("jt") + (1 to 10).map(i => i -> s"str$i").toDF("a", "b").createOrReplaceTempView("jt") withTable("test_parquet_ctas") { sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index 3f6418cbe85d..ac89bbbf8e19 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -74,7 +74,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val schema = spark.table("parquet_compat").schema val rowRDD = spark.sparkContext.parallelize(rows).coalesce(1) - spark.createDataFrame(rowRDD, schema).registerTempTable("data") + spark.createDataFrame(rowRDD, schema).createOrReplaceTempView("data") spark.sql("INSERT INTO TABLE parquet_compat SELECT * FROM data") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala index 78569c58085c..cc05e56d6616 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -32,7 +32,7 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl withSQLConf((SQLConf.HIVE_VERIFY_PARTITION_PATH.key, "true")) { val testData = sparkContext.parallelize( (1 to 10).map(i => TestData(i, i.toString))).toDF() - testData.registerTempTable("testData") + testData.createOrReplaceTempView("testData") val tmpDir = Files.createTempDir() // create the table for test diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 8060ef77e758..7011cd81221c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -115,7 +115,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { sql("DROP TABLE analyzeTable_part").collect() // Try to analyze a temp table - sql("""SELECT * FROM src""").registerTempTable("tempTable") + sql("""SELECT * FROM src""").createOrReplaceTempView("tempTable") intercept[UnsupportedOperationException] { hiveContext.sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala index d1aa5aa93194..d121bcbe15b3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -53,7 +53,7 @@ class UDFSuite sql("USE default") testDF = (1 to 10).map(i => s"sTr$i").toDF("value") - testDF.registerTempTable(testTableName) + testDF.createOrReplaceTempView(testTableName) expectedDF = (1 to 10).map(i => s"STR$i").toDF("value") super.beforeAll() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index c97b3f319758..a2bae2e81f30 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -180,7 +180,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te val emptyDF = spark.createDataFrame( sparkContext.emptyRDD[Row], StructType(StructField("key", StringType) :: StructField("value", IntegerType) :: Nil)) - emptyDF.registerTempTable("emptyTable") + emptyDF.createOrReplaceTempView("emptyTable") // Register UDAFs spark.udf.register("mydoublesum", new MyDoubleSum) @@ -200,7 +200,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te } test("group by function") { - Seq((1, 2)).toDF("a", "b").registerTempTable("data") + Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("data") checkAnswer( sql("SELECT floor(a) AS a, collect_set(b) FROM data GROUP BY floor(a) ORDER BY a"), @@ -783,7 +783,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te (5, 8, 17), (6, 2, 11)).toDF("a", "b", "c") - covar_tab.registerTempTable("covar_tab") + covar_tab.createOrReplaceTempView("covar_tab") checkAnswer( spark.sql( @@ -938,7 +938,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te spark.createDataFrame( sparkContext.parallelize(data, 2), schema) - .registerTempTable("noInputSchemaUDAF") + .createOrReplaceTempView("noInputSchemaUDAF") checkAnswer( spark.sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 17422ca1a02d..131b06aec8c9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -80,7 +80,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") { withTempTable("jt") { val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - hiveContext.read.json(rdd).registerTempTable("jt") + hiveContext.read.json(rdd).createOrReplaceTempView("jt") val outputs = sql( s""" |EXPLAIN EXTENDED diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala index b252c6ee2faa..4d2f190b8e6f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala @@ -29,8 +29,8 @@ class HiveOperatorQueryableSuite extends QueryTest with TestHiveSingleton { test("SPARK-5324 query result of describe command") { hiveContext.loadTestTable("src") - // register a describe command to be a temp table - sql("desc src").registerTempTable("mydesc") + // Creates a temporary view with the output of a describe command + sql("desc src").createOrReplaceTempView("mydesc") checkAnswer( sql("desc mydesc"), Seq( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala index d8d3448adde0..78c0d1f97e7d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -28,7 +28,7 @@ class HivePlanTest extends QueryTest with TestHiveSingleton { import hiveContext.implicits._ test("udf constant folding") { - Seq.empty[Tuple1[Int]].toDF("a").registerTempTable("t") + Seq.empty[Tuple1[Int]].toDF("a").createOrReplaceTempView("t") val optimized = sql("SELECT cos(null) AS c FROM t").queryExecution.optimizedPlan val correctAnswer = sql("SELECT cast(null as double) AS c FROM t").queryExecution.optimizedPlan 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 19f8cb3877b3..2aaaaadb6afa 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 @@ -685,12 +685,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("case sensitivity when query Hive table", "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") - test("case sensitivity: registered table") { + test("case sensitivity: created temporary view") { val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) - testData.toDF().registerTempTable("REGisteredTABle") + testData.toDF().createOrReplaceTempView("REGisteredTABle") assertResult(Array(Row(2, "str2"))) { sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + @@ -715,7 +715,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case ((value, attr), key) => HavingRow(key, value, attr)} - TestHive.sparkContext.parallelize(fixture).toDF().registerTempTable("having_test") + TestHive.sparkContext.parallelize(fixture).toDF().createOrReplaceTempView("having_test") val results = sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() @@ -819,12 +819,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .collect() } - // Describe a registered temporary table. + // Describe a temporary view. val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) - testData.toDF().registerTempTable("test_describe_commands2") + testData.toDF().createOrReplaceTempView("test_describe_commands2") assertResult( Array( @@ -996,9 +996,9 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } - test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { - sparkContext.makeRDD(Seq.empty[LogEntry]).toDF().registerTempTable("rawLogs") - sparkContext.makeRDD(Seq.empty[LogFile]).toDF().registerTempTable("logFiles") + test("SPARK-3414 regression: should store analyzed logical plan when creating a temporary view") { + sparkContext.makeRDD(Seq.empty[LogEntry]).toDF().createOrReplaceTempView("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).toDF().createOrReplaceTempView("logFiles") sql( """ @@ -1009,7 +1009,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { FROM logFiles ) files ON rawLogs.filename = files.name - """).registerTempTable("boom") + """).createOrReplaceTempView("boom") // This should be successfully analyzed sql("SELECT * FROM boom").queryExecution.analyzed diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index dd13b8392880..b2f19d775395 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -32,14 +32,14 @@ class HiveResolutionSuite extends HiveComparisonTest { test("SPARK-3698: case insensitive test for nested data") { read.json(sparkContext.makeRDD( - """{"a": [{"a": {"a": 1}}]}""" :: Nil)).registerTempTable("nested") + """{"a": [{"a": {"a": 1}}]}""" :: Nil)).createOrReplaceTempView("nested") // This should be successfully analyzed sql("SELECT a[0].A.A from nested").queryExecution.analyzed } test("SPARK-5278: check ambiguous reference to fields") { read.json(sparkContext.makeRDD( - """{"a": [{"b": 1, "B": 2}]}""" :: Nil)).registerTempTable("nested") + """{"a": [{"b": 1, "B": 2}]}""" :: Nil)).createOrReplaceTempView("nested") // there are 2 filed matching field name "b", we should report Ambiguous reference error val exception = intercept[AnalysisException] { @@ -78,7 +78,7 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1, 2), Seq(Nested(1, 2))) :: Nil) - .toDF().registerTempTable("caseSensitivityTest") + .toDF().createOrReplaceTempView("caseSensitivityTest") val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") assert(query.schema.fields.map(_.name) === Seq("a", "b", "A", "B", "a", "b", "A", "B"), @@ -89,14 +89,14 @@ class HiveResolutionSuite extends HiveComparisonTest { ignore("case insensitivity with scala reflection joins") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1, 2), Seq(Nested(1, 2))) :: Nil) - .toDF().registerTempTable("caseSensitivityTest") + .toDF().createOrReplaceTempView("caseSensitivityTest") sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") { sparkContext.parallelize(Data(1, 2, Nested(1, 2), Seq(Nested(1, 2))) :: Nil) - .toDF().registerTempTable("nestedRepeatedTest") + .toDF().createOrReplaceTempView("nestedRepeatedTest") assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 8c9c37fecec4..60f8be5e0e82 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -84,7 +84,7 @@ class HiveTableScanSuite extends HiveComparisonTest { sql("""insert into table spark_4959 select "hi" from src limit 1""") table("spark_4959").select( 'col1.as("CaseSensitiveColName"), - 'col1.as("CaseSensitiveColName2")).registerTempTable("spark_4959_2") + 'col1.as("CaseSensitiveColName2")).createOrReplaceTempView("spark_4959_2") assert(sql("select CaseSensitiveColName from spark_4959_2").head() === Row("hi")) assert(sql("select casesensitivecolname from spark_4959_2").head() === Row("hi")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 521964eb4eb5..23b7f6c75b48 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -153,7 +153,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFIntegerToString") { val testData = hiveContext.sparkContext.parallelize( IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() - testData.registerTempTable("integerTable") + testData.createOrReplaceTempView("integerTable") val udfName = classOf[UDFIntegerToString].getName sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '$udfName'") @@ -167,7 +167,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToListString") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListString AS '${classOf[UDFToListString].getName}'") val errMsg = intercept[AnalysisException] { @@ -182,7 +182,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToListInt") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListInt AS '${classOf[UDFToListInt].getName}'") val errMsg = intercept[AnalysisException] { @@ -197,7 +197,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToStringIntMap") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToStringIntMap " + s"AS '${classOf[UDFToStringIntMap].getName}'") @@ -213,7 +213,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToIntIntMap") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToIntIntMap " + s"AS '${classOf[UDFToIntIntMap].getName}'") @@ -232,7 +232,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() - testData.registerTempTable("listListIntTable") + testData.createOrReplaceTempView("listListIntTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") checkAnswer( @@ -247,7 +247,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { val testData = hiveContext.sparkContext.parallelize( ListStringCaseClass(Seq("a", "b", "c")) :: ListStringCaseClass(Seq("d", "e")) :: Nil).toDF() - testData.registerTempTable("listStringTable") + testData.createOrReplaceTempView("listStringTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") checkAnswer( @@ -261,7 +261,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFStringString") { val testData = hiveContext.sparkContext.parallelize( StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF() - testData.registerTempTable("stringTable") + testData.createOrReplaceTempView("stringTable") sql(s"CREATE TEMPORARY FUNCTION testStringStringUDF AS '${classOf[UDFStringString].getName}'") checkAnswer( @@ -283,7 +283,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() - testData.registerTempTable("TwoListTable") + testData.createOrReplaceTempView("TwoListTable") sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") checkAnswer( @@ -295,7 +295,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("Hive UDFs with insufficient number of input arguments should trigger an analysis error") { - Seq((1, 2)).toDF("a", "b").registerTempTable("testUDF") + Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("testUDF") { // HiveSimpleUDF @@ -352,7 +352,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("Hive UDF in group by") { withTempTable("tab1") { - Seq(Tuple1(1451400761)).toDF("test_date").registerTempTable("tab1") + Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") val count = sql("select testUDFToDate(cast(test_date as timestamp))" + " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() 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 ac9a3930fd21..81f3ea8a6e80 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 @@ -102,14 +102,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-6835: udtf in lateral view") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("table1") val query = sql("SELECT c1, v FROM table1 LATERAL VIEW stack(3, 1, c1 + 1, c1 + 2) d AS v") checkAnswer(query, Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: Nil) } test("SPARK-13651: generator outputs shouldn't be resolved from its child's output") { withTempTable("src") { - Seq(("id1", "value1")).toDF("key", "value").registerTempTable("src") + Seq(("id1", "value1")).toDF("key", "value").createOrReplaceTempView("src") val query = sql("SELECT genoutput.* FROM src " + "LATERAL VIEW explode(map('key1', 100, 'key2', 200)) genoutput AS key, value") @@ -135,8 +135,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Order(1, "Atlas", "MTB", 434, "2015-01-07", "John D", "Pacifica", "CA", 20151), Order(11, "Swift", "YFlikr", 137, "2015-01-23", "John D", "Hayward", "CA", 20151)) - orders.toDF.registerTempTable("orders1") - orderUpdates.toDF.registerTempTable("orderupdates1") + orders.toDF.createOrReplaceTempView("orders1") + orderUpdates.toDF.createOrReplaceTempView("orderupdates1") sql( """CREATE TABLE orders( @@ -305,7 +305,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-5371: union with null and sum") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("table1") val query = sql( """ @@ -329,7 +329,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("CTAS with WITH clause") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("table1") sql( """ @@ -346,7 +346,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("explode nested Field") { - Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.registerTempTable("nestedArray") + Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.createOrReplaceTempView("nestedArray") checkAnswer( sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"), Row(1) :: Row(2) :: Row(3) :: Nil) @@ -543,7 +543,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("specifying the column list for CTAS") { - Seq((1, "111111"), (2, "222222")).toDF("key", "value").registerTempTable("mytable1") + Seq((1, "111111"), (2, "222222")).toDF("key", "value").createOrReplaceTempView("mytable1") sql("create table gen__tmp(a int, b string) as select key, value from mytable1") checkAnswer( @@ -598,7 +598,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("double nested data") { sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil) - .toDF().registerTempTable("nested") + .toDF().createOrReplaceTempView("nested") checkAnswer( sql("SELECT f1.f2.f3 FROM nested"), Row(1)) @@ -682,7 +682,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-4963 DataFrame sample on mutable row return wrong result") { sql("SELECT * FROM src WHERE key % 2 = 0") .sample(withReplacement = false, fraction = 0.3) - .registerTempTable("sampled") + .createOrReplaceTempView("sampled") (1 to 10).foreach { i => checkAnswer( sql("SELECT * FROM sampled WHERE key % 2 = 1"), @@ -707,7 +707,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val rowRdd = sparkContext.parallelize(row :: Nil) - hiveContext.createDataFrame(rowRdd, schema).registerTempTable("testTable") + hiveContext.createDataFrame(rowRdd, schema).createOrReplaceTempView("testTable") sql( """CREATE TABLE nullValuesInInnerComplexTypes @@ -733,14 +733,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-4296 Grouping field with Hive UDF as sub expression") { val rdd = sparkContext.makeRDD( """{"a": "str", "b":"1", "c":"1970-01-01 00:00:00"}""" :: Nil) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer( sql("SELECT concat(a, '-', b), year(c) FROM data GROUP BY concat(a, '-', b), year(c)"), Row("str-1", 1970)) dropTempTable("data") - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer(sql("SELECT year(c) + 1 FROM data GROUP BY year(c) + 1"), Row(1971)) dropTempTable("data") @@ -748,14 +748,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("resolve udtf in projection #1") { val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") val df = sql("SELECT explode(a) AS val FROM data") val col = df("val") } test("resolve udtf in projection #2") { val rdd = sparkContext.makeRDD((1 to 2).map(i => s"""{"a":[$i, ${i + 1}]}""")) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer(sql("SELECT explode(map(1, 1)) FROM data LIMIT 1"), Row(1, 1) :: Nil) checkAnswer(sql("SELECT explode(map(1, 1)) as (k1, k2) FROM data LIMIT 1"), Row(1, 1) :: Nil) intercept[AnalysisException] { @@ -770,7 +770,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // TGF with non-TGF in project is allowed in Spark SQL, but not in Hive test("TGF with non-TGF in projection") { val rdd = sparkContext.makeRDD( """{"a": "1", "b":"1"}""" :: Nil) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer( sql("SELECT explode(map(a, b)) as (k1, k2), a, b FROM data"), Row("1", "1", "1", "1") :: Nil) @@ -784,7 +784,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // PreInsertionCasts will actually start to work before ImplicitGenerate and then // generates an invalid query plan. val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") val originalConf = sessionState.convertCTAS setConf(HiveUtils.CONVERT_CTAS, false) @@ -824,7 +824,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Seq.empty[(java.math.BigDecimal, java.math.BigDecimal)] .toDF("d1", "d2") .select($"d1".cast(DecimalType(10, 5)).as("d")) - .registerTempTable("dn") + .createOrReplaceTempView("dn") sql("select d from dn union all select d * 2 from dn") .queryExecution.analyzed @@ -832,27 +832,27 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("Star Expansion - script transform") { val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(100000 === sql("SELECT TRANSFORM (*) USING 'cat' FROM script_trans").count()) } test("test script transform for stdout") { val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(100000 === sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans").count()) } test("test script transform for stderr") { val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(0 === sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat 1>&2' AS (a,b,c) FROM script_trans").count()) } test("test script transform data type") { val data = (1 to 5).map { i => (i, i) } - data.toDF("key", "value").registerTempTable("test") + data.toDF("key", "value").createOrReplaceTempView("test") checkAnswer( sql("""FROM |(FROM test SELECT TRANSFORM(key, value) USING 'cat' AS (`thing1` int, thing2 string)) t @@ -864,7 +864,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { withTempTable("data") { spark.range(1, 5) .select(array($"id", $"id" + 1).as("a"), $"id".as("b"), (lit(10) - $"id").as("c")) - .registerTempTable("data") + .createOrReplaceTempView("data") // case 1: missing sort columns are resolvable if join is true checkAnswer( @@ -888,7 +888,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("test case key when") { - (1 to 5).map(i => (i, i.toString)).toDF("k", "v").registerTempTable("t") + (1 to 5).map(i => (i, i.toString)).toDF("k", "v").createOrReplaceTempView("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) @@ -897,7 +897,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-7269 Check analysis failed in case in-sensitive") { Seq(1, 2, 3).map { i => (i.toString, i.toString) - }.toDF("key", "value").registerTempTable("df_analysis") + }.toDF("key", "value").createOrReplaceTempView("df_analysis") sql("SELECT kEy from df_analysis group by key").collect() sql("SELECT kEy+3 from df_analysis group by key+3").collect() sql("SELECT kEy+3, a.kEy, A.kEy from df_analysis A group by key").collect() @@ -1031,7 +1031,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-8588 HiveTypeCoercion.inConversion fires too early") { val df = createDataFrame(Seq((1, "2014-01-01"), (2, "2015-01-01"), (3, "2016-01-01"))) - df.toDF("id", "datef").registerTempTable("test_SPARK8588") + df.toDF("id", "datef").createOrReplaceTempView("test_SPARK8588") checkAnswer( sql( """ @@ -1046,7 +1046,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-9371: fix the support for special chars in column names for hive context") { read.json(sparkContext.makeRDD( """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) - .registerTempTable("t") + .createOrReplaceTempView("t") checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } @@ -1125,7 +1125,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { spark .range(5) .selectExpr("id AS a", "id AS b") - .registerTempTable("test") + .createOrReplaceTempView("test") checkAnswer( sql( @@ -1143,7 +1143,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { spark .range(5) .selectExpr("id AS a", "id AS b") - .registerTempTable("test") + .createOrReplaceTempView("test") val df = sql( """FROM test @@ -1162,7 +1162,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-10741: Sort on Aggregate using parquet") { withTable("test10741") { withTempTable("src") { - Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF().registerTempTable("src") + Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF().createOrReplaceTempView("src") sql("CREATE TABLE test10741(c1 STRING, c2 INT) STORED AS PARQUET AS SELECT * FROM src") } @@ -1374,7 +1374,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { withTempTable("t1") { spark.range(10) .select(array($"id", $"id" + 1).as("arr"), $"id") - .registerTempTable("source") + .createOrReplaceTempView("source") withTable("dest1", "dest2") { sql("CREATE TABLE dest1 (i INT)") sql("CREATE TABLE dest2 (i INT)") @@ -1407,7 +1407,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { withTempTable("t1", "t2") { val path = dir.getCanonicalPath val ds = spark.range(10) - ds.registerTempTable("t1") + ds.createOrReplaceTempView("t1") sql( s"""CREATE TEMPORARY TABLE t2 @@ -1431,7 +1431,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { "shouldn always be used together with PATH data source option" ) { withTempTable("t") { - spark.range(10).registerTempTable("t") + spark.range(10).createOrReplaceTempView("t") val message = intercept[IllegalArgumentException] { sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala index 4d284e10425d..47ceefb88ebc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala @@ -40,7 +40,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -112,7 +112,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -139,7 +139,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -182,7 +182,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") val e = intercept[AnalysisException] { sql( @@ -203,7 +203,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -232,7 +232,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 11) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql("select month, product, sum(product + 1) over() from windowData order by area"), @@ -301,7 +301,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 11) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -322,7 +322,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi test("window function: multiple window expressions in a single expression") { val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") - nums.registerTempTable("nums") + nums.createOrReplaceTempView("nums") val expected = Row(1, 1, 1, 55, 1, 57) :: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala index 6161412a4977..fed0d11e9d21 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala @@ -90,7 +90,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - read.orc(base.getCanonicalPath).registerTempTable("t") + read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -137,7 +137,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - read.orc(base.getCanonicalPath).registerTempTable("t") + read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -189,7 +189,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B read .option(ConfVars.DEFAULTPARTITIONNAME.varname, defaultPartitionName) .orc(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -231,7 +231,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B read .option(ConfVars.DEFAULTPARTITIONNAME.varname, defaultPartitionName) .orc(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 9a0885822b8d..f83b3a3de2e5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -98,7 +98,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("Creating case class RDD table") { val data = (1 to 100).map(i => (i, s"val_$i")) - sparkContext.parallelize(data).toDF().registerTempTable("t") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("t") withTempTable("t") { checkAnswer(sql("SELECT * FROM t"), data.toDF().collect()) } @@ -223,7 +223,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("appending") { val data = (0 until 10).map(i => (i, i.toString)) - createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withOrcTable(data, "t") { sql("INSERT INTO TABLE t SELECT * FROM tmp") checkAnswer(table("t"), (data ++ data).map(Row.fromTuple)) @@ -233,7 +233,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) - createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withOrcTable(data, "t") { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(table("t"), data.map(Row.fromTuple)) @@ -324,7 +324,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { """.stripMargin) val emptyDF = Seq.empty[(Int, String)].toDF("key", "value").coalesce(1) - emptyDF.registerTempTable("empty") + emptyDF.createOrReplaceTempView("empty") // This creates 1 empty ORC file with Hive ORC SerDe. We are using this trick because // Spark SQL ORC data source always avoids write empty ORC files. @@ -340,7 +340,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { assert(errorMessage.contains("Unable to infer schema for ORC")) val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1) - singleRowDF.registerTempTable("single") + singleRowDF.createOrReplaceTempView("single") spark.sql( s"""INSERT INTO TABLE empty_orc @@ -422,7 +422,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { """.stripMargin) val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1) - singleRowDF.registerTempTable("single") + singleRowDF.createOrReplaceTempView("single") spark.sql( s"""INSERT INTO TABLE dummy_orc diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala index 96a7364437c7..6081d86f4478 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala @@ -51,7 +51,7 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA .makeRDD(1 to 10) .map(i => OrcData(i, s"part-$i")) .toDF() - .registerTempTable(s"orc_temp_table") + .createOrReplaceTempView(s"orc_temp_table") sql( s"""CREATE EXTERNAL TABLE normal_orc( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index f52c6e48c576..7fe158c21846 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -171,8 +171,9 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql(s"ALTER TABLE partitioned_parquet_with_complextypes ADD PARTITION (p=$p)") } - (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").registerTempTable("jt") - (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a").registerTempTable("jt_array") + (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").createOrReplaceTempView("jt") + (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a") + .createOrReplaceTempView("jt_array") setConf(HiveUtils.CONVERT_METASTORE_PARQUET, true) } @@ -541,8 +542,8 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { |STORED AS PARQUET """.stripMargin) - // Temp table to insert data into partitioned table - Seq("foo", "bar").toDF("a").registerTempTable("test_temp") + // Temp view that is used to insert data into partitioned table + Seq("foo", "bar").toDF("a").createOrReplaceTempView("test_temp") sql("INSERT INTO test_added_partitions PARTITION(b='0') SELECT a FROM test_temp") checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 78d2dc28d6b5..a3183f297722 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -91,7 +91,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes yield Row(s"val_$i", s"val_$i", s"val_$i", s"val_$i", 1, 1, 1, 1)) // Self-join - df.registerTempTable("t") + df.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( sql( @@ -337,7 +337,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { intercept[AnalysisException] { @@ -347,7 +347,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t") @@ -459,7 +459,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { intercept[AnalysisException] { @@ -474,7 +474,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { partitionedTestDF.write diff --git a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala index 5df674d60e9c..1c1db72e27ff 100644 --- a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala +++ b/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -66,7 +66,7 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac val res = df3.collect() val expected = Seq((18, 18, 8)).toDF("a", "x", "b").collect() assert(res.toSeq == expected.toSeq) - df3.registerTempTable("mai_table") + df3.createOrReplaceTempView("mai_table") val df4 = hc.table("mai_table") val res2 = df4.collect() assert(res2.toSeq == expected.toSeq) @@ -82,7 +82,7 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac val databases2 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) assert(databases2.toSet == Set("default", "mee_db")) val df = (1 to 10).map { i => ("bob" + i.toString, i) }.toDF("name", "age") - df.registerTempTable("mee_table") + df.createOrReplaceTempView("mee_table") hc.sql("CREATE TABLE moo_table (name string, age int)") hc.sql("INSERT INTO moo_table SELECT * FROM mee_table") assert( From c8be3da66903899fcd743c425c25e32fc356d981 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 17 May 2016 18:07:59 -0700 Subject: [PATCH 247/313] Prepare branch for 2.0.0-preview. --- core/src/main/scala/org/apache/spark/package.scala | 2 +- docs/_config.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index cc5e7ef3ae00..65a3dd2e37e1 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -43,5 +43,5 @@ package org.apache package object spark { // For package docs only - val SPARK_VERSION = "2.0.0-SNAPSHOT" + val SPARK_VERSION = "2.0.0-preview" } diff --git a/docs/_config.yml b/docs/_config.yml index 8bdc68aeeac7..c0a3be7f8208 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,7 +14,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.0-SNAPSHOT +SPARK_VERSION: 2.0.0-preview SPARK_VERSION_SHORT: 2.0.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" From 8f5a04b6299e3a47aca13cbb40e72344c0114860 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 17 May 2016 18:15:42 -0700 Subject: [PATCH 248/313] Preparing Spark release 2.0.0-preview --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/hivecontext-compatibility/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 33 files changed, 33 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 75ac9262cbae..958cb453b554 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5444ae6d70a9..2cb86ea8a4c1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index e736436aec4c..58d7879253fb 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 1fd3af2e6e62..3f8dfe4c20fa 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index bbbb0bd5aa05..03db5b81ff7d 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 14e94eca93b2..e10085136b68 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index d0d1da69ea80..0c2aa8b4b532 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 45f8bfcc05aa..ce5eac514046 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 771da5b9a6e6..3db074aaa780 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 21d40863b77f..b0fc3b943c63 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index ac15b93c048d..51abd3375d49 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 016af934bb89..e22ee74cf686 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f51d334de087..1c716ab46288 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 60e3ff60df06..57d25630ccbe 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 3cc288abeaa2..aff01b6fcf9e 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 4a20b78917ef..da62aead7c6c 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 6fb88ebae5b3..1b04ca860a87 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b5f5ff2854cf..532fb70ba8d1 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index bfb92791de3d..bc12f2b54ee4 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc6c700dd1ec..51eb281c6fb1 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e7303853e656..f536506fc412 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1c6ab2b62d8f..e526ff3c1f76 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c324afb2a2d1..df9681143a6b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/pom.xml b/pom.xml index 9c13af17e4ed..6d1f030da4bb 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index c12d121c6115..5553d3640921 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 1923199f4b86..d0238449ec68 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 2ea980bf20df..cb5a49980879 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 809d36dc69b9..a35affa71160 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c8b20f0afc4e..b9eb5ccca810 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/hivecontext-compatibility/pom.xml b/sql/hivecontext-compatibility/pom.xml index ed9ef8e27919..b8f091eff69c 100644 --- a/sql/hivecontext-compatibility/pom.xml +++ b/sql/hivecontext-compatibility/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 3f6774593644..89c2f8f7e1bd 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 9bb20e138106..001efbca1916 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index e07b93ab9545..fcc94ba0ad4e 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml From b54500913d49b91949716b2c41bd5f637b1795a7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 17 May 2016 18:15:51 -0700 Subject: [PATCH 249/313] Preparing development version 2.0.0-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/hivecontext-compatibility/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 33 files changed, 33 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 958cb453b554..75ac9262cbae 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2cb86ea8a4c1..5444ae6d70a9 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 58d7879253fb..e736436aec4c 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3f8dfe4c20fa..1fd3af2e6e62 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 03db5b81ff7d..bbbb0bd5aa05 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e10085136b68..14e94eca93b2 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 0c2aa8b4b532..d0d1da69ea80 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ce5eac514046..45f8bfcc05aa 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 3db074aaa780..771da5b9a6e6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index b0fc3b943c63..21d40863b77f 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 51abd3375d49..ac15b93c048d 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index e22ee74cf686..016af934bb89 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 1c716ab46288..f51d334de087 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 57d25630ccbe..60e3ff60df06 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index aff01b6fcf9e..3cc288abeaa2 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index da62aead7c6c..4a20b78917ef 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 1b04ca860a87..6fb88ebae5b3 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 532fb70ba8d1..b5f5ff2854cf 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index bc12f2b54ee4..bfb92791de3d 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 51eb281c6fb1..fc6c700dd1ec 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index f536506fc412..e7303853e656 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index e526ff3c1f76..1c6ab2b62d8f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index df9681143a6b..c324afb2a2d1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 6d1f030da4bb..9c13af17e4ed 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 5553d3640921..c12d121c6115 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index d0238449ec68..1923199f4b86 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index cb5a49980879..2ea980bf20df 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a35affa71160..809d36dc69b9 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b9eb5ccca810..c8b20f0afc4e 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hivecontext-compatibility/pom.xml b/sql/hivecontext-compatibility/pom.xml index b8f091eff69c..ed9ef8e27919 100644 --- a/sql/hivecontext-compatibility/pom.xml +++ b/sql/hivecontext-compatibility/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 89c2f8f7e1bd..3f6774593644 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 001efbca1916..9bb20e138106 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index fcc94ba0ad4e..e07b93ab9545 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml From fc97ff5d7f56f85bc5cf2383d4786005e067a0df Mon Sep 17 00:00:00 2001 From: Takuya Kuwahara Date: Wed, 18 May 2016 08:29:47 +0200 Subject: [PATCH 250/313] [SPARK-14978][PYSPARK] PySpark TrainValidationSplitModel should support validationMetrics ## What changes were proposed in this pull request? This pull request includes supporting validationMetrics for TrainValidationSplitModel with Python and test for it. ## How was this patch tested? test in `python/pyspark/ml/tests.py` Author: Takuya Kuwahara Closes #12767 from taku-k/spark-14978. (cherry picked from commit 411c04adb596c514f2634efd5f5d126e12b05df7) Signed-off-by: Nick Pentreath --- python/pyspark/ml/tests.py | 50 ++++++++++++++++++++++++++++++++----- python/pyspark/ml/tuning.py | 13 +++++++--- 2 files changed, 53 insertions(+), 10 deletions(-) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index e3511120bdec..a7c93ac80272 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -610,17 +610,21 @@ def test_fit_minimize_metric(self): iee = InducedErrorEstimator() evaluator = RegressionEvaluator(metricName="rmse") - grid = (ParamGridBuilder() - .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) - .build()) + grid = ParamGridBuilder() \ + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) \ + .build() tvs = TrainValidationSplit(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) tvsModel = tvs.fit(dataset) bestModel = tvsModel.bestModel bestModelMetric = evaluator.evaluate(bestModel.transform(dataset)) + validationMetrics = tvsModel.validationMetrics self.assertEqual(0.0, bestModel.getOrDefault('inducedError'), "Best model should have zero induced error") self.assertEqual(0.0, bestModelMetric, "Best model has RMSE of 0") + self.assertEqual(len(grid), len(validationMetrics), + "validationMetrics has the same size of grid parameter") + self.assertEqual(0.0, min(validationMetrics)) def test_fit_maximize_metric(self): dataset = self.spark.createDataFrame([ @@ -633,17 +637,21 @@ def test_fit_maximize_metric(self): iee = InducedErrorEstimator() evaluator = RegressionEvaluator(metricName="r2") - grid = (ParamGridBuilder() - .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) - .build()) + grid = ParamGridBuilder() \ + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) \ + .build() tvs = TrainValidationSplit(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) tvsModel = tvs.fit(dataset) bestModel = tvsModel.bestModel bestModelMetric = evaluator.evaluate(bestModel.transform(dataset)) + validationMetrics = tvsModel.validationMetrics self.assertEqual(0.0, bestModel.getOrDefault('inducedError'), "Best model should have zero induced error") self.assertEqual(1.0, bestModelMetric, "Best model has R-squared of 1") + self.assertEqual(len(grid), len(validationMetrics), + "validationMetrics has the same size of grid parameter") + self.assertEqual(1.0, max(validationMetrics)) def test_save_load(self): # This tests saving and loading the trained model only. @@ -669,6 +677,36 @@ def test_save_load(self): self.assertEqual(loadedLrModel.uid, lrModel.uid) self.assertEqual(loadedLrModel.intercept, lrModel.intercept) + def test_copy(self): + dataset = self.spark.createDataFrame([ + (10, 10.0), + (50, 50.0), + (100, 100.0), + (500, 500.0)] * 10, + ["feature", "label"]) + + iee = InducedErrorEstimator() + evaluator = RegressionEvaluator(metricName="r2") + + grid = ParamGridBuilder() \ + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) \ + .build() + tvs = TrainValidationSplit(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) + tvsModel = tvs.fit(dataset) + tvsCopied = tvs.copy() + tvsModelCopied = tvsModel.copy() + + self.assertEqual(tvs.getEstimator().uid, tvsCopied.getEstimator().uid, + "Copied TrainValidationSplit has the same uid of Estimator") + + self.assertEqual(tvsModel.bestModel.uid, tvsModelCopied.bestModel.uid) + self.assertEqual(len(tvsModel.validationMetrics), + len(tvsModelCopied.validationMetrics), + "Copied validationMetrics has the same size of the original") + for index in range(len(tvsModel.validationMetrics)): + self.assertEqual(tvsModel.validationMetrics[index], + tvsModelCopied.validationMetrics[index]) + class PersistenceTest(SparkSessionTestCase): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 75789c4d0918..4f7a6b0f7b73 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -379,7 +379,7 @@ def _fit(self, dataset): seed = self.getOrDefault(self.seed) randCol = self.uid + "_rand" df = dataset.select("*", rand(seed).alias(randCol)) - metrics = np.zeros(numModels) + metrics = [0.0] * numModels condition = (df[randCol] >= tRatio) validation = df.filter(condition) train = df.filter(~condition) @@ -392,7 +392,7 @@ def _fit(self, dataset): else: bestIndex = np.argmin(metrics) bestModel = est.fit(dataset, epm[bestIndex]) - return self._copyValues(TrainValidationSplitModel(bestModel)) + return self._copyValues(TrainValidationSplitModel(bestModel, metrics)) @since("2.0.0") def copy(self, extra=None): @@ -424,10 +424,12 @@ class TrainValidationSplitModel(Model, ValidatorParams): .. versionadded:: 2.0.0 """ - def __init__(self, bestModel): + def __init__(self, bestModel, validationMetrics=[]): super(TrainValidationSplitModel, self).__init__() #: best model from cross validation self.bestModel = bestModel + #: evaluated validation metrics + self.validationMetrics = validationMetrics def _transform(self, dataset): return self.bestModel.transform(dataset) @@ -439,13 +441,16 @@ def copy(self, extra=None): and some extra params. This copies the underlying bestModel, creates a deep copy of the embedded paramMap, and copies the embedded and extra parameters over. + And, this creates a shallow copy of the validationMetrics. :param extra: Extra parameters to copy to the new instance :return: Copy of this instance """ if extra is None: extra = dict() - return TrainValidationSplitModel(self.bestModel.copy(extra)) + bestModel = self.bestModel.copy(extra) + validationMetrics = list(self.validationMetrics) + return TrainValidationSplitModel(bestModel, validationMetrics) if __name__ == "__main__": From c66da74b1efdc1089b61d79d51cd32c635ad3b5e Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 18 May 2016 16:00:02 +0800 Subject: [PATCH 251/313] [SPARK-15334][SQL] HiveClient facade not compatible with Hive 0.12 ## What changes were proposed in this pull request? HiveClient facade is not compatible with Hive 0.12. This PR Fixes the following compatibility issues: 1. `org.apache.spark.sql.hive.client.HiveClientImpl` use `AddPartitionDesc(db, table, ignoreIfExists)` to create partitions, however, Hive 0.12 doesn't have this constructor for `AddPartitionDesc`. 2. `HiveClientImpl` uses `PartitionDropOptions` when dropping partition, however, class `PartitionDropOptions` doesn't exist in Hive 0.12. 3. Hive 0.12 doesn't support adding permanent functions. It is not valid to call `org.apache.hadoop.hive.ql.metadata.Hive.createFunction`, `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction`, and `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction` 4. `org.apache.spark.sql.hive.client.VersionsSuite` doesn't have enough test coverage for different hive versions 0.12, 0.13, 0.14, 1.0.0, 1.1.0, 1.2.0. ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13127 from clockfly/versionSuite. (cherry picked from commit 6e02aec44b9e5bc2ada55cb612f26e6ba000c23e) Signed-off-by: Cheng Lian --- .../sql/hive/client/HiveClientImpl.scala | 74 +--- .../spark/sql/hive/client/HiveShim.scala | 182 +++++++- .../spark/sql/hive/client/VersionsSuite.scala | 412 +++++++++++++++--- 3 files changed, 545 insertions(+), 123 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index a4e9f03b4334..af2850d4f568 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -26,14 +26,10 @@ import scala.language.reflectiveCalls import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.{PartitionDropOptions, TableType => HiveTableType} +import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} -import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType} -import org.apache.hadoop.hive.metastore.api.{NoSuchObjectException, PrincipalType} -import org.apache.hadoop.hive.metastore.api.{ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} -import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.UserGroupInformation @@ -41,13 +37,13 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.util.{CausedBy, CircularBuffer, Utils} +import org.apache.spark.util.{CircularBuffer, Utils} /** * A class that wraps the HiveClient and converts its responses to externally visible classes. @@ -400,11 +396,7 @@ private[hive] class HiveClientImpl( table: String, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = withHiveState { - val addPartitionDesc = new AddPartitionDesc(db, table, ignoreIfExists) - parts.foreach { s => - addPartitionDesc.addPartition(s.spec.asJava, s.storage.locationUri.orNull) - } - client.createPartitions(addPartitionDesc) + shim.createPartitions(client, db, table, parts, ignoreIfExists) } override def dropPartitions( @@ -430,10 +422,9 @@ private[hive] class HiveClientImpl( }.distinct var droppedParts = ArrayBuffer.empty[java.util.List[String]] matchingParts.foreach { partition => - val dropOptions = new PartitionDropOptions - dropOptions.ifExists = ignoreIfNotExists try { - client.dropPartition(db, table, partition, dropOptions) + val deleteData = true + client.dropPartition(db, table, partition, deleteData) } catch { case e: Exception => val remainingParts = matchingParts.toBuffer -- droppedParts @@ -629,37 +620,28 @@ private[hive] class HiveClientImpl( } override def createFunction(db: String, func: CatalogFunction): Unit = withHiveState { - client.createFunction(toHiveFunction(func, db)) + shim.createFunction(client, db, func) } override def dropFunction(db: String, name: String): Unit = withHiveState { - client.dropFunction(db, name) + shim.dropFunction(client, db, name) } override def renameFunction(db: String, oldName: String, newName: String): Unit = withHiveState { - val catalogFunc = getFunction(db, oldName) - .copy(identifier = FunctionIdentifier(newName, Some(db))) - val hiveFunc = toHiveFunction(catalogFunc, db) - client.alterFunction(db, oldName, hiveFunc) + shim.renameFunction(client, db, oldName, newName) } override def alterFunction(db: String, func: CatalogFunction): Unit = withHiveState { - client.alterFunction(db, func.identifier.funcName, toHiveFunction(func, db)) + shim.alterFunction(client, db, func) } override def getFunctionOption( - db: String, - name: String): Option[CatalogFunction] = withHiveState { - try { - Option(client.getFunction(db, name)).map(fromHiveFunction) - } catch { - case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => - None - } + db: String, name: String): Option[CatalogFunction] = withHiveState { + shim.getFunctionOption(client, db, name) } override def listFunctions(db: String, pattern: String): Seq[String] = withHiveState { - client.getFunctions(db, pattern).asScala + shim.listFunctions(client, db, pattern) } def addJar(path: String): Unit = { @@ -708,36 +690,6 @@ private[hive] class HiveClientImpl( Utils.classForName(name) .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] - private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { - val resourceUris = f.resources.map { resource => - new ResourceUri( - ResourceType.valueOf(resource.resourceType.resourceType.toUpperCase()), resource.uri) - } - new HiveFunction( - f.identifier.funcName, - db, - f.className, - null, - PrincipalType.USER, - (System.currentTimeMillis / 1000).toInt, - FunctionType.JAVA, - resourceUris.asJava) - } - - private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { - val name = FunctionIdentifier(hf.getFunctionName, Option(hf.getDbName)) - val resources = hf.getResourceUris.asScala.map { uri => - val resourceType = uri.getResourceType() match { - case ResourceType.ARCHIVE => "archive" - case ResourceType.FILE => "file" - case ResourceType.JAR => "jar" - case r => throw new AnalysisException(s"Unknown resource type: $r") - } - FunctionResource(FunctionResourceType.fromString(resourceType), uri.getUri()) - } - new CatalogFunction(name, hf.getClassName, resources) - } - private def toHiveColumn(c: CatalogColumn): FieldSchema = { new FieldSchema(c.name, c.dataType, c.comment.orNull) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 4ecf866f9639..78713c3f0bac 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -27,15 +27,23 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType, NoSuchObjectException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} +import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException +import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{IntegralType, StringType} +import org.apache.spark.util.CausedBy + /** * A shim that defines the interface between [[HiveClientImpl]] and the underlying Hive library used @@ -73,6 +81,13 @@ private[client] sealed abstract class Shim { def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long + def createPartitions( + hive: Hive, + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit + def loadPartition( hive: Hive, loadPath: Path, @@ -100,6 +115,18 @@ private[client] sealed abstract class Shim { holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit + def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit + + def dropFunction(hive: Hive, db: String, name: String): Unit + + def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit + + def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit + + def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] + + def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] + def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit protected def findStaticMethod(klass: Class[_], name: String, args: Class[_]*): Method = { @@ -112,7 +139,6 @@ private[client] sealed abstract class Shim { protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { klass.getMethod(name, args: _*) } - } private[client] class Shim_v0_12 extends Shim with Logging { @@ -144,6 +170,22 @@ private[client] class Shim_v0_12 extends Shim with Logging { classOf[Driver], "getResults", classOf[JArrayList[String]]) + private lazy val createPartitionMethod = + findMethod( + classOf[Hive], + "createPartition", + classOf[Table], + classOf[JMap[String, String]], + classOf[Path], + classOf[JMap[String, String]], + classOf[String], + classOf[String], + JInteger.TYPE, + classOf[JList[Object]], + classOf[String], + classOf[JMap[String, String]], + classOf[JList[Object]], + classOf[JList[Object]]) private lazy val loadPartitionMethod = findMethod( classOf[Hive], @@ -199,6 +241,42 @@ private[client] class Shim_v0_12 extends Shim with Logging { override def setDataLocation(table: Table, loc: String): Unit = setDataLocationMethod.invoke(table, new URI(loc)) + // Follows exactly the same logic of DDLTask.createPartitions in Hive 0.12 + override def createPartitions( + hive: Hive, + database: String, + tableName: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = { + val table = hive.getTable(database, tableName) + parts.foreach { s => + val location = s.storage.locationUri.map(new Path(table.getPath, _)).orNull + val spec = s.spec.asJava + if (hive.getPartition(table, spec, false) != null && ignoreIfExists) { + // Ignore this partition since it already exists and ignoreIfExists == true + } else { + if (location == null && table.isView()) { + throw new HiveException("LOCATION clause illegal for view partition"); + } + + createPartitionMethod.invoke( + hive, + table, + spec, + location, + null, // partParams + null, // inputFormat + null, // outputFormat + -1: JInteger, // numBuckets + null, // cols + null, // serializationLib + null, // serdeParams + null, // bucketCols + null) // sortCols + } + } + } + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq @@ -265,6 +343,30 @@ private[client] class Shim_v0_12 extends Shim with Logging { dropIndexMethod.invoke(hive, dbName, tableName, indexName, true: JBoolean) } + override def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + throw new AnalysisException("Hive 0.12 doesn't support creating permanent functions. " + + "Please use Hive 0.13 or higher.") + } + + def dropFunction(hive: Hive, db: String, name: String): Unit = { + throw new NoSuchPermanentFunctionException(db, name) + } + + def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit = { + throw new NoSuchPermanentFunctionException(db, oldName) + } + + def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + throw new NoSuchPermanentFunctionException(db, func.identifier.funcName) + } + + def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] = { + None + } + + def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { + Seq.empty[String] + } } private[client] class Shim_v0_13 extends Shim_v0_12 { @@ -308,9 +410,85 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { override def setDataLocation(table: Table, loc: String): Unit = setDataLocationMethod.invoke(table, new Path(loc)) + override def createPartitions( + hive: Hive, + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = { + val addPartitionDesc = new AddPartitionDesc(db, table, ignoreIfExists) + parts.foreach { s => + addPartitionDesc.addPartition(s.spec.asJava, s.storage.locationUri.orNull) + } + hive.createPartitions(addPartitionDesc) + } + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq + private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { + val resourceUris = f.resources.map { resource => + new ResourceUri( + ResourceType.valueOf(resource.resourceType.resourceType.toUpperCase()), resource.uri) + } + new HiveFunction( + f.identifier.funcName, + db, + f.className, + null, + PrincipalType.USER, + (System.currentTimeMillis / 1000).toInt, + FunctionType.JAVA, + resourceUris.asJava) + } + + override def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + hive.createFunction(toHiveFunction(func, db)) + } + + override def dropFunction(hive: Hive, db: String, name: String): Unit = { + hive.dropFunction(db, name) + } + + override def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit = { + val catalogFunc = getFunctionOption(hive, db, oldName) + .getOrElse(throw new NoSuchPermanentFunctionException(db, oldName)) + .copy(identifier = FunctionIdentifier(newName, Some(db))) + val hiveFunc = toHiveFunction(catalogFunc, db) + hive.alterFunction(db, oldName, hiveFunc) + } + + override def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + hive.alterFunction(db, func.identifier.funcName, toHiveFunction(func, db)) + } + + private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { + val name = FunctionIdentifier(hf.getFunctionName, Option(hf.getDbName)) + val resources = hf.getResourceUris.asScala.map { uri => + val resourceType = uri.getResourceType() match { + case ResourceType.ARCHIVE => "archive" + case ResourceType.FILE => "file" + case ResourceType.JAR => "jar" + case r => throw new AnalysisException(s"Unknown resource type: $r") + } + FunctionResource(FunctionResourceType.fromString(resourceType), uri.getUri()) + } + new CatalogFunction(name, hf.getClassName, resources) + } + + override def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] = { + try { + Option(hive.getFunction(db, name)).map(fromHiveFunction) + } catch { + case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + None + } + } + + override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { + hive.getFunctions(db, pattern).asScala + } + /** * Converts catalyst expression to the format that Hive's getPartitionsByFilter() expects, i.e. * a string that represents partition predicates like "str_key=\"value\" and int_key=1 ...". 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 a6a5ab3988fc..57e8db7e88fb 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 @@ -17,21 +17,27 @@ package org.apache.spark.sql.hive.client -import java.io.File +import java.io.{ByteArrayOutputStream, File, PrintStream} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.util.VersionInfo import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal, NamedExpression} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.types.IntegerType import org.apache.spark.tags.ExtendedHiveTest -import org.apache.spark.util.Utils +import org.apache.spark.util.{MutableURLClassLoader, Utils} /** * A simple set of tests that call the methods of a [[HiveClient]], loading different version @@ -97,12 +103,6 @@ class VersionsSuite extends SparkFunSuite with Logging { 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. @@ -122,7 +122,7 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'") } - private val versions = Seq("12", "13", "14", "1.0.0", "1.1.0", "1.2.0") + private val versions = Seq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2") private var client: HiveClient = null @@ -130,110 +130,402 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: create client") { client = null System.gc() // Hack to avoid SEGV on some JVM versions. + val hadoopConf = new Configuration(); + hadoopConf.set("test", "success") client = IsolatedClientLoader.forVersion( hiveMetastoreVersion = version, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf = new Configuration(), + hadoopConf, config = buildConf(), ivyPath = ivyPath).createClient() } + def table(database: String, tableName: String): CatalogTable = { + CatalogTable( + identifier = TableIdentifier(tableName, Some(database)), + tableType = CatalogTableType.MANAGED, + schema = Seq(CatalogColumn("key", "int")), + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = Some(classOf[TextInputFormat].getName), + outputFormat = Some(classOf[HiveIgnoreKeyTextOutputFormat[_, _]].getName), + serde = Some(classOf[LazySimpleSerDe].getName()), + compressed = false, + serdeProperties = Map.empty + )) + } + + /////////////////////////////////////////////////////////////////////////// + // Database related API + /////////////////////////////////////////////////////////////////////////// + + val tempDatabasePath = Utils.createTempDir().getCanonicalPath + test(s"$version: createDatabase") { - val db = CatalogDatabase("default", "desc", "loc", Map()) - client.createDatabase(db, ignoreIfExists = true) + val defaultDB = CatalogDatabase("default", "desc", "loc", Map()) + client.createDatabase(defaultDB, ignoreIfExists = true) + val tempDB = CatalogDatabase( + "temporary", description = "test create", tempDatabasePath, Map()) + client.createDatabase(tempDB, ignoreIfExists = true) + } + + test(s"$version: setCurrentDatabase") { + client.setCurrentDatabase("default") + } + + test(s"$version: getDatabase") { + // No exception should be thrown + client.getDatabase("default") + } + + test(s"$version: getDatabaseOption") { + assert(client.getDatabaseOption("default").isDefined) + assert(client.getDatabaseOption("nonexist") == None) } + test(s"$version: listDatabases") { + assert(client.listDatabases("defau.*") == Seq("default")) + } + + test(s"$version: alterDatabase") { + val database = client.getDatabase("temporary").copy(properties = Map("flag" -> "true")) + client.alterDatabase(database) + assert(client.getDatabase("temporary").properties.contains("flag")) + } + + test(s"$version: dropDatabase") { + assert(client.getDatabaseOption("temporary").isDefined) + client.dropDatabase("temporary", ignoreIfNotExists = false, cascade = true) + assert(client.getDatabaseOption("temporary").isEmpty) + } + + /////////////////////////////////////////////////////////////////////////// + // Table related API + /////////////////////////////////////////////////////////////////////////// + test(s"$version: createTable") { - val table = - CatalogTable( - identifier = TableIdentifier("src", Some("default")), - tableType = CatalogTableType.MANAGED, - schema = Seq(CatalogColumn("key", "int")), - storage = CatalogStorageFormat( - locationUri = None, - inputFormat = Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), - outputFormat = Some( - classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), - serde = Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName()), - compressed = false, - serdeProperties = Map.empty - )) - - client.createTable(table, ignoreIfExists = false) + client.createTable(table("default", tableName = "src"), ignoreIfExists = false) + client.createTable(table("default", "temporary"), ignoreIfExists = false) + } + + test(s"$version: loadTable") { + client.loadTable( + emptyDir, + tableName = "src", + replace = false, + holdDDLTime = false) } test(s"$version: getTable") { + // No exception should be thrown client.getTable("default", "src") } - test(s"$version: listTables") { - assert(client.listTables("default") === Seq("src")) + test(s"$version: getTableOption") { + assert(client.getTableOption("default", "src").isDefined) } - test(s"$version: getDatabase") { - client.getDatabase("default") + test(s"$version: alterTable(table: CatalogTable)") { + val newTable = client.getTable("default", "src").copy(properties = Map("changed" -> "")) + client.alterTable(newTable) + assert(client.getTable("default", "src").properties.contains("changed")) } - test(s"$version: alterTable") { - client.alterTable(client.getTable("default", "src")) + test(s"$version: alterTable(tableName: String, table: CatalogTable)") { + val newTable = client.getTable("default", "src").copy(properties = Map("changedAgain" -> "")) + client.alterTable("src", newTable) + assert(client.getTable("default", "src").properties.contains("changedAgain")) } - test(s"$version: set command") { - client.runSqlHive("SET spark.sql.test.key=1") + test(s"$version: listTables(database)") { + assert(client.listTables("default") === Seq("src", "temporary")) + } + + test(s"$version: listTables(database, pattern)") { + assert(client.listTables("default", pattern = "src") === Seq("src")) + assert(client.listTables("default", pattern = "nonexist").isEmpty) + } + + test(s"$version: dropTable") { + client.dropTable("default", tableName = "temporary", ignoreIfNotExists = false) + assert(client.listTables("default") === Seq("src")) } - 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')") + /////////////////////////////////////////////////////////////////////////// + // Partition related API + /////////////////////////////////////////////////////////////////////////// + + val storageFormat = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + compressed = false, + serdeProperties = Map.empty) + + test(s"$version: sql create partitioned table") { + client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key1 INT, key2 INT)") } - test(s"$version: getPartitions") { - client.getPartitions(client.getTable("default", "src_part")) + test(s"$version: createPartitions") { + val partition1 = CatalogTablePartition(Map("key1" -> "1", "key2" -> "1"), storageFormat) + val partition2 = CatalogTablePartition(Map("key1" -> "1", "key2" -> "2"), storageFormat) + client.createPartitions( + "default", "src_part", Seq(partition1, partition2), ignoreIfExists = true) + } + + test(s"$version: getPartitions(catalogTable)") { + assert(2 == client.getPartitions(client.getTable("default", "src_part")).size) } test(s"$version: getPartitionsByFilter") { - client.getPartitionsByFilter(client.getTable("default", "src_part"), Seq(EqualTo( - AttributeReference("key", IntegerType, false)(NamedExpression.newExprId), - Literal(1)))) + // Only one partition [1, 1] for key2 == 1 + val result = client.getPartitionsByFilter(client.getTable("default", "src_part"), + Seq(EqualTo(AttributeReference("key2", IntegerType)(), Literal(1)))) + + // Hive 0.12 doesn't support getPartitionsByFilter, it ignores the filter condition. + if (version != "0.12") { + assert(result.size == 1) + } + } + + test(s"$version: getPartition") { + // No exception should be thrown + client.getPartition("default", "src_part", Map("key1" -> "1", "key2" -> "2")) + } + + test(s"$version: getPartitionOption(db: String, table: String, spec: TablePartitionSpec)") { + val partition = client.getPartitionOption( + "default", "src_part", Map("key1" -> "1", "key2" -> "2")) + assert(partition.isDefined) + } + + test(s"$version: getPartitionOption(table: CatalogTable, spec: TablePartitionSpec)") { + val partition = client.getPartitionOption( + client.getTable("default", "src_part"), Map("key1" -> "1", "key2" -> "2")) + assert(partition.isDefined) + } + + test(s"$version: getPartitions(db: String, table: String)") { + assert(2 == client.getPartitions("default", "src_part", None).size) } test(s"$version: loadPartition") { + val partSpec = new java.util.LinkedHashMap[String, String] + partSpec.put("key1", "1") + partSpec.put("key2", "2") + client.loadPartition( emptyDir, "default.src_part", partSpec, - false, - false, - false, - false) - } - - test(s"$version: loadTable") { - client.loadTable( - emptyDir, - "src", - false, - false) + replace = false, + holdDDLTime = false, + inheritTableSpecs = false, + isSkewedStoreAsSubdir = false) } test(s"$version: loadDynamicPartitions") { + val partSpec = new java.util.LinkedHashMap[String, String] + partSpec.put("key1", "1") + partSpec.put("key2", "") // Dynamic partition + client.loadDynamicPartitions( emptyDir, "default.src_part", partSpec, - false, - 1, + replace = false, + numDP = 1, false, false) } - test(s"$version: create index and reset") { + test(s"$version: renamePartitions") { + val oldSpec = Map("key1" -> "1", "key2" -> "1") + val newSpec = Map("key1" -> "1", "key2" -> "3") + client.renamePartitions("default", "src_part", Seq(oldSpec), Seq(newSpec)) + + // Checks the existence of the new partition (key1 = 1, key2 = 3) + assert(client.getPartitionOption("default", "src_part", newSpec).isDefined) + } + + test(s"$version: alterPartitions") { + val spec = Map("key1" -> "1", "key2" -> "2") + val newLocation = Utils.createTempDir().getPath() + val storage = storageFormat.copy(locationUri = Some(newLocation)) + val partition = CatalogTablePartition(spec, storage) + client.alterPartitions("default", "src_part", Seq(partition)) + assert(client.getPartition("default", "src_part", spec) + .storage.locationUri == Some(newLocation)) + } + + test(s"$version: dropPartitions") { + val spec = Map("key1" -> "1", "key2" -> "3") + client.dropPartitions("default", "src_part", Seq(spec), ignoreIfNotExists = true) + assert(client.getPartitionOption("default", "src_part", spec).isEmpty) + } + + /////////////////////////////////////////////////////////////////////////// + // Function related API + /////////////////////////////////////////////////////////////////////////// + + def function(name: String, className: String): CatalogFunction = { + CatalogFunction( + FunctionIdentifier(name, Some("default")), className, Seq.empty[FunctionResource]) + } + + test(s"$version: createFunction") { + val functionClass = "org.apache.spark.MyFunc1" + if (version == "0.12") { + // Hive 0.12 doesn't support creating permanent functions + intercept[AnalysisException] { + client.createFunction("default", function("func1", functionClass)) + } + } else { + client.createFunction("default", function("func1", functionClass)) + } + } + + test(s"$version: functionExists") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.functionExists("default", "func1") == false) + } else { + assert(client.functionExists("default", "func1") == true) + } + } + + test(s"$version: renameFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.renameFunction("default", "func1", "func2") + } + } else { + client.renameFunction("default", "func1", "func2") + assert(client.functionExists("default", "func2") == true) + } + } + + test(s"$version: alterFunction") { + val functionClass = "org.apache.spark.MyFunc2" + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.alterFunction("default", function("func2", functionClass)) + } + } else { + client.alterFunction("default", function("func2", functionClass)) + } + } + + test(s"$version: getFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.getFunction("default", "func2") + } + } else { + // No exception should be thrown + val func = client.getFunction("default", "func2") + assert(func.className == "org.apache.spark.MyFunc2") + } + } + + test(s"$version: getFunctionOption") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.getFunctionOption("default", "func2").isEmpty) + } else { + assert(client.getFunctionOption("default", "func2").isDefined) + } + } + + test(s"$version: listFunctions") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.listFunctions("default", "fun.*").isEmpty) + } else { + assert(client.listFunctions("default", "fun.*").size == 1) + } + } + + test(s"$version: dropFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't support creating permanent functions + intercept[NoSuchPermanentFunctionException] { + client.dropFunction("default", "func2") + } + } else { + // No exception should be thrown + client.dropFunction("default", "func2") + assert(client.listFunctions("default", "fun.*").size == 0) + } + } + + /////////////////////////////////////////////////////////////////////////// + // SQL related API + /////////////////////////////////////////////////////////////////////////// + + test(s"$version: sql set command") { + client.runSqlHive("SET spark.sql.test.key=1") + } + + test(s"$version: sql create index and reset") { client.runSqlHive("CREATE TABLE indexed_table (key INT)") client.runSqlHive("CREATE INDEX index_1 ON TABLE indexed_table(key) " + "as 'COMPACT' WITH DEFERRED REBUILD") + } + + /////////////////////////////////////////////////////////////////////////// + // Miscellaneous API + /////////////////////////////////////////////////////////////////////////// + + test(s"$version: version") { + assert(client.version.fullVersion.startsWith(version)) + } + + test(s"$version: getConf") { + assert("success" === client.getConf("test", null)) + } + + test(s"$version: setOut") { + client.setOut(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: setInfo") { + client.setInfo(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: setError") { + client.setError(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: newSession") { + val newClient = client.newSession() + assert(newClient != null) + } + + test(s"$version: withHiveState and addJar") { + val newClassPath = "." + client.addJar(newClassPath) + client.withHiveState { + // No exception should be thrown. + // withHiveState changes the classloader to MutableURLClassLoader + val classLoader = Thread.currentThread().getContextClassLoader + .asInstanceOf[MutableURLClassLoader] + + val urls = classLoader.getURLs() + urls.contains(new File(newClassPath).toURI.toURL) + } + } + + test(s"$version: reset") { + // Clears all database, tables, functions... client.reset() + assert(client.listTables("default").isEmpty) } } } From 35c25beaca3e9070ccb6dcca33c415a26e6f1f1c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 18:46:57 +0800 Subject: [PATCH 252/313] [SPARK-15307][SQL] speed up listing files for data source ## What changes were proposed in this pull request? Currently, listing files is very slow if there is thousands files, especially on local file system, because: 1) FileStatus.getPermission() is very slow on local file system, which is launch a subprocess and parse the stdout. 2) Create an JobConf is very expensive (ClassUtil.findContainingJar() is slow). This PR improve these by: 1) Use another constructor of LocatedFileStatus to avoid calling FileStatus.getPermission, the permissions are not used for data sources. 2) Only create an JobConf once within one task. ## How was this patch tested? Manually tests on a partitioned table with 1828 partitions, decrease the time to load the table from 22 seconds to 1.6 seconds (Most of time are spent in merging schema now). Author: Davies Liu Closes #13094 from davies/listing. (cherry picked from commit 33814f887aea339c99e14ce7f14ca6fcc6875015) Signed-off-by: Cheng Lian --- .../datasources/ListingFileCatalog.scala | 9 +++-- .../datasources/fileSourceInterfaces.scala | 38 ++++++++++++++----- 2 files changed, 33 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index 5cee2b9af68c..644e5d65d612 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -77,12 +77,12 @@ class ListingFileCatalog( if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext) } else { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) val statuses: Seq[FileStatus] = paths.flatMap { path => val fs = path.getFileSystem(hadoopConf) logInfo(s"Listing $path on driver") - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) val statuses = { val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) @@ -101,7 +101,8 @@ class ListingFileCatalog( // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should a // a big deal since we always use to `listLeafFilesInParallel` when the number of paths // exceeds threshold. - case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + case f => + HadoopFsRelation.createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) } }.filterNot { status => val name = status.getPath.getName diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index b516297115f0..8d332df02916 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -348,28 +348,40 @@ private[sql] object HadoopFsRelation extends Logging { pathName == "_SUCCESS" || pathName == "_temporary" || pathName.startsWith(".") } + /** + * Create a LocatedFileStatus using FileStatus and block locations. + */ + def createLocatedFileStatus(f: FileStatus, locations: Array[BlockLocation]): LocatedFileStatus = { + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), which is + // very slow on some file system (RawLocalFileSystem, which is launch a subprocess and parse the + // stdout). + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs + } + // We don't filter files/directories whose name start with "_" except "_temporary" here, as // specific data sources may take advantages over them (e.g. Parquet _metadata and // _common_metadata files). "_temporary" directories are explicitly ignored since failed // tasks/jobs may leave partial/corrupted data files there. Files and directories whose name // start with "." are also ignored. - def listLeafFiles(fs: FileSystem, status: FileStatus): Array[FileStatus] = { + def listLeafFiles(fs: FileSystem, status: FileStatus, filter: PathFilter): Array[FileStatus] = { logInfo(s"Listing ${status.getPath}") val name = status.getPath.getName.toLowerCase if (shouldFilterOut(name)) { Array.empty } else { - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(fs.getConf, this.getClass()) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) val statuses = { val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) - val stats = files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) - if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats + val stats = files ++ dirs.flatMap(dir => listLeafFiles(fs, dir, filter)) + if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats } statuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { case f: LocatedFileStatus => f - case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + case f => createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) } } } @@ -403,9 +415,15 @@ private[sql] object HadoopFsRelation extends Logging { val serializableConfiguration = new SerializableConfiguration(hadoopConf) val serializedPaths = paths.map(_.toString) - val fakeStatuses = sparkContext.parallelize(serializedPaths).map(new Path(_)).flatMap { path => - val fs = path.getFileSystem(serializableConfiguration.value) - Try(listLeafFiles(fs, fs.getFileStatus(path))).getOrElse(Array.empty) + val fakeStatuses = sparkContext.parallelize(serializedPaths).mapPartitions { paths => + // Dummy jobconf to get to the pathFilter defined in configuration + // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) + val jobConf = new JobConf(serializableConfiguration.value, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + paths.map(new Path(_)).flatMap { path => + val fs = path.getFileSystem(serializableConfiguration.value) + Try(listLeafFiles(fs, fs.getFileStatus(path), pathFilter)).getOrElse(Array.empty) + } }.map { status => val blockLocations = status match { case f: LocatedFileStatus => From 14751cd0d0d7ed6f64d6ec26d04ac091e1b72717 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 18 May 2016 11:48:46 +0100 Subject: [PATCH 253/313] [SPARK-15322][MLLIB][CORE][SQL] update deprecate accumulator usage into accumulatorV2 in spark project ## What changes were proposed in this pull request? I use Intellj-IDEA to search usage of deprecate SparkContext.accumulator in the whole spark project, and update the code.(except those test code for accumulator method itself) ## How was this patch tested? Exisiting unit tests Author: WeichenXu Closes #13112 from WeichenXu123/update_accuV2_in_mllib. (cherry picked from commit 2f9047b5eb969e0198b8a73e392642ca852ba786) Signed-off-by: Sean Owen --- .../org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 8 ++++---- .../streaming/RecoverableNetworkWordCount.scala | 11 ++++++----- .../scala/org/apache/spark/ml/util/stopwatches.scala | 7 ++++--- .../org/apache/spark/mllib/clustering/KMeans.scala | 4 ++-- .../org/apache/spark/ml/util/StopwatchSuite.scala | 8 ++++---- .../org/apache/spark/sql/DatasetPrimitiveSuite.scala | 8 ++++---- .../scala/org/apache/spark/sql/DatasetSuite.scala | 8 ++++---- 7 files changed, 28 insertions(+), 26 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 8cb0a295b077..58664e77d24a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -65,9 +65,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachAsync") { zeroPartRdd.foreachAsync(i => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 3).foreachAsync { i => - accum += 1 + accum.add(1) }.get() assert(accum.value === 1000) } @@ -75,9 +75,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachPartitionAsync") { zeroPartRdd.foreachPartitionAsync(iter => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 9).foreachPartitionAsync { iter => - accum += 1 + accum.add(1) }.get() assert(accum.value === 9) } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 1bcd85e1d533..acbcb0c4b7b7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -23,11 +23,12 @@ import java.nio.charset.Charset import com.google.common.io.Files -import org.apache.spark.{Accumulator, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext, Time} import org.apache.spark.util.IntParam +import org.apache.spark.util.LongAccumulator /** * Use this singleton to get or register a Broadcast variable. @@ -54,13 +55,13 @@ object WordBlacklist { */ object DroppedWordsCounter { - @volatile private var instance: Accumulator[Long] = null + @volatile private var instance: LongAccumulator = null - def getInstance(sc: SparkContext): Accumulator[Long] = { + def getInstance(sc: SparkContext): LongAccumulator = { if (instance == null) { synchronized { if (instance == null) { - instance = sc.accumulator(0L, "WordsInBlacklistCounter") + instance = sc.longAccumulator("WordsInBlacklistCounter") } } } @@ -124,7 +125,7 @@ object RecoverableNetworkWordCount { // Use blacklist to drop words and use droppedWordsCounter to count them val counts = rdd.filter { case (word, count) => if (blacklist.value.contains(word)) { - droppedWordsCounter += count + droppedWordsCounter.add(count) false } else { true diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala index 8d4174124b5c..e79b1f31643d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala @@ -19,7 +19,8 @@ package org.apache.spark.ml.util import scala.collection.mutable -import org.apache.spark.{Accumulator, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.util.LongAccumulator; /** * Abstract class for stopwatches. @@ -102,12 +103,12 @@ private[spark] class DistributedStopwatch( sc: SparkContext, override val name: String) extends Stopwatch { - private val elapsedTime: Accumulator[Long] = sc.accumulator(0L, s"DistributedStopwatch($name)") + private val elapsedTime: LongAccumulator = sc.longAccumulator(s"DistributedStopwatch($name)") override def elapsed(): Long = elapsedTime.value override protected def add(duration: Long): Unit = { - elapsedTime += duration + elapsedTime.add(duration) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 60f13d27d0a6..38728f2693c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -279,7 +279,7 @@ class KMeans private ( } val activeCenters = activeRuns.map(r => centers(r)).toArray - val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + val costAccums = activeRuns.map(_ => sc.doubleAccumulator) val bcActiveCenters = sc.broadcast(activeCenters) @@ -296,7 +296,7 @@ class KMeans private ( points.foreach { point => (0 until runs).foreach { i => val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) - costAccums(i) += cost + costAccums(i).add(cost) val sum = sums(i)(bestCenter) axpy(1.0, point.vector, sum) counts(i)(bestCenter) += 1 diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala index 9e6bc7193c13..141249a427a4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala @@ -60,9 +60,9 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { test("DistributedStopwatch on executors") { val sw = new DistributedStopwatch(sc, "sw") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => - acc += checkStopwatch(sw) + acc.add(checkStopwatch(sw)) } assert(!sw.isRunning) val elapsed = sw.elapsed() @@ -88,12 +88,12 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { assert(sw.toString === s"{\n local: ${localElapsed}ms,\n spark: ${sparkElapsed}ms\n}") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => sw("local").start() val duration = checkStopwatch(sw("spark")) sw("local").stop() - acc += duration + acc.add(duration) } val localElapsed2 = sw("local").elapsed() assert(localElapsed2 === localElapsed) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index ff022b2dc45e..a634502e2e4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -62,15 +62,15 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { test("foreach") { val ds = Seq(1, 2, 3).toDS() - val acc = sparkContext.accumulator(0) - ds.foreach(acc += _) + val acc = sparkContext.longAccumulator + ds.foreach(acc.add(_)) assert(acc.value == 6) } test("foreachPartition") { val ds = Seq(1, 2, 3).toDS() - val acc = sparkContext.accumulator(0) - ds.foreachPartition(_.foreach(acc +=)) + val acc = sparkContext.longAccumulator + ds.foreachPartition(_.foreach(acc.add(_))) assert(acc.value == 6) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 3b9feae4a31c..b02b714168cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -207,15 +207,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("foreach") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() - val acc = sparkContext.accumulator(0) - ds.foreach(v => acc += v._2) + val acc = sparkContext.longAccumulator + ds.foreach(v => acc.add(v._2)) assert(acc.value == 6) } test("foreachPartition") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() - val acc = sparkContext.accumulator(0) - ds.foreachPartition(_.foreach(v => acc += v._2)) + val acc = sparkContext.longAccumulator + ds.foreachPartition(_.foreach(v => acc.add(v._2))) assert(acc.value == 6) } From a122a3e709479a6510d97dfa81f3d6219bb79de0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 18 May 2016 18:58:24 +0800 Subject: [PATCH 254/313] [SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10 ## What changes were proposed in this pull request? This PR fixes a Scala 2.10 compilation failure introduced in PR #13127. ## How was this patch tested? Jenkins build. Author: Cheng Lian Closes #13166 from liancheng/hotfix-for-scala-2.10. (cherry picked from commit c4a45fd855252f0f7130cdcc08486591fadab7df) Signed-off-by: Cheng Lian --- .../scala/org/apache/spark/sql/hive/client/VersionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 57e8db7e88fb..d46c4e7b2b50 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 @@ -137,7 +137,7 @@ class VersionsSuite extends SparkFunSuite with Logging { hiveMetastoreVersion = version, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf, + hadoopConf = hadoopConf, config = buildConf(), ivyPath = ivyPath).createClient() } From fe0a0686c50743272a841b909cbfe4534350fc18 Mon Sep 17 00:00:00 2001 From: DLucky Date: Wed, 18 May 2016 12:05:21 +0100 Subject: [PATCH 255/313] [SPARK-15346][MLLIB] Reduce duplicate computation in picking initial points mateiz srowen I state that the contribution is my original work and that I license the work to the project under the project's open source license There's some format problems with my last PR, with HyukjinKwon 's help I read the guidance, re-check my code and PR, then run the tests, finally re-submit the PR request here. The related JIRA issue though marked as resolved, this change may relate to it I think. ## Proposed Change After picking each new initial centers, it's unnecessary to compute the distances between all the points and the old ones. Instead this change keeps the distance between all the points and their closest centers, and compare to the distance of them with the new center then update them. ## Test result One can find an easy test way in (https://issues.apache.org/jira/browse/SPARK-6706) I test the KMeans++ method for a small dataset with 16k points, and the whole KMeans|| with a large one with 240k points. The data has 4096 features and I tunes K from 100 to 500. The test environment was on my 4 machine cluster, I also tested a 3M points data on a larger cluster with 25 machines and got similar results, which I would not draw the detail curve. The result of the first two exps are shown below ### Local KMeans++ test: Dataset:4m_ini_center Data_size:16234 Dimension:4096 Lloyd's Iteration = 10 The y-axis is time in sec, the x-axis is tuning the K. ![image](https://cloud.githubusercontent.com/assets/10915169/15175831/d0c92b82-179a-11e6-8b68-4e165fc2fdff.png) ![local_total](https://cloud.githubusercontent.com/assets/10915169/15175957/6b21c3b0-179b-11e6-9741-66dfe4e23eb7.jpg) ### On a larger dataset An improve show in the graph but not commit in this file: In this experiment I also have an improvement for calculation in normalization data (the distance is convert to the cosine distance). As if the data is normalized into (0,1), one improvement in the original vesion for util.MLUtils.fastSauaredDistance would have no effect (the precisionBound 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) will never less then precision in this case). Therefore I design an early terminal method when comparing two distance (used for findClosest). But I don't include this improve in this file, you may only refer to the curves without "normalize" for comparing the results. Dataset:4k24 Data_size:243960 Dimension:4096 Normlize Enlarge Initialize Lloyd's_Iteration NO 1 3 5 YES 10000 3 5 Notice: the normlized data is enlarged to ensure precision The cost time: x-for value of K, y-for time in sec ![4k24_total](https://cloud.githubusercontent.com/assets/10915169/15176635/9a54c0bc-179e-11e6-81c5-238e0c54bce2.jpg) SE for unnormalized data between two version, to ensure the correctness ![4k24_unnorm_se](https://cloud.githubusercontent.com/assets/10915169/15176661/b85dabc8-179e-11e6-9269-fe7d2101dd48.jpg) Here is the SE between normalized data just for reference, it's also correct. ![4k24_norm_se](https://cloud.githubusercontent.com/assets/10915169/15176742/1fbde940-179f-11e6-8290-d24b0dd4a4f7.jpg) Author: DLucky Closes #13133 from mouendless/patch-2. (cherry picked from commit 420b700695fe8bcdda406c34ad48230b9dfc07f1) Signed-off-by: Sean Owen --- .../spark/mllib/clustering/LocalKMeans.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index adf20dc4b8b1..53587670a5db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -46,17 +46,15 @@ private[mllib] object LocalKMeans extends Logging { // Initialize centers by sampling using the k-means++ procedure. centers(0) = pickWeighted(rand, points, weights).toDense + val costArray = points.map(KMeans.fastSquaredDistance(_, centers(0))) + for (i <- 1 until k) { - // Pick the next center with a probability proportional to cost under current centers - val curCenters = centers.view.take(i) - val sum = points.view.zip(weights).map { case (p, w) => - w * KMeans.pointCost(curCenters, p) - }.sum + val sum = costArray.zip(weights).map(p => p._1 * p._2).sum val r = rand.nextDouble() * sum var cumulativeScore = 0.0 var j = 0 while (j < points.length && cumulativeScore < r) { - cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) + cumulativeScore += weights(j) * costArray(j) j += 1 } if (j == 0) { @@ -66,6 +64,12 @@ private[mllib] object LocalKMeans extends Logging { } else { centers(i) = points(j - 1).toDense } + + // update costArray + for (p <- points.indices) { + costArray(p) = math.min(KMeans.fastSquaredDistance(points(p), centers(i)), costArray(p)) + } + } // Run up to maxIterations iterations of Lloyd's algorithm From 7ae006fdef6bd834067c9e80851116bcd4503b7f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 09:44:21 -0700 Subject: [PATCH 256/313] [SPARK-15357] Cooperative spilling should check consumer memory mode ## What changes were proposed in this pull request? Since we support forced spilling for Spillable, which only works in OnHeap mode, different from other SQL operators (could be OnHeap or OffHeap), we should considering the mode of consumer before calling trigger forced spilling. ## How was this patch tested? Add new test. Author: Davies Liu Closes #13151 from davies/fix_mode. (cherry picked from commit 8fb1d1c7f3ed1b62625052a532b7388ebec71bbf) Signed-off-by: Davies Liu --- .../apache/spark/memory/MemoryConsumer.java | 25 ++++-- .../spark/memory/TaskMemoryManager.java | 37 ++++----- .../shuffle/sort/ShuffleExternalSorter.java | 5 +- .../spark/unsafe/map/BytesToBytesMap.java | 2 +- .../unsafe/sort/UnsafeExternalSorter.java | 2 +- .../org/apache/spark/executor/Executor.scala | 10 +-- .../spark/util/collection/Spillable.scala | 4 +- .../spark/memory/TaskMemoryManagerSuite.java | 28 ++++++- .../spark/memory/TestMemoryConsumer.java | 15 ++-- .../sort/PackedRecordPointerSuite.java | 13 +-- .../sort/ShuffleInMemorySorterSuite.java | 4 +- .../sort/UnsafeInMemorySorterSuite.java | 2 +- .../scala/org/apache/spark/FailureSuite.scala | 7 +- .../spark/memory/MemoryManagerSuite.scala | 80 +++++++++++-------- project/MimaExcludes.scala | 1 + .../sql/execution/joins/HashedRelation.scala | 16 ++-- 16 files changed, 145 insertions(+), 106 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 840f13b39464..38a21a896e1f 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -31,15 +31,24 @@ public abstract class MemoryConsumer { protected final TaskMemoryManager taskMemoryManager; private final long pageSize; + private final MemoryMode mode; protected long used; - protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize) { + protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize, MemoryMode mode) { this.taskMemoryManager = taskMemoryManager; this.pageSize = pageSize; + this.mode = mode; } protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { - this(taskMemoryManager, taskMemoryManager.pageSizeBytes()); + this(taskMemoryManager, taskMemoryManager.pageSizeBytes(), MemoryMode.ON_HEAP); + } + + /** + * Returns the memory mode, ON_HEAP or OFF_HEAP. + */ + public MemoryMode getMode() { + return mode; } /** @@ -132,19 +141,19 @@ protected void freePage(MemoryBlock page) { } /** - * Allocates a heap memory of `size`. + * Allocates memory of `size`. */ - public long acquireOnHeapMemory(long size) { - long granted = taskMemoryManager.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this); + public long acquireMemory(long size) { + long granted = taskMemoryManager.acquireExecutionMemory(size, this); used += granted; return granted; } /** - * Release N bytes of heap memory. + * Release N bytes of memory. */ - public void freeOnHeapMemory(long size) { - taskMemoryManager.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this); + public void freeMemory(long size) { + taskMemoryManager.releaseExecutionMemory(size, this); used -= size; } } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index a05a79c88df7..a4a571f15a8c 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -76,9 +76,6 @@ public class TaskMemoryManager { /** Bit mask for the lower 51 bits of a long. */ private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL; - /** Bit mask for the upper 13 bits of a long */ - private static final long MASK_LONG_UPPER_13_BITS = ~MASK_LONG_LOWER_51_BITS; - /** * Similar to an operating system's page table, this array maps page numbers into base object * pointers, allowing us to translate between the hashtable's internal 64-bit address @@ -132,11 +129,10 @@ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { * * @return number of bytes successfully granted (<= N). */ - public long acquireExecutionMemory( - long required, - MemoryMode mode, - MemoryConsumer consumer) { + public long acquireExecutionMemory(long required, MemoryConsumer consumer) { assert(required >= 0); + assert(consumer != null); + MemoryMode mode = consumer.getMode(); // If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap // memory here, then it may not make sense to spill since that would only end up freeing // off-heap memory. This is subject to change, though, so it may be risky to make this @@ -149,10 +145,10 @@ public long acquireExecutionMemory( if (got < required) { // Call spill() on other consumers to release memory for (MemoryConsumer c: consumers) { - if (c != consumer && c.getUsed() > 0) { + if (c != consumer && c.getUsed() > 0 && c.getMode() == mode) { try { long released = c.spill(required - got, consumer); - if (released > 0 && mode == tungstenMemoryMode) { + if (released > 0) { logger.debug("Task {} released {} from {} for {}", taskAttemptId, Utils.bytesToString(released), c, consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); @@ -170,10 +166,10 @@ public long acquireExecutionMemory( } // call spill() on itself - if (got < required && consumer != null) { + if (got < required) { try { long released = consumer.spill(required - got, consumer); - if (released > 0 && mode == tungstenMemoryMode) { + if (released > 0) { logger.debug("Task {} released {} from itself ({})", taskAttemptId, Utils.bytesToString(released), consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); @@ -185,9 +181,7 @@ public long acquireExecutionMemory( } } - if (consumer != null) { - consumers.add(consumer); - } + consumers.add(consumer); logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); return got; } @@ -196,9 +190,9 @@ public long acquireExecutionMemory( /** * Release N bytes of execution memory for a MemoryConsumer. */ - public void releaseExecutionMemory(long size, MemoryMode mode, MemoryConsumer consumer) { + public void releaseExecutionMemory(long size, MemoryConsumer consumer) { logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer); - memoryManager.releaseExecutionMemory(size, taskAttemptId, mode); + memoryManager.releaseExecutionMemory(size, taskAttemptId, consumer.getMode()); } /** @@ -241,12 +235,14 @@ public long pageSizeBytes() { * contains fewer bytes than requested, so callers should verify the size of returned pages. */ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { + assert(consumer != null); + assert(consumer.getMode() == tungstenMemoryMode); if (size > MAXIMUM_PAGE_SIZE_BYTES) { throw new IllegalArgumentException( "Cannot allocate a page with more than " + MAXIMUM_PAGE_SIZE_BYTES + " bytes"); } - long acquired = acquireExecutionMemory(size, tungstenMemoryMode, consumer); + long acquired = acquireExecutionMemory(size, consumer); if (acquired <= 0) { return null; } @@ -255,7 +251,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { synchronized (this) { pageNumber = allocatedPages.nextClearBit(0); if (pageNumber >= PAGE_TABLE_SIZE) { - releaseExecutionMemory(acquired, tungstenMemoryMode, consumer); + releaseExecutionMemory(acquired, consumer); throw new IllegalStateException( "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); } @@ -299,7 +295,7 @@ public void freePage(MemoryBlock page, MemoryConsumer consumer) { } long pageSize = page.size(); memoryManager.tungstenMemoryAllocator().free(page); - releaseExecutionMemory(pageSize, tungstenMemoryMode, consumer); + releaseExecutionMemory(pageSize, consumer); } /** @@ -396,8 +392,7 @@ public long cleanUpAllAllocatedMemory() { Arrays.fill(pageTable, null); } - - // release the memory that is not used by any consumer. + // release the memory that is not used by any consumer (acquired for pages in tungsten mode). memoryManager.releaseExecutionMemory(acquiredButNotUsed, taskAttemptId, tungstenMemoryMode); return memoryManager.releaseAllExecutionMemoryForTask(taskAttemptId); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 2be5a16b2d1e..014aef86b5cc 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -104,8 +104,9 @@ final class ShuffleExternalSorter extends MemoryConsumer { int numPartitions, SparkConf conf, ShuffleWriteMetrics writeMetrics) { - super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, - memoryManager.pageSizeBytes())); + super(memoryManager, + (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), + memoryManager.getTungstenMemoryMode()); this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 6807710f9fef..6c00608302c4 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -182,7 +182,7 @@ public BytesToBytesMap( double loadFactor, long pageSizeBytes, boolean enablePerfMetrics) { - super(taskMemoryManager, pageSizeBytes); + super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; this.serializerManager = serializerManager; diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 7dc050878498..e14a23f4a6a8 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -124,7 +124,7 @@ private UnsafeExternalSorter( long pageSizeBytes, @Nullable UnsafeInMemorySorter existingInMemorySorter, boolean canUseRadixSort) { - super(taskMemoryManager, pageSizeBytes); + super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; this.serializerManager = serializerManager; diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b695aecc13ea..9a017f29f7d2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -281,20 +281,20 @@ private[spark] class Executor( val releasedLocks = env.blockManager.releaseAllLocksForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() - if (freedMemory > 0) { + if (freedMemory > 0 && !threwException) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" - if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false) && !threwException) { + if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { throw new SparkException(errMsg) } else { - logError(errMsg) + logWarning(errMsg) } } - if (releasedLocks.nonEmpty) { + if (releasedLocks.nonEmpty && !threwException) { val errMsg = s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" + releasedLocks.mkString("[", ", ", "]") - if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) { + if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false)) { throw new SparkException(errMsg) } else { logWarning(errMsg) diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index bdcbd22fd814..8183f825592c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -83,7 +83,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = acquireOnHeapMemory(amountToRequest) + val granted = acquireMemory(amountToRequest) myMemoryThreshold += granted // If we were granted too little memory to grow further (either tryToAcquire returned 0, // or we already had more memory than myMemoryThreshold), spill the current collection @@ -131,7 +131,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) * Release our memory back to the execution pool so that other tasks can grab it. */ def releaseMemory(): Unit = { - freeOnHeapMemory(myMemoryThreshold - initialMemoryThreshold) + freeMemory(myMemoryThreshold - initialMemoryThreshold) myMemoryThreshold = initialMemoryThreshold } diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index 127789b632b4..ad755529dec6 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -34,7 +34,8 @@ public void leakedPageMemoryIsDetected() { Long.MAX_VALUE, 1), 0); - manager.allocatePage(4096, null); // leak memory + final MemoryConsumer c = new TestMemoryConsumer(manager); + manager.allocatePage(4096, c); // leak memory Assert.assertEquals(4096, manager.getMemoryConsumptionForThisTask()); Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory()); } @@ -45,7 +46,8 @@ public void encodePageNumberAndOffsetOffHeap() { .set("spark.memory.offHeap.enabled", "true") .set("spark.memory.offHeap.size", "1000"); final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); // In off-heap mode, an offset is an absolute address that may require more than 51 bits to // encode. This test exercises that corner-case: final long offset = ((1L << TaskMemoryManager.OFFSET_BITS) + 10); @@ -58,7 +60,8 @@ public void encodePageNumberAndOffsetOffHeap() { public void encodePageNumberAndOffsetOnHeap() { final TaskMemoryManager manager = new TaskMemoryManager( new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, 64); Assert.assertEquals(dataPage.getBaseObject(), manager.getPage(encodedAddress)); Assert.assertEquals(64, manager.getOffsetInPage(encodedAddress)); @@ -106,6 +109,25 @@ public void cooperativeSpilling() { Assert.assertEquals(0, manager.cleanUpAllAllocatedMemory()); } + @Test + public void shouldNotForceSpillingInDifferentModes() { + final TestMemoryManager memoryManager = new TestMemoryManager(new SparkConf()); + memoryManager.limit(100); + final TaskMemoryManager manager = new TaskMemoryManager(memoryManager, 0); + + TestMemoryConsumer c1 = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + TestMemoryConsumer c2 = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + c1.use(80); + Assert.assertEquals(80, c1.getUsed()); + c2.use(80); + Assert.assertEquals(20, c2.getUsed()); // not enough memory + Assert.assertEquals(80, c1.getUsed()); // not spilled + + c2.use(10); + Assert.assertEquals(10, c2.getUsed()); // spilled + Assert.assertEquals(80, c1.getUsed()); // not spilled + } + @Test public void offHeapConfigurationBackwardsCompatibility() { // Tests backwards-compatibility with the old `spark.unsafe.offHeap` configuration, which diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java index e6e16fff8040..db91329c94cb 100644 --- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -20,8 +20,11 @@ import java.io.IOException; public class TestMemoryConsumer extends MemoryConsumer { + public TestMemoryConsumer(TaskMemoryManager memoryManager, MemoryMode mode) { + super(memoryManager, 1024L, mode); + } public TestMemoryConsumer(TaskMemoryManager memoryManager) { - super(memoryManager); + this(memoryManager, MemoryMode.ON_HEAP); } @Override @@ -32,19 +35,13 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { } void use(long size) { - long got = taskMemoryManager.acquireExecutionMemory( - size, - taskMemoryManager.tungstenMemoryMode, - this); + long got = taskMemoryManager.acquireExecutionMemory(size, this); used += got; } void free(long size) { used -= size; - taskMemoryManager.releaseExecutionMemory( - size, - taskMemoryManager.tungstenMemoryMode, - this); + taskMemoryManager.releaseExecutionMemory(size, this); } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java index fe5abc5c2304..354efe18dbde 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java @@ -22,8 +22,7 @@ import org.junit.Test; import org.apache.spark.SparkConf; -import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.*; import org.apache.spark.unsafe.memory.MemoryBlock; import static org.apache.spark.shuffle.sort.PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES; @@ -38,8 +37,9 @@ public void heap() throws IOException { final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.ON_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); @@ -59,8 +59,9 @@ public void offHeap() throws IOException { .set("spark.memory.offHeap.size", "10000"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.OFF_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java index 278a827644db..694352ee2af4 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java @@ -26,6 +26,7 @@ import org.apache.spark.HashPartitioner; import org.apache.spark.SparkConf; +import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; @@ -71,7 +72,8 @@ public void testBasicSorting() throws Exception { final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, c); final Object baseObject = dataPage.getBaseObject(); final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter( consumer, 4, shouldUseRadixSort()); diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 4a2f65a0ed2b..383c5b3b0884 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -78,7 +78,7 @@ public void testSortingOnlyByIntegerPrefix() throws Exception { final TaskMemoryManager memoryManager = new TaskMemoryManager( new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer); final Object baseObject = dataPage.getBaseObject(); // Write the records into the data page: long position = dataPage.getBaseOffset(); diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 3def8b0b1850..333c23bdaf6d 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{IOException, NotSerializableException, ObjectInputStream} +import org.apache.spark.memory.TestMemoryConsumer import org.apache.spark.util.NonSerializable // Common state shared by FailureSuite-launched tasks. We use a global object @@ -149,7 +150,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // cause is preserved val thrownDueToTaskFailure = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) throw new Exception("intentional task failure") iter }.count() @@ -159,7 +161,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // If the task succeeded but memory was leaked, then the task should fail due to that leak val thrownDueToMemoryLeak = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) iter }.count() } diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 2c4928ab907a..38bf7e5e5aec 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -162,39 +162,42 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft test("single task requesting on-heap execution memory") { val manager = createMemoryManager(1000L) val taskMemoryManager = new TaskMemoryManager(manager, 0) + val c = new TestMemoryConsumer(taskMemoryManager) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(200L, MemoryMode.ON_HEAP, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(200L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) - taskMemoryManager.releaseExecutionMemory(500L, MemoryMode.ON_HEAP, null) - assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 300L) - assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 200L) + taskMemoryManager.releaseExecutionMemory(500L, c) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 300L) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 200L) taskMemoryManager.cleanUpAllAllocatedMemory() - assert(taskMemoryManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) === 1000L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(1000L, c) === 1000L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) } test("two tasks requesting full on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 500 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 500L) assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) // Have both tasks each request 500 bytes more; both should immediately return 0 as they are // both now at 1 / N - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } @@ -203,18 +206,20 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 250 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 250L) assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Have both tasks each request 500 bytes more. // We should only grant 250 bytes to each of them on this second request - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result2, futureTimeout) === 250L) assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 250L) } @@ -223,20 +228,22 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) - t1MemManager.releaseExecutionMemory(250L, MemoryMode.ON_HEAP, null) + t1MemManager.releaseExecutionMemory(250L, c1) // The memory freed from t1 should now be granted to t2. assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory. - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, c2) } assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } @@ -244,21 +251,23 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) // t1 releases all of its memory, so t2 should be able to grab all of the memory t1MemManager.cleanUpAllAllocatedMemory() assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 500L) - val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t2Result3, 200.millis) === 0L) } @@ -267,15 +276,17 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 700L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, c2) } assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 300L) - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, c1) } assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) } @@ -285,17 +296,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft maxOffHeapExecutionMemory = 1000L) val tMemManager = new TaskMemoryManager(memoryManager, 1) - val result1 = Future { tMemManager.acquireExecutionMemory(1000L, MemoryMode.OFF_HEAP, null) } + val c = new TestMemoryConsumer(tMemManager, MemoryMode.OFF_HEAP) + val result1 = Future { tMemManager.acquireExecutionMemory(1000L, c) } assert(ThreadUtils.awaitResult(result1, 200.millis) === 1000L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) - val result2 = Future { tMemManager.acquireExecutionMemory(300L, MemoryMode.OFF_HEAP, null) } + val result2 = Future { tMemManager.acquireExecutionMemory(300L, c) } assert(ThreadUtils.awaitResult(result2, 200.millis) === 0L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) - tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + tMemManager.releaseExecutionMemory(500L, c) assert(tMemManager.getMemoryConsumptionForThisTask === 500L) - tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + tMemManager.releaseExecutionMemory(500L, c) assert(tMemManager.getMemoryConsumptionForThisTask === 0L) } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 45f729704838..4e99a0965780 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -40,6 +40,7 @@ object MimaExcludes { excludePackage("org.spark-project.jetty"), excludePackage("org.apache.spark.unused"), excludePackage("org.apache.spark.unsafe"), + excludePackage("org.apache.spark.memory"), excludePackage("org.apache.spark.util.collection.unsafe"), excludePackage("org.apache.spark.sql.catalyst"), excludePackage("org.apache.spark.sql.execution"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 315ef6a8796f..cb41457b6653 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -398,9 +398,9 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap 0) } - private def acquireMemory(size: Long): Unit = { + private def ensureAcquireMemory(size: Long): Unit = { // do not support spilling - val got = mm.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this) + val got = acquireMemory(size) if (got < size) { freeMemory(got) throw new SparkException(s"Can't acquire $size bytes memory to build hash relation, " + @@ -408,15 +408,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } - private def freeMemory(size: Long): Unit = { - mm.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this) - } - private def init(): Unit = { if (mm != null) { var n = 1 while (n < capacity) n *= 2 - acquireMemory(n * 2 * 8 + (1 << 20)) + ensureAcquireMemory(n * 2 * 8 + (1 << 20)) array = new Array[Long](n * 2) mask = n * 2 - 2 page = new Array[Long](1 << 17) // 1M bytes @@ -538,7 +534,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap if (used >= (1 << 30)) { sys.error("Can not build a HashedRelation that is larger than 8G") } - acquireMemory(used * 8L * 2) + ensureAcquireMemory(used * 8L * 2) val newPage = new Array[Long](used * 2) Platform.copyMemory(page, Platform.LONG_ARRAY_OFFSET, newPage, Platform.LONG_ARRAY_OFFSET, cursor - Platform.LONG_ARRAY_OFFSET) @@ -591,7 +587,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap var old_array = array val n = array.length numKeys = 0 - acquireMemory(n * 2 * 8L) + ensureAcquireMemory(n * 2 * 8L) array = new Array[Long](n * 2) mask = n * 2 - 2 var i = 0 @@ -613,7 +609,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap // Convert to dense mode if it does not require more memory or could fit within L1 cache if (range < array.length || range < 1024) { try { - acquireMemory((range + 1) * 8) + ensureAcquireMemory((range + 1) * 8) } catch { case e: SparkException => // there is no enough memory to convert From 67c54721d35b1a97391a552e739ddcde3405bff9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 May 2016 11:51:50 +0100 Subject: [PATCH 257/313] [MINOR][SQL] Remove unused pattern matching variables in Optimizers. ## What changes were proposed in this pull request? This PR removes unused pattern matching variable in Optimizers in order to improve readability. ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13145 from dongjoon-hyun/remove_unused_pattern_matching_variables. (cherry picked from commit d2f81df1ba67f5d3fc39719d9a67c52d7903e59e) Signed-off-by: Sean Owen --- .../sql/catalyst/optimizer/Optimizer.scala | 33 +++++++++---------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index af7532e0c03e..be9f03d4baaa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -151,9 +151,8 @@ object SamplePushDown extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Push down projection into sample - case Project(projectList, s @ Sample(lb, up, replace, seed, child)) => - Sample(lb, up, replace, seed, - Project(projectList, child))() + case Project(projectList, Sample(lb, up, replace, seed, child)) => + Sample(lb, up, replace, seed, Project(projectList, child))() } } @@ -226,7 +225,7 @@ object LimitPushDown extends Rule[LogicalPlan] { private def stripGlobalLimitIfPresent(plan: LogicalPlan): LogicalPlan = { plan match { - case GlobalLimit(expr, child) => child + case GlobalLimit(_, child) => child case _ => plan } } @@ -259,7 +258,7 @@ object LimitPushDown extends Rule[LogicalPlan] { // - If one side is already limited, stack another limit on top if the new limit is smaller. // The redundant limit will be collapsed by the CombineLimits rule. // - If neither side is limited, limit the side that is estimated to be bigger. - case LocalLimit(exp, join @ Join(left, right, joinType, condition)) => + case LocalLimit(exp, join @ Join(left, right, joinType, _)) => val newJoin = joinType match { case RightOuter => join.copy(right = maybePushLimit(exp, right)) case LeftOuter => join.copy(left = maybePushLimit(exp, left)) @@ -408,7 +407,7 @@ object ColumnPruning extends Rule[LogicalPlan] { p.copy(child = g.copy(join = false)) // Eliminate unneeded attributes from right side of a Left Existence Join. - case j @ Join(left, right, LeftExistence(_), condition) => + case j @ Join(_, right, LeftExistence(_), _) => j.copy(right = prunedChild(right, j.references)) // all the columns will be used to compare, so we can't prune them @@ -440,10 +439,10 @@ object ColumnPruning extends Rule[LogicalPlan] { case w: Window if w.windowExpressions.isEmpty => w.child // Eliminate no-op Projects - case p @ Project(projectList, child) if sameOutput(child.output, p.output) => child + case p @ Project(_, child) if sameOutput(child.output, p.output) => child // Can't prune the columns on LeafNode - case p @ Project(_, l: LeafNode) => p + case p @ Project(_, _: LeafNode) => p // for all other logical plans that inherits the output from it's children case p @ Project(_, child) => @@ -541,7 +540,7 @@ object CollapseProject extends Rule[LogicalPlan] { */ object CollapseRepartition extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case r @ Repartition(numPartitions, shuffle, Repartition(_, _, child)) => + case Repartition(numPartitions, shuffle, Repartition(_, _, child)) => Repartition(numPartitions, shuffle, child) } } @@ -917,7 +916,7 @@ object CombineUnions extends Rule[LogicalPlan] { */ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case ff @ Filter(fc, nf @ Filter(nc, grandChild)) => + case Filter(fc, nf @ Filter(nc, grandChild)) => (ExpressionSet(splitConjunctivePredicates(fc)) -- ExpressionSet(splitConjunctivePredicates(nc))).reduceOption(And) match { case Some(ac) => @@ -1071,9 +1070,9 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { } // two filters should be combine together by other rules - case filter @ Filter(_, f: Filter) => filter + case filter @ Filter(_, _: Filter) => filter // should not push predicates through sample, or will generate different results. - case filter @ Filter(_, s: Sample) => filter + case filter @ Filter(_, _: Sample) => filter case filter @ Filter(condition, u: UnaryNode) if u.expressions.forall(_.deterministic) => pushDownPredicate(filter, u.child) { predicate => @@ -1352,11 +1351,11 @@ object RemoveDispensableExpressions extends Rule[LogicalPlan] { */ object CombineLimits extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case ll @ GlobalLimit(le, nl @ GlobalLimit(ne, grandChild)) => + case GlobalLimit(le, GlobalLimit(ne, grandChild)) => GlobalLimit(Least(Seq(ne, le)), grandChild) - case ll @ LocalLimit(le, nl @ LocalLimit(ne, grandChild)) => + case LocalLimit(le, LocalLimit(ne, grandChild)) => LocalLimit(Least(Seq(ne, le)), grandChild) - case ll @ Limit(le, nl @ Limit(ne, grandChild)) => + case Limit(le, Limit(ne, grandChild)) => Limit(Least(Seq(ne, le)), grandChild) } } @@ -1588,7 +1587,7 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] { */ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case f @ Filter(condition, child) => + case Filter(condition, child) => val (withSubquery, withoutSubquery) = splitConjunctivePredicates(condition).partition(PredicateSubquery.hasPredicateSubquery) @@ -1619,7 +1618,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { val replaced = predicate transformUp { case PredicateSubquery(sub, conditions, nullAware, _) => // TODO: support null-aware join - val exists = AttributeReference("exists", BooleanType, false)() + val exists = AttributeReference("exists", BooleanType, nullable = false)() joined = Join(joined, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) exists } From d005f76e6c3a5a01153c0189e774b9717c1a51f9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 18 May 2016 11:18:33 -0700 Subject: [PATCH 258/313] [SPARK-15342] [SQL] [PYSPARK] PySpark test for non ascii column name does not actually test with unicode column name ## What changes were proposed in this pull request? The PySpark SQL `test_column_name_with_non_ascii` wants to test non-ascii column name. But it doesn't actually test it. We need to construct an unicode explicitly using `unicode` under Python 2. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13134 from viirya/correct-non-ascii-colname-pytest. (cherry picked from commit 3d1e67f903ab3512fcad82b94b1825578f8117c9) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 11 +++++++++-- python/pyspark/sql/types.py | 3 ++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index e86f44281d49..1790432edd5d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1044,8 +1044,15 @@ def test_access_column(self): self.assertRaises(TypeError, lambda: df[{}]) def test_column_name_with_non_ascii(self): - df = self.spark.createDataFrame([(1,)], ["数量"]) - self.assertEqual(StructType([StructField("数量", LongType(), True)]), df.schema) + if sys.version >= '3': + columnName = "数量" + self.assertTrue(isinstance(columnName, str)) + else: + columnName = unicode("数量", "utf-8") + self.assertTrue(isinstance(columnName, unicode)) + schema = StructType([StructField(columnName, LongType(), True)]) + df = self.spark.createDataFrame([(1,)], schema) + self.assertEqual(schema, df.schema) self.assertEqual("DataFrame[数量: bigint]", str(df)) self.assertEqual([("数量", 'bigint')], df.dtypes) self.assertEqual(1, df.select("数量").first()[0]) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 30ab130f2948..7d8d0230b4ab 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -27,7 +27,7 @@ if sys.version >= "3": long = int - unicode = str + basestring = unicode = str from py4j.protocol import register_input_converter from py4j.java_gateway import JavaClass @@ -401,6 +401,7 @@ def __init__(self, name, dataType, nullable=True, metadata=None): False """ assert isinstance(dataType, DataType), "dataType should be DataType" + assert isinstance(name, basestring), "field name should be string" if not isinstance(name, str): name = name.encode('utf-8') self.name = name From 0da8bce0e3fcf6a7f40b5e23e57ce45795926432 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 18 May 2016 21:13:12 +0200 Subject: [PATCH 259/313] [SPARK-14891][ML] Add schema validation for ALS This PR adds schema validation to `ml`'s ALS and ALSModel. Currently, no schema validation was performed as `transformSchema` was never called in `ALS.fit` or `ALSModel.transform`. Furthermore, due to no schema validation, if users passed in Long (or Float etc) ids, they would be silently cast to Int with no warning or error thrown. With this PR, ALS now supports all numeric types for `user`, `item`, and `rating` columns. The rating column is cast to `Float` and the user and item cols are cast to `Int` (as is the case currently) - however for user/item, the cast throws an error if the value is outside integer range. Behavior for rating col is unchanged (as it is not an issue). ## How was this patch tested? New test cases in `ALSSuite`. Author: Nick Pentreath Closes #12762 from MLnick/SPARK-14891-als-validate-schema. (cherry picked from commit e8b79afa024123f9d4ceaf0a1043a7e37d913a8d) Signed-off-by: Nick Pentreath --- .../apache/spark/ml/recommendation/ALS.scala | 55 ++++++++++++----- .../spark/ml/recommendation/ALSSuite.scala | 61 +++++++++++++++++-- .../apache/spark/ml/util/MLTestingUtils.scala | 45 ++++++++++++++ python/pyspark/ml/recommendation.py | 8 +-- 4 files changed, 147 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 509c944fed74..f257382d2205 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -42,7 +42,7 @@ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} @@ -53,24 +53,43 @@ import org.apache.spark.util.random.XORShiftRandom */ private[recommendation] trait ALSModelParams extends Params with HasPredictionCol { /** - * Param for the column name for user ids. + * Param for the column name for user ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "user" * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids") + val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getUserCol: String = $(userCol) /** - * Param for the column name for item ids. + * Param for the column name for item ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "item" * @group param */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids") + val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getItemCol: String = $(itemCol) + + /** + * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is + * out of integer range. + */ + protected val checkedCast = udf { (n: Double) => + if (n > Int.MaxValue || n < Int.MinValue) { + throw new IllegalArgumentException(s"ALS only supports values in Integer range for columns " + + s"${$(userCol)} and ${$(itemCol)}. Value $n was out of Integer range.") + } else { + n.toInt + } + } } /** @@ -193,10 +212,11 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w * @return output schema */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) - val ratingType = schema($(ratingCol)).dataType - require(ratingType == FloatType || ratingType == DoubleType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) + // rating will be cast to Float + SchemaUtils.checkNumericType(schema, $(ratingCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } } @@ -232,6 +252,7 @@ class ALSModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema) // Register a UDF for DataFrame, and then // create a new column named map(predictionCol) by running the predict UDF. val predict = udf { (userFeatures: Seq[Float], itemFeatures: Seq[Float]) => @@ -242,16 +263,19 @@ class ALSModel private[ml] ( } } dataset - .join(userFactors, dataset($(userCol)) === userFactors("id"), "left") - .join(itemFactors, dataset($(itemCol)) === itemFactors("id"), "left") + .join(userFactors, + checkedCast(dataset($(userCol)).cast(DoubleType)) === userFactors("id"), "left") + .join(itemFactors, + checkedCast(dataset($(itemCol)).cast(DoubleType)) === itemFactors("id"), "left") .select(dataset("*"), predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) } @Since("1.3.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } @@ -430,10 +454,13 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] @Since("2.0.0") override def fit(dataset: Dataset[_]): ALSModel = { + transformSchema(dataset.schema) import dataset.sparkSession.implicits._ + val r = if ($(ratingCol) != "") col($(ratingCol)).cast(FloatType) else lit(1.0f) val ratings = dataset - .select(col($(userCol)).cast(IntegerType), col($(itemCol)).cast(IntegerType), r) + .select(checkedCast(col($(userCol)).cast(DoubleType)), + checkedCast(col($(itemCol)).cast(DoubleType)), r) .rdd .map { row => Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index bbfc415cbb9b..59b5edc4013e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types.{FloatType, IntegerType} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -205,7 +206,6 @@ class ALSSuite /** * Generates an explicit feedback dataset for testing ALS. - * * @param numUsers number of users * @param numItems number of items * @param rank rank @@ -246,7 +246,6 @@ class ALSSuite /** * Generates an implicit feedback dataset for testing ALS. - * * @param numUsers number of users * @param numItems number of items * @param rank rank @@ -265,7 +264,6 @@ class ALSSuite /** * Generates random user/item factors, with i.i.d. values drawn from U(a, b). - * * @param size number of users/items * @param rank number of features * @param random random number generator @@ -284,7 +282,6 @@ class ALSSuite /** * Test ALS using the given training/test splits and parameters. - * * @param training training dataset * @param test test dataset * @param rank rank of the matrix factorization @@ -486,6 +483,62 @@ class ALSSuite assert(getFactors(model.userFactors) === getFactors(model2.userFactors)) assert(getFactors(model.itemFactors) === getFactors(model2.itemFactors)) } + + test("input type validation") { + val spark = this.spark + import spark.implicits._ + + // check that ALS can handle all numeric types for rating column + // and user/item columns (when the user/item ids are within Int range) + val als = new ALS().setMaxIter(1).setRank(1) + Seq(("user", IntegerType), ("item", IntegerType), ("rating", FloatType)).foreach { + case (colName, sqlType) => + MLTestingUtils.checkNumericTypesALS(als, spark, colName, sqlType) { + (ex, act) => + ex.userFactors.first().getSeq[Float](1) === act.userFactors.first.getSeq[Float](1) + } { (ex, act, _) => + ex.transform(_: DataFrame).select("prediction").first.getFloat(0) ~== + act.transform(_: DataFrame).select("prediction").first.getFloat(0) absTol 1e-6 + } + } + // check user/item ids falling outside of Int range + val big = Int.MaxValue.toLong + 1 + val small = Int.MinValue.toDouble - 1 + val df = Seq( + (0, 0L, 0d, 1, 1L, 1d, 3.0), + (0, big, small, 0, big, small, 2.0), + (1, 1L, 1d, 0, 0L, 0d, 5.0) + ).toDF("user", "user_big", "user_small", "item", "item_big", "item_small", "rating") + withClue("fit should fail when ids exceed integer range. ") { + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("user_big").as("user"), df("item"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("user_small").as("user"), df("item"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("item_big").as("item"), df("user"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("item_small").as("item"), df("user"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + } + withClue("transform should fail when ids exceed integer range. ") { + val model = als.fit(df) + assert(intercept[SparkException] { + model.transform(df.select(df("user_big").as("user"), df("item"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("user_small").as("user"), df("item"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("item_big").as("item"), df("user"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("item_small").as("item"), df("user"))).first + }.getMessage.contains("was out of Integer range")) + } + } } class ALSCleanerSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index 6aae625fc83f..80b976914cbd 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -22,6 +22,7 @@ import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.recommendation.{ALS, ALSModel} import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.functions._ @@ -58,6 +59,30 @@ object MLTestingUtils extends SparkFunSuite { "Column label must be of type NumericType but was actually of type StringType")) } + def checkNumericTypesALS( + estimator: ALS, + spark: SparkSession, + column: String, + baseType: NumericType) + (check: (ALSModel, ALSModel) => Unit) + (check2: (ALSModel, ALSModel, DataFrame) => Unit): Unit = { + val dfs = genRatingsDFWithNumericCols(spark, column) + val expected = estimator.fit(dfs(baseType)) + val actuals = dfs.keys.filter(_ != baseType).map(t => (t, estimator.fit(dfs(t)))) + actuals.foreach { case (_, actual) => check(expected, actual) } + actuals.foreach { case (t, actual) => check2(expected, actual, dfs(t)) } + + val baseDF = dfs(baseType) + val others = baseDF.columns.toSeq.diff(Seq(column)).map(col(_)) + val cols = Seq(col(column).cast(StringType)) ++ others + val strDF = baseDF.select(cols: _*) + val thrown = intercept[IllegalArgumentException] { + estimator.fit(strDF) + } + assert(thrown.getMessage.contains( + s"$column must be of type NumericType but was actually of type StringType")) + } + def checkNumericTypes[T <: Evaluator](evaluator: T, spark: SparkSession): Unit = { val dfs = genEvaluatorDFWithNumericLabelCol(spark, "label", "prediction") val expected = evaluator.evaluate(dfs(DoubleType)) @@ -116,6 +141,26 @@ object MLTestingUtils extends SparkFunSuite { }.toMap } + def genRatingsDFWithNumericCols( + spark: SparkSession, + column: String): Map[NumericType, DataFrame] = { + val df = spark.createDataFrame(Seq( + (0, 10, 1.0), + (1, 20, 2.0), + (2, 30, 3.0), + (3, 40, 4.0), + (4, 50, 5.0) + )).toDF("user", "item", "rating") + + val others = df.columns.toSeq.diff(Seq(column)).map(col(_)) + val types: Seq[NumericType] = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types.map { t => + val cols = Seq(col(column).cast(t)) ++ others + t -> df.select(cols: _*) + }.toMap + } + def genEvaluatorDFWithNumericLabelCol( spark: SparkSession, labelColName: String = "label", diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index d7cb65846574..86c00d91652d 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -110,10 +110,10 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha typeConverter=TypeConverters.toBoolean) alpha = Param(Params._dummy(), "alpha", "alpha for implicit preference", typeConverter=TypeConverters.toFloat) - userCol = Param(Params._dummy(), "userCol", "column name for user ids", - typeConverter=TypeConverters.toString) - itemCol = Param(Params._dummy(), "itemCol", "column name for item ids", - typeConverter=TypeConverters.toString) + userCol = Param(Params._dummy(), "userCol", "column name for user ids. Ids must be within " + + "the integer value range.", typeConverter=TypeConverters.toString) + itemCol = Param(Params._dummy(), "itemCol", "column name for item ids. Ids must be within " + + "the integer value range.", typeConverter=TypeConverters.toString) ratingCol = Param(Params._dummy(), "ratingCol", "column name for ratings", typeConverter=TypeConverters.toString) nonnegative = Param(Params._dummy(), "nonnegative", From d65707b7f685b6cee4fff4cf3354b6ea279d92bf Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 May 2016 23:19:55 +0100 Subject: [PATCH 260/313] [SPARK-15373][WEB UI] Spark UI should show consistent timezones. ## What changes were proposed in this pull request? Currently, SparkUI shows two timezones in a single page when the timezone of browser is different from the server JVM timezone. The following is an example on Databricks CE which uses 'Etc/UTC' timezone. - The time of `submitted` column of list and pop-up description shows `2016/05/18 00:03:07` - The time of `timeline chart` shows `2016/05/17 17:03:07`. ![Different Timezone](https://issues.apache.org/jira/secure/attachment/12804553/12804553_timezone.png) This PR fixes the **timeline chart** to use the same timezone by the followings. - Upgrade `vis` from 3.9.0(2015-01-16) to 4.16.1(2016-04-18) - Override `moment` of `vis` to get `offset` - Update `AllJobsPage`, `JobPage`, and `StagePage`. ## How was this patch tested? Manual. Run the following command and see the Spark UI's event timelines. ``` $ SPARK_SUBMIT_OPTS="-Dscala.usejavacp=true -Duser.timezone=Etc/UTC" bin/spark-submit --class org.apache.spark.repl.Main ... scala> sql("select 1").head ``` Author: Dongjoon Hyun Closes #13158 from dongjoon-hyun/SPARK-15373. (cherry picked from commit cc6a47dd81fb2fcf2d2ec158c1537bcc27ed1674) Signed-off-by: Sean Owen --- .../apache/spark/ui/static/timeline-view.js | 19 ++++++--- .../org/apache/spark/ui/static/vis.min.css | 2 +- .../org/apache/spark/ui/static/vis.min.js | 41 +++++++++++-------- .../scala/org/apache/spark/ui/UIUtils.scala | 5 ++- .../apache/spark/ui/jobs/AllJobsPage.scala | 2 +- .../org/apache/spark/ui/jobs/JobPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 5 ++- 7 files changed, 49 insertions(+), 28 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index f1beca29c2aa..9ab5684d901f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -15,7 +15,7 @@ * limitations under the License. */ -function drawApplicationTimeline(groupArray, eventObjArray, startTime) { +function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $("#application-timeline")[0]; @@ -26,7 +26,10 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime) { editable: false, showCurrentTime: false, min: startTime, - zoomable: false + zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var applicationTimeline = new vis.Timeline(container); @@ -87,7 +90,7 @@ $(function (){ } }); -function drawJobTimeline(groupArray, eventObjArray, startTime) { +function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $('#job-timeline')[0]; @@ -99,6 +102,9 @@ function drawJobTimeline(groupArray, eventObjArray, startTime) { showCurrentTime: false, min: startTime, zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var jobTimeline = new vis.Timeline(container); @@ -159,7 +165,7 @@ $(function (){ } }); -function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, maxFinishTime) { +function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, maxFinishTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $("#task-assignment-timeline")[0] @@ -173,7 +179,10 @@ function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, ma showCurrentTime: false, min: minLaunchTime, max: maxFinishTime, - zoomable: false + zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var taskTimeline = new vis.Timeline(container) diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.css b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css index a390c40d6757..40d182cfde23 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.min.css +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css @@ -1 +1 @@ -.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}.vis [class*=span]{min-height:0;width:auto}.vis.timeline.root{position:relative;border:1px solid #bfbfbf;overflow:hidden;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel{position:absolute;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right,.vis.timeline .vispanel.top{border:1px #bfbfbf}.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.top{border-left-style:solid;border-right-style:solid}.vis.timeline .background{overflow:hidden}.vis.timeline .vispanel>.content{position:relative}.vis.timeline .vispanel .shadow{position:absolute;width:100%;height:1px;box-shadow:0 0 10px rgba(0,0,0,.8)}.vis.timeline .vispanel .shadow.top{top:-1px;left:0}.vis.timeline .vispanel .shadow.bottom{bottom:-1px;left:0}.vis.timeline .labelset{position:relative;overflow:hidden;box-sizing:border-box}.vis.timeline .labelset .vlabel{position:relative;left:0;top:0;width:100%;color:#4d4d4d;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .labelset .vlabel:last-child{border-bottom:none}.vis.timeline .labelset .vlabel .inner{display:inline-block;padding:5px}.vis.timeline .labelset .vlabel .inner.hidden{padding:0}.vis.timeline .itemset{position:relative;padding:0;margin:0;box-sizing:border-box}.vis.timeline .itemset .background,.vis.timeline .itemset .foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis.timeline .axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis.timeline .foreground .group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .foreground .group:last-child{border-bottom:none}.vis.timeline .item{position:absolute;color:#1A1A1A;border-color:#97B0F8;border-width:1px;background-color:#D5DDF6;display:inline-block;padding:5px}.vis.timeline .item.selected{border-color:#FFC200;background-color:#FFF785;z-index:2}.vis.timeline .editable .item.selected{cursor:move}.vis.timeline .item.point.selected{background-color:#FFF785}.vis.timeline .item.box{text-align:center;border-style:solid;border-radius:2px}.vis.timeline .item.point{background:0 0}.vis.timeline .item.dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis.timeline .item.range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis.timeline .item.background{overflow:hidden;border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis.timeline .item.range .content{position:relative;display:inline-block;max-width:100%;overflow:hidden}.vis.timeline .item.background .content{position:absolute;display:inline-block;overflow:hidden;max-width:100%;margin:5px}.vis.timeline .item.line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis.timeline .item .content{white-space:nowrap;overflow:hidden}.vis.timeline .item .delete{background:url(img/timeline/delete.png) top center no-repeat;position:absolute;width:24px;height:24px;top:0;right:-24px;cursor:pointer}.vis.timeline .item.range .drag-left{position:absolute;width:24px;height:100%;top:0;left:-4px;cursor:w-resize}.vis.timeline .item.range .drag-right{position:absolute;width:24px;height:100%;top:0;right:-4px;cursor:e-resize}.vis.timeline .timeaxis{position:relative;overflow:hidden}.vis.timeline .timeaxis.foreground{top:0;left:0;width:100%}.vis.timeline .timeaxis.background{position:absolute;top:0;left:0;width:100%;height:100%}.vis.timeline .timeaxis .text{position:absolute;color:#4d4d4d;padding:3px;white-space:nowrap}.vis.timeline .timeaxis .text.measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right:0;visibility:hidden}.vis.timeline .timeaxis .grid.vertical{position:absolute;border-left:1px solid}.vis.timeline .timeaxis .grid.minor{border-color:#e5e5e5}.vis.timeline .timeaxis .grid.major{border-color:#bfbfbf}.vis.timeline .currenttime{background-color:#FF7F6E;width:2px;z-index:1}.vis.timeline .customtime{background-color:#6E94FF;width:2px;cursor:move;z-index:1}.vis.timeline .vispanel.background.horizontal .grid.horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis.timeline .vispanel.background.horizontal .grid.minor{border-color:#e5e5e5}.vis.timeline .vispanel.background.horizontal .grid.major{border-color:#bfbfbf}.vis.timeline .dataaxis .yAxis.major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis.timeline .dataaxis .yAxis.major.measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis.timeline .dataaxis .yAxis.minor.measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis.timeline .dataaxis .yAxis.title.measure{padding:0;margin:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title.left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis.timeline .dataaxis .yAxis.title.right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis.timeline .legend{background-color:rgba(247,252,255,.65);padding:5px;border-color:#b3b3b3;border-style:solid;border-width:1px;box-shadow:2px 2px 10px rgba(154,154,154,.55)}.vis.timeline .legendText{white-space:nowrap;display:inline-block}.vis.timeline .graphGroup0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis.timeline .graphGroup1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis.timeline .graphGroup2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis.timeline .graphGroup3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis.timeline .graphGroup4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis.timeline .graphGroup5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis.timeline .graphGroup6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis.timeline .graphGroup7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis.timeline .graphGroup8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#f0f}.vis.timeline .graphGroup9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis.timeline .fill{fill-opacity:.1;stroke:none}.vis.timeline .bar{fill-opacity:.5;stroke-width:1px}.vis.timeline .point{stroke-width:2px;fill-opacity:1}.vis.timeline .legendBackground{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis.timeline .outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis.timeline .iconFill{fill-opacity:.3;stroke:none}div.network-manipulationDiv{border-width:0;border-bottom:1px;border-style:solid;border-color:#d6d9d8;background:#fff;background:-moz-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#fff),color-stop(48%,#fcfcfc),color-stop(50%,#fafafa),color-stop(100%,#fcfcfc));background:-webkit-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-o-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-ms-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:linear-gradient(to bottom,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffff', endColorstr='#fcfcfc', GradientType=0);position:absolute;left:0;top:0;width:100%;height:30px}div.network-manipulation-editMode{position:absolute;left:0;top:0;height:30px;margin-top:20px}div.network-manipulation-closeDiv{position:absolute;right:0;top:0;width:30px;height:30px;background-position:20px 3px;background-repeat:no-repeat;background-image:url(img/network/cross.png);cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-manipulation-closeDiv:hover{opacity:.6}span.network-manipulationUI{font-family:verdana;font-size:12px;-moz-border-radius:15px;border-radius:15px;display:inline-block;background-position:0 0;background-repeat:no-repeat;height:24px;margin:-14px 0 0 10px;vertical-align:middle;cursor:pointer;padding:0 8px;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}span.network-manipulationUI:hover{box-shadow:1px 1px 8px rgba(0,0,0,.2)}span.network-manipulationUI:active{box-shadow:1px 1px 8px rgba(0,0,0,.5)}span.network-manipulationUI.back{background-image:url(img/network/backIcon.png)}span.network-manipulationUI.none:hover{box-shadow:1px 1px 8px transparent;cursor:default}span.network-manipulationUI.none:active{box-shadow:1px 1px 8px transparent}span.network-manipulationUI.none{padding:0}span.network-manipulationUI.notification{margin:2px;font-weight:700}span.network-manipulationUI.add{background-image:url(img/network/addNodeIcon.png)}span.network-manipulationUI.edit{background-image:url(img/network/editIcon.png)}span.network-manipulationUI.edit.editmode{background-color:#fcfcfc;border-style:solid;border-width:1px;border-color:#ccc}span.network-manipulationUI.connect{background-image:url(img/network/connectIcon.png)}span.network-manipulationUI.delete{background-image:url(img/network/deleteIcon.png)}span.network-manipulationLabel{margin:0 0 0 23px;line-height:25px}div.network-seperatorLine{display:inline-block;width:1px;height:20px;background-color:#bdbdbd;margin:5px 7px 0 15px}div.network-navigation_wrapper{position:absolute;left:0;top:0;width:100%;height:100%}div.network-navigation{width:34px;height:34px;-moz-border-radius:17px;border-radius:17px;position:absolute;display:inline-block;background-position:2px 2px;background-repeat:no-repeat;cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-navigation:hover{box-shadow:0 0 3px 3px rgba(56,207,21,.3)}div.network-navigation:active{box-shadow:0 0 1px 3px rgba(56,207,21,.95)}div.network-navigation.up{background-image:url(img/network/upArrow.png);bottom:50px;left:55px}div.network-navigation.down{background-image:url(img/network/downArrow.png);bottom:10px;left:55px}div.network-navigation.left{background-image:url(img/network/leftArrow.png);bottom:10px;left:15px}div.network-navigation.right{background-image:url(img/network/rightArrow.png);bottom:10px;left:95px}div.network-navigation.zoomIn{background-image:url(img/network/plus.png);bottom:10px;right:15px}div.network-navigation.zoomOut{background-image:url(img/network/minus.png);bottom:10px;right:55px}div.network-navigation.zoomExtends{background-image:url(img/network/zoomExtends.png);bottom:50px;right:15px} \ No newline at end of file +.vis-background,.vis-labelset,.vis-timeline{overflow:hidden}.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}.vis [class*=span]{min-height:0;width:auto}div.vis-configuration{position:relative;display:block;float:left;font-size:12px}div.vis-configuration-wrapper{display:block;width:700px}div.vis-configuration-wrapper::after{clear:both;content:"";display:block}div.vis-configuration.vis-config-option-container{display:block;width:495px;background-color:#fff;border:2px solid #f7f8fa;border-radius:4px;margin-top:20px;left:10px;padding-left:5px}div.vis-configuration.vis-config-button{display:block;width:495px;height:25px;vertical-align:middle;line-height:25px;background-color:#f7f8fa;border:2px solid #ceced0;border-radius:4px;margin-top:20px;left:10px;padding-left:5px;cursor:pointer;margin-bottom:30px}div.vis-configuration.vis-config-button.hover{background-color:#4588e6;border:2px solid #214373;color:#fff}div.vis-configuration.vis-config-item{display:block;float:left;width:495px;height:25px;vertical-align:middle;line-height:25px}div.vis-configuration.vis-config-item.vis-config-s2{left:10px;background-color:#f7f8fa;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s3{left:20px;background-color:#e4e9f0;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s4{left:30px;background-color:#cfd8e6;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-header{font-size:18px;font-weight:700}div.vis-configuration.vis-config-label{width:120px;height:25px;line-height:25px}div.vis-configuration.vis-config-label.vis-config-s3{width:110px}div.vis-configuration.vis-config-label.vis-config-s4{width:100px}div.vis-configuration.vis-config-colorBlock{top:1px;width:30px;height:19px;border:1px solid #444;border-radius:2px;padding:0;margin:0;cursor:pointer}input.vis-configuration.vis-config-checkbox{left:-5px}input.vis-configuration.vis-config-rangeinput{position:relative;top:-5px;width:60px;padding:1px;margin:0;pointer-events:none}.vis-panel,.vis-timeline{padding:0;box-sizing:border-box}input.vis-configuration.vis-config-range{-webkit-appearance:none;border:0 solid #fff;background-color:rgba(0,0,0,0);width:300px;height:20px}input.vis-configuration.vis-config-range::-webkit-slider-runnable-track{width:300px;height:5px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(to bottom,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8', GradientType=0 );border:1px solid #999;box-shadow:#aaa 0 0 3px 0;border-radius:3px}input.vis-configuration.vis-config-range::-webkit-slider-thumb{-webkit-appearance:none;border:1px solid #14334b;height:17px;width:17px;border-radius:50%;background:#3876c2;background:-moz-linear-gradient(top,#3876c2 0,#385380 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#3876c2),color-stop(100%,#385380));background:-webkit-linear-gradient(top,#3876c2 0,#385380 100%);background:-o-linear-gradient(top,#3876c2 0,#385380 100%);background:-ms-linear-gradient(top,#3876c2 0,#385380 100%);background:linear-gradient(to bottom,#3876c2 0,#385380 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#3876c2', endColorstr='#385380', GradientType=0 );box-shadow:#111927 0 0 1px 0;margin-top:-7px}input.vis-configuration.vis-config-range:focus{outline:0}input.vis-configuration.vis-config-range:focus::-webkit-slider-runnable-track{background:#9d9d9d;background:-moz-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#9d9d9d),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-o-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:linear-gradient(to bottom,#9d9d9d 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#9d9d9d', endColorstr='#c8c8c8', GradientType=0 )}input.vis-configuration.vis-config-range::-moz-range-track{width:300px;height:10px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(to bottom,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8', GradientType=0 );border:1px solid #999;box-shadow:#aaa 0 0 3px 0;border-radius:3px}input.vis-configuration.vis-config-range::-moz-range-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:-moz-focusring{outline:#fff solid 1px;outline-offset:-1px}input.vis-configuration.vis-config-range::-ms-track{width:300px;height:5px;background:0 0;border-color:transparent;border-width:6px 0;color:transparent}input.vis-configuration.vis-config-range::-ms-fill-lower{background:#777;border-radius:10px}input.vis-configuration.vis-config-range::-ms-fill-upper{background:#ddd;border-radius:10px}input.vis-configuration.vis-config-range::-ms-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:focus::-ms-fill-lower{background:#888}input.vis-configuration.vis-config-range:focus::-ms-fill-upper{background:#ccc}.vis-configuration-popup{position:absolute;background:rgba(57,76,89,.85);border:2px solid #f2faff;line-height:30px;height:30px;width:150px;text-align:center;color:#fff;font-size:14px;border-radius:4px;-webkit-transition:opacity .3s ease-in-out;-moz-transition:opacity .3s ease-in-out;transition:opacity .3s ease-in-out}.vis-configuration-popup:after,.vis-configuration-popup:before{left:100%;top:50%;border:solid transparent;content:" ";height:0;width:0;position:absolute;pointer-events:none}.vis-configuration-popup:after{border-color:rgba(136,183,213,0);border-left-color:rgba(57,76,89,.85);border-width:8px;margin-top:-8px}.vis-configuration-popup:before{border-color:rgba(194,225,245,0);border-left-color:#f2faff;border-width:12px;margin-top:-12px}.vis-timeline{position:relative;border:1px solid #bfbfbf;margin:0}.vis-panel{position:absolute;margin:0}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right,.vis-panel.vis-top{border:1px #bfbfbf}.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-top{border-left-style:solid;border-right-style:solid}.vis-panel>.vis-content{position:relative}.vis-panel .vis-shadow{position:absolute;width:100%;height:1px;box-shadow:0 0 10px rgba(0,0,0,.8)}.vis-itemset,.vis-labelset,.vis-labelset .vis-label{position:relative;box-sizing:border-box}.vis-panel .vis-shadow.vis-top{top:-1px;left:0}.vis-panel .vis-shadow.vis-bottom{bottom:-1px;left:0}.vis-labelset .vis-label{left:0;top:0;width:100%;color:#4d4d4d;border-bottom:1px solid #bfbfbf}.vis-labelset .vis-label.draggable{cursor:pointer}.vis-labelset .vis-label:last-child{border-bottom:none}.vis-labelset .vis-label .vis-inner{display:inline-block;padding:5px}.vis-labelset .vis-label .vis-inner.vis-hidden{padding:0}.vis-itemset{padding:0;margin:0}.vis-itemset .vis-background,.vis-itemset .vis-foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis-axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis-foreground .vis-group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis-foreground .vis-group:last-child{border-bottom:none}.vis-overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-item{position:absolute;color:#1A1A1A;border-color:#97B0F8;border-width:1px;background-color:#D5DDF6;display:inline-block}.vis-item.vis-point.vis-selected,.vis-item.vis-selected{background-color:#FFF785}.vis-item.vis-selected{border-color:#FFC200;z-index:2}.vis-editable.vis-selected{cursor:move}.vis-item.vis-box{text-align:center;border-style:solid;border-radius:2px}.vis-item.vis-point{background:0 0}.vis-item.vis-dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis-item.vis-range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis-item.vis-background{border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis-item .vis-item-overflow{position:relative;width:100%;height:100%;padding:0;margin:0;overflow:hidden}.vis-item .vis-delete,.vis-item .vis-delete-rtl{background:url(img/timeline/delete.png) center no-repeat;height:24px;top:-4px;cursor:pointer}.vis-item.vis-range .vis-item-content{position:relative;display:inline-block}.vis-item.vis-background .vis-item-content{position:absolute;display:inline-block}.vis-item.vis-line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis-item .vis-item-content{white-space:nowrap;box-sizing:border-box;padding:5px}.vis-item .vis-delete{position:absolute;width:24px;right:-24px}.vis-item .vis-delete-rtl{position:absolute;width:24px;left:-24px}.vis-item.vis-range .vis-drag-left{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0;left:-4px;cursor:w-resize}.vis-item.vis-range .vis-drag-right{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0;right:-4px;cursor:e-resize}.vis-range.vis-item.vis-readonly .vis-drag-left,.vis-range.vis-item.vis-readonly .vis-drag-right{cursor:auto}.vis-time-axis{position:relative;overflow:hidden}.vis-time-axis.vis-foreground{top:0;left:0;width:100%}.vis-time-axis.vis-background{position:absolute;top:0;left:0;width:100%;height:100%}.vis-time-axis .vis-text{position:absolute;color:#4d4d4d;padding:3px;overflow:hidden;box-sizing:border-box;white-space:nowrap}.vis-time-axis .vis-text.vis-measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right:0;visibility:hidden}.vis-time-axis .vis-grid.vis-vertical{position:absolute;border-left:1px solid}.vis-time-axis .vis-grid.vis-vertical-rtl{position:absolute;border-right:1px solid}.vis-time-axis .vis-grid.vis-minor{border-color:#e5e5e5}.vis-time-axis .vis-grid.vis-major{border-color:#bfbfbf}.vis-current-time{background-color:#FF7F6E;width:2px;z-index:1}.vis-custom-time{background-color:#6E94FF;width:2px;cursor:move;z-index:1}div.vis-network div.vis-close,div.vis-network div.vis-edit-mode div.vis-button,div.vis-network div.vis-manipulation div.vis-button{cursor:pointer;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;-webkit-touch-callout:none;-khtml-user-select:none}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-minor{border-color:#e5e5e5}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-major{border-color:#bfbfbf}.vis-data-axis .vis-y-axis.vis-major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-major.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-minor.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis-data-axis .vis-y-axis.vis-title.vis-measure{padding:0;margin:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title.vis-left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis-data-axis .vis-y-axis.vis-title.vis-right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis-legend{background-color:rgba(247,252,255,.65);padding:5px;border:1px solid #b3b3b3;box-shadow:2px 2px 10px rgba(154,154,154,.55)}.vis-legend-text{white-space:nowrap;display:inline-block}.vis-graph-group0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis-graph-group1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis-graph-group2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis-graph-group3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis-graph-group4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis-graph-group5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis-graph-group6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis-graph-group7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis-graph-group8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#f0f}.vis-graph-group9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis-timeline .vis-fill{fill-opacity:.1;stroke:none}.vis-timeline .vis-bar{fill-opacity:.5;stroke-width:1px}.vis-timeline .vis-point{stroke-width:2px;fill-opacity:1}.vis-timeline .vis-legend-background{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis-timeline .vis-outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis-timeline .vis-icon-fill{fill-opacity:.3;stroke:none}div.vis-network div.vis-manipulation{border-width:0;border-bottom:1px;border-style:solid;border-color:#d6d9d8;background:#fff;background:-moz-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#fff),color-stop(48%,#fcfcfc),color-stop(50%,#fafafa),color-stop(100%,#fcfcfc));background:-webkit-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-o-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-ms-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:linear-gradient(to bottom,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#ffffff', endColorstr='#fcfcfc', GradientType=0 );padding-top:4px;position:absolute;left:0;top:0;width:100%;height:28px}div.vis-network div.vis-edit-mode{position:absolute;left:0;top:5px;height:30px}div.vis-network div.vis-close{position:absolute;right:0;top:0;width:30px;height:30px;background-position:20px 3px;background-repeat:no-repeat;background-image:url(img/network/cross.png);user-select:none}div.vis-network div.vis-close:hover{opacity:.6}div.vis-network div.vis-edit-mode div.vis-button,div.vis-network div.vis-manipulation div.vis-button{float:left;font-family:verdana;font-size:12px;-moz-border-radius:15px;border-radius:15px;display:inline-block;background-position:0 0;background-repeat:no-repeat;height:24px;margin-left:10px;padding:0 8px;user-select:none}div.vis-network div.vis-manipulation div.vis-button:hover{box-shadow:1px 1px 8px rgba(0,0,0,.2)}div.vis-network div.vis-manipulation div.vis-button:active{box-shadow:1px 1px 8px rgba(0,0,0,.5)}div.vis-network div.vis-manipulation div.vis-button.vis-back{background-image:url(img/network/backIcon.png)}div.vis-network div.vis-manipulation div.vis-button.vis-none:hover{box-shadow:1px 1px 8px transparent;cursor:default}div.vis-network div.vis-manipulation div.vis-button.vis-none:active{box-shadow:1px 1px 8px transparent}div.vis-network div.vis-manipulation div.vis-button.vis-none{padding:0}div.vis-network div.vis-manipulation div.notification{margin:2px;font-weight:700}div.vis-network div.vis-manipulation div.vis-button.vis-add{background-image:url(img/network/addNodeIcon.png)}div.vis-network div.vis-edit-mode div.vis-button.vis-edit,div.vis-network div.vis-manipulation div.vis-button.vis-edit{background-image:url(img/network/editIcon.png)}div.vis-network div.vis-edit-mode div.vis-button.vis-edit.vis-edit-mode{background-color:#fcfcfc;border:1px solid #ccc}div.vis-network div.vis-manipulation div.vis-button.vis-connect{background-image:url(img/network/connectIcon.png)}div.vis-network div.vis-manipulation div.vis-button.vis-delete{background-image:url(img/network/deleteIcon.png)}div.vis-network div.vis-edit-mode div.vis-label,div.vis-network div.vis-manipulation div.vis-label{margin:0 0 0 23px;line-height:25px}div.vis-network div.vis-manipulation div.vis-separator-line{float:left;display:inline-block;width:1px;height:21px;background-color:#bdbdbd;margin:0 7px 0 15px}div.vis-network-tooltip{position:absolute;visibility:hidden;padding:5px;white-space:nowrap;font-family:verdana;font-size:14px;color:#000;background-color:#f5f4ed;-moz-border-radius:3px;-webkit-border-radius:3px;border-radius:3px;border:1px solid #808074;box-shadow:3px 3px 10px rgba(0,0,0,.2);pointer-events:none}div.vis-network div.vis-navigation div.vis-button{width:34px;height:34px;-moz-border-radius:17px;border-radius:17px;position:absolute;display:inline-block;background-position:2px 2px;background-repeat:no-repeat;cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.vis-network div.vis-navigation div.vis-button:hover{box-shadow:0 0 3px 3px rgba(56,207,21,.3)}div.vis-network div.vis-navigation div.vis-button:active{box-shadow:0 0 1px 3px rgba(56,207,21,.95)}div.vis-network div.vis-navigation div.vis-button.vis-up{background-image:url(img/network/upArrow.png);bottom:50px;left:55px}div.vis-network div.vis-navigation div.vis-button.vis-down{background-image:url(img/network/downArrow.png);bottom:10px;left:55px}div.vis-network div.vis-navigation div.vis-button.vis-left{background-image:url(img/network/leftArrow.png);bottom:10px;left:15px}div.vis-network div.vis-navigation div.vis-button.vis-right{background-image:url(img/network/rightArrow.png);bottom:10px;left:95px}div.vis-network div.vis-navigation div.vis-button.vis-zoomIn{background-image:url(img/network/plus.png);bottom:10px;right:15px}div.vis-network div.vis-navigation div.vis-button.vis-zoomOut{background-image:url(img/network/minus.png);bottom:10px;right:55px}div.vis-network div.vis-navigation div.vis-button.vis-zoomExtends{background-image:url(img/network/zoomExtends.png);bottom:50px;right:15px}div.vis-color-picker{position:absolute;top:0;left:30px;margin-top:-140px;margin-left:30px;width:310px;height:444px;z-index:1;padding:10px;border-radius:15px;background-color:#fff;display:none;box-shadow:rgba(0,0,0,.5) 0 0 10px 0}div.vis-color-picker div.vis-arrow{position:absolute;top:147px;left:5px}div.vis-color-picker div.vis-arrow::after,div.vis-color-picker div.vis-arrow::before{right:100%;top:50%;border:solid transparent;content:" ";height:0;width:0;position:absolute;pointer-events:none}div.vis-color-picker div.vis-arrow:after{border-color:rgba(255,255,255,0);border-right-color:#fff;border-width:30px;margin-top:-30px}div.vis-color-picker div.vis-color{position:absolute;width:289px;height:289px;cursor:pointer}div.vis-color-picker div.vis-brightness{position:absolute;top:313px}div.vis-color-picker div.vis-opacity{position:absolute;top:350px}div.vis-color-picker div.vis-selector{position:absolute;top:137px;left:137px;width:15px;height:15px;border-radius:15px;border:1px solid #fff;background:#4c4c4c;background:-moz-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#4c4c4c),color-stop(12%,#595959),color-stop(25%,#666),color-stop(39%,#474747),color-stop(50%,#2c2c2c),color-stop(51%,#000),color-stop(60%,#111),color-stop(76%,#2b2b2b),color-stop(91%,#1c1c1c),color-stop(100%,#131313));background:-webkit-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-o-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-ms-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:linear-gradient(to bottom,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#4c4c4c', endColorstr='#131313', GradientType=0 )}div.vis-color-picker div.vis-initial-color,div.vis-color-picker div.vis-new-color{width:140px;height:20px;top:380px;font-size:10px;color:rgba(0,0,0,.4);line-height:20px;position:absolute;vertical-align:middle}div.vis-color-picker div.vis-new-color{border:1px solid rgba(0,0,0,.1);border-radius:5px;left:159px;text-align:right;padding-right:2px}div.vis-color-picker div.vis-initial-color{border:1px solid rgba(0,0,0,.1);border-radius:5px;left:10px;text-align:left;padding-left:2px}div.vis-color-picker div.vis-label{position:absolute;width:300px;left:10px}div.vis-color-picker div.vis-label.vis-brightness{top:300px}div.vis-color-picker div.vis-label.vis-opacity{top:338px}div.vis-color-picker div.vis-button{position:absolute;width:68px;height:25px;border-radius:10px;vertical-align:middle;text-align:center;line-height:25px;top:410px;border:2px solid #d9d9d9;background-color:#f7f7f7;cursor:pointer}div.vis-color-picker div.vis-button.vis-cancel{left:5px}div.vis-color-picker div.vis-button.vis-load{left:82px}div.vis-color-picker div.vis-button.vis-apply{left:159px}div.vis-color-picker div.vis-button.vis-save{left:236px}div.vis-color-picker input.vis-range{width:290px;height:20px} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js index 2b3b1d60463f..92b8ed75d85f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js @@ -4,11 +4,11 @@ * * A dynamic, browser-based visualization library. * - * @version 3.9.0 - * @date 2015-01-16 + * @version 4.16.1 + * @date 2016-04-18 * * @license - * Copyright (C) 2011-2014 Almende B.V, http://almende.com + * Copyright (C) 2011-2016 Almende B.V, http://almende.com * * Vis.js is dual licensed under both * @@ -22,17 +22,24 @@ * * Vis.js may be distributed under either license. */ -"use strict";!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define(e):"object"==typeof exports?exports.vis=e():t.vis=e()}(this,function(){return function(t){function e(s){if(i[s])return i[s].exports;var o=i[s]={exports:{},id:s,loaded:!1};return t[s].call(o.exports,o,o.exports,e),o.loaded=!0,o.exports}var i={};return e.m=t,e.c=i,e.p="",e(0)}([function(t,e,i){e.util=i(1),e.DOMutil=i(2),e.DataSet=i(3),e.DataView=i(4),e.Queue=i(5),e.Graph3d=i(6),e.graph3d={Camera:i(7),Filter:i(8),Point2d:i(9),Point3d:i(10),Slider:i(11),StepNumber:i(12)},e.Timeline=i(13),e.Graph2d=i(14),e.timeline={DateUtil:i(15),DataStep:i(16),Range:i(17),stack:i(18),TimeStep:i(19),components:{items:{Item:i(31),BackgroundItem:i(32),BoxItem:i(33),PointItem:i(34),RangeItem:i(35)},Component:i(20),CurrentTime:i(21),CustomTime:i(22),DataAxis:i(23),GraphGroup:i(24),Group:i(25),BackgroundGroup:i(26),ItemSet:i(27),Legend:i(28),LineGraph:i(29),TimeAxis:i(30)}},e.Network=i(36),e.network={Edge:i(37),Groups:i(38),Images:i(39),Node:i(40),Popup:i(41),dotparser:i(42),gephiParser:i(43)},e.Graph=function(){throw new Error("Graph is renamed to Network. Please create a graph as new vis.Network(...)")},e.moment=i(44),e.hammer=i(45),e.Hammer=i(45)},function(t,e,i){var s=i(44);e.isNumber=function(t){return t instanceof Number||"number"==typeof t},e.isString=function(t){return t instanceof String||"string"==typeof t},e.isDate=function(t){if(t instanceof Date)return!0;if(e.isString(t)){var i=o.exec(t);if(i)return!0;if(!isNaN(Date.parse(t)))return!0}return!1},e.isDataTable=function(t){return"undefined"!=typeof google&&google.visualization&&google.visualization.DataTable&&t instanceof google.visualization.DataTable},e.randomUUID=function(){var t=function(){return Math.floor(65536*Math.random()).toString(16)};return t()+t()+"-"+t()+"-"+t()+"-"+t()+"-"+t()+t()+t()},e.extend=function(t){for(var e=1,i=arguments.length;i>e;e++){var s=arguments[e];for(var o in s)s.hasOwnProperty(o)&&(t[o]=s[o])}return t},e.selectiveExtend=function(t,e){if(!Array.isArray(t))throw new Error("Array with property names expected as first argument");for(var i=2;ii;i++)if(t[i]!=e[i])return!1;return!0},e.convert=function(t,i){var n;if(void 0===t)return void 0;if(null===t)return null;if(!i)return t;if("string"!=typeof i&&!(i instanceof String))throw new Error("Type must be a string");switch(i){case"boolean":case"Boolean":return Boolean(t);case"number":case"Number":return Number(t.valueOf());case"string":case"String":return String(t);case"Date":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return new Date(t.valueOf());if(s.isMoment(t))return new Date(t.valueOf());if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])):s(t).toDate();throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"Moment":if(e.isNumber(t))return s(t);if(t instanceof Date)return s(t.valueOf());if(s.isMoment(t))return s(t);if(e.isString(t))return n=o.exec(t),s(n?Number(n[1]):t);throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"ISODate":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return t.toISOString();if(s.isMoment(t))return t.toDate().toISOString();if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])).toISOString():new Date(t).toISOString();throw new Error("Cannot convert object of type "+e.getType(t)+" to type ISODate");case"ASPDate":if(e.isNumber(t))return"/Date("+t+")/";if(t instanceof Date)return"/Date("+t.valueOf()+")/";if(e.isString(t)){n=o.exec(t);var r;return r=n?new Date(Number(n[1])).valueOf():new Date(t).valueOf(),"/Date("+r+")/"}throw new Error("Cannot convert object of type "+e.getType(t)+" to type ASPDate");default:throw new Error('Unknown type "'+i+'"')}};var o=/^\/?Date\((\-?\d+)/i;e.getType=function(t){var e=typeof t;return"object"==e?null==t?"null":t instanceof Boolean?"Boolean":t instanceof Number?"Number":t instanceof String?"String":Array.isArray(t)?"Array":t instanceof Date?"Date":"Object":"number"==e?"Number":"boolean"==e?"Boolean":"string"==e?"String":e},e.getAbsoluteLeft=function(t){return t.getBoundingClientRect().left},e.getAbsoluteTop=function(t){return t.getBoundingClientRect().top},e.addClassName=function(t,e){var i=t.className.split(" ");-1==i.indexOf(e)&&(i.push(e),t.className=i.join(" "))},e.removeClassName=function(t,e){var i=t.className.split(" "),s=i.indexOf(e);-1!=s&&(i.splice(s,1),t.className=i.join(" "))},e.forEach=function(t,e){var i,s;if(Array.isArray(t))for(i=0,s=t.length;s>i;i++)e(t[i],i,t);else for(i in t)t.hasOwnProperty(i)&&e(t[i],i,t)},e.toArray=function(t){var e=[];for(var i in t)t.hasOwnProperty(i)&&e.push(t[i]);return e},e.updateProperty=function(t,e,i){return t[e]!==i?(t[e]=i,!0):!1},e.addEventListener=function(t,e,i,s){t.addEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.addEventListener(e,i,s)):t.attachEvent("on"+e,i)},e.removeEventListener=function(t,e,i,s){t.removeEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.removeEventListener(e,i,s)):t.detachEvent("on"+e,i)},e.preventDefault=function(t){t||(t=window.event),t.preventDefault?t.preventDefault():t.returnValue=!1},e.getTarget=function(t){t||(t=window.event);var e;return t.target?e=t.target:t.srcElement&&(e=t.srcElement),void 0!=e.nodeType&&3==e.nodeType&&(e=e.parentNode),e},e.option={},e.option.asBoolean=function(t,e){return"function"==typeof t&&(t=t()),null!=t?0!=t:e||null},e.option.asNumber=function(t,e){return"function"==typeof t&&(t=t()),null!=t?Number(t)||e||null:e||null},e.option.asString=function(t,e){return"function"==typeof t&&(t=t()),null!=t?String(t):e||null},e.option.asSize=function(t,i){return"function"==typeof t&&(t=t()),e.isString(t)?t:e.isNumber(t)?t+"px":i||null},e.option.asElement=function(t,e){return"function"==typeof t&&(t=t()),t||e||null},e.hexToRGB=function(t){var e=/^#?([a-f\d])([a-f\d])([a-f\d])$/i;t=t.replace(e,function(t,e,i,s){return e+e+i+i+s+s});var i=/^#?([a-f\d]{2})([a-f\d]{2})([a-f\d]{2})$/i.exec(t);return i?{r:parseInt(i[1],16),g:parseInt(i[2],16),b:parseInt(i[3],16)}:null},e.RGBToHex=function(t,e,i){return"#"+((1<<24)+(t<<16)+(e<<8)+i).toString(16).slice(1)},e.parseColor=function(t){var i;if(e.isString(t)){if(e.isValidRGB(t)){var s=t.substr(4).substr(0,t.length-5).split(",");t=e.RGBToHex(s[0],s[1],s[2])}if(e.isValidHex(t)){var o=e.hexToHSV(t),n={h:o.h,s:.45*o.s,v:Math.min(1,1.05*o.v)},r={h:o.h,s:Math.min(1,1.25*o.v),v:.6*o.v},a=e.HSVToHex(r.h,r.h,r.v),h=e.HSVToHex(n.h,n.s,n.v);i={background:t,border:a,highlight:{background:h,border:a},hover:{background:h,border:a}}}else i={background:t,border:t,highlight:{background:t,border:t},hover:{background:t,border:t}}}else i={},i.background=t.background||"white",i.border=t.border||i.background,e.isString(t.highlight)?i.highlight={border:t.highlight,background:t.highlight}:(i.highlight={},i.highlight.background=t.highlight&&t.highlight.background||i.background,i.highlight.border=t.highlight&&t.highlight.border||i.border),e.isString(t.hover)?i.hover={border:t.hover,background:t.hover}:(i.hover={},i.hover.background=t.hover&&t.hover.background||i.background,i.hover.border=t.hover&&t.hover.border||i.border);return i},e.RGBToHSV=function(t,e,i){t/=255,e/=255,i/=255;var s=Math.min(t,Math.min(e,i)),o=Math.max(t,Math.max(e,i));if(s==o)return{h:0,s:0,v:s};var n=t==s?e-i:i==s?t-e:i-t,r=t==s?3:i==s?1:5,a=60*(r-n/(o-s))/360,h=(o-s)/o,d=o;return{h:a,s:h,v:d}};var n={split:function(t){var e={};return t.split(";").forEach(function(t){if(""!=t.trim()){var i=t.split(":"),s=i[0].trim(),o=i[1].trim();e[s]=o}}),e},join:function(t){return Object.keys(t).map(function(e){return e+": "+t[e]}).join("; ")}};e.addCssText=function(t,i){var s=n.split(t.style.cssText),o=n.split(i),r=e.extend(s,o);t.style.cssText=n.join(r)},e.removeCssText=function(t,e){var i=n.split(t.style.cssText),s=n.split(e);for(var o in s)s.hasOwnProperty(o)&&delete i[o];t.style.cssText=n.join(i)},e.HSVToRGB=function(t,e,i){var s,o,n,r=Math.floor(6*t),a=6*t-r,h=i*(1-e),d=i*(1-a*e),l=i*(1-(1-a)*e);switch(r%6){case 0:s=i,o=l,n=h;break;case 1:s=d,o=i,n=h;break;case 2:s=h,o=i,n=l;break;case 3:s=h,o=d,n=i;break;case 4:s=l,o=h,n=i;break;case 5:s=i,o=h,n=d}return{r:Math.floor(255*s),g:Math.floor(255*o),b:Math.floor(255*n)}},e.HSVToHex=function(t,i,s){var o=e.HSVToRGB(t,i,s);return e.RGBToHex(o.r,o.g,o.b)},e.hexToHSV=function(t){var i=e.hexToRGB(t);return e.RGBToHSV(i.r,i.g,i.b)},e.isValidHex=function(t){var e=/(^#[0-9A-F]{6}$)|(^#[0-9A-F]{3}$)/i.test(t);return e},e.isValidRGB=function(t){t=t.replace(" ","");var e=/rgb\((\d{1,3}),(\d{1,3}),(\d{1,3})\)/i.test(t);return e},e.selectiveBridgeObject=function(t,i){if("object"==typeof i){for(var s=Object.create(i),o=0;o=r&&o>n;){var h=Math.floor((r+a)/2),d=t[h],l=void 0===s?d[i]:d[i][s],c=e(l);if(0==c)return h;-1==c?r=h+1:a=h-1,n++}return-1},e.binarySearchValue=function(t,e,i,s){for(var o,n,r,a,h=1e4,d=0,l=0,c=t.length-1;c>=l&&h>d;){if(a=Math.floor(.5*(c+l)),o=t[Math.max(0,a-1)][i],n=t[a][i],r=t[Math.min(t.length-1,a+1)][i],n==e)return a;if(e>o&&n>e)return"before"==s?Math.max(0,a-1):a;if(e>n&&r>e)return"before"==s?a:Math.min(t.length-1,a+1);e>n?l=a+1:c=a-1,d++}return-1},e.easeInOutQuad=function(t,e,i,s){var o=i-e;return t/=s/2,1>t?o/2*t*t+e:(t--,-o/2*(t*(t-2)-1)+e)},e.easingFunctions={linear:function(t){return t},easeInQuad:function(t){return t*t},easeOutQuad:function(t){return t*(2-t)},easeInOutQuad:function(t){return.5>t?2*t*t:-1+(4-2*t)*t},easeInCubic:function(t){return t*t*t},easeOutCubic:function(t){return--t*t*t+1},easeInOutCubic:function(t){return.5>t?4*t*t*t:(t-1)*(2*t-2)*(2*t-2)+1},easeInQuart:function(t){return t*t*t*t},easeOutQuart:function(t){return 1- --t*t*t*t},easeInOutQuart:function(t){return.5>t?8*t*t*t*t:1-8*--t*t*t*t},easeInQuint:function(t){return t*t*t*t*t},easeOutQuint:function(t){return 1+--t*t*t*t*t},easeInOutQuint:function(t){return.5>t?16*t*t*t*t*t:1+16*--t*t*t*t*t}}},function(t,e){e.prepareElements=function(t){for(var e in t)t.hasOwnProperty(e)&&(t[e].redundant=t[e].used,t[e].used=[])},e.cleanupElements=function(t){for(var e in t)if(t.hasOwnProperty(e)&&t[e].redundant){for(var i=0;i0?(s=e[t].redundant[0],e[t].redundant.shift()):(s=document.createElementNS("http://www.w3.org/2000/svg",t),i.appendChild(s)):(s=document.createElementNS("http://www.w3.org/2000/svg",t),e[t]={used:[],redundant:[]},i.appendChild(s)),e[t].used.push(s),s},e.getDOMElement=function(t,e,i,s){var o;return e.hasOwnProperty(t)?e[t].redundant.length>0?(o=e[t].redundant[0],e[t].redundant.shift()):(o=document.createElement(t),void 0!==s?i.insertBefore(o,s):i.appendChild(o)):(o=document.createElement(t),e[t]={used:[],redundant:[]},void 0!==s?i.insertBefore(o,s):i.appendChild(o)),e[t].used.push(o),o},e.drawPoint=function(t,i,s,o,n){var r;return"circle"==s.options.drawPoints.style?(r=e.getSVGElement("circle",o,n),r.setAttributeNS(null,"cx",t),r.setAttributeNS(null,"cy",i),r.setAttributeNS(null,"r",.5*s.options.drawPoints.size)):(r=e.getSVGElement("rect",o,n),r.setAttributeNS(null,"x",t-.5*s.options.drawPoints.size),r.setAttributeNS(null,"y",i-.5*s.options.drawPoints.size),r.setAttributeNS(null,"width",s.options.drawPoints.size),r.setAttributeNS(null,"height",s.options.drawPoints.size)),void 0!==s.options.drawPoints.styles&&r.setAttributeNS(null,"style",s.group.options.drawPoints.styles),r.setAttributeNS(null,"class",s.className+" point"),r},e.drawBar=function(t,i,s,o,n,r,a){if(0!=o){0>o&&(o*=-1,i-=o);var h=e.getSVGElement("rect",r,a);h.setAttributeNS(null,"x",t-.5*s),h.setAttributeNS(null,"y",i),h.setAttributeNS(null,"width",s),h.setAttributeNS(null,"height",o),h.setAttributeNS(null,"class",n)}}},function(t,e,i){function s(t,e){if(!t||Array.isArray(t)||o.isDataTable(t)||(e=t,t=null),this._options=e||{},this._data={},this._fieldId=this._options.fieldId||"id",this._type={},this._options.type)for(var i in this._options.type)if(this._options.type.hasOwnProperty(i)){var s=this._options.type[i];this._type[i]="Date"==s||"ISODate"==s||"ASPDate"==s?"Date":s}if(this._options.convert)throw new Error('Option "convert" is deprecated. Use "type" instead.');this._subscribers={},t&&this.add(t),this.setOptions(e)}var o=i(1),n=i(5);s.prototype.setOptions=function(t){t&&void 0!==t.queue&&(t.queue===!1?this._queue&&(this._queue.destroy(),delete this._queue):(this._queue||(this._queue=n.extend(this,{replace:["add","update","remove"]})),"object"==typeof t.queue&&this._queue.setOptions(t.queue)))},s.prototype.on=function(t,e){var i=this._subscribers[t];i||(i=[],this._subscribers[t]=i),i.push({callback:e})},s.prototype.subscribe=s.prototype.on,s.prototype.off=function(t,e){var i=this._subscribers[t];i&&(this._subscribers[t]=i.filter(function(t){return t.callback!=e}))},s.prototype.unsubscribe=s.prototype.off,s.prototype._trigger=function(t,e,i){if("*"==t)throw new Error("Cannot trigger event *");var s=[];t in this._subscribers&&(s=s.concat(this._subscribers[t])),"*"in this._subscribers&&(s=s.concat(this._subscribers["*"]));for(var o=0;or;r++)i=n._addItem(t[r]),s.push(i);else if(o.isDataTable(t))for(var h=this._getColumnNames(t),d=0,l=t.getNumberOfRows();l>d;d++){for(var c={},p=0,u=h.length;u>p;p++){var m=h[p];c[m]=t.getValue(d,p)}i=n._addItem(c),s.push(i)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");i=n._addItem(t),s.push(i)}return s.length&&this._trigger("add",{items:s},e),s},s.prototype.update=function(t,e){var i=[],s=[],n=[],r=this,a=r._fieldId,h=function(t){var e=t[a];r._data[e]?(e=r._updateItem(t),s.push(e),n.push(t)):(e=r._addItem(t),i.push(e))};if(Array.isArray(t))for(var d=0,l=t.length;l>d;d++)h(t[d]);else if(o.isDataTable(t))for(var c=this._getColumnNames(t),p=0,u=t.getNumberOfRows();u>p;p++){for(var m={},f=0,g=c.length;g>f;f++){var v=c[f];m[v]=t.getValue(p,f)}h(m)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");h(t)}return i.length&&this._trigger("add",{items:i},e),s.length&&this._trigger("update",{items:s,data:n},e),i.concat(s)},s.prototype.get=function(){var t,e,i,s,n=this,r=o.getType(arguments[0]);"String"==r||"Number"==r?(t=arguments[0],i=arguments[1],s=arguments[2]):"Array"==r?(e=arguments[0],i=arguments[1],s=arguments[2]):(i=arguments[0],s=arguments[1]);var a;if(i&&i.returnType){var h=["DataTable","Array","Object"];if(a=-1==h.indexOf(i.returnType)?"Array":i.returnType,s&&a!=o.getType(s))throw new Error('Type of parameter "data" ('+o.getType(s)+") does not correspond with specified options.type ("+i.type+")");if("DataTable"==a&&!o.isDataTable(s))throw new Error('Parameter "data" must be a DataTable when options.type is "DataTable"')}else a=s&&"DataTable"==o.getType(s)?"DataTable":"Array";var d,l,c,p,u=i&&i.type||this._options.type,m=i&&i.filter,f=[];if(void 0!=t)d=n._getItem(t,u),m&&!m(d)&&(d=null);else if(void 0!=e)for(c=0,p=e.length;p>c;c++)d=n._getItem(e[c],u),(!m||m(d))&&f.push(d);else for(l in this._data)this._data.hasOwnProperty(l)&&(d=n._getItem(l,u),(!m||m(d))&&f.push(d));if(i&&i.order&&void 0==t&&this._sort(f,i.order),i&&i.fields){var g=i.fields;if(void 0!=t)d=this._filterFields(d,g);else for(c=0,p=f.length;p>c;c++)f[c]=this._filterFields(f[c],g)}if("DataTable"==a){var v=this._getColumnNames(s);if(void 0!=t)n._appendRow(s,v,d);else for(c=0;cc;c++)s.push(f[c]);return s}return f},s.prototype.getIds=function(t){var e,i,s,o,n,r=this._data,a=t&&t.filter,h=t&&t.order,d=t&&t.type||this._options.type,l=[];if(a)if(h){n=[];for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&n.push(o));for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&l.push(o[this._fieldId]));else if(h){n=[];for(s in r)r.hasOwnProperty(s)&&n.push(r[s]);for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=r[s],l.push(o[this._fieldId]));return l},s.prototype.getDataSet=function(){return this},s.prototype.forEach=function(t,e){var i,s,o=e&&e.filter,n=e&&e.type||this._options.type,r=this._data;if(e&&e.order)for(var a=this.get(e),h=0,d=a.length;d>h;h++)i=a[h],s=i[this._fieldId],t(i,s);else for(s in r)r.hasOwnProperty(s)&&(i=this._getItem(s,n),(!o||o(i))&&t(i,s))},s.prototype.map=function(t,e){var i,s=e&&e.filter,o=e&&e.type||this._options.type,n=[],r=this._data;for(var a in r)r.hasOwnProperty(a)&&(i=this._getItem(a,o),(!s||s(i))&&n.push(t(i,a)));return e&&e.order&&this._sort(n,e.order),n},s.prototype._filterFields=function(t,e){var i={};for(var s in t)t.hasOwnProperty(s)&&-1!=e.indexOf(s)&&(i[s]=t[s]);return i},s.prototype._sort=function(t,e){if(o.isString(e)){var i=e;t.sort(function(t,e){var s=t[i],o=e[i];return s>o?1:o>s?-1:0})}else{if("function"!=typeof e)throw new TypeError("Order must be a function or a string");t.sort(e)}},s.prototype.remove=function(t,e){var i,s,o,n=[];if(Array.isArray(t))for(i=0,s=t.length;s>i;i++)o=this._remove(t[i]),null!=o&&n.push(o);else o=this._remove(t),null!=o&&n.push(o);return n.length&&this._trigger("remove",{items:n},e),n},s.prototype._remove=function(t){if(o.isNumber(t)||o.isString(t)){if(this._data[t])return delete this._data[t],t}else if(t instanceof Object){var e=t[this._fieldId];if(e&&this._data[e])return delete this._data[e],e}return null},s.prototype.clear=function(t){var e=Object.keys(this._data);return this._data={},this._trigger("remove",{items:e},t),e},s.prototype.max=function(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||r>s)&&(i=n,s=r)}return i},s.prototype.min=function(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||s>r)&&(i=n,s=r)}return i},s.prototype.distinct=function(t){var e,i=this._data,s=[],n=this._options.type&&this._options.type[t]||null,r=0;for(var a in i)if(i.hasOwnProperty(a)){var h=i[a],d=h[t],l=!1;for(e=0;r>e;e++)if(s[e]==d){l=!0;break}l||void 0===d||(s[r]=d,r++)}if(n)for(e=0;ei;i++)e[i]=t.getColumnId(i)||t.getColumnLabel(i);return e},s.prototype._appendRow=function(t,e,i){for(var s=t.addRow(),o=0,n=e.length;n>o;o++){var r=e[o];t.setValue(s,o,i[r])}},t.exports=s},function(t,e,i){function s(t,e){this._data=null,this._ids={},this._options=e||{},this._fieldId="id",this._subscribers={};var i=this;this.listener=function(){i._onEvent.apply(i,arguments)},this.setData(t)}var o=i(1),n=i(3);s.prototype.setData=function(t){var e,i,s;if(this._data){this._data.unsubscribe&&this._data.unsubscribe("*",this.listener),e=[];for(var o in this._ids)this._ids.hasOwnProperty(o)&&e.push(o);this._ids={},this._trigger("remove",{items:e})}if(this._data=t,this._data){for(this._fieldId=this._options.fieldId||this._data&&this._data.options&&this._data.options.fieldId||"id",e=this._data.getIds({filter:this._options&&this._options.filter}),i=0,s=e.length;s>i;i++)o=e[i],this._ids[o]=!0;this._trigger("add",{items:e}),this._data.on&&this._data.on("*",this.listener)}},s.prototype.get=function(){var t,e,i,s=this,n=o.getType(arguments[0]);"String"==n||"Number"==n||"Array"==n?(t=arguments[0],e=arguments[1],i=arguments[2]):(e=arguments[0],i=arguments[1]);var r=o.extend({},this._options,e);this._options.filter&&e&&e.filter&&(r.filter=function(t){return s._options.filter(t)&&e.filter(t)});var a=[];return void 0!=t&&a.push(t),a.push(r),a.push(i),this._data&&this._data.get.apply(this._data,a)},s.prototype.getIds=function(t){var e;if(this._data){var i,s=this._options.filter;i=t&&t.filter?s?function(e){return s(e)&&t.filter(e)}:t.filter:s,e=this._data.getIds({filter:i,order:t&&t.order})}else e=[];return e},s.prototype.getDataSet=function(){for(var t=this;t instanceof s;)t=t._data;return t||null},s.prototype._onEvent=function(t,e,i){var s,o,n,r,a=e&&e.items,h=this._data,d=[],l=[],c=[];if(a&&h){switch(t){case"add":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r&&(this._ids[n]=!0,d.push(n));break;case"update":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r?this._ids[n]?l.push(n):(this._ids[n]=!0,d.push(n)):this._ids[n]&&(delete this._ids[n],c.push(n));break;case"remove":for(s=0,o=a.length;o>s;s++)n=a[s],this._ids[n]&&(delete this._ids[n],c.push(n))}d.length&&this._trigger("add",{items:d},i),l.length&&this._trigger("update",{items:l},i),c.length&&this._trigger("remove",{items:c},i)}},s.prototype.on=n.prototype.on,s.prototype.off=n.prototype.off,s.prototype._trigger=n.prototype._trigger,s.prototype.subscribe=s.prototype.on,s.prototype.unsubscribe=s.prototype.off,t.exports=s},function(t){function e(t){this.delay=null,this.max=1/0,this._queue=[],this._timeout=null,this._extended=null,this.setOptions(t)}e.prototype.setOptions=function(t){t&&"undefined"!=typeof t.delay&&(this.delay=t.delay),t&&"undefined"!=typeof t.max&&(this.max=t.max),this._flushIfNeeded()},e.extend=function(t,i){var s=new e(i);if(void 0!==t.flush)throw new Error("Target object already has a property flush");t.flush=function(){s.flush()};var o=[{name:"flush",original:void 0}];if(i&&i.replace)for(var n=0;nthis.max&&this.flush(),clearTimeout(this._timeout),this.queue.length>0&&"number"==typeof this.delay){var t=this;this._timeout=setTimeout(function(){t.flush()},this.delay)}},e.prototype.flush=function(){for(;this._queue.length>0;){var t=this._queue.shift();t.fn.apply(t.context||t.fn,t.args||[])}},t.exports=e},function(t,e,i){function s(t,e,i){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");this.containerElement=t,this.width="400px",this.height="400px",this.margin=10,this.defaultXCenter="55%",this.defaultYCenter="50%",this.xLabel="x",this.yLabel="y",this.zLabel="z";var o=function(t){return t};this.xValueLabel=o,this.yValueLabel=o,this.zValueLabel=o,this.filterLabel="time",this.legendLabel="value",this.style=s.STYLE.DOT,this.showPerspective=!0,this.showGrid=!0,this.keepAspectRatio=!0,this.showShadow=!1,this.showGrayBottom=!1,this.showTooltip=!1,this.verticalRatio=.5,this.animationInterval=1e3,this.animationPreload=!1,this.camera=new p,this.eye=new l(0,0,-1),this.dataTable=null,this.dataPoints=null,this.colX=void 0,this.colY=void 0,this.colZ=void 0,this.colValue=void 0,this.colFilter=void 0,this.xMin=0,this.xStep=void 0,this.xMax=1,this.yMin=0,this.yStep=void 0,this.yMax=1,this.zMin=0,this.zStep=void 0,this.zMax=1,this.valueMin=0,this.valueMax=1,this.xBarWidth=1,this.yBarWidth=1,this.colorAxis="#4D4D4D",this.colorGrid="#D3D3D3",this.colorDot="#7DC1FF",this.colorDotBorder="#3267D2",this.create(),this.setOptions(i),e&&this.setData(e)}function o(t){return"clientX"in t?t.clientX:t.targetTouches[0]&&t.targetTouches[0].clientX||0}function n(t){return"clientY"in t?t.clientY:t.targetTouches[0]&&t.targetTouches[0].clientY||0}var r=i(56),a=i(3),h=i(4),d=i(1),l=i(10),c=i(9),p=i(7),u=i(8),m=i(11),f=i(12);r(s.prototype),s.prototype._setScale=function(){this.scale=new l(1/(this.xMax-this.xMin),1/(this.yMax-this.yMin),1/(this.zMax-this.zMin)),this.keepAspectRatio&&(this.scale.x3&&(this.colFilter=3);else{if(this.style!==s.STYLE.DOTCOLOR&&this.style!==s.STYLE.DOTSIZE&&this.style!==s.STYLE.BARCOLOR&&this.style!==s.STYLE.BARSIZE)throw'Unknown style "'+this.style+'"';this.colX=0,this.colY=1,this.colZ=2,this.colValue=3,t.getNumberOfColumns()>4&&(this.colFilter=4)}},s.prototype.getNumberOfRows=function(t){return t.length},s.prototype.getNumberOfColumns=function(t){var e=0;for(var i in t[0])t[0].hasOwnProperty(i)&&e++;return e},s.prototype.getDistinctValues=function(t,e){for(var i=[],s=0;st[s][e]&&(i.min=t[s][e]),i.maxt;t++){var m=(t-p)/(u-p),g=240*m,v=this._hsv2rgb(g,1,1);c.strokeStyle=v,c.beginPath(),c.moveTo(h,r+t),c.lineTo(a,r+t),c.stroke()}c.strokeStyle=this.colorAxis,c.strokeRect(h,r,i,n)}if(this.style===s.STYLE.DOTSIZE&&(c.strokeStyle=this.colorAxis,c.fillStyle=this.colorDot,c.beginPath(),c.moveTo(h,r),c.lineTo(a,r),c.lineTo(a-i+e,d),c.lineTo(h,d),c.closePath(),c.fill(),c.stroke()),this.style===s.STYLE.DOTCOLOR||this.style===s.STYLE.DOTSIZE){var y=5,b=new f(this.valueMin,this.valueMax,(this.valueMax-this.valueMin)/5,!0);for(b.start(),b.getCurrent()0?this.yMin:this.yMax,o=this._convert3Dto2D(new l(x,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(" "+this.xValueLabel(i.getCurrent())+" ",o.x,o.y),i.next()}for(g.lineWidth=1,s=void 0===this.defaultYStep,i=new f(this.yMin,this.yMax,this.yStep,s),i.start(),i.getCurrent()0?this.xMin:this.xMax,o=this._convert3Dto2D(new l(n,i.getCurrent(),this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(" "+this.yValueLabel(i.getCurrent())+" ",o.x,o.y),i.next();for(g.lineWidth=1,s=void 0===this.defaultZStep,i=new f(this.zMin,this.zMax,this.zStep,s),i.start(),i.getCurrent()0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax;!i.end();)t=this._convert3Dto2D(new l(n,r,i.getCurrent())),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(t.x-b,t.y),g.stroke(),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(this.zValueLabel(i.getCurrent())+" ",t.x-5,t.y),i.next();g.lineWidth=1,t=this._convert3Dto2D(new l(n,r,this.zMin)),e=this._convert3Dto2D(new l(n,r,this.zMax)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),g.lineWidth=1,p=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),p=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),g.lineWidth=1,t=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),t=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke();var w=this.xLabel;w.length>0&&(c=.1/this.scale.y,n=(this.xMin+this.xMax)/2,r=Math.cos(_)>0?this.yMin-c:this.yMax+c,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(w,o.x,o.y));var S=this.yLabel;S.length>0&&(d=.1/this.scale.x,n=Math.sin(_)>0?this.xMin-d:this.xMax+d,r=(this.yMin+this.yMax)/2,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(S,o.x,o.y));var M=this.zLabel;M.length>0&&(h=30,n=Math.cos(_)>0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax,a=(this.zMin+this.zMax)/2,o=this._convert3Dto2D(new l(n,r,a)),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(M,o.x-h,o.y))},s.prototype._hsv2rgb=function(t,e,i){var s,o,n,r,a,h;switch(r=i*e,a=Math.floor(t/60),h=r*(1-Math.abs(t/60%2-1)),a){case 0:s=r,o=h,n=0;break;case 1:s=h,o=r,n=0;break;case 2:s=0,o=r,n=h;break;case 3:s=0,o=h,n=r;break;case 4:s=h,o=0,n=r;break;case 5:s=r,o=0,n=h;break;default:s=0,o=0,n=0}return"RGB("+parseInt(255*s)+","+parseInt(255*o)+","+parseInt(255*n)+")"},s.prototype._redrawDataGrid=function(){var t,e,i,o,n,r,a,h,d,c,p,u,m,f=this.frame.canvas,g=f.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(n=0;n0}else r=!0;r?(m=(t.point.z+e.point.z+i.point.z+o.point.z)/4,c=240*(1-(m-this.zMin)*this.scale.z/this.verticalRatio),p=1,this.showShadow?(u=Math.min(1+S.x/M/2,1),a=this._hsv2rgb(c,p,u),h=a):(u=1,a=this._hsv2rgb(c,p,u),h=this.colorAxis)):(a="gray",h=this.colorAxis),d=.5,g.lineWidth=d,g.fillStyle=a,g.strokeStyle=h,g.beginPath(),g.moveTo(t.screen.x,t.screen.y),g.lineTo(e.screen.x,e.screen.y),g.lineTo(o.screen.x,o.screen.y),g.lineTo(i.screen.x,i.screen.y),g.closePath(),g.fill(),g.stroke()}}else for(n=0;np&&(p=0);var u,m,f;this.style===s.STYLE.DOTCOLOR?(u=240*(1-(d.point.value-this.valueMin)*this.scale.value),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)):this.style===s.STYLE.DOTSIZE?(m=this.colorDot,f=this.colorDotBorder):(u=240*(1-(d.point.z-this.zMin)*this.scale.z/this.verticalRatio),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)),i.lineWidth=1,i.strokeStyle=f,i.fillStyle=m,i.beginPath(),i.arc(d.screen.x,d.screen.y,p,0,2*Math.PI,!0),i.fill(),i.stroke()}}},s.prototype._redrawDataBar=function(){var t,e,i,o,n=this.frame.canvas,r=n.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t0&&(t=this.dataPoints[0],s.lineWidth=1,s.strokeStyle="blue",s.beginPath(),s.moveTo(t.screen.x,t.screen.y)),e=1;e0&&s.stroke()}},s.prototype._onMouseDown=function(t){if(t=t||window.event,this.leftButtonDown&&this._onMouseUp(t),this.leftButtonDown=t.which?1===t.which:1===t.button,this.leftButtonDown||this.touchDown){this.startMouseX=o(t),this.startMouseY=n(t),this.startStart=new Date(this.start),this.startEnd=new Date(this.end),this.startArmRotation=this.camera.getArmRotation(),this.frame.style.cursor="move";var e=this;this.onmousemove=function(t){e._onMouseMove(t)},this.onmouseup=function(t){e._onMouseUp(t)},d.addEventListener(document,"mousemove",e.onmousemove),d.addEventListener(document,"mouseup",e.onmouseup),d.preventDefault(t)}},s.prototype._onMouseMove=function(t){t=t||window.event;var e=parseFloat(o(t))-this.startMouseX,i=parseFloat(n(t))-this.startMouseY,s=this.startArmRotation.horizontal+e/200,r=this.startArmRotation.vertical+i/200,a=4,h=Math.sin(a/360*2*Math.PI);Math.abs(Math.sin(s))0?1:0>t?-1:0}var s=e[0],o=e[1],n=e[2],r=i((o.x-s.x)*(t.y-s.y)-(o.y-s.y)*(t.x-s.x)),a=i((n.x-o.x)*(t.y-o.y)-(n.y-o.y)*(t.x-o.x)),h=i((s.x-n.x)*(t.y-n.y)-(s.y-n.y)*(t.x-n.x));return!(0!=r&&0!=a&&r!=a||0!=a&&0!=h&&a!=h||0!=r&&0!=h&&r!=h)},s.prototype._dataPointFromXY=function(t,e){var i,o=100,n=null,r=null,a=null,h=new c(t,e);if(this.style===s.STYLE.BAR||this.style===s.STYLE.BARCOLOR||this.style===s.STYLE.BARSIZE)for(i=this.dataPoints.length-1;i>=0;i--){n=this.dataPoints[i];var d=n.surfaces;if(d)for(var l=d.length-1;l>=0;l--){var p=d[l],u=p.corners,m=[u[0].screen,u[1].screen,u[2].screen],f=[u[2].screen,u[3].screen,u[0].screen];if(this._insideTriangle(h,m)||this._insideTriangle(h,f))return n}}else for(i=0;ib)&&o>b&&(a=b,r=n)}}return r},s.prototype._showTooltip=function(t){var e,i,s;this.tooltip?(e=this.tooltip.dom.content,i=this.tooltip.dom.line,s=this.tooltip.dom.dot):(e=document.createElement("div"),e.style.position="absolute",e.style.padding="10px",e.style.border="1px solid #4d4d4d",e.style.color="#1a1a1a",e.style.background="rgba(255,255,255,0.7)",e.style.borderRadius="2px",e.style.boxShadow="5px 5px 10px rgba(128,128,128,0.5)",i=document.createElement("div"),i.style.position="absolute",i.style.height="40px",i.style.width="0",i.style.borderLeft="1px solid #4d4d4d",s=document.createElement("div"),s.style.position="absolute",s.style.height="0",s.style.width="0",s.style.border="5px solid #4d4d4d",s.style.borderRadius="5px",this.tooltip={dataPoint:null,dom:{content:e,line:i,dot:s}}),this._hideTooltip(),this.tooltip.dataPoint=t,e.innerHTML="function"==typeof this.showTooltip?this.showTooltip(t.point):"
    x:"+t.point.x+"
    y:"+t.point.y+"
    z:"+t.point.z+"
    ",e.style.left="0",e.style.top="0",this.frame.appendChild(e),this.frame.appendChild(i),this.frame.appendChild(s);var o=e.offsetWidth,n=e.offsetHeight,r=i.offsetHeight,a=s.offsetWidth,h=s.offsetHeight,d=t.screen.x-o/2;d=Math.min(Math.max(d,10),this.frame.clientWidth-10-o),i.style.left=t.screen.x+"px",i.style.top=t.screen.y-r+"px",e.style.left=d+"px",e.style.top=t.screen.y-r-n+"px",s.style.left=t.screen.x-a/2+"px",s.style.top=t.screen.y-h/2+"px"},s.prototype._hideTooltip=function(){if(this.tooltip){this.tooltip.dataPoint=null;for(var t in this.tooltip.dom)if(this.tooltip.dom.hasOwnProperty(t)){var e=this.tooltip.dom[t];e&&e.parentNode&&e.parentNode.removeChild(e)}}},t.exports=s},function(t,e,i){function s(){this.armLocation=new o,this.armRotation={},this.armRotation.horizontal=0,this.armRotation.vertical=0,this.armLength=1.7,this.cameraLocation=new o,this.cameraRotation=new o(.5*Math.PI,0,0),this.calculateCameraOrientation()}var o=i(10);s.prototype.setArmLocation=function(t,e,i){this.armLocation.x=t,this.armLocation.y=e,this.armLocation.z=i,this.calculateCameraOrientation()},s.prototype.setArmRotation=function(t,e){void 0!==t&&(this.armRotation.horizontal=t),void 0!==e&&(this.armRotation.vertical=e,this.armRotation.vertical<0&&(this.armRotation.vertical=0),this.armRotation.vertical>.5*Math.PI&&(this.armRotation.vertical=.5*Math.PI)),(void 0!==t||void 0!==e)&&this.calculateCameraOrientation()},s.prototype.getArmRotation=function(){var t={};return t.horizontal=this.armRotation.horizontal,t.vertical=this.armRotation.vertical,t},s.prototype.setArmLength=function(t){void 0!==t&&(this.armLength=t,this.armLength<.71&&(this.armLength=.71),this.armLength>5&&(this.armLength=5),this.calculateCameraOrientation())},s.prototype.getArmLength=function(){return this.armLength},s.prototype.getCameraLocation=function(){return this.cameraLocation},s.prototype.getCameraRotation=function(){return this.cameraRotation},s.prototype.calculateCameraOrientation=function(){this.cameraLocation.x=this.armLocation.x-this.armLength*Math.sin(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.y=this.armLocation.y-this.armLength*Math.cos(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.z=this.armLocation.z+this.armLength*Math.sin(this.armRotation.vertical),this.cameraRotation.x=Math.PI/2-this.armRotation.vertical,this.cameraRotation.y=0,this.cameraRotation.z=-this.armRotation.horizontal},t.exports=s},function(t,e,i){function s(t,e,i){this.data=t,this.column=e,this.graph=i,this.index=void 0,this.value=void 0,this.values=i.getDistinctValues(t.get(),this.column),this.values.sort(function(t,e){return t>e?1:e>t?-1:0}),this.values.length>0&&this.selectValue(0),this.dataPoints=[],this.loaded=!1,this.onLoadCallback=void 0,i.animationPreload?(this.loaded=!1,this.loadInBackground()):this.loaded=!0}var o=i(4);s.prototype.isLoaded=function(){return this.loaded},s.prototype.getLoadedProgress=function(){for(var t=this.values.length,e=0;this.dataPoints[e];)e++;return Math.round(e/t*100)},s.prototype.getLabel=function(){return this.graph.filterLabel},s.prototype.getColumn=function(){return this.column},s.prototype.getSelectedValue=function(){return void 0===this.index?void 0:this.values[this.index]},s.prototype.getValues=function(){return this.values},s.prototype.getValue=function(t){if(t>=this.values.length)throw"Error: index out of range";return this.values[t]},s.prototype._getDataPoints=function(t){if(void 0===t&&(t=this.index),void 0===t)return[];var e;if(this.dataPoints[t])e=this.dataPoints[t];else{var i={};i.column=this.column,i.value=this.values[t];var s=new o(this.data,{filter:function(t){return t[i.column]==i.value}}).get();e=this.graph._getDataPoints(s),this.dataPoints[t]=e}return e},s.prototype.setOnLoadCallback=function(t){this.onLoadCallback=t},s.prototype.selectValue=function(t){if(t>=this.values.length)throw"Error: index out of range";this.index=t,this.value=this.values[t]},s.prototype.loadInBackground=function(t){void 0===t&&(t=0);var e=this.graph.frame;if(t0&&(t--,this.setIndex(t))},s.prototype.next=function(){var t=this.getIndex();t0?this.setIndex(0):this.index=void 0},s.prototype.setIndex=function(t){if(!(ts&&(s=0),s>this.values.length-1&&(s=this.values.length-1),s},s.prototype.indexToLeft=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t/(this.values.length-1)*e,s=i+3;return s},s.prototype._onMouseMove=function(t){var e=t.clientX-this.startClientX,i=this.startSlideX+e,s=this.leftToIndex(i);this.setIndex(s),o.preventDefault()},s.prototype._onMouseUp=function(){this.frame.style.cursor="auto",o.removeEventListener(document,"mousemove",this.onmousemove),o.removeEventListener(document,"mouseup",this.onmouseup),o.preventDefault()},t.exports=s},function(t){function e(t,e,i,s){this._start=0,this._end=0,this._step=1,this.prettyStep=!0,this.precision=5,this._current=0,this.setRange(t,e,i,s)}e.prototype.setRange=function(t,e,i,s){this._start=t?t:0,this._end=e?e:0,this.setStep(i,s)},e.prototype.setStep=function(t,i){void 0===t||0>=t||(void 0!==i&&(this.prettyStep=i),this._step=this.prettyStep===!0?e.calculatePrettyStep(t):t)},e.calculatePrettyStep=function(t){var e=function(t){return Math.log(t)/Math.LN10},i=Math.pow(10,Math.round(e(t))),s=2*Math.pow(10,Math.round(e(t/2))),o=5*Math.pow(10,Math.round(e(t/5))),n=i;return Math.abs(s-t)<=Math.abs(n-t)&&(n=s),Math.abs(o-t)<=Math.abs(n-t)&&(n=o),0>=n&&(n=1),n},e.prototype.getCurrent=function(){return parseFloat(this._current.toPrecision(this.precision))},e.prototype.getStep=function(){return this._step},e.prototype.start=function(){this._current=this._start-this._start%this._step},e.prototype.next=function(){this._current+=this._step},e.prototype.end=function(){return this._current>this._end},t.exports=e},function(t,e,i){function s(t,e,i,r){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var h=r;r=i,i=h}var u=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:u._toScreen.bind(u),toGlobalScreen:u._toGlobalScreen.bind(u),toTime:u._toTime.bind(u),toGlobalTime:u._toGlobalTime.bind(u)}},this.range=new a(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.itemSet=new p(this.body),this.components.push(this.itemSet),this.itemsData=null,this.groupsData=null,r&&this.setOptions(r),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(27);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.itemSet&&this.itemSet.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){if(void 0==this.options.start||void 0==this.options.end)var s=this._getDataRange();var o=void 0!=this.options.start?this.options.start:s.start,a=void 0!=this.options.end?this.options.end:s.end;this.setWindow(o,a,{animate:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.itemSet.setGroups(e)},s.prototype.setSelection=function(t,e){this.itemSet&&this.itemSet.setSelection(t),e&&e.focus&&this.focus(t,e)},s.prototype.getSelection=function(){return this.itemSet&&this.itemSet.getSelection()||[]},s.prototype.focus=function(t,e){if(this.itemsData&&void 0!=t){var i=Array.isArray(t)?t:[t],s=this.itemsData.getDataSet().get(i,{type:{start:"Date",end:"Date"}}),o=null,n=null;if(s.forEach(function(t){var e=t.start.valueOf(),i="end"in t?t.end.valueOf():t.start.valueOf();(null===o||o>e)&&(o=e),(null===n||i>n)&&(n=i)}),null!==o&&null!==n){var r=(o+n)/2,a=Math.max(this.range.end-this.range.start,1.1*(n-o)),h=e&&void 0!==e.animate?e.animate:!0;this.range.setRange(r-a/2,r+a/2,h)}}},s.prototype.getItemRange=function(){var t=this.itemsData.getDataSet(),e=null,i=null;if(t){var s=t.min("start");e=s?o.convert(s.start,"Date").valueOf():null;var n=t.max("start");n&&(i=o.convert(n.start,"Date").valueOf());var r=t.max("end");r&&(i=null==i?o.convert(r.end,"Date").valueOf():Math.max(i,o.convert(r.end,"Date").valueOf()))}return{min:null!=e?new Date(e):null,max:null!=i?new Date(i):null}},t.exports=s},function(t,e,i){function s(t,e,i,s){if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var r=s;s=i,i=r}var h=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:h._toScreen.bind(h),toGlobalScreen:h._toGlobalScreen.bind(h),toTime:h._toTime.bind(h),toGlobalTime:h._toGlobalTime.bind(h)}},this.range=new a(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.linegraph=new p(this.body),this.components.push(this.linegraph),this.itemsData=null,this.groupsData=null,s&&this.setOptions(s),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(29);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.linegraph&&this.linegraph.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){var s=void 0!=this.options.start?this.options.start:null,o=void 0!=this.options.end?this.options.end:null;this.setWindow(s,o,{animate:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.linegraph.setGroups(e)},s.prototype.getLegend=function(t,e,i){return void 0===e&&(e=15),void 0===i&&(i=15),void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].getLegend(e,i):"cannot find group:"+t},s.prototype.isGroupVisible=function(t){return void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].visible&&(void 0===this.linegraph.options.groups.visibility[t]||1==this.linegraph.options.groups.visibility[t]):!1},s.prototype.getItemRange=function(){var t=null,e=null;for(var i in this.linegraph.groups)if(this.linegraph.groups.hasOwnProperty(i)&&1==this.linegraph.groups[i].visible)for(var s=0;sr?r:t,e=null==e?r:r>e?r:e}return{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},t.exports=s},function(t,e,i){var s=i(44);e.convertHiddenOptions=function(t,e){if(t.hiddenDates=[],e&&1==Array.isArray(e)){for(var i=0;i=4*a){var p=0,u=n.clone();switch(i[h].repeat){case"daily":d.day()!=l.day()&&(p=1),d.dayOfYear(o.dayOfYear()),d.year(o.year()),d.subtract(7,"days"),l.dayOfYear(o.dayOfYear()),l.year(o.year()),l.subtract(7-p,"days"),u.add(1,"weeks");break;case"weekly":var m=l.diff(d,"days"),f=d.day();d.date(o.date()),d.month(o.month()),d.year(o.year()),l=d.clone(),d.day(f),l.day(f),l.add(m,"days"),d.subtract(1,"weeks"),l.subtract(1,"weeks"),u.add(1,"weeks");break;case"monthly":d.month()!=l.month()&&(p=1),d.month(o.month()),d.year(o.year()),d.subtract(1,"months"),l.month(o.month()),l.year(o.year()),l.subtract(1,"months"),l.add(p,"months"),u.add(1,"months");break;case"yearly":d.year()!=l.year()&&(p=1),d.year(o.year()),d.subtract(1,"years"),l.year(o.year()),l.subtract(1,"years"),l.add(p,"years"),u.add(1,"years");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}for(;u>d;)switch(t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()}),i[h].repeat){case"daily":d.add(1,"days"),l.add(1,"days");break;case"weekly":d.add(1,"weeks"),l.add(1,"weeks");break;case"monthly":d.add(1,"months"),l.add(1,"months");break;case"yearly":d.add(1,"y"),l.add(1,"y");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()})}}e.removeDuplicates(t);var g=e.isHidden(t.range.start,t.hiddenDates),v=e.isHidden(t.range.end,t.hiddenDates),y=t.range.start,b=t.range.end;1==g.hidden&&(y=1==t.range.startToFront?g.startDate-1:g.endDate+1),1==v.hidden&&(b=1==t.range.endToFront?v.startDate-1:v.endDate+1),(1==g.hidden||1==v.hidden)&&t.range._applyRange(y,b)}},e.removeDuplicates=function(t){for(var e=t.hiddenDates,i=[],s=0;s=e[s].start&&e[o].end<=e[s].end?e[o].remove=!0:e[o].start>=e[s].start&&e[o].start<=e[s].end?(e[s].end=e[o].end,e[o].remove=!0):e[o].end>=e[s].start&&e[o].end<=e[s].end&&(e[s].start=e[o].start,e[o].remove=!0));for(var s=0;s=r&&a>o){i=!0;break}}if(1==i&&o=e&&i>r&&(s+=r-n)}return s},e.correctTimeForHidden=function(t,i,o){return o=s(o).toDate().valueOf(),o-=e.getHiddenDurationBefore(t,i,o)},e.getHiddenDurationBefore=function(t,e,i){var o=0;i=s(i).toDate().valueOf();for(var n=0;n=e.start&&a=a&&(o+=a-r)}return o},e.getAccumulatedHiddenDuration=function(t,e,i){for(var s=0,o=0,n=e.start,r=0;r=e.start&&h=i)break;s+=h-a}}return s},e.snapAwayFromHidden=function(t,i,s,o){var n=e.isHidden(i,t);return 1==n.hidden?0>s?1==o?n.startDate-(n.endDate-i)-1:n.startDate-1:1==o?n.endDate+(i-n.startDate)+1:n.endDate+1:i},e.isHidden=function(t,e){for(var i=0;i=s&&o>t)return{hidden:!0,startDate:s,endDate:o}}return{hidden:!1,startDate:s,endDate:o}}},function(t){function e(t,e,i,s,o,n){this.current=0,this.autoScale=!0,this.stepIndex=0,this.step=1,this.scale=1,this.marginStart,this.marginEnd,this.deadSpace=0,this.majorSteps=[1,2,5,10],this.minorSteps=[.25,.5,1,2],this.alignZeros=n,this.setRange(t,e,i,s,o)}e.prototype.setRange=function(t,e,i,s,o){this._start=void 0===o.min?t:o.min,this._end=void 0===o.max?e:o.max,this._start==this._end&&(this._start-=.75,this._end+=1),1==this.autoScale&&this.setMinimumStep(i,s),this.setFirst(o)},e.prototype.setMinimumStep=function(t,e){var i=this._end-this._start,s=1.2*i,o=t*(s/e),n=Math.round(Math.log(s)/Math.LN10),r=-1,a=Math.pow(10,n),h=0;0>n&&(h=n);for(var d=!1,l=h;Math.abs(l)<=Math.abs(n);l++){a=Math.pow(10,l);for(var c=0;c=o){d=!0,r=c;break}}if(1==d)break}this.stepIndex=r,this.scale=a,this.step=a*this.minorSteps[r]},e.prototype.setFirst=function(t){void 0===t&&(t={});var e=void 0===t.min?this._start-2*this.scale*this.minorSteps[this.stepIndex]:t.min,i=void 0===t.max?this._end+this.scale*this.minorSteps[this.stepIndex]:t.max;this.marginEnd=void 0===t.max?this.roundToMinor(i):t.max,this.marginStart=void 0===t.min?this.roundToMinor(e):t.min,1==this.alignZeros&&(this.marginEnd-this.marginStart)%this.step!=0&&(this.marginEnd+=this.marginEnd%this.step),this.deadSpace=this.roundToMinor(i)-i+this.roundToMinor(e)-e,this.marginRange=this.marginEnd-this.marginStart,this.current=this.marginEnd},e.prototype.roundToMinor=function(t){var e=t-t%(this.scale*this.minorSteps[this.stepIndex]);return t%(this.scale*this.minorSteps[this.stepIndex])>.5*this.scale*this.minorSteps[this.stepIndex]?e+this.scale*this.minorSteps[this.stepIndex]:e},e.prototype.hasNext=function(){return this.current>=this.marginStart},e.prototype.next=function(){var t=this.current;this.current-=this.step,this.current==t&&(this.current=this._end)},e.prototype.previous=function(){this.current+=this.step,this.marginEnd+=this.step,this.marginRange=this.marginEnd-this.marginStart},e.prototype.getCurrent=function(t){var e=Math.abs(this.current)0;s--){if("0"!=i[s]){if("."==i[s]||","==i[s]){i=i.slice(0,s);break}break}i=i.slice(0,s)}}else{var o="",n=i.indexOf("e");if(-1!=n&&(o=i.slice(n),i=i.slice(0,n)),n=Math.max(i.indexOf(","),i.indexOf(".")),-1===n?(0!==t&&(i+="."),n=i.length+t):0!==t&&(n+=t+1),n>i.length)for(var r=n-i.length;r>0;r--)i+="0";else i=i.slice(0,n);i+=o}return i},e.prototype.snap=function(){},e.prototype.isMajor=function(){return this.current%(this.scale*this.majorSteps[this.stepIndex])==0},t.exports=e},function(t,e,i){function s(t,e){var i=a().hours(0).minutes(0).seconds(0).milliseconds(0);this.start=i.clone().add(-3,"days").valueOf(),this.end=i.clone().add(4,"days").valueOf(),this.body=t,this.deltaDifference=0,this.scaleOffset=0,this.startToFront=!1,this.endToFront=!0,this.defaultOptions={start:null,end:null,direction:"horizontal",moveable:!0,zoomable:!0,min:null,max:null,zoomMin:10,zoomMax:31536e10},this.options=r.extend({},this.defaultOptions),this.props={touch:{}},this.animateTimer=null,this.body.emitter.on("panstart",this._onDragStart.bind(this)),this.body.emitter.on("panmove",this._onDrag.bind(this)),this.body.emitter.on("panend",this._onDragEnd.bind(this)),this.body.emitter.on("press",this._onHold.bind(this)),this.body.emitter.on("mousewheel",this._onMouseWheel.bind(this)),this.body.emitter.on("touch",this._onTouch.bind(this)),this.body.emitter.on("pinch",this._onPinch.bind(this)),this.setOptions(e)}function o(t){if("horizontal"!=t&&"vertical"!=t)throw new TypeError('Unknown direction "'+t+'". Choose "horizontal" or "vertical".')}function n(t,e){return{x:t.x-r.getAbsoluteLeft(e),y:t.y-r.getAbsoluteTop(e)}}var r=i(1),a=(i(47),i(44)),h=i(20),d=i(15);s.prototype=new h,s.prototype.setOptions=function(t){if(t){var e=["direction","min","max","zoomMin","zoomMax","moveable","zoomable","activate","hiddenDates"];r.selectiveExtend(e,this.options,t),("start"in t||"end"in t)&&this.setRange(t.start,t.end)}},s.prototype.setRange=function(t,e,i,s){s!==!0&&(s=!1);var o=void 0!=t?r.convert(t,"Date").valueOf():null,n=void 0!=e?r.convert(e,"Date").valueOf():null;if(this._cancelAnimation(),i){var a=this,h=this.start,l=this.end,c="number"==typeof i?i:500,p=(new Date).valueOf(),u=!1,m=function(){if(!a.props.touch.dragging){var t=(new Date).valueOf(),e=t-p,i=e>c,g=i||null===o?o:r.easeInOutQuad(e,h,o,c),v=i||null===n?n:r.easeInOutQuad(e,l,n,c);f=a._applyRange(g,v),d.updateHiddenDates(a.body,a.options.hiddenDates),u=u||f,f&&a.body.emitter.emit("rangechange",{start:new Date(a.start),end:new Date(a.end),byUser:s}),i?u&&a.body.emitter.emit("rangechanged",{start:new Date(a.start),end:new Date(a.end),byUser:s}):a.animateTimer=setTimeout(m,20)}};return m()}var f=this._applyRange(o,n);if(d.updateHiddenDates(this.body,this.options.hiddenDates),f){var g={start:new Date(this.start),end:new Date(this.end),byUser:s};this.body.emitter.emit("rangechange",g),this.body.emitter.emit("rangechanged",g)}},s.prototype._cancelAnimation=function(){this.animateTimer&&(clearTimeout(this.animateTimer),this.animateTimer=null)},s.prototype._applyRange=function(t,e){var i,s=null!=t?r.convert(t,"Date").valueOf():this.start,o=null!=e?r.convert(e,"Date").valueOf():this.end,n=null!=this.options.max?r.convert(this.options.max,"Date").valueOf():null,a=null!=this.options.min?r.convert(this.options.min,"Date").valueOf():null;if(isNaN(s)||null===s)throw new Error('Invalid start "'+t+'"');if(isNaN(o)||null===o)throw new Error('Invalid end "'+e+'"');if(s>o&&(o=s),null!==a&&a>s&&(i=a-s,s+=i,o+=i,null!=n&&o>n&&(o=n)),null!==n&&o>n&&(i=o-n,s-=i,o-=i,null!=a&&a>s&&(s=a)),null!==this.options.zoomMin){var h=parseFloat(this.options.zoomMin);0>h&&(h=0),h>o-s&&(this.end-this.start===h?(s=this.start,o=this.end):(i=h-(o-s),s-=i/2,o+=i/2))}if(null!==this.options.zoomMax){var d=parseFloat(this.options.zoomMax);0>d&&(d=0),o-s>d&&(this.end-this.start===d?(s=this.start,o=this.end):(i=o-s-d,s+=i/2,o-=i/2))}var l=this.start!=s||this.end!=o;return s>=this.start&&s<=this.end||o>=this.start&&o<=this.end||this.start>=s&&this.start<=o||this.end>=s&&this.end<=o||this.body.emitter.emit("checkRangedItems"),this.start=s,this.end=o,l},s.prototype.getRange=function(){return{start:this.start,end:this.end}},s.prototype.conversion=function(t,e){return s.conversion(this.start,this.end,t,e)},s.conversion=function(t,e,i,s){return void 0===s&&(s=0),0!=i&&e-t!=0?{offset:t,scale:i/(e-t-s)}:{offset:0,scale:1}},s.prototype._onDragStart=function(t){this.deltaDifference=0,this.previousDelta=0,this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.dragging=!0,this.body.dom.root&&(this.body.dom.root.style.cursor="move"),t.preventDefault())},s.prototype._onDrag=function(t){if(this.options.moveable&&this.props.touch.allowDragging){var e=this.options.direction;o(e);var i="horizontal"==e?t.deltaX:t.deltaY;i-=this.deltaDifference;var s=this.props.touch.end-this.props.touch.start,n=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end);s-=n;var r="horizontal"==e?this.body.domProps.center.width:this.body.domProps.center.height,a=-i/r*s,h=this.props.touch.start+a,l=this.props.touch.end+a,c=d.snapAwayFromHidden(this.body.hiddenDates,h,this.previousDelta-i,!0),p=d.snapAwayFromHidden(this.body.hiddenDates,l,this.previousDelta-i,!0);if(c!=h||p!=l)return this.deltaDifference+=i,this.props.touch.start=c,this.props.touch.end=p,void this._onDrag(t);this.previousDelta=i,this._applyRange(h,l),this.body.emitter.emit("rangechange",{start:new Date(this.start),end:new Date(this.end),byUser:!0}),t.preventDefault()}},s.prototype._onDragEnd=function(){this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.dragging=!1,this.body.dom.root&&(this.body.dom.root.style.cursor="auto"),this.body.emitter.emit("rangechanged",{start:new Date(this.start),end:new Date(this.end),byUser:!0}))},s.prototype._onMouseWheel=function(t){if(this.options.zoomable&&this.options.moveable){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i;i=0>e?1-e/5:1/(1+e/5);var s=n({x:t.pageX,y:t.pageY},this.body.dom.center),o=this._pointerToDate(s);this.zoom(i,o,e)}t.preventDefault()}},s.prototype._onTouch=function(){this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.allowDragging=!0,this.props.touch.center=null,this.scaleOffset=0,this.deltaDifference=0},s.prototype._onHold=function(){this.props.touch.allowDragging=!1},s.prototype._onPinch=function(t){if(this.options.zoomable&&this.options.moveable){this.props.touch.allowDragging=!1,this.props.touch.center||(this.props.touch.center=n(t.center,this.body.dom.center));var e=1/(t.scale+this.scaleOffset),i=this._pointerToDate(this.props.touch.center),s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),o=d.getHiddenDurationBefore(this.body.hiddenDates,this,i),r=s-o,a=i-o+(this.props.touch.start-(i-o))*e,h=i+r+(this.props.touch.end-(i+r))*e;this.startToFront=0>=1-e,this.endToFront=0>=e-1;var l=d.snapAwayFromHidden(this.body.hiddenDates,a,1-e,!0),c=d.snapAwayFromHidden(this.body.hiddenDates,h,e-1,!0);(l!=a||c!=h)&&(this.props.touch.start=l,this.props.touch.end=c,this.scaleOffset=1-t.scale,a=l,h=c),this.setRange(a,h,!1,!0),this.startToFront=!1,this.endToFront=!0,t.preventDefault()}},s.prototype._pointerToDate=function(t){var e,i=this.options.direction;if(o(i),"horizontal"==i)return this.body.util.toTime(t.x).valueOf();var s=this.body.domProps.center.height;return e=this.conversion(s),t.y/e.scale+e.offset},s.prototype.zoom=function(t,e,i){null==e&&(e=(this.start+this.end)/2);var s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),o=d.getHiddenDurationBefore(this.body.hiddenDates,this,e),n=s-o,r=e-o+(this.start-(e-o))*t,a=e+n+(this.end-(e+n))*t;this.startToFront=i>0?!1:!0,this.endToFront=-i>0?!1:!0;var h=d.snapAwayFromHidden(this.body.hiddenDates,r,i,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,-i,!0);(h!=r||l!=a)&&(r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0},s.prototype.move=function(t){var e=this.end-this.start,i=this.start+e*t,s=this.end+e*t;this.start=i,this.end=s},s.prototype.moveTo=function(t){var e=(this.start+this.end)/2,i=e-t,s=this.start-i,o=this.end-i;this.setRange(s,o)},t.exports=s},function(t,e){var i=.001;e.orderByStart=function(t){t.sort(function(t,e){return t.data.start-e.data.start})},e.orderByEnd=function(t){t.sort(function(t,e){var i="end"in t.data?t.data.end:t.data.start,s="end"in e.data?e.data.end:e.data.start;return i-s})},e.stack=function(t,i,s){var o,n;if(s)for(o=0,n=t.length;n>o;o++)t[o].top=null;for(o=0,n=t.length;n>o;o++){var r=t[o];if(r.stack&&null===r.top){r.top=i.axis;do{for(var a=null,h=0,d=t.length;d>h;h++){var l=t[h];if(null!==l.top&&l!==r&&l.stack&&e.collision(r,l,i.item)){a=l;break}}null!=a&&(r.top=a.top+a.height+i.item.vertical)}while(a)}}},e.nostack=function(t,e,i){var s,o,n;for(s=0,o=t.length;o>s;s++)if(void 0!==t[s].data.subgroup){n=e.axis;for(var r in i)i.hasOwnProperty(r)&&1==i[r].visible&&i[r].indexe.left&&t.top-s.vertical+ie.top}},function(t,e,i){function s(t,e,i,o){this.current=new Date,this._start=new Date,this._end=new Date,this.autoScale=!0,this.scale="day",this.step=1,this.setRange(t,e,i),this.switchedDay=!1,this.switchedMonth=!1,this.switchedYear=!1,this.hiddenDates=o,void 0===o&&(this.hiddenDates=[]),this.format=s.FORMAT}var o=i(44),n=i(15),r=i(1);s.FORMAT={minorLabels:{millisecond:"SSS",second:"s",minute:"HH:mm",hour:"HH:mm",weekday:"ddd D",day:"D",month:"MMM",year:"YYYY"},majorLabels:{millisecond:"HH:mm:ss",second:"D MMMM HH:mm",minute:"ddd D MMMM",hour:"ddd D MMMM",weekday:"MMMM YYYY",day:"MMMM YYYY",month:"YYYY",year:""}},s.prototype.setFormat=function(t){var e=r.deepExtend({},s.FORMAT);this.format=r.deepExtend(e,t)},s.prototype.setRange=function(t,e,i){if(!(t instanceof Date&&e instanceof Date))throw"No legal start or end date in method setRange";this._start=void 0!=t?new Date(t.valueOf()):new Date,this._end=void 0!=e?new Date(e.valueOf()):new Date,this.autoScale&&this.setMinimumStep(i)},s.prototype.first=function(){this.current=new Date(this._start.valueOf()),this.roundToMinor()},s.prototype.roundToMinor=function(){switch(this.scale){case"year":this.current.setFullYear(this.step*Math.floor(this.current.getFullYear()/this.step)),this.current.setMonth(0);case"month":this.current.setDate(1);case"day":case"weekday":this.current.setHours(0);case"hour":this.current.setMinutes(0);case"minute":this.current.setSeconds(0);case"second":this.current.setMilliseconds(0)}if(1!=this.step)switch(this.scale){case"millisecond":this.current.setMilliseconds(this.current.getMilliseconds()-this.current.getMilliseconds()%this.step);break;case"second":this.current.setSeconds(this.current.getSeconds()-this.current.getSeconds()%this.step);break;case"minute":this.current.setMinutes(this.current.getMinutes()-this.current.getMinutes()%this.step);break;case"hour":this.current.setHours(this.current.getHours()-this.current.getHours()%this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()-1-(this.current.getDate()-1)%this.step+1);break;case"month":this.current.setMonth(this.current.getMonth()-this.current.getMonth()%this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()-this.current.getFullYear()%this.step)}},s.prototype.hasNext=function(){return this.current.valueOf()<=this._end.valueOf()},s.prototype.next=function(){var t=this.current.valueOf();if(this.current.getMonth()<6)switch(this.scale){case"millisecond":this.current=new Date(this.current.valueOf()+this.step);break;case"second":this.current=new Date(this.current.valueOf()+1e3*this.step);break;case"minute":this.current=new Date(this.current.valueOf()+1e3*this.step*60);break;case"hour":this.current=new Date(this.current.valueOf()+1e3*this.step*60*60);var e=this.current.getHours();this.current.setHours(e-e%this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()+this.step);break;case"month":this.current.setMonth(this.current.getMonth()+this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()+this.step)}else switch(this.scale){case"millisecond":this.current=new Date(this.current.valueOf()+this.step);break;case"second":this.current.setSeconds(this.current.getSeconds()+this.step);break;case"minute":this.current.setMinutes(this.current.getMinutes()+this.step); -break;case"hour":this.current.setHours(this.current.getHours()+this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()+this.step);break;case"month":this.current.setMonth(this.current.getMonth()+this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()+this.step)}if(1!=this.step)switch(this.scale){case"millisecond":this.current.getMilliseconds()0&&(this.step=e),this.autoScale=!1},s.prototype.setAutoScale=function(t){this.autoScale=t},s.prototype.setMinimumStep=function(t){if(void 0!=t){var e=31104e6,i=2592e6,s=864e5,o=36e5,n=6e4,r=1e3,a=1;1e3*e>t&&(this.scale="year",this.step=1e3),500*e>t&&(this.scale="year",this.step=500),100*e>t&&(this.scale="year",this.step=100),50*e>t&&(this.scale="year",this.step=50),10*e>t&&(this.scale="year",this.step=10),5*e>t&&(this.scale="year",this.step=5),e>t&&(this.scale="year",this.step=1),3*i>t&&(this.scale="month",this.step=3),i>t&&(this.scale="month",this.step=1),5*s>t&&(this.scale="day",this.step=5),2*s>t&&(this.scale="day",this.step=2),s>t&&(this.scale="day",this.step=1),s/2>t&&(this.scale="weekday",this.step=1),4*o>t&&(this.scale="hour",this.step=4),o>t&&(this.scale="hour",this.step=1),15*n>t&&(this.scale="minute",this.step=15),10*n>t&&(this.scale="minute",this.step=10),5*n>t&&(this.scale="minute",this.step=5),n>t&&(this.scale="minute",this.step=1),15*r>t&&(this.scale="second",this.step=15),10*r>t&&(this.scale="second",this.step=10),5*r>t&&(this.scale="second",this.step=5),r>t&&(this.scale="second",this.step=1),200*a>t&&(this.scale="millisecond",this.step=200),100*a>t&&(this.scale="millisecond",this.step=100),50*a>t&&(this.scale="millisecond",this.step=50),10*a>t&&(this.scale="millisecond",this.step=10),5*a>t&&(this.scale="millisecond",this.step=5),a>t&&(this.scale="millisecond",this.step=1)}},s.prototype.snap=function(t){var e=new Date(t.valueOf());if("year"==this.scale){var i=e.getFullYear()+Math.round(e.getMonth()/12);e.setFullYear(Math.round(i/this.step)*this.step),e.setMonth(0),e.setDate(0),e.setHours(0),e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("month"==this.scale)e.getDate()>15?(e.setDate(1),e.setMonth(e.getMonth()+1)):e.setDate(1),e.setHours(0),e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0);else if("day"==this.scale){switch(this.step){case 5:case 2:e.setHours(24*Math.round(e.getHours()/24));break;default:e.setHours(12*Math.round(e.getHours()/12))}e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("weekday"==this.scale){switch(this.step){case 5:case 2:e.setHours(12*Math.round(e.getHours()/12));break;default:e.setHours(6*Math.round(e.getHours()/6))}e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("hour"==this.scale){switch(this.step){case 4:e.setMinutes(60*Math.round(e.getMinutes()/60));break;default:e.setMinutes(30*Math.round(e.getMinutes()/30))}e.setSeconds(0),e.setMilliseconds(0)}else if("minute"==this.scale){switch(this.step){case 15:case 10:e.setMinutes(5*Math.round(e.getMinutes()/5)),e.setSeconds(0);break;case 5:e.setSeconds(60*Math.round(e.getSeconds()/60));break;default:e.setSeconds(30*Math.round(e.getSeconds()/30))}e.setMilliseconds(0)}else if("second"==this.scale)switch(this.step){case 15:case 10:e.setSeconds(5*Math.round(e.getSeconds()/5)),e.setMilliseconds(0);break;case 5:e.setMilliseconds(1e3*Math.round(e.getMilliseconds()/1e3));break;default:e.setMilliseconds(500*Math.round(e.getMilliseconds()/500))}else if("millisecond"==this.scale){var s=this.step>5?this.step/2:1;e.setMilliseconds(Math.round(e.getMilliseconds()/s)*s)}return e},s.prototype.isMajor=function(){if(1==this.switchedYear)switch(this.switchedYear=!1,this.scale){case"year":case"month":case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedMonth)switch(this.switchedMonth=!1,this.scale){case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedDay)switch(this.switchedDay=!1,this.scale){case"millisecond":case"second":case"minute":case"hour":return!0;default:return!1}switch(this.scale){case"millisecond":return 0==this.current.getMilliseconds();case"second":return 0==this.current.getSeconds();case"minute":return 0==this.current.getHours()&&0==this.current.getMinutes();case"hour":return 0==this.current.getHours();case"weekday":case"day":return 1==this.current.getDate();case"month":return 0==this.current.getMonth();case"year":return!1;default:return!1}},s.prototype.getLabelMinor=function(t){void 0==t&&(t=this.current);var e=this.format.minorLabels[this.scale];return e&&e.length>0?o(t).format(e):""},s.prototype.getLabelMajor=function(t){void 0==t&&(t=this.current);var e=this.format.majorLabels[this.scale];return e&&e.length>0?o(t).format(e):""},s.prototype.getClassName=function(){function t(t){return t/h%2==0?" even":" odd"}function e(t){return t.isSame(new Date,"day")?" today":t.isSame(o().add(1,"day"),"day")?" tomorrow":t.isSame(o().add(-1,"day"),"day")?" yesterday":""}function i(t){return t.isSame(new Date,"week")?" current-week":""}function s(t){return t.isSame(new Date,"month")?" current-month":""}function n(t){return t.isSame(new Date,"year")?" current-year":""}var r=o(this.current),a=r.locale?r.locale("en"):r.lang("en"),h=this.step;switch(this.scale){case"millisecond":return t(a.milliseconds()).trim();case"second":return t(a.seconds()).trim();case"minute":return t(a.minutes()).trim();case"hour":var d=a.hours();return 4==this.step&&(d=d+"-"+(d+4)),d+"h"+e(a)+t(a.hours());case"weekday":return a.format("dddd").toLowerCase()+e(a)+i(a)+t(a.date());case"day":var l=a.date(),c=a.format("MMMM").toLowerCase();return"day"+l+" "+c+s(a)+t(l-1);case"month":return a.format("MMMM").toLowerCase()+s(a)+t(a.month());case"year":var p=a.year();return"year"+p+n(a)+t(p);default:return""}},t.exports=s},function(t){function e(){this.options=null,this.props=null}e.prototype.setOptions=function(t){t&&util.extend(this.options,t)},e.prototype.redraw=function(){return!1},e.prototype.destroy=function(){},e.prototype._isResized=function(){var t=this.props._previousWidth!==this.props.width||this.props._previousHeight!==this.props.height;return this.props._previousWidth=this.props.width,this.props._previousHeight=this.props.height,t},t.exports=e},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={showCurrentTime:!0,locales:a,locale:"en"},this.options=o.extend({},this.defaultOptions),this.offset=0,this._create(),this.setOptions(e)}var o=i(1),n=i(20),r=i(44),a=i(48);s.prototype=new n,s.prototype._create=function(){var t=document.createElement("div");t.className="currenttime",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t},s.prototype.destroy=function(){this.options.showCurrentTime=!1,this.redraw(),this.body=null},s.prototype.setOptions=function(t){t&&o.selectiveExtend(["showCurrentTime","locale","locales"],this.options,t)},s.prototype.redraw=function(){if(this.options.showCurrentTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar),this.start());var e=new Date((new Date).valueOf()+this.offset),i=this.body.util.toScreen(e),s=this.options.locales[this.options.locale],o=s.current+" "+s.time+": "+r(e).format("dddd, MMMM Do YYYY, H:mm:ss");o=o.charAt(0).toUpperCase()+o.substring(1),this.bar.style.left=i+"px",this.bar.title=o}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),this.stop();return!1},s.prototype.start=function(){function t(){e.stop();var i=e.body.range.conversion(e.body.domProps.center.width).scale,s=1/i/10;30>s&&(s=30),s>1e3&&(s=1e3),e.redraw(),e.currentTimeTimer=setTimeout(t,s)}var e=this;t()},s.prototype.stop=function(){void 0!==this.currentTimeTimer&&(clearTimeout(this.currentTimeTimer),delete this.currentTimeTimer)},s.prototype.setCurrentTime=function(t){var e=o.convert(t,"Date").valueOf(),i=(new Date).valueOf();this.offset=e-i,this.redraw()},s.prototype.getCurrentTime=function(){return new Date((new Date).valueOf()+this.offset)},t.exports=s},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={showCustomTime:!1,locales:h,locale:"en"},this.options=n.extend({},this.defaultOptions),this.customTime=new Date,this.eventParams={},this._create(),this.setOptions(e)}var o=i(45),n=i(1),r=i(20),a=i(44),h=i(48);s.prototype=new r,s.prototype.setOptions=function(t){t&&n.selectiveExtend(["showCustomTime","locale","locales"],this.options,t)},s.prototype._create=function(){var t=document.createElement("div");t.className="customtime",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t;var e=document.createElement("div");e.style.position="relative",e.style.top="0px",e.style.left="-10px",e.style.height="100%",e.style.width="20px",t.appendChild(e),this.hammer=new o(e),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.on("pan",function(t){t.preventDefault()})},s.prototype.destroy=function(){this.options.showCustomTime=!1,this.redraw(),this.hammer.enable(!1),this.hammer=null,this.body=null},s.prototype.redraw=function(){if(this.options.showCustomTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar));var e=this.body.util.toScreen(this.customTime),i=this.options.locales[this.options.locale],s=i.time+": "+a(this.customTime).format("dddd, MMMM Do YYYY, H:mm:ss");s=s.charAt(0).toUpperCase()+s.substring(1),this.bar.style.left=e+"px",this.bar.title=s}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar);return!1},s.prototype.setCustomTime=function(t){this.customTime=n.convert(t,"Date"),this.redraw()},s.prototype.getCustomTime=function(){return new Date(this.customTime.valueOf())},s.prototype._onDragStart=function(t){this.eventParams.dragging=!0,this.eventParams.customTime=this.customTime,t.stopPropagation(),t.preventDefault()},s.prototype._onDrag=function(t){if(this.eventParams.dragging){var e=this.body.util.toScreen(this.eventParams.customTime)+t.deltaX,i=this.body.util.toTime(e);this.setCustomTime(i),this.body.emitter.emit("timechange",{time:new Date(this.customTime.valueOf())}),t.stopPropagation(),t.preventDefault()}},s.prototype._onDragEnd=function(t){this.eventParams.dragging&&(this.body.emitter.emit("timechanged",{time:new Date(this.customTime.valueOf())}),t.stopPropagation(),t.preventDefault())},t.exports=s},function(t,e,i){function s(t,e,i,s){this.id=o.randomUUID(),this.body=t,this.defaultOptions={orientation:"left",showMinorLabels:!0,showMajorLabels:!0,icons:!0,majorLinesOffset:7,minorLinesOffset:4,labelOffsetX:10,labelOffsetY:2,iconWidth:20,width:"40px",visible:!0,alignZeros:!0,customRange:{left:{min:void 0,max:void 0},right:{min:void 0,max:void 0}},title:{left:{text:void 0},right:{text:void 0}},format:{left:{decimals:void 0},right:{decimals:void 0}}},this.linegraphOptions=s,this.linegraphSVG=i,this.props={},this.DOMelements={lines:{},labels:{},title:{}},this.dom={},this.range={start:0,end:0},this.options=o.extend({},this.defaultOptions),this.conversionFactor=1,this.setOptions(e),this.width=Number((""+this.options.width).replace("px","")),this.minWidth=this.width,this.height=this.linegraphSVG.offsetHeight,this.hidden=!1,this.stepPixels=25,this.stepPixelsForced=25,this.zeroCrossing=-1,this.lineOffset=0,this.master=!0,this.svgElements={},this.iconsRemoved=!1,this.groups={},this.amountOfGroups=0,this._create();var n=this;this.body.emitter.on("verticalDrag",function(){n.dom.lineContainer.style.top=n.body.domProps.scrollTop+"px"})}var o=i(1),n=i(2),r=i(20),a=i(16);s.prototype=new r,s.prototype.addGroup=function(t,e){this.groups.hasOwnProperty(t)||(this.groups[t]=e),this.amountOfGroups+=1},s.prototype.updateGroup=function(t,e){this.groups[t]=e},s.prototype.removeGroup=function(t){this.groups.hasOwnProperty(t)&&(delete this.groups[t],this.amountOfGroups-=1)},s.prototype.setOptions=function(t){if(t){var e=!1;this.options.orientation!=t.orientation&&void 0!==t.orientation&&(e=!0);var i=["orientation","showMinorLabels","showMajorLabels","icons","majorLinesOffset","minorLinesOffset","labelOffsetX","labelOffsetY","iconWidth","width","visible","customRange","title","format","alignZeros"];o.selectiveExtend(i,this.options,t),this.minWidth=Number((""+this.options.width).replace("px","")),1==e&&this.dom.frame&&(this.hide(),this.show())}},s.prototype._create=function(){this.dom.frame=document.createElement("div"),this.dom.frame.style.width=this.options.width,this.dom.frame.style.height=this.height,this.dom.lineContainer=document.createElement("div"),this.dom.lineContainer.style.width="100%",this.dom.lineContainer.style.height=this.height,this.dom.lineContainer.style.position="relative",this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="absolute",this.svg.style.top="0px",this.svg.style.height="100%",this.svg.style.width="100%",this.svg.style.display="block",this.dom.frame.appendChild(this.svg)},s.prototype._redrawGroupIcons=function(){n.prepareElements(this.svgElements);var t,e=this.options.iconWidth,i=15,s=4,o=s+.5*i;t="left"==this.options.orientation?s:this.width-e-s;for(var r in this.groups)this.groups.hasOwnProperty(r)&&(1!=this.groups[r].visible||void 0!==this.linegraphOptions.visibility[r]&&1!=this.linegraphOptions.visibility[r]||(this.groups[r].drawIcon(t,o,this.svgElements,this.svg,e,i),o+=i+s));n.cleanupElements(this.svgElements),this.iconsRemoved=!1},s.prototype._cleanupIcons=function(){0==this.iconsRemoved&&(n.prepareElements(this.svgElements),n.cleanupElements(this.svgElements),this.iconsRemoved=!0)},s.prototype.show=function(){this.hidden=!1,this.dom.frame.parentNode||("left"==this.options.orientation?this.body.dom.left.appendChild(this.dom.frame):this.body.dom.right.appendChild(this.dom.frame)),this.dom.lineContainer.parentNode||this.body.dom.backgroundHorizontal.appendChild(this.dom.lineContainer)},s.prototype.hide=function(){this.hidden=!0,this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame),this.dom.lineContainer.parentNode&&this.dom.lineContainer.parentNode.removeChild(this.dom.lineContainer)},s.prototype.setRange=function(t,e){0==this.master&&1==this.options.alignZeros&&-1!=this.zeroCrossing&&t>0&&(t=0),this.range.start=t,this.range.end=e},s.prototype.redraw=function(){var t=!1,e=0;this.dom.lineContainer.style.top=this.body.domProps.scrollTop+"px";for(var i in this.groups)this.groups.hasOwnProperty(i)&&(1!=this.groups[i].visible||void 0!==this.linegraphOptions.visibility[i]&&1!=this.linegraphOptions.visibility[i]||e++);if(0==this.amountOfGroups||0==e)this.hide();else{this.show(),this.height=Number(this.linegraphSVG.style.height.replace("px","")),this.dom.lineContainer.style.height=this.height+"px",this.width=1==this.options.visible?Number((""+this.options.width).replace("px","")):0;var s=this.props,o=this.dom.frame;o.className="dataaxis",this._calculateCharSize();var n=this.options.orientation,r=this.options.showMinorLabels,a=this.options.showMajorLabels;s.minorLabelHeight=r?s.minorCharHeight:0,s.majorLabelHeight=a?s.majorCharHeight:0,s.minorLineWidth=this.body.dom.backgroundHorizontal.offsetWidth-this.lineOffset-this.width+2*this.options.minorLinesOffset,s.minorLineHeight=1,s.majorLineWidth=this.body.dom.backgroundHorizontal.offsetWidth-this.lineOffset-this.width+2*this.options.majorLinesOffset,s.majorLineHeight=1,"left"==n?(o.style.top="0",o.style.left="0",o.style.bottom="",o.style.width=this.width+"px",o.style.height=this.height+"px",this.props.width=this.body.domProps.left.width,this.props.height=this.body.domProps.left.height):(o.style.top="",o.style.bottom="0",o.style.left="0",o.style.width=this.width+"px",o.style.height=this.height+"px",this.props.width=this.body.domProps.right.width,this.props.height=this.body.domProps.right.height),t=this._redrawLabels(),t=this._isResized()||t,1==this.options.icons?this._redrawGroupIcons():this._cleanupIcons(),this._redrawTitle(n)}return t},s.prototype._redrawLabels=function(){var t=!1;n.prepareElements(this.DOMelements.lines),n.prepareElements(this.DOMelements.labels);var e=this.options.orientation,i=this.master?this.props.majorCharHeight||10:this.stepPixelsForced,s=new a(this.range.start,this.range.end,i,this.dom.frame.offsetHeight,this.options.customRange[this.options.orientation],0==this.master&&this.options.alignZeros);this.step=s;var o=(this.dom.frame.offsetHeight-s.deadSpace*(this.dom.frame.offsetHeight/s.marginRange))/((s.marginRange-s.deadSpace)/s.step);this.stepPixels=o;var r=this.height/o,h=0;if(0==this.master){o=this.stepPixelsForced,h=Math.round(this.dom.frame.offsetHeight/o-r);for(var d=0;.5*h>d;d++)s.previous();if(r=this.height/o,-1!=this.zeroCrossing&&1==this.options.alignZeros){var l=s.marginEnd/s.step-this.zeroCrossing;if(l>0)for(var d=0;l>d;d++)s.next();else if(0>l)for(var d=0;-l>d;d++)s.previous()}}else r+=.25;this.valueAtZero=s.marginEnd;var c,p=0,u=1;void 0!==this.options.format[e]&&(c=this.options.format[e].decimals),this.maxLabelSize=0;for(var m=0;u=0&&this._redrawLabel(m-2,s.getCurrent(c),e,"yAxis major",this.props.majorCharHeight),this._redrawLine(m,e,"grid horizontal major",this.options.majorLinesOffset,this.props.majorLineWidth)):this._redrawLine(m,e,"grid horizontal minor",this.options.minorLinesOffset,this.props.minorLineWidth),1==this.master&&0==s.current&&(this.zeroCrossing=u),u++}this.conversionFactor=0==this.master?m/(this.valueAtZero-s.current):this.dom.frame.offsetHeight/s.marginRange;var g=0;void 0!==this.options.title[e]&&void 0!==this.options.title[e].text&&(g=this.props.titleCharHeight);var v=1==this.options.icons?Math.max(this.options.iconWidth,g)+this.options.labelOffsetX+15:g+this.options.labelOffsetX+15;return this.maxLabelSize>this.width-v&&1==this.options.visible?(this.width=this.maxLabelSize+v,this.options.width=this.width+"px",n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),this.redraw(),t=!0):this.maxLabelSizethis.minWidth?(this.width=Math.max(this.minWidth,this.maxLabelSize+v),this.options.width=this.width+"px",n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),this.redraw(),t=!0):(n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),t=!1),t},s.prototype.convertValue=function(t){var e=this.valueAtZero-t,i=e*this.conversionFactor;return i},s.prototype._redrawLabel=function(t,e,i,s,o){var r=n.getDOMElement("div",this.DOMelements.labels,this.dom.frame);r.className=s,r.innerHTML=e,"left"==i?(r.style.left="-"+this.options.labelOffsetX+"px",r.style.textAlign="right"):(r.style.right="-"+this.options.labelOffsetX+"px",r.style.textAlign="left"),r.style.top=t-.5*o+this.options.labelOffsetY+"px",e+="";var a=Math.max(this.props.majorCharWidth,this.props.minorCharWidth);this.maxLabelSized;d++){var c=this.visibleItems[d];c.repositionY(e)}return s},s.prototype._calculateHeight=function(t){var e,i=this.visibleItems;this.resetSubgroups();var s=this;if(i.length){var n=i[0].top,r=i[0].top+i[0].height;if(o.forEach(i,function(t){n=Math.min(n,t.top),r=Math.max(r,t.top+t.height),void 0!==t.data.subgroup&&(s.subgroups[t.data.subgroup].height=Math.max(s.subgroups[t.data.subgroup].height,t.height),s.subgroups[t.data.subgroup].visible=!0)}),n>t.axis){var a=n-t.axis;r-=a,o.forEach(i,function(t){t.top-=a})}e=r+t.item.vertical/2}else e=t.axis+t.item.vertical;return e=Math.max(e,this.props.label.height)},s.prototype.show=function(){this.dom.label.parentNode||this.itemSet.dom.labelSet.appendChild(this.dom.label),this.dom.foreground.parentNode||this.itemSet.dom.foreground.appendChild(this.dom.foreground),this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background),this.dom.axis.parentNode||this.itemSet.dom.axis.appendChild(this.dom.axis)},s.prototype.hide=function(){var t=this.dom.label;t.parentNode&&t.parentNode.removeChild(t);var e=this.dom.foreground;e.parentNode&&e.parentNode.removeChild(e);var i=this.dom.background;i.parentNode&&i.parentNode.removeChild(i);var s=this.dom.axis;s.parentNode&&s.parentNode.removeChild(s)},s.prototype.add=function(t){if(this.items[t.id]=t,t.setParent(this),void 0!==t.data.subgroup&&(void 0===this.subgroups[t.data.subgroup]&&(this.subgroups[t.data.subgroup]={height:0,visible:!1,index:this.subgroupIndex,items:[]},this.subgroupIndex++),this.subgroups[t.data.subgroup].items.push(t)),this.orderSubgroups(),-1==this.visibleItems.indexOf(t)){var e=this.itemSet.body.range;this._checkIfVisible(t,this.visibleItems,e)}},s.prototype.orderSubgroups=function(){if(void 0!==this.subgroupOrderer){var t=[];if("string"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push({subgroup:e,sortField:this.subgroups[e].items[0].data[this.subgroupOrderer]});t.sort(function(t,e){return t.sortField-e.sortField})}else if("function"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push(this.subgroups[e].items[0].data);t.sort(this.subgroupOrderer)}if(t.length>0)for(var i=0;it?-1:l>=t?0:1};if(e.length>0)for(n=0;nl}),1==this.checkRangedItems)for(this.checkRangedItems=!1,n=0;nl})}for(n=0;n=0&&(n=e[r],!o(n));r--)void 0===s[n.id]&&(s[n.id]=!0,i.push(n));for(r=t+1;rs;s++){var n=this.visibleItems[s];n.repositionY(e)}return i},s.prototype.show=function(){this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background)},t.exports=s},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={type:null,orientation:"bottom",align:"auto",stack:!0,groupOrder:null,selectable:!0,editable:{updateTime:!1,updateGroup:!1,add:!1,remove:!1},onAdd:function(t,e){e(t)},onUpdate:function(t,e){e(t)},onMove:function(t,e){e(t)},onRemove:function(t,e){e(t)},onMoving:function(t,e){e(t)},margin:{item:{horizontal:10,vertical:10},axis:20},padding:5},this.options=n.extend({},this.defaultOptions),this.itemOptions={type:{start:"Date",end:"Date"}},this.conversion={toScreen:t.util.toScreen,toTime:t.util.toTime},this.dom={},this.props={},this.hammer=null;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e){i._onAdd(e.items)},update:function(t,e){i._onUpdate(e.items)},remove:function(t,e){i._onRemove(e.items)}},this.groupListeners={add:function(t,e){i._onAddGroups(e.items)},update:function(t,e){i._onUpdateGroups(e.items)},remove:function(t,e){i._onRemoveGroups(e.items)}},this.items={},this.groups={},this.groupIds=[],this.selection=[],this.stackDirty=!0,this.touchParams={},this._create(),this.setOptions(e)}var o=i(45),n=i(1),r=i(3),a=i(4),h=i(20),d=i(25),l=i(26),c=i(33),p=i(34),u=i(35),m=i(32),f="__ungrouped__",g="__background__";s.prototype=new h,s.types={background:m,box:c,range:u,point:p},s.prototype._create=function(){var t=document.createElement("div");t.className="itemset",t["timeline-itemset"]=this,this.dom.frame=t;var e=document.createElement("div");e.className="background",t.appendChild(e),this.dom.background=e;var i=document.createElement("div");i.className="foreground",t.appendChild(i),this.dom.foreground=i;var s=document.createElement("div");s.className="axis",this.dom.axis=s;var n=document.createElement("div");n.className="labelset",this.dom.labelSet=n,this._updateUngrouped();var r=new l(g,null,this);r.show(),this.groups[g]=r,this.hammer=new o(this.body.dom.centerContainer),this.hammer.on("hammer.input",function(t){t.isFirst&&this._onTouch(t)}.bind(this)),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.on("tap",this._onSelectItem.bind(this)),this.hammer.on("press",this._onMultiSelectItem.bind(this)),this.hammer.on("doubletap",this._onAddItem.bind(this)),this.show()},s.prototype.setOptions=function(t){if(t){var e=["type","align","orientation","padding","stack","selectable","groupOrder","dataAttributes","template","hide"];n.selectiveExtend(e,this.options,t),"margin"in t&&("number"==typeof t.margin?(this.options.margin.axis=t.margin,this.options.margin.item.horizontal=t.margin,this.options.margin.item.vertical=t.margin):"object"==typeof t.margin&&(n.selectiveExtend(["axis"],this.options.margin,t.margin),"item"in t.margin&&("number"==typeof t.margin.item?(this.options.margin.item.horizontal=t.margin.item,this.options.margin.item.vertical=t.margin.item):"object"==typeof t.margin.item&&n.selectiveExtend(["horizontal","vertical"],this.options.margin.item,t.margin.item)))),"editable"in t&&("boolean"==typeof t.editable?(this.options.editable.updateTime=t.editable,this.options.editable.updateGroup=t.editable,this.options.editable.add=t.editable,this.options.editable.remove=t.editable):"object"==typeof t.editable&&n.selectiveExtend(["updateTime","updateGroup","add","remove"],this.options.editable,t.editable));var i=function(e){var i=t[e];if(i){if(!(i instanceof Function))throw new Error("option "+e+" must be a function "+e+"(item, callback)");this.options[e]=i}}.bind(this);["onAdd","onUpdate","onRemove","onMove","onMoving"].forEach(i),this.markDirty()}},s.prototype.markDirty=function(){this.groupIds=[],this.stackDirty=!0},s.prototype.destroy=function(){this.hide(),this.setItems(null),this.setGroups(null),this.hammer=null,this.body=null,this.conversion=null},s.prototype.hide=function(){this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame),this.dom.axis.parentNode&&this.dom.axis.parentNode.removeChild(this.dom.axis),this.dom.labelSet.parentNode&&this.dom.labelSet.parentNode.removeChild(this.dom.labelSet)},s.prototype.show=function(){this.dom.frame.parentNode||this.body.dom.center.appendChild(this.dom.frame),this.dom.axis.parentNode||this.body.dom.backgroundVertical.appendChild(this.dom.axis),this.dom.labelSet.parentNode||this.body.dom.left.appendChild(this.dom.labelSet)},s.prototype.setSelection=function(t){var e,i,s,o;for(void 0==t&&(t=[]),Array.isArray(t)||(t=[t]),e=0,i=this.selection.length;i>e;e++)s=this.selection[e],o=this.items[s],o&&o.unselect();for(this.selection=[],e=0,i=t.length;i>e;e++)s=t[e],o=this.items[s],o&&(this.selection.push(s),o.select())},s.prototype.getSelection=function(){return this.selection.concat([])},s.prototype.getVisibleItems=function(){var t=this.body.range.getRange(),e=this.body.util.toScreen(t.start),i=this.body.util.toScreen(t.end),s=[];for(var o in this.groups)if(this.groups.hasOwnProperty(o))for(var n=this.groups[o],r=n.visibleItems,a=0;ae&&s.push(h.id)}return s},s.prototype._deselect=function(t){for(var e=this.selection,i=0,s=e.length;s>i;i++)if(e[i]==t){e.splice(i,1);break}},s.prototype.redraw=function(){var t=this.options.margin,e=this.body.range,i=n.option.asSize,s=this.options,o=s.orientation,r=!1,a=this.dom.frame,h=s.editable.updateTime||s.editable.updateGroup;this.props.top=this.body.domProps.top.height+this.body.domProps.border.top,this.props.left=this.body.domProps.left.width+this.body.domProps.border.left,a.className="itemset"+(h?" editable":""),r=this._orderGroups()||r;var d=e.end-e.start,l=d!=this.lastVisibleInterval||this.props.width!=this.props.lastWidth;l&&(this.stackDirty=!0),this.lastVisibleInterval=d,this.props.lastWidth=this.props.width;var c=this.stackDirty,p=this._firstGroup(),u={item:t.item,axis:t.axis},m={item:t.item,axis:t.item.vertical/2},f=0,v=t.axis+t.item.vertical;return this.groups[g].redraw(e,m,c),n.forEach(this.groups,function(t){var i=t==p?u:m,s=t.redraw(e,i,c);r=s||r,f+=t.height}),f=Math.max(f,v),this.stackDirty=!1,a.style.height=i(f),this.props.width=a.offsetWidth,this.props.height=f,this.dom.axis.style.top=i("top"==o?this.body.domProps.top.height+this.body.domProps.border.top:this.body.domProps.top.height+this.body.domProps.centerContainer.height),this.dom.axis.style.left="0",r=this._isResized()||r},s.prototype._firstGroup=function(){var t="top"==this.options.orientation?0:this.groupIds.length-1,e=this.groupIds[t],i=this.groups[e]||this.groups[f];return i||null},s.prototype._updateUngrouped=function(){{var t,e,i=this.groups[f];this.groups[g]}if(this.groupsData){if(i){i.hide(),delete this.groups[f];for(e in this.items)if(this.items.hasOwnProperty(e)){t=this.items[e],t.parent&&t.parent.remove(t);var s=this._getGroupId(t.data),o=this.groups[s];o&&o.add(t)||t.hide()}}}else if(!i){var n=null,r=null;i=new d(n,r,this),this.groups[f]=i;for(e in this.items)this.items.hasOwnProperty(e)&&(t=this.items[e],i.add(t));i.show()}},s.prototype.getLabelSet=function(){return this.dom.labelSet},s.prototype.setItems=function(t){var e,i=this,s=this.itemsData;if(t){if(!(t instanceof r||t instanceof a))throw new TypeError("Data must be an instance of DataSet or DataView");this.itemsData=t}else this.itemsData=null;if(s&&(n.forEach(this.itemListeners,function(t,e){s.off(e,t)}),e=s.getIds(),this._onRemove(e)),this.itemsData){var o=this.id;n.forEach(this.itemListeners,function(t,e){i.itemsData.on(e,t,o)}),e=this.itemsData.getIds(),this._onAdd(e),this._updateUngrouped()}},s.prototype.getItems=function(){return this.itemsData},s.prototype.setGroups=function(t){var e,i=this;if(this.groupsData&&(n.forEach(this.groupListeners,function(t,e){i.groupsData.unsubscribe(e,t)}),e=this.groupsData.getIds(),this.groupsData=null,this._onRemoveGroups(e)),t){if(!(t instanceof r||t instanceof a))throw new TypeError("Data must be an instance of DataSet or DataView");this.groupsData=t}else this.groupsData=null;if(this.groupsData){var s=this.id;n.forEach(this.groupListeners,function(t,e){i.groupsData.on(e,t,s)}),e=this.groupsData.getIds(),this._onAddGroups(e)}this._updateUngrouped(),this._order(),this.body.emitter.emit("change",{queue:!0})},s.prototype.getGroups=function(){return this.groupsData},s.prototype.removeItem=function(t){var e=this.itemsData.get(t),i=this.itemsData.getDataSet();e&&this.options.onRemove(e,function(e){e&&i.remove(t)})},s.prototype._getType=function(t){return t.type||this.options.type||(t.end?"range":"box")},s.prototype._getGroupId=function(t){var e=this._getType(t);return"background"==e&&void 0==t.group?g:this.groupsData?t.group:f},s.prototype._onUpdate=function(t){var e=this;t.forEach(function(t){var i=e.itemsData.get(t,e.itemOptions),o=e.items[t],n=e._getType(i),r=s.types[n];if(o&&(r&&o instanceof r?e._updateItem(o,i):(e._removeItem(o),o=null)),!o){if(!r)throw new TypeError("rangeoverflow"==n?'Item type "rangeoverflow" is deprecated. Use css styling instead: .vis.timeline .item.range .content {overflow: visible;}':'Unknown item type "'+n+'"');o=new r(i,e.conversion,e.options),o.id=t,e._addItem(o)}}),this._order(),this.stackDirty=!0,this.body.emitter.emit("change",{queue:!0})},s.prototype._onAdd=s.prototype._onUpdate,s.prototype._onRemove=function(t){var e=0,i=this;t.forEach(function(t){var s=i.items[t];s&&(e++,i._removeItem(s))}),e&&(this._order(),this.stackDirty=!0,this.body.emitter.emit("change",{queue:!0}))},s.prototype._order=function(){n.forEach(this.groups,function(t){t.order()})},s.prototype._onUpdateGroups=function(t){this._onAddGroups(t)},s.prototype._onAddGroups=function(t){var e=this;t.forEach(function(t){var i=e.groupsData.get(t),s=e.groups[t];if(s)s.setData(i);else{if(t==f||t==g)throw new Error("Illegal group id. "+t+" is a reserved id.");var o=Object.create(e.options);n.extend(o,{height:null}),s=new d(t,i,e),e.groups[t]=s;for(var r in e.items)if(e.items.hasOwnProperty(r)){var a=e.items[r];a.data.group==t&&s.add(a)}s.order(),s.show()}}),this.body.emitter.emit("change",{queue:!0})},s.prototype._onRemoveGroups=function(t){var e=this.groups;t.forEach(function(t){var i=e[t];i&&(i.hide(),delete e[t])}),this.markDirty(),this.body.emitter.emit("change",{queue:!0})},s.prototype._orderGroups=function(){if(this.groupsData){var t=this.groupsData.getIds({order:this.options.groupOrder}),e=!n.equalArray(t,this.groupIds);if(e){var i=this.groups;t.forEach(function(t){i[t].hide()}),t.forEach(function(t){i[t].show()}),this.groupIds=t}return e}return!1},s.prototype._addItem=function(t){this.items[t.id]=t;var e=this._getGroupId(t.data),i=this.groups[e];i&&i.add(t)},s.prototype._updateItem=function(t,e){var i=t.data.group;if(t.setData(e),i!=t.data.group){var s=this.groups[i];s&&s.remove(t);var o=this._getGroupId(t.data),n=this.groups[o];n&&n.add(t)}},s.prototype._removeItem=function(t){t.hide(),delete this.items[t.id];var e=this.selection.indexOf(t.id);-1!=e&&this.selection.splice(e,1),t.parent&&t.parent.remove(t)},s.prototype._constructByEndArray=function(t){for(var e=[],i=0;i0||o.length>0)&&this.body.emitter.emit("select",{items:a})}},s.prototype._onAddItem=function(t){if(this.options.selectable&&this.options.editable.add){var e=this,i=this.body.util.snap||null,o=s.itemFromTarget(t);if(o){var r=e.itemsData.get(o.id);this.options.onUpdate(r,function(t){t&&e.itemsData.getDataSet().update(t)})}else{var a=n.getAbsoluteLeft(this.dom.frame),h=t.center.x-a,d=this.body.util.toTime(h),l={start:i?i(d):d,content:"new item"};if("range"===this.options.type){var c=this.body.util.toTime(h+this.props.width/5);l.end=i?i(c):c}l[this.itemsData._fieldId]=n.randomUUID();var p=s.groupFromTarget(t);p&&(l.group=p.groupId),this.options.onAdd(l,function(t){t&&e.itemsData.getDataSet().add(t)})}}},s.prototype._onMultiSelectItem=function(t){if(this.options.selectable){var e,i=s.itemFromTarget(t);if(i){e=this.getSelection();var o=t.srcEvent&&t.srcEvent.shiftKey||!1;if(o){e.push(i.id);var n=s._getItemRange(this.itemsData.get(e,this.itemOptions));e=[];for(var r in this.items)if(this.items.hasOwnProperty(r)){var a=this.items[r],h=a.data.start,d=void 0!==a.data.end?a.data.end:h;h>=n.min&&d<=n.max&&e.push(a.id)}}else{var l=e.indexOf(i.id);-1==l?e.push(i.id):e.splice(l,1)}this.setSelection(e),this.body.emitter.emit("select",{items:this.getSelection()})}}},s._getItemRange=function(t){var e=null,i=null;return t.forEach(function(t){(null==i||t.starte)&&(e=t.end):(null==e||t.start>e)&&(e=t.start)}),{min:i,max:e}},s.itemFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-item"))return e["timeline-item"];e=e.parentNode}return null},s.groupFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-group"))return e["timeline-group"];e=e.parentNode}return null},s.itemSetFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-itemset"))return e["timeline-itemset"];e=e.parentNode}return null},t.exports=s},function(t,e,i){function s(t,e,i,s){this.body=t,this.defaultOptions={enabled:!0,icons:!0,iconSize:20,iconSpacing:6,left:{visible:!0,position:"top-left"},right:{visible:!0,position:"top-left"}},this.side=i,this.options=o.extend({},this.defaultOptions),this.linegraphOptions=s,this.svgElements={},this.dom={},this.groups={},this.amountOfGroups=0,this._create(),this.setOptions(e)}var o=i(1),n=i(2),r=i(20);s.prototype=new r,s.prototype.clear=function(){this.groups={},this.amountOfGroups=0},s.prototype.addGroup=function(t,e){this.groups.hasOwnProperty(t)||(this.groups[t]=e),this.amountOfGroups+=1},s.prototype.updateGroup=function(t,e){this.groups[t]=e},s.prototype.removeGroup=function(t){this.groups.hasOwnProperty(t)&&(delete this.groups[t],this.amountOfGroups-=1)},s.prototype._create=function(){this.dom.frame=document.createElement("div"),this.dom.frame.className="legend",this.dom.frame.style.position="absolute",this.dom.frame.style.top="10px",this.dom.frame.style.display="block",this.dom.textArea=document.createElement("div"),this.dom.textArea.className="legendText",this.dom.textArea.style.position="relative",this.dom.textArea.style.top="0px",this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="absolute",this.svg.style.top="0px",this.svg.style.width=this.options.iconSize+5+"px",this.svg.style.height="100%",this.dom.frame.appendChild(this.svg),this.dom.frame.appendChild(this.dom.textArea)},s.prototype.hide=function(){this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame)},s.prototype.show=function(){this.dom.frame.parentNode||this.body.dom.center.appendChild(this.dom.frame)},s.prototype.setOptions=function(t){var e=["enabled","orientation","icons","left","right"];o.selectiveDeepExtend(e,this.options,t)},s.prototype.redraw=function(){var t=0;for(var e in this.groups)this.groups.hasOwnProperty(e)&&(1!=this.groups[e].visible||void 0!==this.linegraphOptions.visibility[e]&&1!=this.linegraphOptions.visibility[e]||t++);if(0==this.options[this.side].visible||0==this.amountOfGroups||0==this.options.enabled||0==t)this.hide();else{if(this.show(),"top-left"==this.options[this.side].position||"bottom-left"==this.options[this.side].position?(this.dom.frame.style.left="4px",this.dom.frame.style.textAlign="left",this.dom.textArea.style.textAlign="left",this.dom.textArea.style.left=this.options.iconSize+15+"px",this.dom.textArea.style.right="",this.svg.style.left="0px",this.svg.style.right=""):(this.dom.frame.style.right="4px",this.dom.frame.style.textAlign="right",this.dom.textArea.style.textAlign="right",this.dom.textArea.style.right=this.options.iconSize+15+"px",this.dom.textArea.style.left="",this.svg.style.right="0px",this.svg.style.left=""),"top-left"==this.options[this.side].position||"top-right"==this.options[this.side].position)this.dom.frame.style.top=4-Number(this.body.dom.center.style.top.replace("px",""))+"px",this.dom.frame.style.bottom="";else{var i=this.body.domProps.center.height-this.body.domProps.centerContainer.height;this.dom.frame.style.bottom=4+i+Number(this.body.dom.center.style.top.replace("px",""))+"px",this.dom.frame.style.top=""}0==this.options.icons?(this.dom.frame.style.width=this.dom.textArea.offsetWidth+10+"px",this.dom.textArea.style.right="",this.dom.textArea.style.left="",this.svg.style.width="0px"):(this.dom.frame.style.width=this.options.iconSize+15+this.dom.textArea.offsetWidth+10+"px",this.drawLegendIcons());var s="";for(var e in this.groups)this.groups.hasOwnProperty(e)&&(1!=this.groups[e].visible||void 0!==this.linegraphOptions.visibility[e]&&1!=this.linegraphOptions.visibility[e]||(s+=this.groups[e].content+"
    "));this.dom.textArea.innerHTML=s,this.dom.textArea.style.lineHeight=.75*this.options.iconSize+this.options.iconSpacing+"px"}},s.prototype.drawLegendIcons=function(){if(this.dom.frame.parentNode){n.prepareElements(this.svgElements);var t=window.getComputedStyle(this.dom.frame).paddingTop,e=Number(t.replace("px","")),i=e,s=this.options.iconSize,o=.75*this.options.iconSize,r=e+.5*o+3;this.svg.style.width=s+5+e+"px";for(var a in this.groups)this.groups.hasOwnProperty(a)&&(1!=this.groups[a].visible||void 0!==this.linegraphOptions.visibility[a]&&1!=this.linegraphOptions.visibility[a]||(this.groups[a].drawIcon(i,r,this.svgElements,this.svg,s,o),r+=o+this.options.iconSpacing));n.cleanupElements(this.svgElements)}},t.exports=s},function(t,e,i){function s(t,e){this.id=o.randomUUID(),this.body=t,this.defaultOptions={yAxisOrientation:"left",defaultGroup:"default",sort:!0,sampling:!0,graphHeight:"400px",shaded:{enabled:!1,orientation:"bottom"},style:"line",barChart:{width:50,handleOverlap:"overlap",align:"center"},catmullRom:{enabled:!0,parametrization:"centripetal",alpha:.5},drawPoints:{enabled:!0,size:6,style:"square"},dataAxis:{showMinorLabels:!0,showMajorLabels:!0,icons:!1,width:"40px",visible:!0,alignZeros:!0,customRange:{left:{min:void 0,max:void 0},right:{min:void 0,max:void 0}}},legend:{enabled:!1,icons:!0,left:{visible:!0,position:"top-left"},right:{visible:!0,position:"top-right"}},groups:{visibility:{}}},this.options=o.extend({},this.defaultOptions),this.dom={},this.props={},this.hammer=null,this.groups={},this.abortedGraphUpdate=!1,this.updateSVGheight=!1,this.updateSVGheightOnResize=!1;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e){i._onAdd(e.items)},update:function(t,e){i._onUpdate(e.items)},remove:function(t,e){i._onRemove(e.items)}},this.groupListeners={add:function(t,e){i._onAddGroups(e.items)},update:function(t,e){i._onUpdateGroups(e.items)},remove:function(t,e){i._onRemoveGroups(e.items)}},this.items={},this.selection=[],this.lastStart=this.body.range.start,this.touchParams={},this.svgElements={},this.setOptions(e),this.groupsUsingDefaultStyles=[0],this.COUNTER=0,this.body.emitter.on("rangechanged",function(){i.lastStart=i.body.range.start,i.svg.style.left=o.option.asSize(-i.props.width),i.redraw.call(i,!0)}),this._create(),this.framework={svg:this.svg,svgElements:this.svgElements,options:this.options,groups:this.groups},this.body.emitter.emit("change")}var o=i(1),n=i(2),r=i(3),a=i(4),h=i(20),d=i(23),l=i(24),c=i(28),p=i(52),u="__ungrouped__";s.prototype=new h,s.prototype._create=function(){var t=document.createElement("div");t.className="LineGraph",this.dom.frame=t,this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="relative",this.svg.style.height=(""+this.options.graphHeight).replace("px","")+"px",this.svg.style.display="block",t.appendChild(this.svg),this.options.dataAxis.orientation="left",this.yAxisLeft=new d(this.body,this.options.dataAxis,this.svg,this.options.groups),this.options.dataAxis.orientation="right",this.yAxisRight=new d(this.body,this.options.dataAxis,this.svg,this.options.groups),delete this.options.dataAxis.orientation,this.legendLeft=new c(this.body,this.options.legend,"left",this.options.groups),this.legendRight=new c(this.body,this.options.legend,"right",this.options.groups),this.show()},s.prototype.setOptions=function(t){if(t){var e=["sampling","defaultGroup","height","graphHeight","yAxisOrientation","style","barChart","dataAxis","sort","groups"];void 0===t.graphHeight&&void 0!==t.height&&void 0!==this.body.domProps.centerContainer.height?(this.updateSVGheight=!0,this.updateSVGheightOnResize=!0):void 0!==this.body.domProps.centerContainer.height&&void 0!==t.graphHeight&&parseInt((t.graphHeight+"").replace("px",""))0){var d=this.body.util.toGlobalTime(-this.body.domProps.root.width),l=this.body.util.toGlobalTime(2*this.body.domProps.root.width),c={};for(this._getRelevantData(a,c,d,l),this._applySampling(a,c),e=0;eu&&console.log("WARNING: there may be an infinite loop in the _updateGraph emitter cycle."),this.COUNTER=0,this.abortedGraphUpdate=!1,e=0;e0)for(r=0;rs){d.push(h);break}d.push(h)}}else for(a=0;ai&&h.x0)for(var s=0;s0){var n=1,r=o.length,a=this.body.util.toGlobalScreen(o[o.length-1].x)-this.body.util.toGlobalScreen(o[0].x),h=r/a;n=Math.min(Math.ceil(.2*r),Math.max(1,Math.round(h)));for(var d=[],l=0;r>l;l+=n)d.push(o[l]);e[t[s]]=d}}},s.prototype._getYRanges=function(t,e,i){var s,o,n,r,a=[],h=[];if(t.length>0){for(n=0;n0&&(o=this.groups[t[n]],"stack"==r.barChart.handleOverlap&&"bar"==r.style?"left"==r.yAxisOrientation?a=a.concat(o.getYRange(s)):h=h.concat(o.getYRange(s)):i[t[n]]=o.getYRange(s,t[n]));p.getStackedBarYRange(a,i,t,"__barchartLeft","left"),p.getStackedBarYRange(h,i,t,"__barchartRight","right")}},s.prototype._updateYAxis=function(t,e){var i,s,o=!1,n=!1,r=!1,a=1e9,h=1e9,d=-1e9,l=-1e9;if(t.length>0){for(var c=0;ci?i:a,d=s>d?s:d):(r=!0,h=h>i?i:h,l=s>l?s:l));1==n&&this.yAxisLeft.setRange(a,d),1==r&&this.yAxisRight.setRange(h,l)}return o=this._toggleAxisVisiblity(n,this.yAxisLeft)||o,o=this._toggleAxisVisiblity(r,this.yAxisRight)||o,1==r&&1==n?(this.yAxisLeft.drawIcons=!0,this.yAxisRight.drawIcons=!0):(this.yAxisLeft.drawIcons=!1,this.yAxisRight.drawIcons=!1),this.yAxisRight.master=!n,0==this.yAxisRight.master?(this.yAxisLeft.lineOffset=1==r?this.yAxisRight.width:0,o=this.yAxisLeft.redraw()||o,this.yAxisRight.stepPixelsForced=this.yAxisLeft.stepPixels,this.yAxisRight.zeroCrossing=this.yAxisLeft.zeroCrossing,o=this.yAxisRight.redraw()||o):o=this.yAxisRight.redraw()||o,-1!=t.indexOf("__barchartLeft")&&t.splice(t.indexOf("__barchartLeft"),1),-1!=t.indexOf("__barchartRight")&&t.splice(t.indexOf("__barchartRight"),1),o},s.prototype._toggleAxisVisiblity=function(t,e){var i=!1;return 0==t?e.dom.frame.parentNode&&0==e.hidden&&(e.hide(),i=!0):e.dom.frame.parentNode||1!=e.hidden||(e.show(),i=!0),i},s.prototype._convertXcoordinates=function(t){for(var e,i,s=[],o=this.body.util.toScreen,n=0;ny;)y++,l=h.getCurrent(),c=h.isMajor(),u=h.getClassName(),f=m,m=this.body.util.toScreen(l),g=m-f,p&&(p.style.width=g+"px"),this.options.showMinorLabels&&this._repaintMinorText(m,h.getLabelMinor(),t,u),c&&this.options.showMajorLabels?(m>0&&(void 0==v&&(v=m),this._repaintMajorText(m,h.getLabelMajor(),t,u)),p=this._repaintMajorLine(m,t,u)):p=this._repaintMinorLine(m,t,u),h.next();if(this.options.showMajorLabels){var b=this.body.util.toTime(0),_=h.getLabelMajor(b),x=_.length*(this.props.majorCharWidth||10)+10;(void 0==v||v>x)&&this._repaintMajorText(0,_,t,u)}o.forEach(this.dom.redundant,function(t){for(;t.length;){var e=t.pop();e&&e.parentNode&&e.parentNode.removeChild(e)}})},s.prototype._repaintMinorText=function(t,e,i,s){var o=this.dom.redundant.minorTexts.shift();if(!o){var n=document.createTextNode("");o=document.createElement("div"),o.appendChild(n),this.dom.foreground.appendChild(o)}this.dom.minorTexts.push(o),o.childNodes[0].nodeValue=e,o.style.top="top"==i?this.props.majorLabelHeight+"px":"0",o.style.left=t+"px",o.className="text minor "+s},s.prototype._repaintMajorText=function(t,e,i,s){var o=this.dom.redundant.majorTexts.shift();if(!o){var n=document.createTextNode(e);o=document.createElement("div"),o.appendChild(n),this.dom.foreground.appendChild(o)}this.dom.majorTexts.push(o),o.childNodes[0].nodeValue=e,o.className="text major "+s,o.style.top="top"==i?"0":this.props.minorLabelHeight+"px",o.style.left=t+"px"},s.prototype._repaintMinorLine=function(t,e,i){var s=this.dom.redundant.lines.shift();s||(s=document.createElement("div"),this.dom.background.appendChild(s)),this.dom.lines.push(s);var o=this.props;return s.style.top="top"==e?o.majorLabelHeight+"px":this.body.domProps.top.height+"px",s.style.height=o.minorLineHeight+"px",s.style.left=t-o.minorLineWidth/2+"px",s.className="grid vertical minor "+i,s},s.prototype._repaintMajorLine=function(t,e,i){var s=this.dom.redundant.lines.shift();s||(s=document.createElement("div"),this.dom.background.appendChild(s)),this.dom.lines.push(s);var o=this.props;return s.style.top="top"==e?"0":this.body.domProps.top.height+"px",s.style.left=t-o.majorLineWidth/2+"px",s.style.height=o.majorLineHeight+"px",s.className="grid vertical major "+i,s},s.prototype._calculateCharSize=function(){this.dom.measureCharMinor||(this.dom.measureCharMinor=document.createElement("DIV"),this.dom.measureCharMinor.className="text minor measure",this.dom.measureCharMinor.style.position="absolute",this.dom.measureCharMinor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMinor)),this.props.minorCharHeight=this.dom.measureCharMinor.clientHeight,this.props.minorCharWidth=this.dom.measureCharMinor.clientWidth,this.dom.measureCharMajor||(this.dom.measureCharMajor=document.createElement("DIV"),this.dom.measureCharMajor.className="text major measure",this.dom.measureCharMajor.style.position="absolute",this.dom.measureCharMajor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMajor)),this.props.majorCharHeight=this.dom.measureCharMajor.clientHeight,this.props.majorCharWidth=this.dom.measureCharMajor.clientWidth},s.prototype.snap=function(t){return this.step.snap(t)},t.exports=s},function(t,e,i){function s(t,e,i){this.id=null,this.parent=null,this.data=t,this.dom=null,this.conversion=e||{},this.options=i||{},this.selected=!1,this.displayed=!1,this.dirty=!0,this.top=null,this.left=null,this.width=null,this.height=null}var o=i(45),n=i(1);s.prototype.stack=!0,s.prototype.select=function(){this.selected=!0,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.unselect=function(){this.selected=!1,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.setData=function(t){this.data=t,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.setParent=function(t){this.displayed?(this.hide(),this.parent=t,this.parent&&this.show()):this.parent=t},s.prototype.isVisible=function(){return!1},s.prototype.show=function(){return!1},s.prototype.hide=function(){return!1},s.prototype.redraw=function(){},s.prototype.repositionX=function(){},s.prototype.repositionY=function(){},s.prototype._repaintDeleteButton=function(t){if(this.selected&&this.options.editable.remove&&!this.dom.deleteButton){var e=this,i=document.createElement("div");i.className="delete",i.title="Delete this item",new o(i).on("tap",function(t){e.parent.removeFromDataSet(e),t.stopPropagation(),t.preventDefault()}),t.appendChild(i),this.dom.deleteButton=i}else!this.selected&&this.dom.deleteButton&&(this.dom.deleteButton.parentNode&&this.dom.deleteButton.parentNode.removeChild(this.dom.deleteButton),this.dom.deleteButton=null)},s.prototype._updateContents=function(t){var e;if(this.options.template){var i=this.parent.itemSet.itemsData.get(this.id);e=this.options.template(i)}else e=this.data.content;if(e!==this.content){if(e instanceof Element)t.innerHTML="",t.appendChild(e);else if(void 0!=e)t.innerHTML=e;else if("background"!=this.data.type||void 0!==this.data.content)throw new Error('Property "content" missing in item '+this.id);this.content=e}},s.prototype._updateTitle=function(t){null!=this.data.title?t.title=this.data.title||"":t.removeAttribute("title")},s.prototype._updateDataAttributes=function(t){if(this.options.dataAttributes&&this.options.dataAttributes.length>0){var e=[];if(Array.isArray(this.options.dataAttributes))e=this.options.dataAttributes;else{if("all"!=this.options.dataAttributes)return;e=Object.keys(this.data)}for(var i=0;it.start},s.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.content=document.createElement("div"),t.content.className="content",t.box.appendChild(t.content),this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.background;if(!e)throw new Error("Cannot redraw item: parent has no background container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.content),this._updateDataAttributes(this.dom.content),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.props.content.width=this.dom.content.offsetWidth,this.height=0,this.dirty=!1}},s.prototype.show=r.prototype.show,s.prototype.hide=r.prototype.hide,s.prototype.repositionX=r.prototype.repositionX,s.prototype.repositionY=function(t){var e="top"===this.options.orientation;this.dom.content.style.top=e?"":"0",this.dom.content.style.bottom=e?"0":"";var i;if(void 0!==this.data.subgroup){var s=this.data.subgroup,o=this.parent.subgroups,r=o[s].index;if(1==e){i=this.parent.subgroups[s].height+t.item.vertical,i+=0==r?t.axis-.5*t.item.vertical:0;var a=this.parent.top;for(var h in o)o.hasOwnProperty(h)&&1==o[h].visible&&o[h].indexr&&(a+=o[h].height+t.item.vertical);i=this.parent.subgroups[s].height+t.item.vertical,this.dom.box.style.top=a+"px",this.dom.box.style.bottom=""}}else this.parent instanceof n?(i=Math.max(this.parent.height,this.parent.itemSet.body.domProps.center.height,this.parent.itemSet.body.domProps.centerContainer.height),this.dom.box.style.top=e?"0":"",this.dom.box.style.bottom=e?"":"0"):(i=this.parent.height,this.dom.box.style.top=this.parent.top+"px",this.dom.box.style.bottom="");this.dom.box.style.height=i+"px"},t.exports=s},function(t,e,i){function s(t,e,i){if(this.props={dot:{width:0,height:0},line:{width:0,height:0}},t&&void 0==t.start)throw new Error('Property "start" missing in item '+t);o.call(this,t,e,i)}{var o=i(31);i(1)}s.prototype=new o(null,null,null),s.prototype.isVisible=function(t){var e=(t.end-t.start)/4;return this.data.start>t.start-e&&this.data.startt.start-e&&this.data.startt.start},s.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.content=document.createElement("div"),t.content.className="content",t.box.appendChild(t.content),t.box["timeline-item"]=this,this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.foreground;if(!e)throw new Error("Cannot redraw item: parent has no foreground container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.box),this._updateDataAttributes(this.dom.box),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.dom.content.style.maxWidth="none",this.props.content.width=this.dom.content.offsetWidth,this.height=this.dom.box.offsetHeight,this.dom.content.style.maxWidth="",this.dirty=!1}this._repaintDeleteButton(t.box),this._repaintDragLeft(),this._repaintDragRight()},s.prototype.show=function(){this.displayed||this.redraw()},s.prototype.hide=function(){if(this.displayed){var t=this.dom.box;t.parentNode&&t.parentNode.removeChild(t),this.top=null,this.left=null,this.displayed=!1}},s.prototype.repositionX=function(){var t,e,i=this.parent.width,s=this.conversion.toScreen(this.data.start),o=this.conversion.toScreen(this.data.end);-i>s&&(s=-i),o>2*i&&(o=2*i);var n=Math.max(o-s,1);switch(this.overflow?(this.left=s,this.width=n+this.props.content.width,e=this.props.content.width):(this.left=s,this.width=n,e=Math.min(o-s-2*this.options.padding,this.props.content.width)),this.dom.box.style.left=this.left+"px",this.dom.box.style.width=n+"px",this.options.align){case"left":this.dom.content.style.left="0";break;case"right":this.dom.content.style.left=Math.max(n-e-2*this.options.padding,0)+"px";break;case"center":this.dom.content.style.left=Math.max((n-e-2*this.options.padding)/2,0)+"px";break;default:t=this.overflow?o>0?Math.max(-s,0):-e:0>s?Math.min(-s,o-s-e-2*this.options.padding):0,this.dom.content.style.left=t+"px"}},s.prototype.repositionY=function(){var t=this.options.orientation,e=this.dom.box;e.style.top="top"==t?this.top+"px":this.parent.height-this.top-this.height+"px"},s.prototype._repaintDragLeft=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragLeft){var t=document.createElement("div");t.className="drag-left",t.dragLeftItem=this,this.dom.box.appendChild(t),this.dom.dragLeft=t}else!this.selected&&this.dom.dragLeft&&(this.dom.dragLeft.parentNode&&this.dom.dragLeft.parentNode.removeChild(this.dom.dragLeft),this.dom.dragLeft=null)},s.prototype._repaintDragRight=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragRight){var t=document.createElement("div");t.className="drag-right",t.dragRightItem=this,this.dom.box.appendChild(t),this.dom.dragRight=t}else!this.selected&&this.dom.dragRight&&(this.dom.dragRight.parentNode&&this.dom.dragRight.parentNode.removeChild(this.dom.dragRight),this.dom.dragRight=null)},t.exports=s},function(t,e,i){function s(t,e,i){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");this._determineBrowserMethod(),this._initializeMixinLoaders(),this.containerElement=t,this.renderRefreshRate=60,this.renderTimestep=1e3/this.renderRefreshRate,this.renderTime=0,this.physicsTime=0,this.runDoubleSpeed=!1,this.physicsDiscreteStepsize=.5,this.initializing=!0,this.triggerFunctions={add:null,edit:null,editEdge:null,connect:null,del:null},this.defaultOptions={nodes:{mass:1,radiusMin:10,radiusMax:30,radius:10,shape:"ellipse",image:void 0,widthMin:16,widthMax:64,fontColor:"black",fontSize:14,fontFace:"verdana",fontFill:void 0,fontStrokeWidth:0,fontStrokeColor:"white",level:-1,color:{border:"#2B7CE9",background:"#97C2FC",highlight:{border:"#2B7CE9",background:"#D2E5FF"},hover:{border:"#2B7CE9",background:"#D2E5FF"}},group:void 0,borderWidth:1,borderWidthSelected:void 0},edges:{widthMin:1,widthMax:15,width:1,widthSelectionMultiplier:2,hoverWidth:1.5,style:"line",color:{color:"#848484",highlight:"#848484",hover:"#848484"},fontColor:"#343434",fontSize:14,fontFace:"arial",fontFill:"white",fontStrokeWidth:0,fontStrokeColor:"white",labelAlignment:"horizontal",arrowScaleFactor:1,dash:{length:10,gap:5,altLength:void 0},inheritColor:"from"},configurePhysics:!1,physics:{barnesHut:{enabled:!0,thetaInverted:2,gravitationalConstant:-2e3,centralGravity:.3,springLength:95,springConstant:.04,damping:.09},repulsion:{centralGravity:0,springLength:200,springConstant:.05,nodeDistance:100,damping:.09},hierarchicalRepulsion:{enabled:!1,centralGravity:0,springLength:100,springConstant:.01,nodeDistance:150,damping:.09},damping:null,centralGravity:null,springLength:null,springConstant:null},clustering:{enabled:!1,initialMaxNodes:100,clusterThreshold:500,reduceToNodes:300,chainThreshold:.4,clusterEdgeThreshold:20,sectorThreshold:100,screenSizeThreshold:.2,fontSizeMultiplier:4,maxFontSize:1e3,forceAmplification:.1,distanceAmplification:.1,edgeGrowth:20,nodeScaling:{width:1,height:1,radius:1},maxNodeSizeIncrements:600,activeAreaBoxSize:80,clusterLevelDifference:2},navigation:{enabled:!1},keyboard:{enabled:!1,speed:{x:10,y:10,zoom:.02}},dataManipulation:{enabled:!1,initiallyVisible:!1},hierarchicalLayout:{enabled:!1,levelSeparation:150,nodeSpacing:100,direction:"UD",layout:"hubsize"},freezeForStabilization:!1,smoothCurves:{enabled:!0,dynamic:!0,type:"continuous",roundness:.5},maxVelocity:30,minVelocity:.1,stabilize:!0,stabilizationIterations:1e3,zoomExtentOnStabilize:!0,locale:"en",locales:_,tooltip:{delay:300,fontColor:"black",fontSize:14,fontFace:"verdana",color:{border:"#666",background:"#FFFFC6"}},dragNetwork:!0,dragNodes:!0,zoomable:!0,hover:!1,hideEdgesOnDrag:!1,hideNodesOnDrag:!1,width:"100%",height:"100%",selectable:!0},this.constants=a.extend({},this.defaultOptions),this.pixelRatio=1,this.hoverObj={nodes:{},edges:{}},this.controlNodesActive=!1,this.navigationHammers={existing:[],_new:[]},this.animationSpeed=1/this.renderRefreshRate,this.animationEasingFunction="easeInOutQuint",this.easingTime=0,this.sourceScale=0,this.targetScale=0,this.sourceTranslation=0,this.targetTranslation=0,this.lockedOnNodeId=null,this.lockedOnNodeOffset=null,this.touchTime=0;var o=this;this.groups=new u,this.images=new m,this.images.setOnloadCallback(function(){o._redraw()}),this.xIncrement=0,this.yIncrement=0,this.zoomIncrement=0,this._loadPhysicsSystem(),this._create(),this._loadSectorSystem(),this._loadClusterSystem(),this._loadSelectionSystem(),this._loadHierarchySystem(),this._setTranslation(this.frame.clientWidth/2,this.frame.clientHeight/2),this._setScale(1),this.setOptions(i),this.freezeSimulation=!1,this.cachedFunctions={},this.startedStabilization=!1,this.stabilized=!1,this.stabilizationIterations=null,this.draggingNodes=!1,this.calculationNodes={},this.calculationNodeIndices=[],this.nodeIndices=[],this.nodes={},this.edges={},this.canvasTopLeft={x:0,y:0},this.canvasBottomRight={x:0,y:0},this.pointerPosition={x:0,y:0},this.areaCenter={},this.scale=1,this.previousScale=this.scale,this.nodesData=null,this.edgesData=null,this.nodesListeners={add:function(t,e){o._addNodes(e.items),o.start()},update:function(t,e){o._updateNodes(e.items,e.data),o.start()},remove:function(t,e){o._removeNodes(e.items),o.start()}},this.edgesListeners={add:function(t,e){o._addEdges(e.items),o.start()},update:function(t,e){o._updateEdges(e.items),o.start()},remove:function(t,e){o._removeEdges(e.items),o.start()}},this.moving=!0,this.timer=void 0,this.setData(e,this.constants.clustering.enabled||this.constants.hierarchicalLayout.enabled),this.initializing=!1,1==this.constants.hierarchicalLayout.enabled?this._setupHierarchicalLayout():0==this.constants.stabilize&&this.zoomExtent(void 0,!0,this.constants.clustering.enabled),this.constants.clustering.enabled&&this.startWithClustering()}var o=i(56),n=i(45),r=i(58),a=i(1),h=i(47),d=i(3),l=i(4),c=i(42),p=i(43),u=i(38),m=i(39),f=i(40),g=i(37),v=i(41),y=i(54),b=i(55),_=i(49);i(50),o(s.prototype),s.prototype._determineBrowserMethod=function(){var t=navigator.userAgent.toLowerCase();this.requiresTimeout=!1,-1!=t.indexOf("msie 9.0")?this.requiresTimeout=!0:-1!=t.indexOf("safari")&&t.indexOf("chrome")<=-1&&(this.requiresTimeout=!0)},s.prototype._getScriptPath=function(){for(var t=document.getElementsByTagName("script"),e=0;et.boundingBox.left&&(s=t.boundingBox.left),ot.boundingBox.bottom&&(e=t.boundingBox.bottom),i=this.constants.clustering.initialMaxNodes?49.07548/(n+142.05338)+91444e-8:12.662/(n+7.4147)+.0964822:1==this.constants.clustering.enabled&&n>=this.constants.clustering.initialMaxNodes?77.5271985/(n+187.266146)+476710517e-13:30.5062972/(n+19.93597763)+.08413486;var r=Math.min(this.frame.canvas.clientWidth/600,this.frame.canvas.clientHeight/600);s*=r}else{var a=1.1*Math.abs(o.maxX-o.minX),h=1.1*Math.abs(o.maxY-o.minY),d=this.frame.canvas.clientWidth/a,l=this.frame.canvas.clientHeight/h;s=l>=d?d:l}s>1&&(s=1);var c=this._findCenter(o);if(0==i){var p={position:c,scale:s,animation:t};this.moveTo(p),this.moving=!0,this.start()}else c.x*=s,c.y*=s,c.x-=.5*this.frame.canvas.clientWidth,c.y-=.5*this.frame.canvas.clientHeight,this._setScale(s),this._setTranslation(-c.x,-c.y)},s.prototype._updateNodeIndexList=function(){this._clearNodeIndexList();for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&this.nodeIndices.push(t)},s.prototype.setData=function(t,e){if(void 0===e&&(e=!1),this.initializing=!0,t&&t.dot&&(t.nodes||t.edges))throw new SyntaxError('Data must contain either parameter "dot" or parameter pair "nodes" and "edges", but not both.');if(1==this.constants.dataManipulation.enabled&&this._createManipulatorBar(),this.setOptions(t&&t.options),t&&t.dot){if(t&&t.dot){var i=c.DOTToGraph(t.dot);return void this.setData(i)}}else if(t&&t.gephi){if(t&&t.gephi){var s=p.parseGephi(t.gephi);return void this.setData(s)}}else this._setNodes(t&&t.nodes),this._setEdges(t&&t.edges);this._putDataInSector(),0==e&&(1==this.constants.hierarchicalLayout.enabled?(this._resetLevels(),this._setupHierarchicalLayout()):this.constants.stabilize&&this._stabilize(),this.start()),this.initializing=!1},s.prototype.setOptions=function(t){if(t){var e,i=["nodes","edges","smoothCurves","hierarchicalLayout","clustering","navigation","keyboard","dataManipulation","onAdd","onEdit","onEditEdge","onConnect","onDelete","clickToUse"];if(a.selectiveNotDeepExtend(i,this.constants,t),a.selectiveNotDeepExtend(["color"],this.constants.nodes,t.nodes),a.selectiveNotDeepExtend(["color","length"],this.constants.edges,t.edges),t.physics&&(a.mergeOptions(this.constants.physics,t.physics,"barnesHut"),a.mergeOptions(this.constants.physics,t.physics,"repulsion"),t.physics.hierarchicalRepulsion)){this.constants.hierarchicalLayout.enabled=!0,this.constants.physics.hierarchicalRepulsion.enabled=!0,this.constants.physics.barnesHut.enabled=!1;for(e in t.physics.hierarchicalRepulsion)t.physics.hierarchicalRepulsion.hasOwnProperty(e)&&(this.constants.physics.hierarchicalRepulsion[e]=t.physics.hierarchicalRepulsion[e]) -}if(t.onAdd&&(this.triggerFunctions.add=t.onAdd),t.onEdit&&(this.triggerFunctions.edit=t.onEdit),t.onEditEdge&&(this.triggerFunctions.editEdge=t.onEditEdge),t.onConnect&&(this.triggerFunctions.connect=t.onConnect),t.onDelete&&(this.triggerFunctions.del=t.onDelete),a.mergeOptions(this.constants,t,"smoothCurves"),a.mergeOptions(this.constants,t,"hierarchicalLayout"),a.mergeOptions(this.constants,t,"clustering"),a.mergeOptions(this.constants,t,"navigation"),a.mergeOptions(this.constants,t,"keyboard"),a.mergeOptions(this.constants,t,"dataManipulation"),t.dataManipulation&&(this.editMode=this.constants.dataManipulation.initiallyVisible),t.edges&&(void 0!==t.edges.color&&(a.isString(t.edges.color)?(this.constants.edges.color={},this.constants.edges.color.color=t.edges.color,this.constants.edges.color.highlight=t.edges.color,this.constants.edges.color.hover=t.edges.color):(void 0!==t.edges.color.color&&(this.constants.edges.color.color=t.edges.color.color),void 0!==t.edges.color.highlight&&(this.constants.edges.color.highlight=t.edges.color.highlight),void 0!==t.edges.color.hover&&(this.constants.edges.color.hover=t.edges.color.hover)),this.constants.edges.inheritColor=!1),t.edges.fontColor||void 0!==t.edges.color&&(a.isString(t.edges.color)?this.constants.edges.fontColor=t.edges.color:void 0!==t.edges.color.color&&(this.constants.edges.fontColor=t.edges.color.color))),t.nodes&&t.nodes.color){var s=a.parseColor(t.nodes.color);this.constants.nodes.color.background=s.background,this.constants.nodes.color.border=s.border,this.constants.nodes.color.highlight.background=s.highlight.background,this.constants.nodes.color.highlight.border=s.highlight.border,this.constants.nodes.color.hover.background=s.hover.background,this.constants.nodes.color.hover.border=s.hover.border}if(t.groups)for(var o in t.groups)if(t.groups.hasOwnProperty(o)){var n=t.groups[o];this.groups.add(o,n)}if(t.tooltip){for(e in t.tooltip)t.tooltip.hasOwnProperty(e)&&(this.constants.tooltip[e]=t.tooltip[e]);t.tooltip.color&&(this.constants.tooltip.color=a.parseColor(t.tooltip.color))}if("clickToUse"in t&&(t.clickToUse?this.activator||(this.activator=new b(this.frame),this.activator.on("change",this._createKeyBinds.bind(this))):this.activator&&(this.activator.destroy(),delete this.activator)),t.labels)throw new Error('Option "labels" is deprecated. Use options "locale" and "locales" instead.');this._loadPhysicsSystem(),this._loadNavigationControls(),this._loadManipulationSystem(),this._configureSmoothCurves(),this._createKeyBinds(),this.setSize(this.constants.width,this.constants.height),this.moving=!0,this.start()}},s.prototype._create=function(){for(;this.containerElement.hasChildNodes();)this.containerElement.removeChild(this.containerElement.firstChild);if(this.frame=document.createElement("div"),this.frame.className="vis network-frame",this.frame.style.position="relative",this.frame.style.overflow="hidden",this.frame.canvas=document.createElement("canvas"),this.frame.canvas.style.position="relative",this.frame.appendChild(this.frame.canvas),this.frame.canvas.getContext){var t=this.frame.canvas.getContext("2d");this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1),this.frame.canvas.getContext("2d").setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0)}else{var e=document.createElement("DIV");e.style.color="red",e.style.fontWeight="bold",e.style.padding="10px",e.innerHTML="Error: your browser does not support HTML canvas",this.frame.canvas.appendChild(e)}var i=this;this.drag={},this.pinch={},this.hammer=new n(this.frame.canvas),this.hammer.get("pinch").set({enable:!0}),this.hammer.on("tap",i._onTap.bind(i)),this.hammer.on("doubletap",i._onDoubleTap.bind(i)),this.hammer.on("press",i._onHold.bind(i)),this.hammer.on("pinch",i._onPinch.bind(i)),h.onTouch(this.hammer,i._onTouch.bind(i)),this.hammer.on("panstart",i._onDragStart.bind(i)),this.hammer.on("panmove",i._onDrag.bind(i)),this.hammer.on("panend",i._onDragEnd.bind(i)),this.frame.canvas.addEventListener("mousemove",i._onMouseMoveTitle.bind(i)),this.frame.canvas.addEventListener("mousewheel",i._onMouseWheel.bind(i)),this.frame.canvas.addEventListener("DOMMouseScroll",i._onMouseWheel.bind(i)),this.containerElement.appendChild(this.frame)},s.prototype._createKeyBinds=function(){var t=this;void 0!==this.keycharm&&this.keycharm.destroy(),this.keycharm=r(),this.keycharm.reset(),this.constants.keyboard.enabled&&this.isActive()&&(this.keycharm.bind("up",this._moveUp.bind(t),"keydown"),this.keycharm.bind("up",this._yStopMoving.bind(t),"keyup"),this.keycharm.bind("down",this._moveDown.bind(t),"keydown"),this.keycharm.bind("down",this._yStopMoving.bind(t),"keyup"),this.keycharm.bind("left",this._moveLeft.bind(t),"keydown"),this.keycharm.bind("left",this._xStopMoving.bind(t),"keyup"),this.keycharm.bind("right",this._moveRight.bind(t),"keydown"),this.keycharm.bind("right",this._xStopMoving.bind(t),"keyup"),this.keycharm.bind("=",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("=",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("num+",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("num+",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("num-",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("num-",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("-",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("-",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("[",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("[",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("]",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("]",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("pageup",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("pageup",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("pagedown",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("pagedown",this._stopZoom.bind(t),"keyup")),1==this.constants.dataManipulation.enabled&&(this.keycharm.bind("esc",this._createManipulatorBar.bind(t)),this.keycharm.bind("delete",this._deleteSelected.bind(t)))},s.prototype.destroy=function(){this.start=function(){},this.redraw=function(){},this.timer=!1,this._cleanupPhysicsConfiguration(),this.keycharm.reset(),this.hammer.destroy(),this.off(),this._recursiveDOMDelete(this.containerElement)},s.prototype._recursiveDOMDelete=function(t){for(;1==t.hasChildNodes();)this._recursiveDOMDelete(t.firstChild),t.removeChild(t.firstChild)},s.prototype._getPointer=function(t){return{x:t.x-a.getAbsoluteLeft(this.frame.canvas),y:t.y-a.getAbsoluteTop(this.frame.canvas)}},s.prototype._onTouch=function(t){(new Date).valueOf()-this.touchTime>100&&(this.drag.pointer=this._getPointer(t.center),this.drag.pinched=!1,this.pinch.scale=this._getScale(),this.touchTime=(new Date).valueOf(),this._handleTouch(this.drag.pointer))},s.prototype._onDragStart=function(t){this._handleDragStart(t)},s.prototype._handleDragStart=function(t){void 0===this.drag.pointer&&this._onTouch(t);var e=this._getNodeAt(this.drag.pointer);if(this.drag.dragging=!0,this.drag.selection=[],this.drag.translation=this._getTranslation(),this.drag.nodeId=null,this.draggingNodes=!1,null!=e&&1==this.constants.dragNodes){this.draggingNodes=!0,this.drag.nodeId=e.id,e.isSelected()||this._selectObject(e,!1),this.emit("dragStart",{nodeIds:this.getSelection().nodes});for(var i in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(i)){var s=this.selectionObj.nodes[i],o={id:s.id,node:s,x:s.x,y:s.y,xFixed:s.xFixed,yFixed:s.yFixed};s.xFixed=!0,s.yFixed=!0,this.drag.selection.push(o)}}t.preventDefault()},s.prototype._onDrag=function(t){this._handleOnDrag(t)},s.prototype._handleOnDrag=function(t){if(!this.drag.pinched){this.releaseNode();var e=this._getPointer(t.center),i=this,s=this.drag,o=s.selection;if(o&&o.length&&1==this.constants.dragNodes){var n=e.x-s.pointer.x,r=e.y-s.pointer.y;o.forEach(function(t){var e=t.node;t.xFixed||(e.x=i._XconvertDOMtoCanvas(i._XconvertCanvasToDOM(t.x)+n)),t.yFixed||(e.y=i._YconvertDOMtoCanvas(i._YconvertCanvasToDOM(t.y)+r))}),this.moving||(this.moving=!0,this.start())}else if(1==this.constants.dragNetwork){if(void 0===this.drag.pointer)return void this._handleDragStart(t);var a=e.x-this.drag.pointer.x,h=e.y-this.drag.pointer.y;this._setTranslation(this.drag.translation.x+a,this.drag.translation.y+h),this._redraw()}t.preventDefault()}},s.prototype._onDragEnd=function(t){this._handleDragEnd(t)},s.prototype._handleDragEnd=function(t){this.drag.dragging=!1;var e=this.drag.selection;e&&e.length?(e.forEach(function(t){t.node.xFixed=t.xFixed,t.node.yFixed=t.yFixed}),this.moving=!0,this.start()):this._redraw(),0==this.draggingNodes?this.emit("dragEnd",{nodeIds:[]}):this.emit("dragEnd",{nodeIds:this.getSelection().nodes}),t.preventDefault()},s.prototype._onTap=function(t){var e=this._getPointer(t.center);this.pointerPosition=e,this._handleTap(e)},s.prototype._onDoubleTap=function(t){var e=this._getPointer(t.center);this._handleDoubleTap(e)},s.prototype._onHold=function(t){var e=this._getPointer(t.center);this.pointerPosition=e,this._handleOnHold(e)},s.prototype._onRelease=function(t){var e=this._getPointer(t.center);this._handleOnRelease(e)},s.prototype._onPinch=function(t){var e=this._getPointer(t.center);this.drag.pinched=!0,"scale"in this.pinch||(this.pinch.scale=1);var i=this.pinch.scale*t.scale;this._zoom(i,e)},s.prototype._zoom=function(t,e){if(1==this.constants.zoomable){var i=this._getScale();1e-5>t&&(t=1e-5),t>10&&(t=10);var s=null;void 0!==this.drag&&1==this.drag.dragging&&(s=this.DOMtoCanvas(this.drag.pointer));var o=this._getTranslation(),n=t/i,r=(1-n)*e.x+o.x*n,a=(1-n)*e.y+o.y*n;if(this.areaCenter={x:this._XconvertDOMtoCanvas(e.x),y:this._YconvertDOMtoCanvas(e.y)},this._setScale(t),this._setTranslation(r,a),this.updateClustersDefault(),null!=s){var h=this.canvasToDOM(s);this.drag.pointer.x=h.x,this.drag.pointer.y=h.y}return this._redraw(),t>i?this.emit("zoom",{direction:"+"}):this.emit("zoom",{direction:"-"}),t}},s.prototype._onMouseWheel=function(t){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i=this._getScale(),s=e/10;0>e&&(s/=1-s),i*=1+s;var o=this._getPointer({x:t.pageX,y:t.pageY});this._zoom(i,o)}t.preventDefault()},s.prototype._onMouseMoveTitle=function(t){var e=this._getPointer({x:t.pageX,y:t.pageY});this.popupObj&&this._checkHidePopup(e);var i=this,s=function(){i._checkShowPopup(e)};if(this.popupTimer&&clearInterval(this.popupTimer),this.drag.dragging||(this.popupTimer=setTimeout(s,this.constants.tooltip.delay)),1==this.constants.hover){for(var o in this.hoverObj.edges)this.hoverObj.edges.hasOwnProperty(o)&&(this.hoverObj.edges[o].hover=!1,delete this.hoverObj.edges[o]);var n=this._getNodeAt(e);null==n&&(n=this._getEdgeAt(e)),null!=n&&this._hoverObject(n);for(var r in this.hoverObj.nodes)this.hoverObj.nodes.hasOwnProperty(r)&&(n instanceof f&&n.id!=r||n instanceof g||null==n)&&(this._blurObject(this.hoverObj.nodes[r]),delete this.hoverObj.nodes[r]);this.redraw()}},s.prototype._checkShowPopup=function(t){var e,i={left:this._XconvertDOMtoCanvas(t.x),top:this._YconvertDOMtoCanvas(t.y),right:this._XconvertDOMtoCanvas(t.x),bottom:this._YconvertDOMtoCanvas(t.y)},s=this.popupObj,o=!1;if(void 0==this.popupObj){var n=this.nodes,r=[];for(e in n)if(n.hasOwnProperty(e)){var a=n[e];a.isOverlappingWith(i)&&void 0!==a.getTitle()&&r.push(e)}r.length>0&&(this.popupObj=this.nodes[r[r.length-1]],o=!0)}if(void 0===this.popupObj&&0==o){var h=this.edges,d=[];for(e in h)if(h.hasOwnProperty(e)){var l=h[e];l.connected&&void 0!==l.getTitle()&&l.isOverlappingWith(i)&&d.push(e)}d.length>0&&(this.popupObj=this.edges[d[d.length-1]])}if(this.popupObj){if(this.popupObj!=s){var c=this;c.popup||(c.popup=new v(c.frame,c.constants.tooltip)),c.popup.setPosition(t.x-3,t.y-3),c.popup.setText(c.popupObj.getTitle()),c.popup.show()}}else this.popup&&this.popup.hide()},s.prototype._checkHidePopup=function(t){this.popupObj&&this._getNodeAt(t)||(this.popupObj=void 0,this.popup&&this.popup.hide())},s.prototype.setSize=function(t,e){var i=!1,s=this.frame.canvas.width,o=this.frame.canvas.height;t!=this.constants.width||e!=this.constants.height||this.frame.style.width!=t||this.frame.style.height!=e?(this.frame.style.width=t,this.frame.style.height=e,this.frame.canvas.style.width="100%",this.frame.canvas.style.height="100%",this.frame.canvas.width=this.frame.canvas.clientWidth*this.pixelRatio,this.frame.canvas.height=this.frame.canvas.clientHeight*this.pixelRatio,this.constants.width=t,this.constants.height=e,i=!0):(this.frame.canvas.width!=this.frame.canvas.clientWidth*this.pixelRatio&&(this.frame.canvas.width=this.frame.canvas.clientWidth*this.pixelRatio,i=!0),this.frame.canvas.height!=this.frame.canvas.clientHeight*this.pixelRatio&&(this.frame.canvas.height=this.frame.canvas.clientHeight*this.pixelRatio,i=!0)),1==i&&this.emit("resize",{width:this.frame.canvas.width*this.pixelRatio,height:this.frame.canvas.height*this.pixelRatio,oldWidth:s*this.pixelRatio,oldHeight:o*this.pixelRatio})},s.prototype._setNodes=function(t){var e=this.nodesData;if(t instanceof d||t instanceof l)this.nodesData=t;else if(Array.isArray(t))this.nodesData=new d,this.nodesData.add(t);else{if(t)throw new TypeError("Array or DataSet expected");this.nodesData=new d}if(e&&a.forEach(this.nodesListeners,function(t,i){e.off(i,t)}),this.nodes={},this.nodesData){var i=this;a.forEach(this.nodesListeners,function(t,e){i.nodesData.on(e,t)});var s=this.nodesData.getIds();this._addNodes(s)}this._updateSelection()},s.prototype._addNodes=function(t){for(var e,i=0,s=t.length;s>i;i++){e=t[i];var o=this.nodesData.get(e),n=new f(o,this.images,this.groups,this.constants);if(this.nodes[e]=n,!(0!=n.xFixed&&0!=n.yFixed||null!==n.x&&null!==n.y)){var r=1*t.length+10,a=2*Math.PI*Math.random();0==n.xFixed&&(n.x=r*Math.cos(a)),0==n.yFixed&&(n.y=r*Math.sin(a))}this.moving=!0}this._updateNodeIndexList(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes(),this._reconnectEdges(),this._updateValueRange(this.nodes),this.updateLabels()},s.prototype._updateNodes=function(t,e){for(var i=this.nodes,s=0,o=t.length;o>s;s++){var n=t[s],r=i[n],a=e[s];r?r.setProperties(a,this.constants):(r=new f(properties,this.images,this.groups,this.constants),i[n]=r)}this.moving=!0,1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateNodeIndexList(),this._updateValueRange(i)},s.prototype._removeNodes=function(t){for(var e=this.nodes,i=0,s=t.length;s>i;i++){var o=t[i];delete e[o]}this._updateNodeIndexList(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes(),this._reconnectEdges(),this._updateSelection(),this._updateValueRange(e)},s.prototype._setEdges=function(t){var e=this.edgesData;if(t instanceof d||t instanceof l)this.edgesData=t;else if(Array.isArray(t))this.edgesData=new d,this.edgesData.add(t);else{if(t)throw new TypeError("Array or DataSet expected");this.edgesData=new d}if(e&&a.forEach(this.edgesListeners,function(t,i){e.off(i,t)}),this.edges={},this.edgesData){var i=this;a.forEach(this.edgesListeners,function(t,e){i.edgesData.on(e,t)});var s=this.edgesData.getIds();this._addEdges(s)}this._reconnectEdges()},s.prototype._addEdges=function(t){for(var e=this.edges,i=this.edgesData,s=0,o=t.length;o>s;s++){var n=t[s],r=e[n];r&&r.disconnect();var a=i.get(n,{showInternalIds:!0});e[n]=new g(a,this,this.constants)}this.moving=!0,this._updateValueRange(e),this._createBezierNodes(),this._updateCalculationNodes(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout())},s.prototype._updateEdges=function(t){for(var e=this.edges,i=this.edgesData,s=0,o=t.length;o>s;s++){var n=t[s],r=i.get(n),a=e[n];a?(a.disconnect(),a.setProperties(r,this.constants),a.connect()):(a=new g(r,this,this.constants),this.edges[n]=a)}this._createBezierNodes(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this.moving=!0,this._updateValueRange(e)},s.prototype._removeEdges=function(t){for(var e=this.edges,i=0,s=t.length;s>i;i++){var o=t[i],n=e[o];n&&(null!=n.via&&delete this.sectors.support.nodes[n.via.id],n.disconnect(),delete e[o])}this.moving=!0,this._updateValueRange(e),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes()},s.prototype._reconnectEdges=function(){var t,e=this.nodes,i=this.edges;for(t in e)e.hasOwnProperty(t)&&(e[t].edges=[],e[t].dynamicEdges=[]);for(t in i)if(i.hasOwnProperty(t)){var s=i[t];s.from=null,s.to=null,s.connect()}},s.prototype._updateValueRange=function(t){var e,i=void 0,s=void 0;for(e in t)if(t.hasOwnProperty(e)){var o=t[e].getValue();void 0!==o&&(i=void 0===i?o:Math.min(o,i),s=void 0===s?o:Math.max(o,s))}if(void 0!==i&&void 0!==s)for(e in t)t.hasOwnProperty(e)&&t[e].setValueRange(i,s)},s.prototype.redraw=function(){this.setSize(this.constants.width,this.constants.height),this._redraw()},s.prototype._redraw=function(t){var e=this.frame.canvas.getContext("2d");e.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0);var i=this.frame.canvas.width*this.pixelRatio,s=this.frame.canvas.height*this.pixelRatio;e.clearRect(0,0,i,s),e.save(),e.translate(this.translation.x,this.translation.y),e.scale(this.scale,this.scale),this.canvasTopLeft={x:this._XconvertDOMtoCanvas(0),y:this._YconvertDOMtoCanvas(0)},this.canvasBottomRight={x:this._XconvertDOMtoCanvas(this.frame.canvas.clientWidth*this.pixelRatio),y:this._YconvertDOMtoCanvas(this.frame.canvas.clientHeight*this.pixelRatio)},1!=t&&(this._doInAllSectors("_drawAllSectorNodes",e),(0==this.drag.dragging||void 0===this.drag.dragging||0==this.constants.hideEdgesOnDrag)&&this._doInAllSectors("_drawEdges",e)),(0==this.drag.dragging||void 0===this.drag.dragging||0==this.constants.hideNodesOnDrag)&&this._doInAllSectors("_drawNodes",e,!1),1!=t&&1==this.controlNodesActive&&this._doInAllSectors("_drawControlNodes",e),e.restore(),1==t&&e.clearRect(0,0,i,s)},s.prototype._setTranslation=function(t,e){void 0===this.translation&&(this.translation={x:0,y:0}),void 0!==t&&(this.translation.x=t),void 0!==e&&(this.translation.y=e),this.emit("viewChanged")},s.prototype._getTranslation=function(){return{x:this.translation.x,y:this.translation.y}},s.prototype._setScale=function(t){this.scale=t},s.prototype._getScale=function(){return this.scale},s.prototype._XconvertDOMtoCanvas=function(t){return(t-this.translation.x)/this.scale},s.prototype._XconvertCanvasToDOM=function(t){return t*this.scale+this.translation.x},s.prototype._YconvertDOMtoCanvas=function(t){return(t-this.translation.y)/this.scale},s.prototype._YconvertCanvasToDOM=function(t){return t*this.scale+this.translation.y},s.prototype.canvasToDOM=function(t){return{x:this._XconvertCanvasToDOM(t.x),y:this._YconvertCanvasToDOM(t.y)}},s.prototype.DOMtoCanvas=function(t){return{x:this._XconvertDOMtoCanvas(t.x),y:this._YconvertDOMtoCanvas(t.y)}},s.prototype._drawNodes=function(t,e){void 0===e&&(e=!1);var i=this.nodes,s=[];for(var o in i)i.hasOwnProperty(o)&&(i[o].setScaleAndPos(this.scale,this.canvasTopLeft,this.canvasBottomRight),i[o].isSelected()?s.push(o):(i[o].inArea()||e)&&i[o].draw(t));for(var n=0,r=s.length;r>n;n++)(i[s[n]].inArea()||e)&&i[s[n]].draw(t)},s.prototype._drawEdges=function(t){var e=this.edges;for(var i in e)if(e.hasOwnProperty(i)){var s=e[i];s.setScale(this.scale),s.connected&&e[i].draw(t)}},s.prototype._drawControlNodes=function(t){var e=this.edges;for(var i in e)e.hasOwnProperty(i)&&e[i]._drawControlNodes(t)},s.prototype._stabilize=function(){1==this.constants.freezeForStabilization&&this._freezeDefinedNodes();for(var t=0;this.moving&&t0)for(t in i)i.hasOwnProperty(t)&&(i[t].discreteStepLimited(e,this.constants.maxVelocity),s=!0);else for(t in i)i.hasOwnProperty(t)&&(i[t].discreteStep(e),s=!0);if(1==s){var o=this.constants.minVelocity/Math.max(this.scale,.05);return o>.5*this.constants.maxVelocity?!0:this._isMoving(o)}return!1},s.prototype._revertPhysicsState=function(){var t=this.nodes;for(var e in t)t.hasOwnProperty(e)&&t[e].revertPosition()},s.prototype._revertPhysicsTick=function(){this._doInAllActiveSectors("_revertPhysicsState"),1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic&&this._doInSupportSector("_revertPhysicsState")},s.prototype._physicsTick=function(){if(!this.freezeSimulation&&1==this.moving){var t=!1,e=!1;this._doInAllActiveSectors("_initializeForceCalculation");var i=this._doInAllActiveSectors("_discreteStepNodes");1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic&&(e=this._doInSupportSector("_discreteStepNodes"));for(var s=0;s2*e||1==this.runDoubleSpeed)&&1==this.moving&&(this._physicsTick(),0!=this.renderTime&&(this.runDoubleSpeed=!0));var i=Date.now();this._redraw(),this.renderTime=Date.now()-i,this.start()},"undefined"!=typeof window&&(window.requestAnimationFrame=window.requestAnimationFrame||window.mozRequestAnimationFrame||window.webkitRequestAnimationFrame||window.msRequestAnimationFrame),s.prototype.start=function(){if(1==this.moving||0!=this.xIncrement||0!=this.yIncrement||0!=this.zoomIncrement)this.timer||(this.timer=1==this.requiresTimeout?window.setTimeout(this._animationStep.bind(this),this.renderTimestep):window.requestAnimationFrame(this._animationStep.bind(this)));else if(this._redraw(),this.stabilizationIterations>1){var t=this,e={iterations:t.stabilizationIterations};this.stabilizationIterations=0,this.startedStabilization=!1,setTimeout(function(){t.emit("stabilized",e)},0)}else this.stabilizationIterations=0},s.prototype._handleNavigation=function(){if(0!=this.xIncrement||0!=this.yIncrement){var t=this._getTranslation();this._setTranslation(t.x+this.xIncrement,t.y+this.yIncrement)}if(0!=this.zoomIncrement){var e={x:this.frame.canvas.clientWidth/2,y:this.frame.canvas.clientHeight/2};this._zoom(this.scale*(1+this.zoomIncrement),e)}},s.prototype.toggleFreeze=function(){0==this.freezeSimulation?this.freezeSimulation=!0:(this.freezeSimulation=!1,this.start())},s.prototype._configureSmoothCurves=function(t){if(void 0===t&&(t=!0),1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic){this._createBezierNodes();for(var e in this.sectors.support.nodes)this.sectors.support.nodes.hasOwnProperty(e)&&void 0===this.edges[this.sectors.support.nodes[e].parentEdgeId]&&delete this.sectors.support.nodes[e]}else{this.sectors.support.nodes={};for(var i in this.edges)this.edges.hasOwnProperty(i)&&(this.edges[i].via=null)}this._updateCalculationNodes(),t||(this.moving=!0,this.start())},s.prototype._createBezierNodes=function(){if(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic)for(var t in this.edges)if(this.edges.hasOwnProperty(t)){var e=this.edges[t];if(null==e.via){var i="edgeId:".concat(e.id);this.sectors.support.nodes[i]=new f({id:i,mass:1,shape:"circle",image:"",internalMultiplier:1},{},{},this.constants),e.via=this.sectors.support.nodes[i],e.via.parentEdgeId=e.id,e.positionBezierNode()}}},s.prototype._initializeMixinLoaders=function(){for(var t in y)y.hasOwnProperty(t)&&(s.prototype[t]=y[t])},s.prototype.storePosition=function(){console.log("storePosition is deprecated: use .storePositions() from now on."),this.storePositions()},s.prototype.storePositions=function(){var t=[];for(var e in this.nodes)if(this.nodes.hasOwnProperty(e)){var i=this.nodes[e],s=!this.nodes.xFixed,o=!this.nodes.yFixed;(this.nodesData._data[e].x!=Math.round(i.x)||this.nodesData._data[e].y!=Math.round(i.y))&&t.push({id:e,x:Math.round(i.x),y:Math.round(i.y),allowedToMoveX:s,allowedToMoveY:o})}this.nodesData.update(t)},s.prototype.getPositions=function(t){var e={};if(void 0!==t){if(1==Array.isArray(t)){for(var i=0;i=1&&(this.easingTime=0,this._redraw=null!=this.lockedOnNodeId?this._lockedRedraw:this._classicRedraw,this.emit("animationFinished"))},s.prototype._classicRedraw=function(){},s.prototype.isActive=function(){return!this.activator||this.activator.active},s.prototype.setScale=function(){return this._setScale()},s.prototype.getScale=function(){return this._getScale()},s.prototype.getCenterCoordinates=function(){return this.DOMtoCanvas({x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight})},s.prototype.getBoundingBox=function(t){return void 0!==this.nodes[t]?this.nodes[t].boundingBox:void 0},t.exports=s},function(t,e,i){function s(t,e,i){if(!e)throw"No network provided";var s=["edges","physics"],n=o.selectiveBridgeObject(s,i);this.options=n.edges,this.physics=n.physics,this.options.smoothCurves=i.smoothCurves,this.network=e,this.id=void 0,this.fromId=void 0,this.toId=void 0,this.title=void 0,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier,this.value=void 0,this.selected=!1,this.hover=!1,this.labelDimensions={top:0,left:0,width:0,height:0,yLine:0},this.dirtyLabel=!0,this.from=null,this.to=null,this.via=null,this.fromBackup=null,this.toBackup=null,this.originalFromId=[],this.originalToId=[],this.connected=!1,this.widthFixed=!1,this.lengthFixed=!1,this.setProperties(t),this.controlNodesEnabled=!1,this.controlNodes={from:null,to:null,positions:{}},this.connectedNode=null}var o=i(1),n=i(40);s.prototype.setProperties=function(t){if(t){var e=["style","fontSize","fontFace","fontColor","fontFill","fontStrokeWidth","fontStrokeColor","width","widthSelectionMultiplier","hoverWidth","arrowScaleFactor","dash","inheritColor","labelAlignment"];switch(o.selectiveDeepExtend(e,this.options,t),void 0!==t.from&&(this.fromId=t.from),void 0!==t.to&&(this.toId=t.to),void 0!==t.id&&(this.id=t.id),void 0!==t.label&&(this.label=t.label,this.dirtyLabel=!0),void 0!==t.title&&(this.title=t.title),void 0!==t.value&&(this.value=t.value),void 0!==t.length&&(this.physics.springLength=t.length),void 0!==t.color&&(this.options.inheritColor=!1,o.isString(t.color)?(this.options.color.color=t.color,this.options.color.highlight=t.color):(void 0!==t.color.color&&(this.options.color.color=t.color.color),void 0!==t.color.highlight&&(this.options.color.highlight=t.color.highlight),void 0!==t.color.hover&&(this.options.color.hover=t.color.hover))),this.connect(),this.widthFixed=this.widthFixed||void 0!==t.width,this.lengthFixed=this.lengthFixed||void 0!==t.length,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier,this.options.style){case"line":this.draw=this._drawLine;break;case"arrow":this.draw=this._drawArrow;break;case"arrow-center":this.draw=this._drawArrowCenter;break;case"dash-line":this.draw=this._drawDashLine;break;default:this.draw=this._drawLine}}},s.prototype.connect=function(){this.disconnect(),this.from=this.network.nodes[this.fromId]||null,this.to=this.network.nodes[this.toId]||null,this.connected=this.from&&this.to,this.connected?(this.from.attachEdge(this),this.to.attachEdge(this)):(this.from&&this.from.detachEdge(this),this.to&&this.to.detachEdge(this))},s.prototype.disconnect=function(){this.from&&(this.from.detachEdge(this),this.from=null),this.to&&(this.to.detachEdge(this),this.to=null),this.connected=!1},s.prototype.getTitle=function(){return"function"==typeof this.title?this.title():this.title -},s.prototype.getValue=function(){return this.value},s.prototype.setValueRange=function(t,e){if(!this.widthFixed&&void 0!==this.value){var i=(this.options.widthMax-this.options.widthMin)/(e-t);this.options.width=(this.value-t)*i+this.options.widthMin,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier}},s.prototype.draw=function(){throw"Method draw not initialized in edge"},s.prototype.isOverlappingWith=function(t){if(this.connected){var e=10,i=this.from.x,s=this.from.y,o=this.to.x,n=this.to.y,r=t.left,a=t.top,h=this._getDistanceToEdge(i,s,o,n,r,a);return e>h}return!1},s.prototype._getColor=function(){var t=this.options.color;return"to"==this.options.inheritColor?t={highlight:this.to.options.color.highlight.border,hover:this.to.options.color.hover.border,color:this.to.options.color.border}:("from"==this.options.inheritColor||1==this.options.inheritColor)&&(t={highlight:this.from.options.color.highlight.border,hover:this.from.options.color.hover.border,color:this.from.options.color.border}),1==this.selected?t.highlight:1==this.hover?t.hover:t.color},s.prototype._drawLine=function(t){if(t.strokeStyle=this._getColor(),t.lineWidth=this._getLineWidth(),this.from!=this.to){var e,i=this._line(t);if(this.label){if(1==this.options.smoothCurves.enabled&&null!=i){var s=.5*(.5*(this.from.x+i.x)+.5*(this.to.x+i.x)),o=.5*(.5*(this.from.y+i.y)+.5*(this.to.y+i.y));e={x:s,y:o}}else e=this._pointOnLine(.5);this._label(t,this.label,e.x,e.y)}}else{var n,r,a=this.physics.springLength/4,h=this.from;h.width||h.resize(t),h.width>h.height?(n=h.x+h.width/2,r=h.y-a):(n=h.x+a,r=h.y-h.height/2),this._circle(t,n,r,a),e=this._pointOnCircle(n,r,a,.5),this._label(t,this.label,e.x,e.y)}},s.prototype._getLineWidth=function(){return 1==this.selected?Math.max(Math.min(this.widthSelected,this.options.widthMax),.3*this.networkScaleInv):1==this.hover?Math.max(Math.min(this.options.hoverWidth,this.options.widthMax),.3*this.networkScaleInv):Math.max(this.options.width,.3*this.networkScaleInv)},s.prototype._getViaCoordinates=function(){if(1==this.options.smoothCurves.dynamic&&1==this.options.smoothCurves.enabled)return this.via;if(0==this.options.smoothCurves.enabled)return{x:0,y:0};var t=null,e=null,i=this.options.smoothCurves.roundness,s=this.options.smoothCurves.type,o=Math.abs(this.from.x-this.to.x),n=Math.abs(this.from.y-this.to.y);return"discrete"==s||"diagonalCross"==s?Math.abs(this.from.x-this.to.x)this.to.y?this.from.xthis.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n)),"discrete"==s&&(t=i*n>o?this.from.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>this.to.y?this.from.xthis.to.x&&(t=this.from.x-i*o,e=this.from.y-i*o):this.from.ythis.to.x&&(t=this.from.x-i*o,e=this.from.y+i*o)),"discrete"==s&&(e=i*o>n?this.from.y:e)):"straightCross"==s?Math.abs(this.from.x-this.to.x)Math.abs(this.from.y-this.to.y)&&(t=this.from.xthis.to.y?this.from.xthis.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n,t=this.to.x>t?this.to.x:t):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n,t=this.to.x>t?this.to.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>this.to.y?this.from.xe?this.to.y:e):this.from.x>this.to.x&&(t=this.from.x-i*o,e=this.from.y-i*o,e=this.to.y>e?this.to.y:e):this.from.ythis.to.x&&(t=this.from.x-i*o,e=this.from.y+i*o,e=this.to.yd;d++){var l=t.measureText(n[d]).width;h=l>h?l:h}var c=this.options.fontSize*r,p=i-h/2,u=s-c/2;this.labelDimensions={top:u,left:p,width:h,height:c,yLine:o}}var o=this.labelDimensions.yLine;t.save(),"horizontal"!=this.options.labelAlignment&&(t.translate(i,o),this._rotateForLabelAlignment(t),i=0,o=0),this._drawLabelRect(t),this._drawLabelText(t,i,o,n,r,a),t.restore()}},s.prototype._rotateForLabelAlignment=function(t){var e=this.from.y-this.to.y,i=this.from.x-this.to.x,s=Math.atan2(e,i);(-1>s&&0>i||s>0&&0>i)&&(s+=Math.PI),t.rotate(s)},s.prototype._drawLabelRect=function(t){if(void 0!==this.options.fontFill&&null!==this.options.fontFill&&"none"!==this.options.fontFill){t.fillStyle=this.options.fontFill;var e=2;"line-center"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,.5*-this.labelDimensions.height,this.labelDimensions.width,this.labelDimensions.height):"line-above"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,-(this.labelDimensions.height+e),this.labelDimensions.width,this.labelDimensions.height):"line-below"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,e,this.labelDimensions.width,this.labelDimensions.height):t.fillRect(this.labelDimensions.left,this.labelDimensions.top,this.labelDimensions.width,this.labelDimensions.height)}},s.prototype._drawLabelText=function(t,e,i,s,o,n){if(t.fillStyle=this.options.fontColor||"black",t.textAlign="center","horizontal"!=this.options.labelAlignment){var r=2;"line-above"==this.options.labelAlignment?(t.textBaseline="alphabetic",i-=2*r):"line-below"==this.options.labelAlignment?(t.textBaseline="hanging",i+=2*r):t.textBaseline="middle"}else t.textBaseline="middle";this.options.fontStrokeWidth>0&&(t.lineWidth=this.options.fontStrokeWidth,t.strokeStyle=this.options.fontStrokeColor,t.lineJoin="round");for(var a=0;o>a;a++)this.options.fontStrokeWidth>0&&t.strokeText(s[a],e,i),t.fillText(s[a],e,i),i+=n},s.prototype._drawDashLine=function(t){t.strokeStyle=this._getColor(),t.lineWidth=this._getLineWidth();var e=null;if(void 0!==t.setLineDash){t.save();var i=[0];i=void 0!==this.options.dash.length&&void 0!==this.options.dash.gap?[this.options.dash.length,this.options.dash.gap]:[5,5],t.setLineDash(i),t.lineDashOffset=0,e=this._line(t),t.setLineDash([0]),t.lineDashOffset=0,t.restore()}else t.beginPath(),t.lineCap="round",void 0!==this.options.dash.altLength?t.dashedLine(this.from.x,this.from.y,this.to.x,this.to.y,[this.options.dash.length,this.options.dash.gap,this.options.dash.altLength,this.options.dash.gap]):void 0!==this.options.dash.length&&void 0!==this.options.dash.gap?t.dashedLine(this.from.x,this.from.y,this.to.x,this.to.y,[this.options.dash.length,this.options.dash.gap]):(t.moveTo(this.from.x,this.from.y),t.lineTo(this.to.x,this.to.y)),t.stroke();if(this.label){var s;if(1==this.options.smoothCurves.enabled&&null!=e){var o=.5*(.5*(this.from.x+e.x)+.5*(this.to.x+e.x)),n=.5*(.5*(this.from.y+e.y)+.5*(this.to.y+e.y));s={x:o,y:n}}else s=this._pointOnLine(.5);this._label(t,this.label,s.x,s.y)}},s.prototype._pointOnLine=function(t){return{x:(1-t)*this.from.x+t*this.to.x,y:(1-t)*this.from.y+t*this.to.y}},s.prototype._pointOnCircle=function(t,e,i,s){var o=2*(s-3/8)*Math.PI;return{x:t+i*Math.cos(o),y:e-i*Math.sin(o)}},s.prototype._drawArrowCenter=function(t){var e;if(t.strokeStyle=this._getColor(),t.fillStyle=t.strokeStyle,t.lineWidth=this._getLineWidth(),this.from!=this.to){var i=this._line(t),s=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),o=(10+5*this.options.width)*this.options.arrowScaleFactor;if(1==this.options.smoothCurves.enabled&&null!=i){var n=.5*(.5*(this.from.x+i.x)+.5*(this.to.x+i.x)),r=.5*(.5*(this.from.y+i.y)+.5*(this.to.y+i.y));e={x:n,y:r}}else e=this._pointOnLine(.5);t.arrow(e.x,e.y,s,o),t.fill(),t.stroke(),this.label&&this._label(t,this.label,e.x,e.y)}else{var a,h,d=.25*Math.max(100,this.physics.springLength),l=this.from;l.width||l.resize(t),l.width>l.height?(a=l.x+.5*l.width,h=l.y-d):(a=l.x+d,h=l.y-.5*l.height),this._circle(t,a,h,d);var s=.2*Math.PI,o=(10+5*this.options.width)*this.options.arrowScaleFactor;e=this._pointOnCircle(a,h,d,.5),t.arrow(e.x,e.y,s,o),t.fill(),t.stroke(),this.label&&(e=this._pointOnCircle(a,h,d,.5),this._label(t,this.label,e.x,e.y))}},s.prototype._pointOnBezier=function(t){var e=this._getViaCoordinates(),i=Math.pow(1-t,2)*this.from.x+2*t*(1-t)*e.x+Math.pow(t,2)*this.to.x,s=Math.pow(1-t,2)*this.from.y+2*t*(1-t)*e.y+Math.pow(t,2)*this.to.y;return{x:i,y:s}},s.prototype._findBorderPosition=function(t,e){var i,s,o,n,r,a=10,h=0,d=0,l=1,c=.2,p=this.to;for(1==t&&(p=this.from);l>=d&&a>h;){var u=.5*(d+l);if(i=this._pointOnBezier(u),s=Math.atan2(p.y-i.y,p.x-i.x),o=p.distanceToBorder(e,s),n=Math.sqrt(Math.pow(i.x-p.x,2)+Math.pow(i.y-p.y,2)),r=o-n,Math.abs(r)r?0==t?d=u:l=u:0==t?l=u:d=u,h++}return i.t=u,i},s.prototype._drawArrow=function(t){t.strokeStyle=this._getColor(),t.fillStyle=t.strokeStyle,t.lineWidth=this._getLineWidth();var e,i,s;if(this.from!=this.to){if(this._line(t),1==this.options.smoothCurves.enabled){var o=this._getViaCoordinates();s=this._findBorderPosition(!1,t);var n=this._pointOnBezier(Math.max(0,s.t-.1));e=Math.atan2(s.y-n.y,s.x-n.x)}else{e=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x);var r=this.to.x-this.from.x,a=this.to.y-this.from.y,h=Math.sqrt(r*r+a*a),d=this.to.distanceToBorder(t,e),l=(h-d)/h;s={},s.x=(1-l)*this.from.x+l*this.to.x,s.y=(1-l)*this.from.y+l*this.to.y}if(i=(10+5*this.options.width)*this.options.arrowScaleFactor,t.arrow(s.x,s.y,e,i),t.fill(),t.stroke(),this.label){var c;c=1==this.options.smoothCurves.enabled&&null!=o?this._pointOnBezier(.5):this._pointOnLine(.5),this._label(t,this.label,c.x,c.y)}}else{var p,u,m,f=this.from,g=.25*Math.max(100,this.physics.springLength);f.width||f.resize(t),f.width>f.height?(p=f.x+.5*f.width,u=f.y-g,m={x:p,y:f.y,angle:.9*Math.PI}):(p=f.x+g,u=f.y-.5*f.height,m={x:f.x,y:u,angle:.6*Math.PI}),t.beginPath(),t.arc(p,u,g,0,2*Math.PI,!1),t.stroke();var i=(10+5*this.options.width)*this.options.arrowScaleFactor;t.arrow(m.x,m.y,m.angle,i),t.fill(),t.stroke(),this.label&&(c=this._pointOnCircle(p,u,g,.5),this._label(t,this.label,c.x,c.y))}},s.prototype._getDistanceToEdge=function(t,e,i,s,o,n){var r=0;if(this.from!=this.to)if(1==this.options.smoothCurves.enabled){var a,h;if(1==this.options.smoothCurves.enabled&&1==this.options.smoothCurves.dynamic)a=this.via.x,h=this.via.y;else{var d=this._getViaCoordinates();a=d.x,h=d.y}var l,c,p,u,m,f,g,v=1e9;for(c=0;10>c;c++)p=.1*c,u=Math.pow(1-p,2)*t+2*p*(1-p)*a+Math.pow(p,2)*i,m=Math.pow(1-p,2)*e+2*p*(1-p)*h+Math.pow(p,2)*s,c>0&&(l=this._getDistanceToLine(f,g,u,m,o,n),v=v>l?l:v),f=u,g=m;r=v}else r=this._getDistanceToLine(t,e,i,s,o,n);else{var u,m,y,b,_=.25*this.physics.springLength,x=this.from;x.width>x.height?(u=x.x+.5*x.width,m=x.y-_):(u=x.x+_,m=x.y-.5*x.height),y=u-o,b=m-n,r=Math.abs(Math.sqrt(y*y+b*b)-_)}return this.labelDimensions.lefto&&this.labelDimensions.topn?0:r},s.prototype._getDistanceToLine=function(t,e,i,s,o,n){var r=i-t,a=s-e,h=r*r+a*a,d=((o-t)*r+(n-e)*a)/h;d>1?d=1:0>d&&(d=0);var l=t+d*r,c=e+d*a,p=l-o,u=c-n;return Math.sqrt(p*p+u*u)},s.prototype.setScale=function(t){this.networkScaleInv=1/t},s.prototype.select=function(){this.selected=!0},s.prototype.unselect=function(){this.selected=!1},s.prototype.positionBezierNode=function(){null!==this.via&&null!==this.from&&null!==this.to?(this.via.x=.5*(this.from.x+this.to.x),this.via.y=.5*(this.from.y+this.to.y)):(this.via.x=0,this.via.y=0)},s.prototype._drawControlNodes=function(t){if(1==this.controlNodesEnabled){if(null===this.controlNodes.from&&null===this.controlNodes.to){var e="edgeIdFrom:".concat(this.id),i="edgeIdTo:".concat(this.id),s={nodes:{group:"",radius:7,borderWidth:2,borderWidthSelected:2},physics:{damping:0},clustering:{maxNodeSizeIncrements:0,nodeScaling:{width:0,height:0,radius:0}}};this.controlNodes.from=new n({id:e,shape:"dot",color:{background:"#ff0000",border:"#3c3c3c",highlight:{background:"#07f968"}}},{},{},s),this.controlNodes.to=new n({id:i,shape:"dot",color:{background:"#ff0000",border:"#3c3c3c",highlight:{background:"#07f968"}}},{},{},s)}this.controlNodes.positions={},0==this.controlNodes.from.selected&&(this.controlNodes.positions.from=this.getControlNodeFromPosition(t),this.controlNodes.from.x=this.controlNodes.positions.from.x,this.controlNodes.from.y=this.controlNodes.positions.from.y),0==this.controlNodes.to.selected&&(this.controlNodes.positions.to=this.getControlNodeToPosition(t),this.controlNodes.to.x=this.controlNodes.positions.to.x,this.controlNodes.to.y=this.controlNodes.positions.to.y),this.controlNodes.from.draw(t),this.controlNodes.to.draw(t)}else this.controlNodes={from:null,to:null,positions:{}}},s.prototype._enableControlNodes=function(){this.fromBackup=this.from,this.toBackup=this.to,this.controlNodesEnabled=!0},s.prototype._disableControlNodes=function(){this.fromId=this.from.id,this.toId=this.to.id,this.fromId!=this.fromBackup.id?this.fromBackup.detachEdge(this):this.toId!=this.toBackup.id&&this.toBackup.detachEdge(this),this.fromBackup=null,this.toBackup=null,this.controlNodesEnabled=!1},s.prototype._getSelectedControlNode=function(t,e){var i=this.controlNodes.positions,s=Math.sqrt(Math.pow(t-i.from.x,2)+Math.pow(e-i.from.y,2)),o=Math.sqrt(Math.pow(t-i.to.x,2)+Math.pow(e-i.to.y,2));return 15>s?(this.connectedNode=this.from,this.from=this.controlNodes.from,this.controlNodes.from):15>o?(this.connectedNode=this.to,this.to=this.controlNodes.to,this.controlNodes.to):null},s.prototype._restoreControlNodes=function(){1==this.controlNodes.from.selected?(this.from=this.connectedNode,this.connectedNode=null,this.controlNodes.from.unselect()):1==this.controlNodes.to.selected&&(this.to=this.connectedNode,this.connectedNode=null,this.controlNodes.to.unselect())},s.prototype.getControlNodeFromPosition=function(t){var e;if(1==this.options.smoothCurves.enabled)e=this._findBorderPosition(!0,t);else{var i=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),s=this.to.x-this.from.x,o=this.to.y-this.from.y,n=Math.sqrt(s*s+o*o),r=this.from.distanceToBorder(t,i+Math.PI),a=(n-r)/n;e={},e.x=a*this.from.x+(1-a)*this.to.x,e.y=a*this.from.y+(1-a)*this.to.y}return e},s.prototype.getControlNodeToPosition=function(t){var e;if(1==this.options.smoothCurves.enabled)e=this._findBorderPosition(!1,t);else{var i=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),s=this.to.x-this.from.x,o=this.to.y-this.from.y,n=Math.sqrt(s*s+o*o),r=this.to.distanceToBorder(t,i),a=(n-r)/n;e={},e.x=(1-a)*this.from.x+a*this.to.x,e.y=(1-a)*this.from.y+a*this.to.y}return e},t.exports=s},function(t,e,i){function s(){this.clear(),this.defaultIndex=0}i(1);s.DEFAULT=[{border:"#2B7CE9",background:"#97C2FC",highlight:{border:"#2B7CE9",background:"#D2E5FF"},hover:{border:"#2B7CE9",background:"#D2E5FF"}},{border:"#FFA500",background:"#FFFF00",highlight:{border:"#FFA500",background:"#FFFFA3"},hover:{border:"#FFA500",background:"#FFFFA3"}},{border:"#FA0A10",background:"#FB7E81",highlight:{border:"#FA0A10",background:"#FFAFB1"},hover:{border:"#FA0A10",background:"#FFAFB1"}},{border:"#41A906",background:"#7BE141",highlight:{border:"#41A906",background:"#A1EC76"},hover:{border:"#41A906",background:"#A1EC76"}},{border:"#E129F0",background:"#EB7DF4",highlight:{border:"#E129F0",background:"#F0B3F5"},hover:{border:"#E129F0",background:"#F0B3F5"}},{border:"#7C29F0",background:"#AD85E4",highlight:{border:"#7C29F0",background:"#D3BDF0"},hover:{border:"#7C29F0",background:"#D3BDF0"}},{border:"#C37F00",background:"#FFA807",highlight:{border:"#C37F00",background:"#FFCA66"},hover:{border:"#C37F00",background:"#FFCA66"}},{border:"#4220FB",background:"#6E6EFD",highlight:{border:"#4220FB",background:"#9B9BFD"},hover:{border:"#4220FB",background:"#9B9BFD"}},{border:"#FD5A77",background:"#FFC0CB",highlight:{border:"#FD5A77",background:"#FFD1D9"},hover:{border:"#FD5A77",background:"#FFD1D9"}},{border:"#4AD63A",background:"#C2FABC",highlight:{border:"#4AD63A",background:"#E6FFE3"},hover:{border:"#4AD63A",background:"#E6FFE3"}}],s.prototype.clear=function(){this.groups={},this.groups.length=function(){var t=0;for(var e in this)this.hasOwnProperty(e)&&t++;return t}},s.prototype.get=function(t){var e=this.groups[t];if(void 0==e){var i=this.defaultIndex%s.DEFAULT.length;this.defaultIndex++,e={},e.color=s.DEFAULT[i],this.groups[t]=e}return e},s.prototype.add=function(t,e){return this.groups[t]=e,e},t.exports=s},function(t){function e(){this.images={},this.imageBroken={},this.callback=void 0}e.prototype.setOnloadCallback=function(t){this.callback=t},e.prototype.load=function(t,e){var i=this.images[t];if(void 0===i){var s=this;i=new Image,i.onload=function(){0==this.width&&(document.body.appendChild(this),this.width=this.offsetWidth,this.height=this.offsetHeight,document.body.removeChild(this)),s.callback&&(s.images[t]=i,s.callback(this))},i.onerror=function(){void 0===e?(console.error("Could not load image:",t),delete this.src,s.callback&&s.callback(this)):s.imageBroken[t]===!0?(console.error("Could not load brokenImage:",e),delete this.src,s.callback&&s.callback(this)):(this.src=e,s.imageBroken[t]=!0)},i.src=t}return i},t.exports=e},function(t,e,i){function s(t,e,i,s){var n=o.selectiveBridgeObject(["nodes"],s);this.options=n.nodes,this.selected=!1,this.hover=!1,this.edges=[],this.dynamicEdges=[],this.reroutedEdges={},this.fontDrawThreshold=3,this.id=void 0,this.allowedToMoveX=!1,this.allowedToMoveY=!1,this.xFixed=!1,this.yFixed=!1,this.horizontalAlignLeft=!0,this.verticalAlignTop=!0,this.baseRadiusValue=s.nodes.radius,this.radiusFixed=!1,this.level=-1,this.preassignedLevel=!1,this.hierarchyEnumerated=!1,this.labelDimensions={top:0,left:0,width:0,height:0,yLine:0},this.boundingBox={top:0,left:0,right:0,bottom:0},this.imagelist=e,this.grouplist=i,this.fx=0,this.fy=0,this.vx=0,this.vy=0,this.x=null,this.y=null,this.previousState={vx:0,vy:0,x:0,y:0},this.damping=s.physics.damping,this.fixedData={x:null,y:null},this.setProperties(t,n),this.resetCluster(),this.dynamicEdgesLength=0,this.clusterSession=0,this.clusterSizeWidthFactor=s.clustering.nodeScaling.width,this.clusterSizeHeightFactor=s.clustering.nodeScaling.height,this.clusterSizeRadiusFactor=s.clustering.nodeScaling.radius,this.maxNodeSizeIncrements=s.clustering.maxNodeSizeIncrements,this.growthIndicator=0,this.networkScaleInv=1,this.networkScale=1,this.canvasTopLeft={x:-300,y:-300},this.canvasBottomRight={x:300,y:300},this.parentEdgeId=null}var o=i(1);s.prototype.revertPosition=function(){this.x=this.previousState.x,this.y=this.previousState.y,this.vx=this.previousState.vx,this.vy=this.previousState.vy},s.prototype.resetCluster=function(){this.formationScale=void 0,this.clusterSize=1,this.containedNodes={},this.containedEdges={},this.clusterSessions=[]},s.prototype.attachEdge=function(t){-1==this.edges.indexOf(t)&&this.edges.push(t),-1==this.dynamicEdges.indexOf(t)&&this.dynamicEdges.push(t),this.dynamicEdgesLength=this.dynamicEdges.length},s.prototype.detachEdge=function(t){var e=this.edges.indexOf(t);-1!=e&&this.edges.splice(e,1),e=this.dynamicEdges.indexOf(t),-1!=e&&this.dynamicEdges.splice(e,1),this.dynamicEdgesLength=this.dynamicEdges.length},s.prototype.setProperties=function(t,e){if(t){var i=["borderWidth","borderWidthSelected","shape","image","brokenImage","radius","fontColor","fontSize","fontFace","fontFill","fontStrokeWidth","fontStrokeColor","group","mass"];if(o.selectiveDeepExtend(i,this.options,t),void 0!==t.id&&(this.id=t.id),void 0!==t.label&&(this.label=t.label,this.originalLabel=t.label),void 0!==t.title&&(this.title=t.title),void 0!==t.x&&(this.x=t.x),void 0!==t.y&&(this.y=t.y),void 0!==t.value&&(this.value=t.value),void 0!==t.level&&(this.level=t.level,this.preassignedLevel=!0),void 0!==t.horizontalAlignLeft&&(this.horizontalAlignLeft=t.horizontalAlignLeft),void 0!==t.verticalAlignTop&&(this.verticalAlignTop=t.verticalAlignTop),void 0!==t.triggerFunction&&(this.triggerFunction=t.triggerFunction),void 0===this.id)throw"Node must have an id";if("number"==typeof this.options.group||"string"==typeof this.options.group&&""!=this.options.group){var s=this.grouplist.get(this.options.group);o.deepExtend(this.options,s),this.options.color=o.parseColor(this.options.color)}if(void 0!==t.radius&&(this.baseRadiusValue=this.options.radius),void 0!==t.color&&(this.options.color=o.parseColor(t.color)),void 0!==this.options.image&&""!=this.options.image){if(!this.imagelist)throw"No imagelist provided";this.imageObj=this.imagelist.load(this.options.image,this.options.brokenImage)}switch(void 0!==t.allowedToMoveX?(this.xFixed=!t.allowedToMoveX,this.allowedToMoveX=t.allowedToMoveX):void 0!==t.x&&0==this.allowedToMoveX&&(this.xFixed=!0),void 0!==t.allowedToMoveY?(this.yFixed=!t.allowedToMoveY,this.allowedToMoveY=t.allowedToMoveY):void 0!==t.y&&0==this.allowedToMoveY&&(this.yFixed=!0),this.radiusFixed=this.radiusFixed||void 0!==t.radius,("image"===this.options.shape||"circularImage"===this.options.shape)&&(this.options.radiusMin=e.nodes.widthMin,this.options.radiusMax=e.nodes.widthMax),this.options.shape){case"database":this.draw=this._drawDatabase,this.resize=this._resizeDatabase;break;case"box":this.draw=this._drawBox,this.resize=this._resizeBox;break;case"circle":this.draw=this._drawCircle,this.resize=this._resizeCircle;break;case"ellipse":this.draw=this._drawEllipse,this.resize=this._resizeEllipse;break;case"image":this.draw=this._drawImage,this.resize=this._resizeImage;break;case"circularImage":this.draw=this._drawCircularImage,this.resize=this._resizeCircularImage;break;case"text":this.draw=this._drawText,this.resize=this._resizeText;break;case"dot":this.draw=this._drawDot,this.resize=this._resizeShape;break;case"square":this.draw=this._drawSquare,this.resize=this._resizeShape;break;case"triangle":this.draw=this._drawTriangle,this.resize=this._resizeShape;break;case"triangleDown":this.draw=this._drawTriangleDown,this.resize=this._resizeShape;break;case"star":this.draw=this._drawStar,this.resize=this._resizeShape;break;default:this.draw=this._drawEllipse,this.resize=this._resizeEllipse}this._reset()}},s.prototype.select=function(){this.selected=!0,this._reset()},s.prototype.unselect=function(){this.selected=!1,this._reset()},s.prototype.clearSizeCache=function(){this._reset()},s.prototype._reset=function(){this.width=void 0,this.height=void 0},s.prototype.getTitle=function(){return"function"==typeof this.title?this.title():this.title},s.prototype.distanceToBorder=function(t,e){var i=1;switch(this.width||this.resize(t),this.options.shape){case"circle":case"dot":return this.options.radius+i;case"ellipse":var s=this.width/2,o=this.height/2,n=Math.sin(e)*s,r=Math.cos(e)*o;return s*o/Math.sqrt(n*n+r*r);case"box":case"image":case"text":default:return this.width?Math.min(Math.abs(this.width/2/Math.cos(e)),Math.abs(this.height/2/Math.sin(e)))+i:0}},s.prototype._setForce=function(t,e){this.fx=t,this.fy=e},s.prototype._addForce=function(t,e){this.fx+=t,this.fy+=e},s.prototype.storeState=function(){this.previousState.x=this.x,this.previousState.y=this.y,this.previousState.vx=this.vx,this.previousState.vy=this.vy},s.prototype.discreteStep=function(t){if(this.storeState(),this.xFixed)this.fx=0,this.vx=0;else{var e=this.damping*this.vx,i=(this.fx-e)/this.options.mass;this.vx+=i*t,this.x+=this.vx*t}if(this.yFixed)this.fy=0,this.vy=0;else{var s=this.damping*this.vy,o=(this.fy-s)/this.options.mass;this.vy+=o*t,this.y+=this.vy*t}},s.prototype.discreteStepLimited=function(t,e){if(this.storeState(),this.xFixed)this.fx=0,this.vx=0;else{var i=this.damping*this.vx,s=(this.fx-i)/this.options.mass;this.vx+=s*t,this.vx=Math.abs(this.vx)>e?this.vx>0?e:-e:this.vx,this.x+=this.vx*t}if(this.yFixed)this.fy=0,this.vy=0;else{var o=this.damping*this.vy,n=(this.fy-o)/this.options.mass;this.vy+=n*t,this.vy=Math.abs(this.vy)>e?this.vy>0?e:-e:this.vy,this.y+=this.vy*t}},s.prototype.isFixed=function(){return this.xFixed&&this.yFixed},s.prototype.isMoving=function(t){var e=Math.sqrt(Math.pow(this.vx,2)+Math.pow(this.vy,2));return e>t},s.prototype.isSelected=function(){return this.selected},s.prototype.getValue=function(){return this.value},s.prototype.getDistance=function(t,e){var i=this.x-t,s=this.y-e;return Math.sqrt(i*i+s*s)},s.prototype.setValueRange=function(t,e){if(!this.radiusFixed&&void 0!==this.value)if(e==t)this.options.radius=(this.options.radiusMin+this.options.radiusMax)/2;else{var i=(this.options.radiusMax-this.options.radiusMin)/(e-t);this.options.radius=(this.value-t)*i+this.options.radiusMin}this.baseRadiusValue=this.options.radius},s.prototype.draw=function(){throw"Draw method not initialized for node"},s.prototype.resize=function(){throw"Resize method not initialized for node"},s.prototype.isOverlappingWith=function(t){return this.leftt.left&&this.topt.top},s.prototype._resizeImage=function(){if(!this.width||!this.height){var t,e;if(this.value){this.options.radius=this.baseRadiusValue;var i=this.imageObj.height/this.imageObj.width;void 0!==i?(t=this.options.radius||this.imageObj.width,e=this.options.radius*i||this.imageObj.height):(t=0,e=0)}else t=this.imageObj.width,e=this.imageObj.height;this.width=t,this.height=e,this.growthIndicator=0,this.width>0&&this.height>0&&(this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-t)}},s.prototype._drawImageAtPosition=function(t){if(0!=this.imageObj.width){if(this.clusterSize>1){var e=this.clusterSize>1?10:0;e*=this.networkScaleInv,e=Math.min(.2*this.width,e),t.globalAlpha=.5,t.drawImage(this.imageObj,this.left-e,this.top-e,this.width+2*e,this.height+2*e)}t.globalAlpha=1,t.drawImage(this.imageObj,this.left,this.top,this.width,this.height)}},s.prototype._drawImageLabel=function(t){var e,i=0;if(this.height){i=this.height/2;var s=this.getTextSize(t);s.lineCount>=1&&(i+=s.height/2,i+=3)}e=this.y+i,this._label(t,this.label,this.x,e,void 0)},s.prototype._drawImage=function(t){this._resizeImage(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._drawImageAtPosition(t),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._drawImageLabel(t),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height)},s.prototype._resizeCircularImage=function(t){if(this.imageObj.src&&this.imageObj.width&&this.imageObj.height)this._swapToImageResizeWhenImageLoaded&&(this.width=0,this.height=0,delete this._swapToImageResizeWhenImageLoaded),this._resizeImage(t);else if(!this.width){var e=2*this.options.radius;this.width=e,this.height=e,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.options.radius-.5*e,this._swapToImageResizeWhenImageLoaded=!0}},s.prototype._drawCircularImage=function(t){this._resizeCircularImage(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=this.left+this.width/2,i=this.top+this.height/2,s=Math.abs(this.height/2);this._drawRawCircle(t,e,i,s),t.save(),t.circle(this.x,this.y,s),t.stroke(),t.clip(),this._drawImageAtPosition(t),t.restore(),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this._drawImageLabel(t),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height)},s.prototype._resizeBox=function(t){if(!this.width){var e=5,i=this.getTextSize(t);this.width=i.width+2*e,this.height=i.height+2*e,this.width+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.growthIndicator=this.width-(i.width+2*e)}},s.prototype._drawBox=function(t){this._resizeBox(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=2.5,i=this.options.borderWidth,s=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.roundRect(this.left-2*t.lineWidth,this.top-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth,this.options.radius),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.roundRect(this.left,this.top,this.width,this.height,this.options.radius),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y)},s.prototype._resizeDatabase=function(t){if(!this.width){var e=5,i=this.getTextSize(t),s=i.width+2*e;this.width=s,this.height=s,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-s}},s.prototype._drawDatabase=function(t){this._resizeDatabase(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=2.5,i=this.options.borderWidth,s=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.database(this.x-this.width/2-2*t.lineWidth,this.y-.5*this.height-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.database(this.x-this.width/2,this.y-.5*this.height,this.width,this.height),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y) -},s.prototype._resizeCircle=function(t){if(!this.width){var e=5,i=this.getTextSize(t),s=Math.max(i.width,i.height)+2*e;this.options.radius=s/2,this.width=s,this.height=s,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.options.radius-.5*s}},s.prototype._drawRawCircle=function(t,e,i,s){var o=2.5,n=this.options.borderWidth,r=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?r:n)+(this.clusterSize>1?o:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.circle(e,i,s+2*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?r:n)+(this.clusterSize>1?o:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.circle(this.x,this.y,s),t.fill(),t.stroke()},s.prototype._drawCircle=function(t){this._resizeCircle(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._drawRawCircle(t,this.x,this.y,this.options.radius),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this._label(t,this.label,this.x,this.y)},s.prototype._resizeEllipse=function(t){if(!this.width){var e=this.getTextSize(t);this.width=1.5*e.width,this.height=2*e.height,this.width1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.ellipse(this.left-2*t.lineWidth,this.top-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.ellipse(this.left,this.top,this.width,this.height),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y)},s.prototype._drawDot=function(t){this._drawShape(t,"circle")},s.prototype._drawTriangle=function(t){this._drawShape(t,"triangle")},s.prototype._drawTriangleDown=function(t){this._drawShape(t,"triangleDown")},s.prototype._drawSquare=function(t){this._drawShape(t,"square")},s.prototype._drawStar=function(t){this._drawShape(t,"star")},s.prototype._resizeShape=function(){if(!this.width){this.options.radius=this.baseRadiusValue;var t=2*this.options.radius;this.width=t,this.height=t,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-t}},s.prototype._drawShape=function(t,e){this._resizeShape(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var i=2.5,s=this.options.borderWidth,o=this.options.borderWidthSelected||2*this.options.borderWidth,n=2;switch(e){case"dot":n=2;break;case"square":n=2;break;case"triangle":n=3;break;case"triangleDown":n=3;break;case"star":n=4}t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?o:s)+(this.clusterSize>1?i:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t[e](this.x,this.y,this.options.radius+n*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?o:s)+(this.clusterSize>1?i:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t[e](this.x,this.y,this.options.radius),t.fill(),t.stroke(),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this.label&&(this._label(t,this.label,this.x,this.y+this.height/2,void 0,"hanging",!0),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height))},s.prototype._resizeText=function(t){if(!this.width){var e=5,i=this.getTextSize(t);this.width=i.width+2*e,this.height=i.height+2*e,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-(i.width+2*e)}},s.prototype._drawText=function(t){this._resizeText(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._label(t,this.label,this.x,this.y),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height},s.prototype._label=function(t,e,i,s,o,n,r){if(e&&Number(this.options.fontSize)*this.networkScale>this.fontDrawThreshold){t.font=(this.selected?"bold ":"")+this.options.fontSize+"px "+this.options.fontFace;var a=e.split("\n"),h=a.length,d=Number(this.options.fontSize),l=s+(1-h)/2*d;1==r&&(l=s+(1-h)/(2*d));for(var c=t.measureText(a[0]).width,p=1;h>p;p++){var u=t.measureText(a[p]).width;c=u>c?u:c}var m=this.options.fontSize*h,f=i-c/2,g=s-m/2;"hanging"==n&&(g+=.5*d,g+=4,l+=4),this.labelDimensions={top:g,left:f,width:c,height:m,yLine:l},void 0!==this.options.fontFill&&null!==this.options.fontFill&&"none"!==this.options.fontFill&&(t.fillStyle=this.options.fontFill,t.fillRect(f,g,c,m)),t.fillStyle=this.options.fontColor||"black",t.textAlign=o||"center",t.textBaseline=n||"middle",this.options.fontStrokeWidth>0&&(t.lineWidth=this.options.fontStrokeWidth,t.strokeStyle=this.options.fontStrokeColor,t.lineJoin="round");for(var p=0;h>p;p++)this.options.fontStrokeWidth&&t.strokeText(a[p],i,l),t.fillText(a[p],i,l),l+=d}},s.prototype.getTextSize=function(t){if(void 0!==this.label){t.font=(this.selected?"bold ":"")+this.options.fontSize+"px "+this.options.fontFace;for(var e=this.label.split("\n"),i=(Number(this.options.fontSize)+4)*e.length,s=0,o=0,n=e.length;n>o;o++)s=Math.max(s,t.measureText(e[o]).width);return{width:s,height:i,lineCount:e.length}}return{width:0,height:0,lineCount:0}},s.prototype.inArea=function(){return void 0!==this.width?this.x+this.width*this.networkScaleInv>=this.canvasTopLeft.x&&this.x-this.width*this.networkScaleInv=this.canvasTopLeft.y&&this.y-this.height*this.networkScaleInv=this.canvasTopLeft.x&&this.x=this.canvasTopLeft.y&&this.ys&&(n=s-e-this.padding),no&&(r=o-i-this.padding),ri;i++)if(e.id===r.nodes[i].id){o=r.nodes[i];break}for(o||(o={id:e.id},t.node&&(o.attr=a(o.attr,t.node))),i=n.length-1;i>=0;i--){var h=n[i];h.nodes||(h.nodes=[]),-1==h.nodes.indexOf(o)&&h.nodes.push(o)}e.attr&&(o.attr=a(o.attr,e.attr))}function l(t,e){if(t.edges||(t.edges=[]),t.edges.push(e),t.edge){var i=a({},t.edge);e.attr=a(i,e.attr)}}function c(t,e,i,s,o){var n={from:e,to:i,type:s};return t.edge&&(n.attr=a({},t.edge)),n.attr=a(n.attr||{},o),n}function p(){for(N=D.NULL,k="";" "==E||" "==E||"\n"==E||"\r"==E;)o();do{var t=!1;if("#"==E){for(var e=O-1;" "==T.charAt(e)||" "==T.charAt(e);)e--;if("\n"==T.charAt(e)||""==T.charAt(e)){for(;""!=E&&"\n"!=E;)o();t=!0}}if("/"==E&&"/"==n()){for(;""!=E&&"\n"!=E;)o();t=!0}if("/"==E&&"*"==n()){for(;""!=E;){if("*"==E&&"/"==n()){o(),o();break}o()}t=!0}for(;" "==E||" "==E||"\n"==E||"\r"==E;)o()}while(t);if(""==E)return void(N=D.DELIMITER);var i=E+n();if(C[i])return N=D.DELIMITER,k=i,o(),void o();if(C[E])return N=D.DELIMITER,k=E,void o();if(r(E)||"-"==E){for(k+=E,o();r(E);)k+=E,o();return"false"==k?k=!1:"true"==k?k=!0:isNaN(Number(k))||(k=Number(k)),void(N=D.IDENTIFIER)}if('"'==E){for(o();""!=E&&('"'!=E||'"'==E&&'"'==n());)k+=E,'"'==E&&o(),o();if('"'!=E)throw x('End of string " expected');return o(),void(N=D.IDENTIFIER)}for(N=D.UNKNOWN;""!=E;)k+=E,o();throw new SyntaxError('Syntax error in part "'+w(k,30)+'"')}function u(){var t={};if(s(),p(),"strict"==k&&(t.strict=!0,p()),("graph"==k||"digraph"==k)&&(t.type=k,p()),N==D.IDENTIFIER&&(t.id=k,p()),"{"!=k)throw x("Angle bracket { expected");if(p(),m(t),"}"!=k)throw x("Angle bracket } expected");if(p(),""!==k)throw x("End of file expected");return p(),delete t.node,delete t.edge,delete t.graph,t}function m(t){for(;""!==k&&"}"!=k;)f(t),";"==k&&p()}function f(t){var e=g(t);if(e)return void b(t,e);var i=v(t);if(!i){if(N!=D.IDENTIFIER)throw x("Identifier expected");var s=k;if(p(),"="==k){if(p(),N!=D.IDENTIFIER)throw x("Identifier expected");t[s]=k,p()}else y(t,s)}}function g(t){var e=null;if("subgraph"==k&&(e={},e.type="subgraph",p(),N==D.IDENTIFIER&&(e.id=k,p())),"{"==k){if(p(),e||(e={}),e.parent=t,e.node=t.node,e.edge=t.edge,e.graph=t.graph,m(e),"}"!=k)throw x("Angle bracket } expected");p(),delete e.node,delete e.edge,delete e.graph,delete e.parent,t.subgraphs||(t.subgraphs=[]),t.subgraphs.push(e)}return e}function v(t){return"node"==k?(p(),t.node=_(),"node"):"edge"==k?(p(),t.edge=_(),"edge"):"graph"==k?(p(),t.graph=_(),"graph"):null}function y(t,e){var i={id:e},s=_();s&&(i.attr=s),d(t,i),b(t,e)}function b(t,e){for(;"->"==k||"--"==k;){var i,s=k;p();var o=g(t);if(o)i=o;else{if(N!=D.IDENTIFIER)throw x("Identifier or subgraph expected");i=k,d(t,{id:i}),p()}var n=_(),r=c(t,e,i,s,n);l(t,r),e=i}}function _(){for(var t=null;"["==k;){for(p(),t={};""!==k&&"]"!=k;){if(N!=D.IDENTIFIER)throw x("Attribute name expected");var e=k;if(p(),"="!=k)throw x("Equal sign = expected");if(p(),N!=D.IDENTIFIER)throw x("Attribute value expected");var i=k;h(t,e,i),p(),","==k&&p()}if("]"!=k)throw x("Bracket ] expected");p()}return t}function x(t){return new SyntaxError(t+', got "'+w(k,30)+'" (char '+O+")")}function w(t,e){return t.length<=e?t:t.substr(0,27)+"..."}function S(t,e,i){Array.isArray(t)?t.forEach(function(t){Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}):Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}function M(t){var e=i(t),s={nodes:[],edges:[],options:{}};if(e.nodes&&e.nodes.forEach(function(t){var e={id:t.id,label:String(t.label||t.id)};a(e,t.attr),e.image&&(e.shape="image"),s.nodes.push(e)}),e.edges){var o=function(t){var e={from:t.from,to:t.to};return a(e,t.attr),e.style="->"==t.type?"arrow":"line",e};e.edges.forEach(function(t){var e,i;e=t.from instanceof Object?t.from.nodes:{id:t.from},i=t.to instanceof Object?t.to.nodes:{id:t.to},t.from instanceof Object&&t.from.edges&&t.from.edges.forEach(function(t){var e=o(t);s.edges.push(e)}),S(e,i,function(e,i){var n=c(s,e.id,i.id,t.type,t.attr),r=o(n);s.edges.push(r)}),t.to instanceof Object&&t.to.edges&&t.to.edges.forEach(function(t){var e=o(t);s.edges.push(e)})})}return e.attr&&(s.options=e.attr),s}var D={NULL:0,DELIMITER:1,IDENTIFIER:2,UNKNOWN:3},C={"{":!0,"}":!0,"[":!0,"]":!0,";":!0,"=":!0,",":!0,"->":!0,"--":!0},T="",O=0,E="",k="",N=D.NULL,I=/[a-zA-Z_0-9.:#]/;e.parseDOT=i,e.DOTToGraph=M},function(t,e){function i(t,e){var i=[],s=[];this.options={edges:{inheritColor:!0},nodes:{allowedToMove:!1,parseColor:!1}},void 0!==e&&(this.options.nodes.allowedToMove=e.allowedToMove|!1,this.options.nodes.parseColor=e.parseColor|!1,this.options.edges.inheritColor=e.inheritColor|!0);for(var o=t.edges,n=t.nodes,r=0;r=s&&(s=864e5),e=new Date(e.valueOf()-.05*s),i=new Date(i.valueOf()+.05*s)}return{start:e,end:i}},s.prototype.setWindow=function(t,e,i){var s=i&&void 0!==i.animate?i.animate:!0;if(1==arguments.length){var o=arguments[0];this.range.setRange(o.start,o.end,s)}else this.range.setRange(t,e,s)},s.prototype.moveTo=function(t,e){var i=this.range.end-this.range.start,s=r.convert(t,"Date").valueOf(),o=s-i/2,n=s+i/2,a=e&&void 0!==e.animate?e.animate:!0;this.range.setRange(o,n,a)},s.prototype.getWindow=function(){var t=this.range.getRange();return{start:new Date(t.start),end:new Date(t.end)}},s.prototype.redraw=function(){var t=!1,e=this.options,i=this.props,s=this.dom;if(s){h.updateHiddenDates(this.body,this.options.hiddenDates),"top"==e.orientation?(r.addClassName(s.root,"top"),r.removeClassName(s.root,"bottom")):(r.removeClassName(s.root,"top"),r.addClassName(s.root,"bottom")),s.root.style.maxHeight=r.option.asSize(e.maxHeight,""),s.root.style.minHeight=r.option.asSize(e.minHeight,""),s.root.style.width=r.option.asSize(e.width,""),i.border.left=(s.centerContainer.offsetWidth-s.centerContainer.clientWidth)/2,i.border.right=i.border.left,i.border.top=(s.centerContainer.offsetHeight-s.centerContainer.clientHeight)/2,i.border.bottom=i.border.top;var o=s.root.offsetHeight-s.root.clientHeight,n=s.root.offsetWidth-s.root.clientWidth;0===s.centerContainer.clientHeight&&(i.border.left=i.border.top,i.border.right=i.border.left),0===s.root.clientHeight&&(n=o),i.center.height=s.center.offsetHeight,i.left.height=s.left.offsetHeight,i.right.height=s.right.offsetHeight,i.top.height=s.top.clientHeight||-i.border.top,i.bottom.height=s.bottom.clientHeight||-i.border.bottom;var a=Math.max(i.left.height,i.center.height,i.right.height),d=i.top.height+a+i.bottom.height+o+i.border.top+i.border.bottom;s.root.style.height=r.option.asSize(e.height,d+"px"),i.root.height=s.root.offsetHeight,i.background.height=i.root.height-o;var l=i.root.height-i.top.height-i.bottom.height-o;i.centerContainer.height=l,i.leftContainer.height=l,i.rightContainer.height=i.leftContainer.height,i.root.width=s.root.offsetWidth,i.background.width=i.root.width-n,i.left.width=s.leftContainer.clientWidth||-i.border.left,i.leftContainer.width=i.left.width,i.right.width=s.rightContainer.clientWidth||-i.border.right,i.rightContainer.width=i.right.width;var c=i.root.width-i.left.width-i.right.width-n;i.center.width=c,i.centerContainer.width=c,i.top.width=c,i.bottom.width=c,s.background.style.height=i.background.height+"px",s.backgroundVertical.style.height=i.background.height+"px",s.backgroundHorizontal.style.height=i.centerContainer.height+"px",s.centerContainer.style.height=i.centerContainer.height+"px",s.leftContainer.style.height=i.leftContainer.height+"px",s.rightContainer.style.height=i.rightContainer.height+"px",s.background.style.width=i.background.width+"px",s.backgroundVertical.style.width=i.centerContainer.width+"px",s.backgroundHorizontal.style.width=i.background.width+"px",s.centerContainer.style.width=i.center.width+"px",s.top.style.width=i.top.width+"px",s.bottom.style.width=i.bottom.width+"px",s.background.style.left="0",s.background.style.top="0",s.backgroundVertical.style.left=i.left.width+i.border.left+"px",s.backgroundVertical.style.top="0",s.backgroundHorizontal.style.left="0",s.backgroundHorizontal.style.top=i.top.height+"px",s.centerContainer.style.left=i.left.width+"px",s.centerContainer.style.top=i.top.height+"px",s.leftContainer.style.left="0",s.leftContainer.style.top=i.top.height+"px",s.rightContainer.style.left=i.left.width+i.center.width+"px",s.rightContainer.style.top=i.top.height+"px",s.top.style.left=i.left.width+"px",s.top.style.top="0",s.bottom.style.left=i.left.width+"px",s.bottom.style.top=i.top.height+i.centerContainer.height+"px",this._updateScrollTop();var p=this.props.scrollTop;"bottom"==e.orientation&&(p+=Math.max(this.props.centerContainer.height-this.props.center.height-this.props.border.top-this.props.border.bottom,0)),s.center.style.left="0",s.center.style.top=p+"px",s.left.style.left="0",s.left.style.top=p+"px",s.right.style.left="0",s.right.style.top=p+"px";var u=0==this.props.scrollTop?"hidden":"",m=this.props.scrollTop==this.props.scrollTopMin?"hidden":"";if(s.shadowTop.style.visibility=u,s.shadowBottom.style.visibility=m,s.shadowTopLeft.style.visibility=u,s.shadowBottomLeft.style.visibility=m,s.shadowTopRight.style.visibility=u,s.shadowBottomRight.style.visibility=m,this.components.forEach(function(e){t=e.redraw()||t}),t){var f=3;this.redrawCount0&&(this.props.scrollTop=0),this.props.scrollTops;s++){var o=s%2===0?1.3*i:.5*i;this.lineTo(t+o*Math.sin(2*s*Math.PI/10),e-o*Math.cos(2*s*Math.PI/10))}this.closePath()},CanvasRenderingContext2D.prototype.roundRect=function(t,e,i,s,o){var n=Math.PI/180;0>i-2*o&&(o=i/2),0>s-2*o&&(o=s/2),this.beginPath(),this.moveTo(t+o,e),this.lineTo(t+i-o,e),this.arc(t+i-o,e+o,o,270*n,360*n,!1),this.lineTo(t+i,e+s-o),this.arc(t+i-o,e+s-o,o,0,90*n,!1),this.lineTo(t+o,e+s),this.arc(t+o,e+s-o,o,90*n,180*n,!1),this.lineTo(t,e+o),this.arc(t+o,e+o,o,180*n,270*n,!1)},CanvasRenderingContext2D.prototype.ellipse=function(t,e,i,s){var o=.5522848,n=i/2*o,r=s/2*o,a=t+i,h=e+s,d=t+i/2,l=e+s/2; -this.beginPath(),this.moveTo(t,l),this.bezierCurveTo(t,l-r,d-n,e,d,e),this.bezierCurveTo(d+n,e,a,l-r,a,l),this.bezierCurveTo(a,l+r,d+n,h,d,h),this.bezierCurveTo(d-n,h,t,l+r,t,l)},CanvasRenderingContext2D.prototype.database=function(t,e,i,s){var o=1/3,n=i,r=s*o,a=.5522848,h=n/2*a,d=r/2*a,l=t+n,c=e+r,p=t+n/2,u=e+r/2,m=e+(s-r/2),f=e+s;this.beginPath(),this.moveTo(l,u),this.bezierCurveTo(l,u+d,p+h,c,p,c),this.bezierCurveTo(p-h,c,t,u+d,t,u),this.bezierCurveTo(t,u-d,p-h,e,p,e),this.bezierCurveTo(p+h,e,l,u-d,l,u),this.lineTo(l,m),this.bezierCurveTo(l,m+d,p+h,f,p,f),this.bezierCurveTo(p-h,f,t,m+d,t,m),this.lineTo(t,u)},CanvasRenderingContext2D.prototype.arrow=function(t,e,i,s){var o=t-s*Math.cos(i),n=e-s*Math.sin(i),r=t-.9*s*Math.cos(i),a=e-.9*s*Math.sin(i),h=o+s/3*Math.cos(i+.5*Math.PI),d=n+s/3*Math.sin(i+.5*Math.PI),l=o+s/3*Math.cos(i-.5*Math.PI),c=n+s/3*Math.sin(i-.5*Math.PI);this.beginPath(),this.moveTo(t,e),this.lineTo(h,d),this.lineTo(r,a),this.lineTo(l,c),this.closePath()},CanvasRenderingContext2D.prototype.dashedLine=function(t,e,i,s,o){o||(o=[10,5]),0==p&&(p=.001);var n=o.length;this.moveTo(t,e);for(var r=i-t,a=s-e,h=a/r,d=Math.sqrt(r*r+a*a),l=0,c=!0;d>=.1;){var p=o[l++%n];p>d&&(p=d);var u=Math.sqrt(p*p/(1+h*h));0>r&&(u=-u),t+=u,e+=h*u,this[c?"lineTo":"moveTo"](t,e),d-=p,c=!c}})},function(t,e,i){function s(t,e){this.groupId=t,this.options=e}var o=i(2),n=i(53);s.prototype.getYRange=function(t){for(var e=t[0].y,i=t[0].y,s=0;st[s].y?t[s].y:e,i=i0){var r,a,h=Number(i.svg.style.height.replace("px",""));if(r=o.getSVGElement("path",i.svgElements,i.svg),r.setAttributeNS(null,"class",e.className),void 0!==e.style&&r.setAttributeNS(null,"style",e.style),a=1==e.options.catmullRom.enabled?s._catmullRom(t,e):s._linear(t),1==e.options.shaded.enabled){var d,l=o.getSVGElement("path",i.svgElements,i.svg);d="top"==e.options.shaded.orientation?"M"+t[0].x+",0 "+a+"L"+t[t.length-1].x+",0":"M"+t[0].x+","+h+" "+a+"L"+t[t.length-1].x+","+h,l.setAttributeNS(null,"class",e.className+" fill"),void 0!==e.options.shaded.style&&l.setAttributeNS(null,"style",e.options.shaded.style),l.setAttributeNS(null,"d",d)}r.setAttributeNS(null,"d","M"+a),1==e.options.drawPoints.enabled&&n.draw(t,e,i)}},s._catmullRomUniform=function(t){for(var e,i,s,o,n,r,a=Math.round(t[0].x)+","+Math.round(t[0].y)+" ",h=1/6,d=t.length,l=0;d-1>l;l++)e=0==l?t[0]:t[l-1],i=t[l],s=t[l+1],o=d>l+2?t[l+2]:s,n={x:(-e.x+6*i.x+s.x)*h,y:(-e.y+6*i.y+s.y)*h},r={x:(i.x+6*s.x-o.x)*h,y:(i.y+6*s.y-o.y)*h},a+="C"+n.x+","+n.y+" "+r.x+","+r.y+" "+s.x+","+s.y+" ";return a},s._catmullRom=function(t,e){var i=e.options.catmullRom.alpha;if(0==i||void 0===i)return this._catmullRomUniform(t);for(var s,o,n,r,a,h,d,l,c,p,u,m,f,g,v,y,b,_,x,w=Math.round(t[0].x)+","+Math.round(t[0].y)+" ",S=t.length,M=0;S-1>M;M++)s=0==M?t[0]:t[M-1],o=t[M],n=t[M+1],r=S>M+2?t[M+2]:n,d=Math.sqrt(Math.pow(s.x-o.x,2)+Math.pow(s.y-o.y,2)),l=Math.sqrt(Math.pow(o.x-n.x,2)+Math.pow(o.y-n.y,2)),c=Math.sqrt(Math.pow(n.x-r.x,2)+Math.pow(n.y-r.y,2)),g=Math.pow(c,i),y=Math.pow(c,2*i),v=Math.pow(l,i),b=Math.pow(l,2*i),x=Math.pow(d,i),_=Math.pow(d,2*i),p=2*_+3*x*v+b,u=2*y+3*g*v+b,m=3*x*(x+v),m>0&&(m=1/m),f=3*g*(g+v),f>0&&(f=1/f),a={x:(-b*s.x+p*o.x+_*n.x)*m,y:(-b*s.y+p*o.y+_*n.y)*m},h={x:(y*o.x+u*n.x-b*r.x)*f,y:(y*o.y+u*n.y-b*r.y)*f},0==a.x&&0==a.y&&(a=o),0==h.x&&0==h.y&&(h=n),w+="C"+a.x+","+a.y+" "+h.x+","+h.y+" "+n.x+","+n.y+" ";return w},s._linear=function(t){for(var e="",i=0;it[s].y?t[s].y:e,i=i0&&(n=Math.min(n,Math.abs(c[d-1].x-r))),a=s._getSafeDrawData(n,h,m);else{var g=d+(p[r].amount-p[r].resolved),v=d-(p[r].resolved+1);g0&&(n=Math.min(n,Math.abs(c[v].x-r))),a=s._getSafeDrawData(n,h,m),p[r].resolved+=1,"stack"==h.options.barChart.handleOverlap?(f=p[r].accumulated,p[r].accumulated+=h.zeroPosition-c[d].y):"sideBySide"==h.options.barChart.handleOverlap&&(a.width=a.width/p[r].amount,a.offset+=p[r].resolved*a.width-.5*a.width*(p[r].amount+1),"left"==h.options.barChart.align?a.offset-=.5*a.width:"right"==h.options.barChart.align&&(a.offset+=.5*a.width))}o.drawBar(c[d].x+a.offset,c[d].y-f,a.width,h.zeroPosition-c[d].y,h.className+" bar",i.svgElements,i.svg),1==h.options.drawPoints.enabled&&o.drawPoint(c[d].x+a.offset,c[d].y,h,i.svgElements,i.svg)}},s._getDataIntersections=function(t,e){for(var i,s=0;s0&&(i=Math.min(i,Math.abs(e[s-1].x-e[s].x))),0==i&&(void 0===t[e[s].x]&&(t[e[s].x]={amount:0,resolved:0,accumulated:0}),t[e[s].x].amount+=1)},s._getSafeDrawData=function(t,e,i){var s,o;return t0?(s=i>t?i:t,o=0,"left"==e.options.barChart.align?o-=.5*t:"right"==e.options.barChart.align&&(o+=.5*t)):(s=e.options.barChart.width,o=0,"left"==e.options.barChart.align?o-=.5*e.options.barChart.width:"right"==e.options.barChart.align&&(o+=.5*e.options.barChart.width)),{width:s,offset:o}},s.getStackedBarYRange=function(t,e,i,o,n){if(t.length>0){t.sort(function(t,e){return t.x==e.x?t.groupId-e.groupId:t.x-e.x});var r={};s._getDataIntersections(r,t),e[o]=s._getStackedBarYRange(r,t),e[o].yAxisOrientation=n,i.push(o)}},s._getStackedBarYRange=function(t,e){for(var i,s=e[0].y,o=e[0].y,n=0;ne[n].y?e[n].y:s,o=ot[r].accumulated?t[r].accumulated:s,o=ot[s].y?t[s].y:e,i=is;++s)i[s].apply(this,e)}return this},e.prototype.listeners=function(t){return this._callbacks=this._callbacks||{},this._callbacks[t]||[]},e.prototype.hasListeners=function(t){return!!this.listeners(t).length}},function(t,e,i){var s;(function(t,o){(function(n){function r(t,e,i){switch(arguments.length){case 2:return null!=t?t:e;case 3:return null!=t?t:null!=e?e:i;default:throw new Error("Implement me")}}function a(t,e){return Ie.call(t,e)}function h(){return{empty:!1,unusedTokens:[],unusedInput:[],overflow:-2,charsLeftOver:0,nullInput:!1,invalidMonth:null,invalidFormat:!1,userInvalidated:!1,iso:!1}}function d(t){Ce.suppressDeprecationWarnings===!1&&"undefined"!=typeof console&&console.warn&&console.warn("Deprecation warning: "+t)}function l(t,e){var i=!0;return b(function(){return i&&(d(t),i=!1),e.apply(this,arguments)},e)}function c(t,e){Si[t]||(d(e),Si[t]=!0)}function p(t,e){return function(i){return w(t.call(this,i),e)}}function u(t,e){return function(i){return this.localeData().ordinal(t.call(this,i),e)}}function m(t,e){var i,s,o=12*(e.year()-t.year())+(e.month()-t.month()),n=t.clone().add(o,"months");return 0>e-n?(i=t.clone().add(o-1,"months"),s=(e-n)/(n-i)):(i=t.clone().add(o+1,"months"),s=(e-n)/(i-n)),-(o+s)}function f(t,e,i){var s;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?(s=t.isPM(i),s&&12>e&&(e+=12),s||12!==e||(e=0),e):e}function g(){}function v(t,e){e!==!1&&F(t),_(this,t),this._d=new Date(+t._d),Di===!1&&(Di=!0,Ce.updateOffset(this),Di=!1)}function y(t){var e=N(t),i=e.year||0,s=e.quarter||0,o=e.month||0,n=e.week||0,r=e.day||0,a=e.hour||0,h=e.minute||0,d=e.second||0,l=e.millisecond||0;this._milliseconds=+l+1e3*d+6e4*h+36e5*a,this._days=+r+7*n,this._months=+o+3*s+12*i,this._data={},this._locale=Ce.localeData(),this._bubble()}function b(t,e){for(var i in e)a(e,i)&&(t[i]=e[i]);return a(e,"toString")&&(t.toString=e.toString),a(e,"valueOf")&&(t.valueOf=e.valueOf),t}function _(t,e){var i,s,o;if("undefined"!=typeof e._isAMomentObject&&(t._isAMomentObject=e._isAMomentObject),"undefined"!=typeof e._i&&(t._i=e._i),"undefined"!=typeof e._f&&(t._f=e._f),"undefined"!=typeof e._l&&(t._l=e._l),"undefined"!=typeof e._strict&&(t._strict=e._strict),"undefined"!=typeof e._tzm&&(t._tzm=e._tzm),"undefined"!=typeof e._isUTC&&(t._isUTC=e._isUTC),"undefined"!=typeof e._offset&&(t._offset=e._offset),"undefined"!=typeof e._pf&&(t._pf=e._pf),"undefined"!=typeof e._locale&&(t._locale=e._locale),Ye.length>0)for(i in Ye)s=Ye[i],o=e[s],"undefined"!=typeof o&&(t[s]=o);return t}function x(t){return 0>t?Math.ceil(t):Math.floor(t)}function w(t,e,i){for(var s=""+Math.abs(t),o=t>=0;s.lengths;s++)(i&&t[s]!==e[s]||!i&&L(t[s])!==L(e[s]))&&r++;return r+n}function k(t){if(t){var e=t.toLowerCase().replace(/(.)s$/,"$1");t=gi[t]||vi[e]||e}return t}function N(t){var e,i,s={};for(i in t)a(t,i)&&(e=k(i),e&&(s[e]=t[i]));return s}function I(t){var e,i;if(0===t.indexOf("week"))e=7,i="day";else{if(0!==t.indexOf("month"))return;e=12,i="month"}Ce[t]=function(s,o){var r,a,h=Ce._locale[t],d=[];if("number"==typeof s&&(o=s,s=n),a=function(t){var e=Ce().utc().set(i,t);return h.call(Ce._locale,e,s||"")},null!=o)return a(o);for(r=0;e>r;r++)d.push(a(r));return d}}function L(t){var e=+t,i=0;return 0!==e&&isFinite(e)&&(i=e>=0?Math.floor(e):Math.ceil(e)),i}function z(t,e){return new Date(Date.UTC(t,e+1,0)).getUTCDate()}function P(t,e,i){return me(Ce([t,11,31+e-i]),e,i).week}function A(t){return R(t)?366:365}function R(t){return t%4===0&&t%100!==0||t%400===0}function F(t){var e;t._a&&-2===t._pf.overflow&&(e=t._a[ze]<0||t._a[ze]>11?ze:t._a[Pe]<1||t._a[Pe]>z(t._a[Le],t._a[ze])?Pe:t._a[Ae]<0||t._a[Ae]>24||24===t._a[Ae]&&(0!==t._a[Re]||0!==t._a[Fe]||0!==t._a[He])?Ae:t._a[Re]<0||t._a[Re]>59?Re:t._a[Fe]<0||t._a[Fe]>59?Fe:t._a[He]<0||t._a[He]>999?He:-1,t._pf._overflowDayOfYear&&(Le>e||e>Pe)&&(e=Pe),t._pf.overflow=e)}function H(t){return null==t._isValid&&(t._isValid=!isNaN(t._d.getTime())&&t._pf.overflow<0&&!t._pf.empty&&!t._pf.invalidMonth&&!t._pf.nullInput&&!t._pf.invalidFormat&&!t._pf.userInvalidated,t._strict&&(t._isValid=t._isValid&&0===t._pf.charsLeftOver&&0===t._pf.unusedTokens.length&&t._pf.bigHour===n)),t._isValid}function B(t){return t?t.toLowerCase().replace("_","-"):t}function Y(t){for(var e,i,s,o,n=0;n0;){if(s=W(o.slice(0,e).join("-")))return s;if(i&&i.length>=e&&E(o,i,!0)>=e-1)break;e--}n++}return null}function W(t){var e=null;if(!Be[t]&&We)try{e=Ce.locale(),!function(){var t=new Error('Cannot find module "./locale"');throw t.code="MODULE_NOT_FOUND",t}(),Ce.locale(e)}catch(i){}return Be[t]}function G(t,e){var i,s;return e._isUTC?(i=e.clone(),s=(Ce.isMoment(t)||O(t)?+t:+Ce(t))-+i,i._d.setTime(+i._d+s),Ce.updateOffset(i,!1),i):Ce(t).local()}function j(t){return t.match(/\[[\s\S]/)?t.replace(/^\[|\]$/g,""):t.replace(/\\/g,"")}function U(t){var e,i,s=t.match(Ve);for(e=0,i=s.length;i>e;e++)s[e]=wi[s[e]]?wi[s[e]]:j(s[e]);return function(o){var n="";for(e=0;i>e;e++)n+=s[e]instanceof Function?s[e].call(o,t):s[e];return n}}function V(t,e){return t.isValid()?(e=X(e,t.localeData()),yi[e]||(yi[e]=U(e)),yi[e](t)):t.localeData().invalidDate()}function X(t,e){function i(t){return e.longDateFormat(t)||t}var s=5;for(Xe.lastIndex=0;s>=0&&Xe.test(t);)t=t.replace(Xe,i),Xe.lastIndex=0,s-=1;return t}function q(t,e){var i,s=e._strict;switch(t){case"Q":return oi;case"DDDD":return ri;case"YYYY":case"GGGG":case"gggg":return s?ai:Qe;case"Y":case"G":case"g":return di;case"YYYYYY":case"YYYYY":case"GGGGG":case"ggggg":return s?hi:Ke;case"S":if(s)return oi;case"SS":if(s)return ni;case"SSS":if(s)return ri;case"DDD":return Ze;case"MMM":case"MMMM":case"dd":case"ddd":case"dddd":return Je;case"a":case"A":return e._locale._meridiemParse;case"x":return ii;case"X":return si;case"Z":case"ZZ":return ti;case"T":return ei;case"SSSS":return $e;case"MM":case"DD":case"YY":case"GG":case"gg":case"HH":case"hh":case"mm":case"ss":case"ww":case"WW":return s?ni:qe;case"M":case"D":case"d":case"H":case"h":case"m":case"s":case"w":case"W":case"e":case"E":return qe;case"Do":return s?e._locale._ordinalParse:e._locale._ordinalParseLenient;default:return i=new RegExp(se(ie(t.replace("\\","")),"i"))}}function Z(t){t=t||"";var e=t.match(ti)||[],i=e[e.length-1]||[],s=(i+"").match(mi)||["-",0,0],o=+(60*s[1])+L(s[2]);return"+"===s[0]?o:-o}function Q(t,e,i){var s,o=i._a;switch(t){case"Q":null!=e&&(o[ze]=3*(L(e)-1));break;case"M":case"MM":null!=e&&(o[ze]=L(e)-1);break;case"MMM":case"MMMM":s=i._locale.monthsParse(e,t,i._strict),null!=s?o[ze]=s:i._pf.invalidMonth=e;break;case"D":case"DD":null!=e&&(o[Pe]=L(e));break;case"Do":null!=e&&(o[Pe]=L(parseInt(e.match(/\d{1,2}/)[0],10)));break;case"DDD":case"DDDD":null!=e&&(i._dayOfYear=L(e));break;case"YY":o[Le]=Ce.parseTwoDigitYear(e);break;case"YYYY":case"YYYYY":case"YYYYYY":o[Le]=L(e);break;case"a":case"A":i._meridiem=e;break;case"h":case"hh":i._pf.bigHour=!0;case"H":case"HH":o[Ae]=L(e);break;case"m":case"mm":o[Re]=L(e);break;case"s":case"ss":o[Fe]=L(e);break;case"S":case"SS":case"SSS":case"SSSS":o[He]=L(1e3*("0."+e));break;case"x":i._d=new Date(L(e));break;case"X":i._d=new Date(1e3*parseFloat(e));break;case"Z":case"ZZ":i._useUTC=!0,i._tzm=Z(e);break;case"dd":case"ddd":case"dddd":s=i._locale.weekdaysParse(e),null!=s?(i._w=i._w||{},i._w.d=s):i._pf.invalidWeekday=e;break;case"w":case"ww":case"W":case"WW":case"d":case"e":case"E":t=t.substr(0,1);case"gggg":case"GGGG":case"GGGGG":t=t.substr(0,2),e&&(i._w=i._w||{},i._w[t]=L(e));break;case"gg":case"GG":i._w=i._w||{},i._w[t]=Ce.parseTwoDigitYear(e)}}function K(t){var e,i,s,o,n,a,h;e=t._w,null!=e.GG||null!=e.W||null!=e.E?(n=1,a=4,i=r(e.GG,t._a[Le],me(Ce(),1,4).year),s=r(e.W,1),o=r(e.E,1)):(n=t._locale._week.dow,a=t._locale._week.doy,i=r(e.gg,t._a[Le],me(Ce(),n,a).year),s=r(e.w,1),null!=e.d?(o=e.d,n>o&&++s):o=null!=e.e?e.e+n:n),h=fe(i,s,o,a,n),t._a[Le]=h.year,t._dayOfYear=h.dayOfYear}function $(t){var e,i,s,o,n=[];if(!t._d){for(s=te(t),t._w&&null==t._a[Pe]&&null==t._a[ze]&&K(t),t._dayOfYear&&(o=r(t._a[Le],s[Le]),t._dayOfYear>A(o)&&(t._pf._overflowDayOfYear=!0),i=le(o,0,t._dayOfYear),t._a[ze]=i.getUTCMonth(),t._a[Pe]=i.getUTCDate()),e=0;3>e&&null==t._a[e];++e)t._a[e]=n[e]=s[e];for(;7>e;e++)t._a[e]=n[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[Ae]&&0===t._a[Re]&&0===t._a[Fe]&&0===t._a[He]&&(t._nextDay=!0,t._a[Ae]=0),t._d=(t._useUTC?le:de).apply(null,n),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[Ae]=24)}}function J(t){var e;t._d||(e=N(t._i),t._a=[e.year,e.month,e.day||e.date,e.hour,e.minute,e.second,e.millisecond],$(t))}function te(t){var e=new Date;return t._useUTC?[e.getUTCFullYear(),e.getUTCMonth(),e.getUTCDate()]:[e.getFullYear(),e.getMonth(),e.getDate()]}function ee(t){if(t._f===Ce.ISO_8601)return void ne(t);t._a=[],t._pf.empty=!0;var e,i,s,o,r,a=""+t._i,h=a.length,d=0;for(s=X(t._f,t._locale).match(Ve)||[],e=0;e0&&t._pf.unusedInput.push(r),a=a.slice(a.indexOf(i)+i.length),d+=i.length),wi[o]?(i?t._pf.empty=!1:t._pf.unusedTokens.push(o),Q(o,i,t)):t._strict&&!i&&t._pf.unusedTokens.push(o);t._pf.charsLeftOver=h-d,a.length>0&&t._pf.unusedInput.push(a),t._pf.bigHour===!0&&t._a[Ae]<=12&&(t._pf.bigHour=n),t._a[Ae]=f(t._locale,t._a[Ae],t._meridiem),$(t),F(t)}function ie(t){return t.replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,function(t,e,i,s,o){return e||i||s||o})}function se(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}function oe(t){var e,i,s,o,n;if(0===t._f.length)return t._pf.invalidFormat=!0,void(t._d=new Date(0/0));for(o=0;on)&&(s=n,i=e));b(t,i||e)}function ne(t){var e,i,s=t._i,o=li.exec(s);if(o){for(t._pf.iso=!0,e=0,i=pi.length;i>e;e++)if(pi[e][1].exec(s)){t._f=pi[e][0]+(o[6]||" ");break}for(e=0,i=ui.length;i>e;e++)if(ui[e][1].exec(s)){t._f+=ui[e][0];break}s.match(ti)&&(t._f+="Z"),ee(t)}else t._isValid=!1}function re(t){ne(t),t._isValid===!1&&(delete t._isValid,Ce.createFromInputFallback(t))}function ae(t,e){var i,s=[];for(i=0;it&&a.setFullYear(t),a}function le(t){var e=new Date(Date.UTC.apply(null,arguments));return 1970>t&&e.setUTCFullYear(t),e}function ce(t,e){if("string"==typeof t)if(isNaN(t)){if(t=e.weekdaysParse(t),"number"!=typeof t)return null}else t=parseInt(t,10);return t}function pe(t,e,i,s,o){return o.relativeTime(e||1,!!i,t,s)}function ue(t,e,i){var s=Ce.duration(t).abs(),o=Ne(s.as("s")),n=Ne(s.as("m")),r=Ne(s.as("h")),a=Ne(s.as("d")),h=Ne(s.as("M")),d=Ne(s.as("y")),l=o0,l[4]=i,pe.apply({},l)}function me(t,e,i){var s,o=i-e,n=i-t.day();return n>o&&(n-=7),o-7>n&&(n+=7),s=Ce(t).add(n,"d"),{week:Math.ceil(s.dayOfYear()/7),year:s.year()}}function fe(t,e,i,s,o){var n,r,a=le(t,0,1).getUTCDay();return a=0===a?7:a,i=null!=i?i:o,n=o-a+(a>s?7:0)-(o>a?7:0),r=7*(e-1)+(i-o)+n+1,{year:r>0?t:t-1,dayOfYear:r>0?r:A(t-1)+r}}function ge(t){var e,i=t._i,s=t._f;return t._locale=t._locale||Ce.localeData(t._l),null===i||s===n&&""===i?Ce.invalid({nullInput:!0}):("string"==typeof i&&(t._i=i=t._locale.preparse(i)),Ce.isMoment(i)?new v(i,!0):(s?T(s)?oe(t):ee(t):he(t),e=new v(t),e._nextDay&&(e.add(1,"d"),e._nextDay=n),e))}function ve(t,e){var i,s;if(1===e.length&&T(e[0])&&(e=e[0]),!e.length)return Ce();for(i=e[0],s=1;s=0?"+":"-";return e+w(Math.abs(t),6)},gg:function(){return w(this.weekYear()%100,2)},gggg:function(){return w(this.weekYear(),4)},ggggg:function(){return w(this.weekYear(),5)},GG:function(){return w(this.isoWeekYear()%100,2)},GGGG:function(){return w(this.isoWeekYear(),4)},GGGGG:function(){return w(this.isoWeekYear(),5)},e:function(){return this.weekday()},E:function(){return this.isoWeekday()},a:function(){return this.localeData().meridiem(this.hours(),this.minutes(),!0)},A:function(){return this.localeData().meridiem(this.hours(),this.minutes(),!1)},H:function(){return this.hours()},h:function(){return this.hours()%12||12},m:function(){return this.minutes()},s:function(){return this.seconds()},S:function(){return L(this.milliseconds()/100)},SS:function(){return w(L(this.milliseconds()/10),2)},SSS:function(){return w(this.milliseconds(),3)},SSSS:function(){return w(this.milliseconds(),3)},Z:function(){var t=this.utcOffset(),e="+";return 0>t&&(t=-t,e="-"),e+w(L(t/60),2)+":"+w(L(t)%60,2)},ZZ:function(){var t=this.utcOffset(),e="+";return 0>t&&(t=-t,e="-"),e+w(L(t/60),2)+w(L(t)%60,2)},z:function(){return this.zoneAbbr()},zz:function(){return this.zoneName()},x:function(){return this.valueOf()},X:function(){return this.unix()},Q:function(){return this.quarter()}},Si={},Mi=["months","monthsShort","weekdays","weekdaysShort","weekdaysMin"],Di=!1;_i.length;)Oe=_i.pop(),wi[Oe+"o"]=u(wi[Oe],Oe);for(;xi.length;)Oe=xi.pop(),wi[Oe+Oe]=p(wi[Oe],2);wi.DDDD=p(wi.DDD,3),b(g.prototype,{set:function(t){var e,i;for(i in t)e=t[i],"function"==typeof e?this[i]=e:this["_"+i]=e;this._ordinalParseLenient=new RegExp(this._ordinalParse.source+"|"+/\d{1,2}/.source)},_months:"January_February_March_April_May_June_July_August_September_October_November_December".split("_"),months:function(t){return this._months[t.month()]},_monthsShort:"Jan_Feb_Mar_Apr_May_Jun_Jul_Aug_Sep_Oct_Nov_Dec".split("_"),monthsShort:function(t){return this._monthsShort[t.month()]},monthsParse:function(t,e,i){var s,o,n;for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),s=0;12>s;s++){if(o=Ce.utc([2e3,s]),i&&!this._longMonthsParse[s]&&(this._longMonthsParse[s]=new RegExp("^"+this.months(o,"").replace(".","")+"$","i"),this._shortMonthsParse[s]=new RegExp("^"+this.monthsShort(o,"").replace(".","")+"$","i")),i||this._monthsParse[s]||(n="^"+this.months(o,"")+"|^"+this.monthsShort(o,""),this._monthsParse[s]=new RegExp(n.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[s].test(t))return s;if(i&&"MMM"===e&&this._shortMonthsParse[s].test(t))return s;if(!i&&this._monthsParse[s].test(t))return s}},_weekdays:"Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),weekdays:function(t){return this._weekdays[t.day()]},_weekdaysShort:"Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),weekdaysShort:function(t){return this._weekdaysShort[t.day()]},_weekdaysMin:"Su_Mo_Tu_We_Th_Fr_Sa".split("_"),weekdaysMin:function(t){return this._weekdaysMin[t.day()]},weekdaysParse:function(t){var e,i,s;for(this._weekdaysParse||(this._weekdaysParse=[]),e=0;7>e;e++)if(this._weekdaysParse[e]||(i=Ce([2e3,1]).day(e),s="^"+this.weekdays(i,"")+"|^"+this.weekdaysShort(i,"")+"|^"+this.weekdaysMin(i,""),this._weekdaysParse[e]=new RegExp(s.replace(".",""),"i")),this._weekdaysParse[e].test(t))return e},_longDateFormat:{LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY LT",LLLL:"dddd, MMMM D, YYYY LT"},longDateFormat:function(t){var e=this._longDateFormat[t]; -return!e&&this._longDateFormat[t.toUpperCase()]&&(e=this._longDateFormat[t.toUpperCase()].replace(/MMMM|MM|DD|dddd/g,function(t){return t.slice(1)}),this._longDateFormat[t]=e),e},isPM:function(t){return"p"===(t+"").toLowerCase().charAt(0)},_meridiemParse:/[ap]\.?m?\.?/i,meridiem:function(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"},_calendar:{sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},calendar:function(t,e,i){var s=this._calendar[t];return"function"==typeof s?s.apply(e,[i]):s},_relativeTime:{future:"in %s",past:"%s ago",s:"a few seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},relativeTime:function(t,e,i,s){var o=this._relativeTime[i];return"function"==typeof o?o(t,e,i,s):o.replace(/%d/i,t)},pastFuture:function(t,e){var i=this._relativeTime[t>0?"future":"past"];return"function"==typeof i?i(e):i.replace(/%s/i,e)},ordinal:function(t){return this._ordinal.replace("%d",t)},_ordinal:"%d",_ordinalParse:/\d{1,2}/,preparse:function(t){return t},postformat:function(t){return t},week:function(t){return me(t,this._week.dow,this._week.doy).week},_week:{dow:0,doy:6},firstDayOfWeek:function(){return this._week.dow},firstDayOfYear:function(){return this._week.doy},_invalidDate:"Invalid date",invalidDate:function(){return this._invalidDate}}),Ce=function(t,e,i,s){var o;return"boolean"==typeof i&&(s=i,i=n),o={},o._isAMomentObject=!0,o._i=t,o._f=e,o._l=i,o._strict=s,o._isUTC=!1,o._pf=h(),ge(o)},Ce.suppressDeprecationWarnings=!1,Ce.createFromInputFallback=l("moment construction falls back to js Date. This is discouraged and will be removed in upcoming major release. Please refer to https://github.com/moment/moment/issues/1407 for more info.",function(t){t._d=new Date(t._i+(t._useUTC?" UTC":""))}),Ce.min=function(){var t=[].slice.call(arguments,0);return ve("isBefore",t)},Ce.max=function(){var t=[].slice.call(arguments,0);return ve("isAfter",t)},Ce.utc=function(t,e,i,s){var o;return"boolean"==typeof i&&(s=i,i=n),o={},o._isAMomentObject=!0,o._useUTC=!0,o._isUTC=!0,o._l=i,o._i=t,o._f=e,o._strict=s,o._pf=h(),ge(o).utc()},Ce.unix=function(t){return Ce(1e3*t)},Ce.duration=function(t,e){var i,s,o,n,r=t,h=null;return Ce.isDuration(t)?r={ms:t._milliseconds,d:t._days,M:t._months}:"number"==typeof t?(r={},e?r[e]=t:r.milliseconds=t):(h=je.exec(t))?(i="-"===h[1]?-1:1,r={y:0,d:L(h[Pe])*i,h:L(h[Ae])*i,m:L(h[Re])*i,s:L(h[Fe])*i,ms:L(h[He])*i}):(h=Ue.exec(t))?(i="-"===h[1]?-1:1,o=function(t){var e=t&&parseFloat(t.replace(",","."));return(isNaN(e)?0:e)*i},r={y:o(h[2]),M:o(h[3]),d:o(h[4]),h:o(h[5]),m:o(h[6]),s:o(h[7]),w:o(h[8])}):null==r?r={}:"object"==typeof r&&("from"in r||"to"in r)&&(n=M(Ce(r.from),Ce(r.to)),r={},r.ms=n.milliseconds,r.M=n.months),s=new y(r),Ce.isDuration(t)&&a(t,"_locale")&&(s._locale=t._locale),s},Ce.version=Ee,Ce.defaultFormat=ci,Ce.ISO_8601=function(){},Ce.momentProperties=Ye,Ce.updateOffset=function(){},Ce.relativeTimeThreshold=function(t,e){return bi[t]===n?!1:e===n?bi[t]:(bi[t]=e,!0)},Ce.lang=l("moment.lang is deprecated. Use moment.locale instead.",function(t,e){return Ce.locale(t,e)}),Ce.locale=function(t,e){var i;return t&&(i="undefined"!=typeof e?Ce.defineLocale(t,e):Ce.localeData(t),i&&(Ce.duration._locale=Ce._locale=i)),Ce._locale._abbr},Ce.defineLocale=function(t,e){return null!==e?(e.abbr=t,Be[t]||(Be[t]=new g),Be[t].set(e),Ce.locale(t),Be[t]):(delete Be[t],null)},Ce.langData=l("moment.langData is deprecated. Use moment.localeData instead.",function(t){return Ce.localeData(t)}),Ce.localeData=function(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return Ce._locale;if(!T(t)){if(e=W(t))return e;t=[t]}return Y(t)},Ce.isMoment=function(t){return t instanceof v||null!=t&&a(t,"_isAMomentObject")},Ce.isDuration=function(t){return t instanceof y};for(Oe=Mi.length-1;Oe>=0;--Oe)I(Mi[Oe]);Ce.normalizeUnits=function(t){return k(t)},Ce.invalid=function(t){var e=Ce.utc(0/0);return null!=t?b(e._pf,t):e._pf.userInvalidated=!0,e},Ce.parseZone=function(){return Ce.apply(null,arguments).parseZone()},Ce.parseTwoDigitYear=function(t){return L(t)+(L(t)>68?1900:2e3)},Ce.isDate=O,b(Ce.fn=v.prototype,{clone:function(){return Ce(this)},valueOf:function(){return+this._d-6e4*(this._offset||0)},unix:function(){return Math.floor(+this/1e3)},toString:function(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")},toDate:function(){return this._offset?new Date(+this):this._d},toISOString:function(){var t=Ce(this).utc();return 00:!1},parsingFlags:function(){return b({},this._pf)},invalidAt:function(){return this._pf.overflow},utc:function(t){return this.utcOffset(0,t)},local:function(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(this._dateUtcOffset(),"m")),this},format:function(t){var e=V(this,t||Ce.defaultFormat);return this.localeData().postformat(e)},add:D(1,"add"),subtract:D(-1,"subtract"),diff:function(t,e,i){var s,o,n=G(t,this),r=6e4*(n.utcOffset()-this.utcOffset());return e=k(e),"year"===e||"month"===e||"quarter"===e?(o=m(this,n),"quarter"===e?o/=3:"year"===e&&(o/=12)):(s=this-n,o="second"===e?s/1e3:"minute"===e?s/6e4:"hour"===e?s/36e5:"day"===e?(s-r)/864e5:"week"===e?(s-r)/6048e5:s),i?o:x(o)},from:function(t,e){return Ce.duration({to:this,from:t}).locale(this.locale()).humanize(!e)},fromNow:function(t){return this.from(Ce(),t)},calendar:function(t){var e=t||Ce(),i=G(e,this).startOf("day"),s=this.diff(i,"days",!0),o=-6>s?"sameElse":-1>s?"lastWeek":0>s?"lastDay":1>s?"sameDay":2>s?"nextDay":7>s?"nextWeek":"sameElse";return this.format(this.localeData().calendar(o,this,Ce(e)))},isLeapYear:function(){return R(this.year())},isDST:function(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()},day:function(t){var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=ce(t,this.localeData()),this.add(t-e,"d")):e},month:xe("Month",!0),startOf:function(t){switch(t=k(t)){case"year":this.month(0);case"quarter":case"month":this.date(1);case"week":case"isoWeek":case"day":this.hours(0);case"hour":this.minutes(0);case"minute":this.seconds(0);case"second":this.milliseconds(0)}return"week"===t?this.weekday(0):"isoWeek"===t&&this.isoWeekday(1),"quarter"===t&&this.month(3*Math.floor(this.month()/3)),this},endOf:function(t){return t=k(t),t===n||"millisecond"===t?this:this.startOf(t).add(1,"isoWeek"===t?"week":t).subtract(1,"ms")},isAfter:function(t,e){var i;return e=k("undefined"!=typeof e?e:"millisecond"),"millisecond"===e?(t=Ce.isMoment(t)?t:Ce(t),+this>+t):(i=Ce.isMoment(t)?+t:+Ce(t),i<+this.clone().startOf(e))},isBefore:function(t,e){var i;return e=k("undefined"!=typeof e?e:"millisecond"),"millisecond"===e?(t=Ce.isMoment(t)?t:Ce(t),+t>+this):(i=Ce.isMoment(t)?+t:+Ce(t),+this.clone().endOf(e)t?this:t}),max:l("moment().max is deprecated, use moment.max instead. https://github.com/moment/moment/issues/1548",function(t){return t=Ce.apply(null,arguments),t>this?this:t}),zone:l("moment().zone is deprecated, use moment().utcOffset instead. https://github.com/moment/moment/issues/1779",function(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()}),utcOffset:function(t,e){var i,s=this._offset||0;return null!=t?("string"==typeof t&&(t=Z(t)),Math.abs(t)<16&&(t=60*t),!this._isUTC&&e&&(i=this._dateUtcOffset()),this._offset=t,this._isUTC=!0,null!=i&&this.add(i,"m"),s!==t&&(!e||this._changeInProgress?C(this,Ce.duration(t-s,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,Ce.updateOffset(this,!0),this._changeInProgress=null)),this):this._isUTC?s:this._dateUtcOffset()},isLocal:function(){return!this._isUTC},isUtcOffset:function(){return this._isUTC},isUtc:function(){return this._isUTC&&0===this._offset},zoneAbbr:function(){return this._isUTC?"UTC":""},zoneName:function(){return this._isUTC?"Coordinated Universal Time":""},parseZone:function(){return this._tzm?this.utcOffset(this._tzm):"string"==typeof this._i&&this.utcOffset(Z(this._i)),this},hasAlignedHourOffset:function(t){return t=t?Ce(t).utcOffset():0,(this.utcOffset()-t)%60===0},daysInMonth:function(){return z(this.year(),this.month())},dayOfYear:function(t){var e=Ne((Ce(this).startOf("day")-Ce(this).startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")},quarter:function(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)},weekYear:function(t){var e=me(this,this.localeData()._week.dow,this.localeData()._week.doy).year;return null==t?e:this.add(t-e,"y")},isoWeekYear:function(t){var e=me(this,1,4).year;return null==t?e:this.add(t-e,"y")},week:function(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")},isoWeek:function(t){var e=me(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")},weekday:function(t){var e=(this.day()+7-this.localeData()._week.dow)%7;return null==t?e:this.add(t-e,"d")},isoWeekday:function(t){return null==t?this.day()||7:this.day(this.day()%7?t:t-7)},isoWeeksInYear:function(){return P(this.year(),1,4)},weeksInYear:function(){var t=this.localeData()._week;return P(this.year(),t.dow,t.doy)},get:function(t){return t=k(t),this[t]()},set:function(t,e){var i;if("object"==typeof t)for(i in t)this.set(i,t[i]);else t=k(t),"function"==typeof this[t]&&this[t](e);return this},locale:function(t){var e;return t===n?this._locale._abbr:(e=Ce.localeData(t),null!=e&&(this._locale=e),this)},lang:l("moment().lang() is deprecated. Instead, use moment().localeData() to get the language configuration. Use moment().locale() to change languages.",function(t){return t===n?this.localeData():this.locale(t)}),localeData:function(){return this._locale},_dateUtcOffset:function(){return 15*-Math.round(this._d.getTimezoneOffset()/15)}}),Ce.fn.millisecond=Ce.fn.milliseconds=xe("Milliseconds",!1),Ce.fn.second=Ce.fn.seconds=xe("Seconds",!1),Ce.fn.minute=Ce.fn.minutes=xe("Minutes",!1),Ce.fn.hour=Ce.fn.hours=xe("Hours",!0),Ce.fn.date=xe("Date",!0),Ce.fn.dates=l("dates accessor is deprecated. Use date instead.",xe("Date",!0)),Ce.fn.year=xe("FullYear",!0),Ce.fn.years=l("years accessor is deprecated. Use year instead.",xe("FullYear",!0)),Ce.fn.days=Ce.fn.day,Ce.fn.months=Ce.fn.month,Ce.fn.weeks=Ce.fn.week,Ce.fn.isoWeeks=Ce.fn.isoWeek,Ce.fn.quarters=Ce.fn.quarter,Ce.fn.toJSON=Ce.fn.toISOString,Ce.fn.isUTC=Ce.fn.isUtc,b(Ce.duration.fn=y.prototype,{_bubble:function(){var t,e,i,s=this._milliseconds,o=this._days,n=this._months,r=this._data,a=0;r.milliseconds=s%1e3,t=x(s/1e3),r.seconds=t%60,e=x(t/60),r.minutes=e%60,i=x(e/60),r.hours=i%24,o+=x(i/24),a=x(we(o)),o-=x(Se(a)),n+=x(o/30),o%=30,a+=x(n/12),n%=12,r.days=o,r.months=n,r.years=a},abs:function(){return this._milliseconds=Math.abs(this._milliseconds),this._days=Math.abs(this._days),this._months=Math.abs(this._months),this._data.milliseconds=Math.abs(this._data.milliseconds),this._data.seconds=Math.abs(this._data.seconds),this._data.minutes=Math.abs(this._data.minutes),this._data.hours=Math.abs(this._data.hours),this._data.months=Math.abs(this._data.months),this._data.years=Math.abs(this._data.years),this},weeks:function(){return x(this.days()/7)},valueOf:function(){return this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*L(this._months/12)},humanize:function(t){var e=ue(this,!t,this.localeData());return t&&(e=this.localeData().pastFuture(+this,e)),this.localeData().postformat(e)},add:function(t,e){var i=Ce.duration(t,e);return this._milliseconds+=i._milliseconds,this._days+=i._days,this._months+=i._months,this._bubble(),this},subtract:function(t,e){var i=Ce.duration(t,e);return this._milliseconds-=i._milliseconds,this._days-=i._days,this._months-=i._months,this._bubble(),this},get:function(t){return t=k(t),this[t.toLowerCase()+"s"]()},as:function(t){var e,i;if(t=k(t),"month"===t||"year"===t)return e=this._days+this._milliseconds/864e5,i=this._months+12*we(e),"month"===t?i:i/12;switch(e=this._days+Math.round(Se(this._months/12)),t){case"week":return e/7+this._milliseconds/6048e5;case"day":return e+this._milliseconds/864e5;case"hour":return 24*e+this._milliseconds/36e5;case"minute":return 24*e*60+this._milliseconds/6e4;case"second":return 24*e*60*60+this._milliseconds/1e3;case"millisecond":return Math.floor(24*e*60*60*1e3)+this._milliseconds;default:throw new Error("Unknown unit "+t)}},lang:Ce.fn.lang,locale:Ce.fn.locale,toIsoString:l("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",function(){return this.toISOString()}),toISOString:function(){var t=Math.abs(this.years()),e=Math.abs(this.months()),i=Math.abs(this.days()),s=Math.abs(this.hours()),o=Math.abs(this.minutes()),n=Math.abs(this.seconds()+this.milliseconds()/1e3);return this.asSeconds()?(this.asSeconds()<0?"-":"")+"P"+(t?t+"Y":"")+(e?e+"M":"")+(i?i+"D":"")+(s||o||n?"T":"")+(s?s+"H":"")+(o?o+"M":"")+(n?n+"S":""):"P0D"},localeData:function(){return this._locale},toJSON:function(){return this.toISOString()}}),Ce.duration.fn.toString=Ce.duration.fn.toISOString;for(Oe in fi)a(fi,Oe)&&Me(Oe.toLowerCase());Ce.duration.fn.asMilliseconds=function(){return this.as("ms")},Ce.duration.fn.asSeconds=function(){return this.as("s")},Ce.duration.fn.asMinutes=function(){return this.as("m")},Ce.duration.fn.asHours=function(){return this.as("h")},Ce.duration.fn.asDays=function(){return this.as("d")},Ce.duration.fn.asWeeks=function(){return this.as("weeks")},Ce.duration.fn.asMonths=function(){return this.as("M")},Ce.duration.fn.asYears=function(){return this.as("y")},Ce.locale("en",{ordinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10,i=1===L(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th";return t+i}}),We?o.exports=Ce:(s=function(t,e,i){return i.config&&i.config()&&i.config().noGlobal===!0&&(ke.moment=Te),Ce}.call(e,i,e,o),!(s!==n&&(o.exports=s)),De(!0))}).call(this)}).call(e,function(){return this}(),i(72)(t))},function(t,e){var i,s,o;!function(n,r){s=[],i=r,o="function"==typeof i?i.apply(e,s):i,!(void 0!==o&&(t.exports=o))}(this,function(){function t(t){var e,i=t&&t.preventDefault||!1,s=t&&t.container||window,o={},n={keydown:{},keyup:{}},r={};for(e=97;122>=e;e++)r[String.fromCharCode(e)]={code:65+(e-97),shift:!1};for(e=65;90>=e;e++)r[String.fromCharCode(e)]={code:e,shift:!0};for(e=0;9>=e;e++)r[""+e]={code:48+e,shift:!1};for(e=1;12>=e;e++)r["F"+e]={code:111+e,shift:!1};for(e=0;9>=e;e++)r["num"+e]={code:96+e,shift:!1};r["num*"]={code:106,shift:!1},r["num+"]={code:107,shift:!1},r["num-"]={code:109,shift:!1},r["num/"]={code:111,shift:!1},r["num."]={code:110,shift:!1},r.left={code:37,shift:!1},r.up={code:38,shift:!1},r.right={code:39,shift:!1},r.down={code:40,shift:!1},r.space={code:32,shift:!1},r.enter={code:13,shift:!1},r.shift={code:16,shift:void 0},r.esc={code:27,shift:!1},r.backspace={code:8,shift:!1},r.tab={code:9,shift:!1},r.ctrl={code:17,shift:!1},r.alt={code:18,shift:!1},r["delete"]={code:46,shift:!1},r.pageup={code:33,shift:!1},r.pagedown={code:34,shift:!1},r["="]={code:187,shift:!1},r["-"]={code:189,shift:!1},r["]"]={code:221,shift:!1},r["["]={code:219,shift:!1};var a=function(t){d(t,"keydown")},h=function(t){d(t,"keyup")},d=function(t,e){if(void 0!==n[e][t.keyCode]){for(var s=n[e][t.keyCode],o=0;o0?i._handlers[t]=s:(i._off(t,o),delete i._handlers[t]))}),i},i.destroy=function(){var t=i.element;delete t.hammer,i._handlers={},i._destroy()},i}})},function(t,e,i){var s;!function(o,n,r,a){function h(t,e,i){return setTimeout(m(t,i),e)}function d(t,e,i){return Array.isArray(t)?(l(t,i[e],i),!0):!1}function l(t,e,i){var s;if(t)if(t.forEach)t.forEach(e,i);else if(t.length!==a)for(s=0;s-1}function x(t){return t.trim().split(/\s+/g)}function w(t,e,i){if(t.indexOf&&!i)return t.indexOf(e);for(var s=0;si[e]}):s.sort()),s}function D(t,e){for(var i,s,o=e[0].toUpperCase()+e.slice(1),n=0;n1&&!i.firstMultiple?i.firstMultiple=z(e):1===o&&(i.firstMultiple=!1);var n=i.firstInput,r=i.firstMultiple,a=r?r.center:n.center,h=e.center=P(s);e.timeStamp=ve(),e.deltaTime=e.timeStamp-n.timeStamp,e.angle=H(a,h),e.distance=F(a,h),I(i,e),e.offsetDirection=R(e.deltaX,e.deltaY),e.scale=r?Y(r.pointers,s):1,e.rotation=r?B(r.pointers,s):0,L(i,e);var d=t.element;b(e.srcEvent.target,d)&&(d=e.srcEvent.target),e.target=d}function I(t,e){var i=e.center,s=t.offsetDelta||{},o=t.prevDelta||{},n=t.prevInput||{};(e.eventType===Oe||n.eventType===ke)&&(o=t.prevDelta={x:n.deltaX||0,y:n.deltaY||0},s=t.offsetDelta={x:i.x,y:i.y}),e.deltaX=o.x+(i.x-s.x),e.deltaY=o.y+(i.y-s.y)}function L(t,e){var i,s,o,n,r=t.lastInterval||e,h=e.timeStamp-r.timeStamp;if(e.eventType!=Ne&&(h>Te||r.velocity===a)){var d=r.deltaX-e.deltaX,l=r.deltaY-e.deltaY,c=A(h,d,l);s=c.x,o=c.y,i=ge(c.x)>ge(c.y)?c.x:c.y,n=R(d,l),t.lastInterval=e}else i=r.velocity,s=r.velocityX,o=r.velocityY,n=r.direction;e.velocity=i,e.velocityX=s,e.velocityY=o,e.direction=n}function z(t){for(var e=[],i=0;io;)i+=t[o].clientX,s+=t[o].clientY,o++;return{x:fe(i/e),y:fe(s/e)}}function A(t,e,i){return{x:e/t||0,y:i/t||0}}function R(t,e){return t===e?Ie:ge(t)>=ge(e)?t>0?Le:ze:e>0?Pe:Ae}function F(t,e,i){i||(i=Be);var s=e[i[0]]-t[i[0]],o=e[i[1]]-t[i[1]];return Math.sqrt(s*s+o*o)}function H(t,e,i){i||(i=Be);var s=e[i[0]]-t[i[0]],o=e[i[1]]-t[i[1]];return 180*Math.atan2(o,s)/Math.PI}function B(t,e){return H(e[1],e[0],Ye)-H(t[1],t[0],Ye)}function Y(t,e){return F(e[0],e[1],Ye)/F(t[0],t[1],Ye)}function W(){this.evEl=Ge,this.evWin=je,this.allow=!0,this.pressed=!1,O.apply(this,arguments)}function G(){this.evEl=Xe,this.evWin=qe,O.apply(this,arguments),this.store=this.manager.session.pointerEvents=[]}function j(){this.evTarget=Qe,this.evWin=Ke,this.started=!1,O.apply(this,arguments)}function U(t,e){var i=S(t.touches),s=S(t.changedTouches);return e&(ke|Ne)&&(i=M(i.concat(s),"identifier",!0)),[i,s]}function V(){this.evTarget=Je,this.targetIds={},O.apply(this,arguments)}function X(t,e){var i=S(t.touches),s=this.targetIds;if(e&(Oe|Ee)&&1===i.length)return s[i[0].identifier]=!0,[i,i];var o,n,r=S(t.changedTouches),a=[],h=this.target;if(n=i.filter(function(t){return b(t.target,h)}),e===Oe)for(o=0;oa&&(e.push(t),a=e.length-1):o&(ke|Ne)&&(i=!0),0>a||(e[a]=t,this.callback(this.manager,o,{pointers:e,changedPointers:[t],pointerType:n,srcEvent:t}),i&&e.splice(a,1))}});var Ze={touchstart:Oe,touchmove:Ee,touchend:ke,touchcancel:Ne},Qe="touchstart",Ke="touchstart touchmove touchend touchcancel";u(j,O,{handler:function(t){var e=Ze[t.type];if(e===Oe&&(this.started=!0),this.started){var i=U.call(this,t,e);e&(ke|Ne)&&i[0].length-i[1].length===0&&(this.started=!1),this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Se,srcEvent:t})}}});var $e={touchstart:Oe,touchmove:Ee,touchend:ke,touchcancel:Ne},Je="touchstart touchmove touchend touchcancel";u(V,O,{handler:function(t){var e=$e[t.type],i=X.call(this,t,e);i&&this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Se,srcEvent:t})}}),u(q,O,{handler:function(t,e,i){var s=i.pointerType==Se,o=i.pointerType==De;if(s)this.mouse.allow=!1;else if(o&&!this.mouse.allow)return;e&(ke|Ne)&&(this.mouse.allow=!0),this.callback(t,e,i)},destroy:function(){this.touch.destroy(),this.mouse.destroy()}});var ti=D(ue.style,"touchAction"),ei=ti!==a,ii="compute",si="auto",oi="manipulation",ni="none",ri="pan-x",ai="pan-y";Z.prototype={set:function(t){t==ii&&(t=this.compute()),ei&&(this.manager.element.style[ti]=t),this.actions=t.toLowerCase().trim()},update:function(){this.set(this.manager.options.touchAction)},compute:function(){var t=[];return l(this.manager.recognizers,function(e){f(e.options.enable,[e])&&(t=t.concat(e.getTouchAction()))}),Q(t.join(" "))},preventDefaults:function(t){if(!ei){var e=t.srcEvent,i=t.offsetDirection;if(this.manager.session.prevented)return void e.preventDefault();var s=this.actions,o=_(s,ni),n=_(s,ai),r=_(s,ri);return o||n&&i&Re||r&&i&Fe?this.preventSrc(e):void 0}},preventSrc:function(t){this.manager.session.prevented=!0,t.preventDefault()}};var hi=1,di=2,li=4,ci=8,pi=ci,ui=16,mi=32;K.prototype={defaults:{},set:function(t){return c(this.options,t),this.manager&&this.manager.touchAction.update(),this},recognizeWith:function(t){if(d(t,"recognizeWith",this))return this;var e=this.simultaneous;return t=te(t,this),e[t.id]||(e[t.id]=t,t.recognizeWith(this)),this},dropRecognizeWith:function(t){return d(t,"dropRecognizeWith",this)?this:(t=te(t,this),delete this.simultaneous[t.id],this)},requireFailure:function(t){if(d(t,"requireFailure",this))return this;var e=this.requireFail;return t=te(t,this),-1===w(e,t)&&(e.push(t),t.requireFailure(this)),this},dropRequireFailure:function(t){if(d(t,"dropRequireFailure",this))return this;t=te(t,this);var e=w(this.requireFail,t);return e>-1&&this.requireFail.splice(e,1),this},hasRequireFailures:function(){return this.requireFail.length>0},canRecognizeWith:function(t){return!!this.simultaneous[t.id]},emit:function(t){function e(e){i.manager.emit(i.options.event+(e?$(s):""),t)}var i=this,s=this.state;ci>s&&e(!0),e(),s>=ci&&e(!0)},tryEmit:function(t){return this.canEmit()?this.emit(t):void(this.state=mi)},canEmit:function(){for(var t=0;tn?Le:ze,i=n!=this.pX,s=Math.abs(t.deltaX)):(o=0===r?Ie:0>r?Pe:Ae,i=r!=this.pY,s=Math.abs(t.deltaY))),t.direction=o,i&&s>e.threshold&&o&e.direction},attrTest:function(t){return ee.prototype.attrTest.call(this,t)&&(this.state&di||!(this.state&di)&&this.directionTest(t))},emit:function(t){this.pX=t.deltaX,this.pY=t.deltaY;var e=J(t.direction);e&&this.manager.emit(this.options.event+e,t),this._super.emit.call(this,t)}}),u(se,ee,{defaults:{event:"pinch",threshold:0,pointers:2},getTouchAction:function(){return[ni]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.scale-1)>this.options.threshold||this.state&di)},emit:function(t){if(this._super.emit.call(this,t),1!==t.scale){var e=t.scale<1?"in":"out";this.manager.emit(this.options.event+e,t)}}}),u(oe,K,{defaults:{event:"press",pointers:1,time:500,threshold:5},getTouchAction:function(){return[si]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,s=t.distancee.time;if(this._input=t,!s||!i||t.eventType&(ke|Ne)&&!o)this.reset();else if(t.eventType&Oe)this.reset(),this._timer=h(function(){this.state=pi,this.tryEmit() -},e.time,this);else if(t.eventType&ke)return pi;return mi},reset:function(){clearTimeout(this._timer)},emit:function(t){this.state===pi&&(t&&t.eventType&ke?this.manager.emit(this.options.event+"up",t):(this._input.timeStamp=ve(),this.manager.emit(this.options.event,this._input)))}}),u(ne,ee,{defaults:{event:"rotate",threshold:0,pointers:2},getTouchAction:function(){return[ni]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.rotation)>this.options.threshold||this.state&di)}}),u(re,ee,{defaults:{event:"swipe",threshold:10,velocity:.65,direction:Re|Fe,pointers:1},getTouchAction:function(){return ie.prototype.getTouchAction.call(this)},attrTest:function(t){var e,i=this.options.direction;return i&(Re|Fe)?e=t.velocity:i&Re?e=t.velocityX:i&Fe&&(e=t.velocityY),this._super.attrTest.call(this,t)&&i&t.direction&&t.distance>this.options.threshold&&ge(e)>this.options.velocity&&t.eventType&ke},emit:function(t){var e=J(t.direction);e&&this.manager.emit(this.options.event+e,t),this.manager.emit(this.options.event,t)}}),u(ae,K,{defaults:{event:"tap",pointers:1,taps:1,interval:300,time:250,threshold:2,posThreshold:10},getTouchAction:function(){return[oi]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,s=t.distancet&&s>o;)o%3==0?(this.forceAggregateHubs(!0),this.normalizeClusterLevels()):this.increaseClusterLevel(),i=this.nodeIndices.length,o+=1;o>0&&1==e&&this.repositionNodes(),this._updateCalculationNodes()},e.openCluster=function(t){var e=this.moving;if(t.clusterSize>this.constants.clustering.sectorThreshold&&this._nodeInActiveArea(t)&&("default"!=this._sector()||1!=this.nodeIndices.length)){this._addSector(t);for(var i=0;this.nodeIndices.lengthi;)this.decreaseClusterLevel(),i+=1}else this._expandClusterNode(t,!1,!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this._updateCalculationNodes(),this.updateLabels();this.moving!=e&&this.start()},e.updateClustersDefault=function(){1==this.constants.clustering.enabled&&this.updateClusters(0,!1,!1)},e.increaseClusterLevel=function(){this.updateClusters(-1,!1,!0)},e.decreaseClusterLevel=function(){this.updateClusters(1,!1,!0)},e.updateClusters=function(t,e,i,s){var o=this.moving,n=this.nodeIndices.length;this.previousScale>this.scale&&0==t&&this._collapseSector(),this.previousScale>this.scale||-1==t?this._formClusters(i):(this.previousScalethis.scale||-1==t)&&(this._aggregateHubs(i),this._updateNodeIndexList()),(this.previousScale>this.scale||-1==t)&&(this.handleChains(),this._updateNodeIndexList()),this.previousScale=this.scale,this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.lengththis.constants.clustering.chainThreshold&&this._reduceAmountOfChains(1-this.constants.clustering.chainThreshold/t)},e._aggregateHubs=function(t){this._getHubSize(),this._formClustersByHub(t,!1)},e.forceAggregateHubs=function(t){var e=this.moving,i=this.nodeIndices.length;this._aggregateHubs(!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.length!=i&&(this.clusterSession+=1),(0==t||void 0===t)&&this.moving!=e&&this.start()},e._openClustersBySize=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];1==e.inView()&&(e.width*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientWidth||e.height*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientHeight)&&this.openCluster(e)}},e._openClusters=function(t,e){for(var i=0;i1&&(t.clusterSizei)){var r=n.from,a=n.to;n.to.options.mass>n.from.options.mass&&(r=n.to,a=n.from),1==a.dynamicEdgesLength?this._addToCluster(r,a,!1):1==r.dynamicEdgesLength&&this._addToCluster(a,r,!1)}}},e._forceClustersByZoom=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];if(1==e.dynamicEdgesLength&&0!=e.dynamicEdges.length){var i=e.dynamicEdges[0],s=i.toId==e.id?this.nodes[i.fromId]:this.nodes[i.toId];e.id!=s.id&&(s.options.mass>e.options.mass?this._addToCluster(s,e,!0):this._addToCluster(e,s,!0))}}},e._clusterToSmallestNeighbour=function(t){for(var e=-1,i=null,s=0;so.clusterSessions.length&&(e=o.clusterSessions.length,i=o)}null!=o&&void 0!==this.nodes[o.id]&&this._addToCluster(o,t,!0)},e._formClustersByHub=function(t,e){for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&this._formClusterFromHub(this.nodes[i],t,e)},e._formClusterFromHub=function(t,e,i,s){if(void 0===s&&(s=0),t.dynamicEdgesLength>=this.hubThreshold&&0==i||t.dynamicEdgesLength==this.hubThreshold&&1==i){for(var o,n,r,a=this.constants.clustering.clusterEdgeThreshold/this.scale,h=!1,d=[],l=t.dynamicEdges.length,c=0;l>c;c++)d.push(t.dynamicEdges[c].id);if(0==e)for(h=!1,c=0;l>c;c++){var p=this.edges[d[c]];if(void 0!==p&&p.connected&&p.toId!=p.fromId&&(o=p.to.x-p.from.x,n=p.to.y-p.from.y,r=Math.sqrt(o*o+n*n),a>r)){h=!0;break}}if(!e&&h||e)for(c=0;l>c;c++)if(p=this.edges[d[c]],void 0!==p){var u=this.nodes[p.fromId==t.id?p.toId:p.fromId];u.dynamicEdges.length<=this.hubThreshold+s&&u.id!=t.id&&this._addToCluster(t,u,e)}}},e._addToCluster=function(t,e,i){t.containedNodes[e.id]=e;for(var s=0;s1)for(var s=0;s1&&(e.label="[".concat(String(e.clusterSize),"]"))}for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(e=this.nodes[t],1==e.clusterSize&&(e.label=void 0!==e.originalLabel?e.originalLabel:String(e.id)))},e.normalizeClusterLevels=function(){var t,e=0,i=1e9,s=0;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(s=this.nodes[t].clusterSessions.length,s>e&&(e=s),i>s&&(i=s));if(e-i>this.constants.clustering.clusterLevelDifference){var o=this.nodeIndices.length,n=e-this.constants.clustering.clusterLevelDifference;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&this.nodes[t].clusterSessions.lengths&&(s=n.dynamicEdgesLength),t+=n.dynamicEdgesLength,e+=Math.pow(n.dynamicEdgesLength,2),i+=1}t/=i,e/=i;var r=e-Math.pow(t,2),a=Math.sqrt(r);this.hubThreshold=Math.floor(t+2*a),this.hubThreshold>s&&(this.hubThreshold=s)},e._reduceAmountOfChains=function(t){this.hubThreshold=2;var e=Math.floor(this.nodeIndices.length*t);for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&e>0&&(this._formClusterFromHub(this.nodes[i],!0,!0,1),e-=1)},e._getChainFraction=function(){var t=0,e=0;for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&(2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&(t+=1),e+=1);return t/e}},function(t,e,i){var s=i(1),o=i(40);e._putDataInSector=function(){this.sectors.active[this._sector()].nodes=this.nodes,this.sectors.active[this._sector()].edges=this.edges,this.sectors.active[this._sector()].nodeIndices=this.nodeIndices},e._switchToSector=function(t,e){void 0===e||"active"==e?this._switchToActiveSector(t):this._switchToFrozenSector(t)},e._switchToActiveSector=function(t){this.nodeIndices=this.sectors.active[t].nodeIndices,this.nodes=this.sectors.active[t].nodes,this.edges=this.sectors.active[t].edges},e._switchToSupportSector=function(){this.nodeIndices=this.sectors.support.nodeIndices,this.nodes=this.sectors.support.nodes,this.edges=this.sectors.support.edges},e._switchToFrozenSector=function(t){this.nodeIndices=this.sectors.frozen[t].nodeIndices,this.nodes=this.sectors.frozen[t].nodes,this.edges=this.sectors.frozen[t].edges},e._loadLatestSector=function(){this._switchToSector(this._sector())},e._sector=function(){return this.activeSector[this.activeSector.length-1]},e._previousSector=function(){if(this.activeSector.length>1)return this.activeSector[this.activeSector.length-2];throw new TypeError("there are not enough sectors in the this.activeSector array.")},e._setActiveSector=function(t){this.activeSector.push(t)},e._forgetLastSector=function(){this.activeSector.pop()},e._createNewSector=function(t){this.sectors.active[t]={nodes:{},edges:{},nodeIndices:[],formationScale:this.scale,drawingNode:void 0},this.sectors.active[t].drawingNode=new o({id:t,color:{background:"#eaefef",border:"495c5e"}},{},{},this.constants),this.sectors.active[t].drawingNode.clusterSize=2},e._deleteActiveSector=function(t){delete this.sectors.active[t]},e._deleteFrozenSector=function(t){delete this.sectors.frozen[t]},e._freezeSector=function(t){this.sectors.frozen[t]=this.sectors.active[t],this._deleteActiveSector(t)},e._activateSector=function(t){this.sectors.active[t]=this.sectors.frozen[t],this._deleteFrozenSector(t)},e._mergeThisWithFrozen=function(t){for(var e in this.nodes)this.nodes.hasOwnProperty(e)&&(this.sectors.frozen[t].nodes[e]=this.nodes[e]);for(var i in this.edges)this.edges.hasOwnProperty(i)&&(this.sectors.frozen[t].edges[i]=this.edges[i]);for(var s=0;s1?this[t](o[0],o[1]):this[t](e))}return this._loadLatestSector(),i},e._doInSupportSector=function(t,e){var i=!1;if(void 0===e)this._switchToSupportSector(),i=this[t]();else{this._switchToSupportSector();var s=Array.prototype.splice.call(arguments,1);i=s.length>1?this[t](s[0],s[1]):this[t](e)}return this._loadLatestSector(),i},e._doInAllFrozenSectors=function(t,e){if(void 0===e)for(var i in this.sectors.frozen)this.sectors.frozen.hasOwnProperty(i)&&(this._switchToFrozenSector(i),this[t]());else for(var i in this.sectors.frozen)if(this.sectors.frozen.hasOwnProperty(i)){this._switchToFrozenSector(i);var s=Array.prototype.splice.call(arguments,1);s.length>1?this[t](s[0],s[1]):this[t](e)}this._loadLatestSector()},e._doInAllSectors=function(t,e){var i=Array.prototype.splice.call(arguments,1);void 0===e?(this._doInAllActiveSectors(t),this._doInAllFrozenSectors(t)):i.length>1?(this._doInAllActiveSectors(t,i[0],i[1]),this._doInAllFrozenSectors(t,i[0],i[1])):(this._doInAllActiveSectors(t,e),this._doInAllFrozenSectors(t,e))},e._clearNodeIndexList=function(){var t=this._sector();this.sectors.active[t].nodeIndices=[],this.nodeIndices=this.sectors.active[t].nodeIndices},e._drawSectorNodes=function(t,e){var i,s=1e9,o=-1e9,n=1e9,r=-1e9;for(var a in this.sectors[e])if(this.sectors[e].hasOwnProperty(a)&&void 0!==this.sectors[e][a].drawingNode){this._switchToSector(a,e),s=1e9,o=-1e9,n=1e9,r=-1e9;for(var h in this.nodes)this.nodes.hasOwnProperty(h)&&(i=this.nodes[h],i.resize(t),n>i.x-.5*i.width&&(n=i.x-.5*i.width),ri.y-.5*i.height&&(s=i.y-.5*i.height),o0?this.nodes[i[i.length-1]]:null},e._getEdgesOverlappingWith=function(t,e){var i=this.edges;for(var s in i)i.hasOwnProperty(s)&&i[s].isOverlappingWith(t)&&e.push(s)},e._getAllEdgesOverlappingWith=function(t){var e=[];return this._doInAllActiveSectors("_getEdgesOverlappingWith",t,e),e},e._getEdgeAt=function(t){var e=this._pointerToPositionObject(t),i=this._getAllEdgesOverlappingWith(e);return i.length>0?this.edges[i[i.length-1]]:null},e._addToSelection=function(t){t instanceof s?this.selectionObj.nodes[t.id]=t:this.selectionObj.edges[t.id]=t},e._addToHover=function(t){t instanceof s?this.hoverObj.nodes[t.id]=t:this.hoverObj.edges[t.id]=t},e._removeFromSelection=function(t){t instanceof s?delete this.selectionObj.nodes[t.id]:delete this.selectionObj.edges[t.id]},e._unselectAll=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].unselect();for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&this.selectionObj.edges[i].unselect();this.selectionObj={nodes:{},edges:{}},0==t&&this.emit("select",this.getSelection())},e._unselectClusters=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].clusterSize>1&&(this.selectionObj.nodes[e].unselect(),this._removeFromSelection(this.selectionObj.nodes[e]));0==t&&this.emit("select",this.getSelection())},e._getSelectedNodeCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedNode=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return this.selectionObj.nodes[t];return null},e._getSelectedEdge=function(){for(var t in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(t))return this.selectionObj.edges[t];return null},e._getSelectedEdgeCount=function(){var t=0;for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedObjectCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&(t+=1);return t},e._selectionIsEmpty=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return!1;for(var e in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(e))return!1;return!0},e._clusterInSelection=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].clusterSize>1)return!0;return!1},e._selectConnectedEdges=function(t){for(var e=0;ei;i++){o=t[i];var n=this.nodes[o];if(!n)throw new RangeError('Node with id "'+o+'" not found');this._selectObject(n,!0,!0,e,!0)}this.redraw()},e.selectEdges=function(t){var e,i,s;if(!t||void 0==t.length)throw"Selection must be an array with ids";for(this._unselectAll(!0),e=0,i=t.length;i>e;e++){s=t[e];var o=this.edges[s];if(!o)throw new RangeError('Edge with id "'+s+'" not found');this._selectObject(o,!0,!0,!1,!0)}this.redraw()},e._updateSelection=function(){for(var t in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(t)&&(this.nodes.hasOwnProperty(t)||delete this.selectionObj.nodes[t]);for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(this.edges.hasOwnProperty(e)||delete this.selectionObj.edges[e])}},function(t,e,i){var s=i(1),o=i(40),n=i(37);e._clearManipulatorBar=function(){this._recursiveDOMDelete(this.manipulationDiv),this.manipulationDOM={},this._manipulationReleaseOverload=function(){},delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode,this.controlNodesActive=!1,this.freezeSimulation=!1},e._restoreOverloadedFunctions=function(){for(var t in this.cachedFunctions)this.cachedFunctions.hasOwnProperty(t)&&(this[t]=this.cachedFunctions[t],delete this.cachedFunctions[t])},e._toggleEditMode=function(){this.editMode=!this.editMode;var t=this.manipulationDiv,e=this.closeDiv,i=this.editModeDiv;1==this.editMode?(t.style.display="block",e.style.display="block",i.style.display="none",e.onclick=this._toggleEditMode.bind(this)):(t.style.display="none",e.style.display="none",i.style.display="block",e.onclick=null),this._createManipulatorBar()},e._createManipulatorBar=function(){this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];if(void 0!==this.edgeBeingEdited&&(this.edgeBeingEdited._disableControlNodes(),this.edgeBeingEdited=void 0,this.selectedControlNode=null,this.controlNodesActive=!1,this._redraw()),this._restoreOverloadedFunctions(),this.freezeSimulation=!1,this.blockConnectingEdgeSelection=!1,this.forceAppendSelection=!1,this.manipulationDOM={},1==this.editMode){for(;this.manipulationDiv.hasChildNodes();)this.manipulationDiv.removeChild(this.manipulationDiv.firstChild);this.manipulationDOM.addNodeSpan=document.createElement("span"),this.manipulationDOM.addNodeSpan.className="network-manipulationUI add",this.manipulationDOM.addNodeLabelSpan=document.createElement("span"),this.manipulationDOM.addNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addNodeLabelSpan.innerHTML=t.addNode,this.manipulationDOM.addNodeSpan.appendChild(this.manipulationDOM.addNodeLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.addEdgeSpan=document.createElement("span"),this.manipulationDOM.addEdgeSpan.className="network-manipulationUI connect",this.manipulationDOM.addEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.addEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addEdgeLabelSpan.innerHTML=t.addEdge,this.manipulationDOM.addEdgeSpan.appendChild(this.manipulationDOM.addEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.addNodeSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.addEdgeSpan),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?(this.manipulationDOM.seperatorLineDiv2=document.createElement("div"),this.manipulationDOM.seperatorLineDiv2.className="network-seperatorLine",this.manipulationDOM.editNodeSpan=document.createElement("span"),this.manipulationDOM.editNodeSpan.className="network-manipulationUI edit",this.manipulationDOM.editNodeLabelSpan=document.createElement("span"),this.manipulationDOM.editNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editNodeLabelSpan.innerHTML=t.editNode,this.manipulationDOM.editNodeSpan.appendChild(this.manipulationDOM.editNodeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv2),this.manipulationDiv.appendChild(this.manipulationDOM.editNodeSpan)):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.seperatorLineDiv3=document.createElement("div"),this.manipulationDOM.seperatorLineDiv3.className="network-seperatorLine",this.manipulationDOM.editEdgeSpan=document.createElement("span"),this.manipulationDOM.editEdgeSpan.className="network-manipulationUI edit",this.manipulationDOM.editEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.editEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editEdgeLabelSpan.innerHTML=t.editEdge,this.manipulationDOM.editEdgeSpan.appendChild(this.manipulationDOM.editEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv3),this.manipulationDiv.appendChild(this.manipulationDOM.editEdgeSpan)),0==this._selectionIsEmpty()&&(this.manipulationDOM.seperatorLineDiv4=document.createElement("div"),this.manipulationDOM.seperatorLineDiv4.className="network-seperatorLine",this.manipulationDOM.deleteSpan=document.createElement("span"),this.manipulationDOM.deleteSpan.className="network-manipulationUI delete",this.manipulationDOM.deleteLabelSpan=document.createElement("span"),this.manipulationDOM.deleteLabelSpan.className="network-manipulationLabel",this.manipulationDOM.deleteLabelSpan.innerHTML=t.del,this.manipulationDOM.deleteSpan.appendChild(this.manipulationDOM.deleteLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv4),this.manipulationDiv.appendChild(this.manipulationDOM.deleteSpan)),this.manipulationDOM.addNodeSpan.onclick=this._createAddNodeToolbar.bind(this),this.manipulationDOM.addEdgeSpan.onclick=this._createAddEdgeToolbar.bind(this),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?this.manipulationDOM.editNodeSpan.onclick=this._editNode.bind(this):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.editEdgeSpan.onclick=this._createEditEdgeToolbar.bind(this)),0==this._selectionIsEmpty()&&(this.manipulationDOM.deleteSpan.onclick=this._deleteSelected.bind(this)),this.closeDiv.onclick=this._toggleEditMode.bind(this); -var e=this;this.boundFunction=e._createManipulatorBar,this.on("select",this.boundFunction)}else{for(;this.editModeDiv.hasChildNodes();)this.editModeDiv.removeChild(this.editModeDiv.firstChild);this.manipulationDOM.editModeSpan=document.createElement("span"),this.manipulationDOM.editModeSpan.className="network-manipulationUI edit editmode",this.manipulationDOM.editModeLabelSpan=document.createElement("span"),this.manipulationDOM.editModeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editModeLabelSpan.innerHTML=t.edit,this.manipulationDOM.editModeSpan.appendChild(this.manipulationDOM.editModeLabelSpan),this.editModeDiv.appendChild(this.manipulationDOM.editModeSpan),this.manipulationDOM.editModeSpan.onclick=this._toggleEditMode.bind(this)}},e._createAddNodeToolbar=function(){this._clearManipulatorBar(),this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.addDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._addNode,this.on("select",this.boundFunction)},e._createAddEdgeToolbar=function(){this._clearManipulatorBar(),this._unselectAll(!0),this.freezeSimulation=!0,this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this._unselectAll(),this.forceAppendSelection=!1,this.blockConnectingEdgeSelection=!0,this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.edgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._handleConnect,this.on("select",this.boundFunction),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleDragEnd=this._handleDragEnd,this._handleTouch=this._handleConnect,this._manipulationReleaseOverload=function(){},this._handleDragStart=function(){},this._handleDragEnd=this._finishConnect,this._redraw()},e._createEditEdgeToolbar=function(){this._clearManipulatorBar(),this.controlNodesActive=!0,this.boundFunction&&this.off("select",this.boundFunction),this.edgeBeingEdited=this._getSelectedEdge(),this.edgeBeingEdited._enableControlNodes();var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.editEdgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleTap=this._handleTap,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleTouch=this._selectControlNode,this._handleTap=function(){},this._handleOnDrag=this._controlNodeDrag,this._handleDragStart=function(){},this._manipulationReleaseOverload=this._releaseControlNode,this._redraw()},e._selectControlNode=function(t){this.edgeBeingEdited.controlNodes.from.unselect(),this.edgeBeingEdited.controlNodes.to.unselect(),this.selectedControlNode=this.edgeBeingEdited._getSelectedControlNode(this._XconvertDOMtoCanvas(t.x),this._YconvertDOMtoCanvas(t.y)),null!==this.selectedControlNode&&(this.selectedControlNode.select(),this.freezeSimulation=!0),this._redraw()},e._controlNodeDrag=function(t){var e=this._getPointer(t.center);null!==this.selectedControlNode&&void 0!==this.selectedControlNode&&(this.selectedControlNode.x=this._XconvertDOMtoCanvas(e.x),this.selectedControlNode.y=this._YconvertDOMtoCanvas(e.y)),this._redraw()},e._releaseControlNode=function(t){var e=this._getNodeAt(t);null!==e?(1==this.edgeBeingEdited.controlNodes.from.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(e.id,this.edgeBeingEdited.to.id),this.edgeBeingEdited.controlNodes.from.unselect()),1==this.edgeBeingEdited.controlNodes.to.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(this.edgeBeingEdited.from.id,e.id),this.edgeBeingEdited.controlNodes.to.unselect())):this.edgeBeingEdited._restoreControlNodes(),this.freezeSimulation=!1,this._redraw()},e._handleConnect=function(t){if(0==this._getSelectedNodeCount()){var e=this._getNodeAt(t);if(null!=e)if(e.clusterSize>1)alert(this.constants.locales[this.constants.locale].createEdgeError);else{this._selectObject(e,!1);var i=this.sectors.support.nodes;i.targetNode=new o({id:"targetNode"},{},{},this.constants);var s=i.targetNode;s.x=e.x,s.y=e.y,this.edges.connectionEdge=new n({id:"connectionEdge",from:e.id,to:s.id},this,this.constants);var r=this.edges.connectionEdge;r.from=e,r.connected=!0,r.options.smoothCurves={enabled:!0,dynamic:!1,type:"continuous",roundness:.5},r.selected=!0,r.to=s,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleOnDrag=function(t){var e=this._getPointer(t.center),i=this.edges.connectionEdge;i.to.x=this._XconvertDOMtoCanvas(e.x),i.to.y=this._YconvertDOMtoCanvas(e.y)},this.moving=!0,this.start()}}},e._finishConnect=function(t){if(1==this._getSelectedNodeCount()){var e=this._getPointer(t.center);this._handleOnDrag=this.cachedFunctions._handleOnDrag,delete this.cachedFunctions._handleOnDrag;var i=this.edges.connectionEdge.fromId;delete this.edges.connectionEdge,delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode;var s=this._getNodeAt(e);null!=s&&(s.clusterSize>1?alert(this.constants.locales[this.constants.locale].createEdgeError):(this._createEdge(i,s.id),this._createManipulatorBar())),this._unselectAll()}},e._addNode=function(){if(this._selectionIsEmpty()&&1==this.editMode){var t=this._pointerToPositionObject(this.pointerPosition),e={id:s.randomUUID(),x:t.left,y:t.top,label:"new",allowedToMoveX:!0,allowedToMoveY:!0};if(this.triggerFunctions.add){if(2!=this.triggerFunctions.add.length)throw new Error("The function for add does not support two arguments (data,callback)");var i=this;this.triggerFunctions.add(e,function(t){i.nodesData.add(t),i._createManipulatorBar(),i.moving=!0,i.start()})}else this.nodesData.add(e),this._createManipulatorBar(),this.moving=!0,this.start()}},e._createEdge=function(t,e){if(1==this.editMode){var i={from:t,to:e};if(this.triggerFunctions.connect){if(2!=this.triggerFunctions.connect.length)throw new Error("The function for connect does not support two arguments (data,callback)");var s=this;this.triggerFunctions.connect(i,function(t){s.edgesData.add(t),s.moving=!0,s.start()})}else this.edgesData.add(i),this.moving=!0,this.start()}},e._editEdge=function(t,e){if(1==this.editMode){var i={id:this.edgeBeingEdited.id,from:t,to:e};if(this.triggerFunctions.editEdge){if(2!=this.triggerFunctions.editEdge.length)throw new Error("The function for edit does not support two arguments (data, callback)");var s=this;this.triggerFunctions.editEdge(i,function(t){s.edgesData.update(t),s.moving=!0,s.start()})}else this.edgesData.update(i),this.moving=!0,this.start()}},e._editNode=function(){if(!this.triggerFunctions.edit||1!=this.editMode)throw new Error("No edit function has been bound to this button");var t=this._getSelectedNode(),e={id:t.id,label:t.label,group:t.options.group,shape:t.options.shape,color:{background:t.options.color.background,border:t.options.color.border,highlight:{background:t.options.color.highlight.background,border:t.options.color.highlight.border}}};if(2!=this.triggerFunctions.edit.length)throw new Error("The function for edit does not support two arguments (data, callback)");var i=this;this.triggerFunctions.edit(e,function(t){i.nodesData.update(t),i._createManipulatorBar(),i.moving=!0,i.start()})},e._deleteSelected=function(){if(!this._selectionIsEmpty()&&1==this.editMode)if(this._clusterInSelection())alert(this.constants.locales[this.constants.locale].deleteClusterError);else{var t=this.getSelectedNodes(),e=this.getSelectedEdges();if(this.triggerFunctions.del){var i=this,s={nodes:t,edges:e};if(2!=this.triggerFunctions.del.length)throw new Error("The function for delete does not support two arguments (data, callback)");this.triggerFunctions.del(s,function(t){i.edgesData.remove(t.edges),i.nodesData.remove(t.nodes),i._unselectAll(),i.moving=!0,i.start()})}else this.edgesData.remove(e),this.nodesData.remove(t),this._unselectAll(),this.moving=!0,this.start()}}},function(t,e,i){var s=(i(1),i(47)),o=i(45);e._cleanNavigation=function(){if(0!=this.navigationHammers.existing.length){for(var t=0;t0){var t,e,i=0,s=!1,o=!1;for(e in this.nodes)this.nodes.hasOwnProperty(e)&&(t=this.nodes[e],-1!=t.level?s=!0:o=!0,is&&(n.xFixed=!1,n.x=i[n.level].minPos,r=!0):n.yFixed&&n.level>s&&(n.yFixed=!1,n.y=i[n.level].minPos,r=!0),1==r&&(i[n.level].minPos+=i[n.level].nodeSpacing,n.edges.length>1&&this._placeBranchNodes(n.edges,n.id,i,n.level))}},e._setLevel=function(t,e,i){for(var s=0;st)&&(o.level=t,o.edges.length>1&&this._setLevel(t+1,o.edges,o.id))}},e._setLevelDirected=function(t,e,i){this.nodes[i].hierarchyEnumerated=!0;for(var s,o,n=0;n1&&s.hierarchyEnumerated===!1&&this._setLevelDirected(s.level,s.edges,s.id)},e._restoreNodes=function(){for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.nodes[t].xFixed=!1,this.nodes[t].yFixed=!1)}},function(t,e,i){function s(){this.constants.smoothCurves.enabled=!this.constants.smoothCurves.enabled;var t=document.getElementById("graph_toggleSmooth");t.style.background=1==this.constants.smoothCurves.enabled?"#A4FF56":"#FF8532",this._configureSmoothCurves(!1)}function o(){for(var t in this.calculationNodes)this.calculationNodes.hasOwnProperty(t)&&(this.calculationNodes[t].vx=0,this.calculationNodes[t].vy=0,this.calculationNodes[t].fx=0,this.calculationNodes[t].fy=0);1==this.constants.hierarchicalLayout.enabled?(this._setupHierarchicalLayout(),a.call(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),a.call(this,"graph_H_cg",1,"physics_centralGravity"),a.call(this,"graph_H_sc",1,"physics_springConstant"),a.call(this,"graph_H_sl",1,"physics_springLength"),a.call(this,"graph_H_damp",1,"physics_damping")):this.repositionNodes(),this.moving=!0,this.start()}function n(){var t="No options are required, default values used.",e=[],i=document.getElementById("graph_physicsMethod1"),s=document.getElementById("graph_physicsMethod2");if(1==i.checked){if(this.constants.physics.barnesHut.gravitationalConstant!=this.backupConstants.physics.barnesHut.gravitationalConstant&&e.push("gravitationalConstant: "+this.constants.physics.barnesHut.gravitationalConstant),this.constants.physics.centralGravity!=this.backupConstants.physics.barnesHut.centralGravity&&e.push("centralGravity: "+this.constants.physics.centralGravity),this.constants.physics.springLength!=this.backupConstants.physics.barnesHut.springLength&&e.push("springLength: "+this.constants.physics.springLength),this.constants.physics.springConstant!=this.backupConstants.physics.barnesHut.springConstant&&e.push("springConstant: "+this.constants.physics.springConstant),this.constants.physics.damping!=this.backupConstants.physics.barnesHut.damping&&e.push("damping: "+this.constants.physics.damping),0!=e.length){t="var options = {",t+="physics: {barnesHut: {";for(var o=0;othis.constants.clustering.clusterThreshold&&1==this.constants.clustering.enabled&&this.clusterToFit(this.constants.clustering.reduceToNodes,!1),this._calculateForces())},e._calculateForces=function(){this._calculateGravitationalForces(),this._calculateNodeForces(),this.constants.physics.springConstant>0&&(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic?this._calculateSpringForcesWithSupport():1==this.constants.physics.hierarchicalRepulsion.enabled?this._calculateHierarchicalSpringForces():this._calculateSpringForces())},e._updateCalculationNodes=function(){if(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic){this.calculationNodes={},this.calculationNodeIndices=[];for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.calculationNodes[t]=this.nodes[t]);var e=this.sectors.support.nodes;for(var i in e)e.hasOwnProperty(i)&&(this.edges.hasOwnProperty(e[i].parentEdgeId)?this.calculationNodes[i]=e[i]:e[i]._setForce(0,0));for(var s in this.calculationNodes)this.calculationNodes.hasOwnProperty(s)&&this.calculationNodeIndices.push(s)}else this.calculationNodes=this.nodes,this.calculationNodeIndices=this.nodeIndices},e._calculateGravitationalForces=function(){var t,e,i,s,o,n=this.calculationNodes,r=this.constants.physics.centralGravity,a=0;for(o=0;oSimulation Mode:Barnes HutRepulsionHierarchical
    Options:
    ',this.containerElement.parentElement.insertBefore(this.physicsConfiguration,this.containerElement),this.optionsDiv=document.createElement("div"),this.optionsDiv.style.fontSize="14px",this.optionsDiv.style.fontFamily="verdana",this.containerElement.parentElement.insertBefore(this.optionsDiv,this.containerElement); -var e;e=document.getElementById("graph_BH_gc"),e.onchange=a.bind(this,"graph_BH_gc",-1,"physics_barnesHut_gravitationalConstant"),e=document.getElementById("graph_BH_cg"),e.onchange=a.bind(this,"graph_BH_cg",1,"physics_centralGravity"),e=document.getElementById("graph_BH_sc"),e.onchange=a.bind(this,"graph_BH_sc",1,"physics_springConstant"),e=document.getElementById("graph_BH_sl"),e.onchange=a.bind(this,"graph_BH_sl",1,"physics_springLength"),e=document.getElementById("graph_BH_damp"),e.onchange=a.bind(this,"graph_BH_damp",1,"physics_damping"),e=document.getElementById("graph_R_nd"),e.onchange=a.bind(this,"graph_R_nd",1,"physics_repulsion_nodeDistance"),e=document.getElementById("graph_R_cg"),e.onchange=a.bind(this,"graph_R_cg",1,"physics_centralGravity"),e=document.getElementById("graph_R_sc"),e.onchange=a.bind(this,"graph_R_sc",1,"physics_springConstant"),e=document.getElementById("graph_R_sl"),e.onchange=a.bind(this,"graph_R_sl",1,"physics_springLength"),e=document.getElementById("graph_R_damp"),e.onchange=a.bind(this,"graph_R_damp",1,"physics_damping"),e=document.getElementById("graph_H_nd"),e.onchange=a.bind(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),e=document.getElementById("graph_H_cg"),e.onchange=a.bind(this,"graph_H_cg",1,"physics_centralGravity"),e=document.getElementById("graph_H_sc"),e.onchange=a.bind(this,"graph_H_sc",1,"physics_springConstant"),e=document.getElementById("graph_H_sl"),e.onchange=a.bind(this,"graph_H_sl",1,"physics_springLength"),e=document.getElementById("graph_H_damp"),e.onchange=a.bind(this,"graph_H_damp",1,"physics_damping"),e=document.getElementById("graph_H_direction"),e.onchange=a.bind(this,"graph_H_direction",t,"hierarchicalLayout_direction"),e=document.getElementById("graph_H_levsep"),e.onchange=a.bind(this,"graph_H_levsep",1,"hierarchicalLayout_levelSeparation"),e=document.getElementById("graph_H_nspac"),e.onchange=a.bind(this,"graph_H_nspac",1,"hierarchicalLayout_nodeSpacing");var i=document.getElementById("graph_physicsMethod1"),d=document.getElementById("graph_physicsMethod2"),l=document.getElementById("graph_physicsMethod3");d.checked=!0,this.constants.physics.barnesHut.enabled&&(i.checked=!0),this.constants.hierarchicalLayout.enabled&&(l.checked=!0);var c=document.getElementById("graph_toggleSmooth"),p=document.getElementById("graph_repositionNodes"),u=document.getElementById("graph_generateOptions");c.onclick=s.bind(this),p.onclick=o.bind(this),u.onclick=n.bind(this),c.style.background=1==this.constants.smoothCurves&&0==this.constants.dynamicSmoothCurves?"#A4FF56":"#FF8532",r.apply(this),i.onchange=r.bind(this),d.onchange=r.bind(this),l.onchange=r.bind(this)}},e._overWriteGraphConstants=function(t,e){var i=t.split("_");1==i.length?this.constants[i[0]]=e:2==i.length?this.constants[i[0]][i[1]]=e:3==i.length&&(this.constants[i[0]][i[1]][i[2]]=e)}},function(t){function e(t){throw new Error("Cannot find module '"+t+"'.")}e.keys=function(){return[]},e.resolve=e,t.exports=e,e.id=68},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l,c=this.calculationNodes,p=this.calculationNodeIndices,u=-2/3,m=4/3,f=this.constants.physics.repulsion.nodeDistance,g=f;for(d=0;di&&(r=.5*g>i?1:v*i+m,r*=0==n?1:1+n*this.constants.clustering.forceAmplification,r/=Math.max(i,.01*g),s=t*r,o=e*r,a.fx-=s,a.fy-=o,h.fx+=s,h.fy+=o)}}},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l=this.calculationNodes,c=this.calculationNodeIndices,p=this.constants.physics.hierarchicalRepulsion.nodeDistance;for(h=0;hi?-Math.pow(u*i,2)+Math.pow(u*p,2):0,0==i?i=.01:n/=i,s=t*n,o=e*n,r.fx-=s,r.fy-=o,a.fx+=s,a.fy+=o}},e._calculateHierarchicalSpringForces=function(){for(var t,e,i,s,o,n,r,a,h,d=this.edges,l=this.calculationNodes,c=this.calculationNodeIndices,p=0;pn;n++)t=e[i[n]],t.options.mass>0&&(this._getForceContribution(o.root.children.NW,t),this._getForceContribution(o.root.children.NE,t),this._getForceContribution(o.root.children.SW,t),this._getForceContribution(o.root.children.SE,t))}},e._getForceContribution=function(t,e){if(t.childrenCount>0){var i,s,o;if(i=t.centerOfMass.x-e.x,s=t.centerOfMass.y-e.y,o=Math.sqrt(i*i+s*s),o*t.calcSize>this.constants.physics.barnesHut.thetaInverted){0==o&&(o=.1*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}else if(4==t.childrenCount)this._getForceContribution(t.children.NW,e),this._getForceContribution(t.children.NE,e),this._getForceContribution(t.children.SW,e),this._getForceContribution(t.children.SE,e);else if(t.children.data.id!=e.id){0==o&&(o=.5*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}}},e._formBarnesHutTree=function(t,e){for(var i,s=e.length,o=Number.MAX_VALUE,n=Number.MAX_VALUE,r=-Number.MAX_VALUE,a=-Number.MAX_VALUE,h=0;s>h;h++){var d=t[e[h]].x,l=t[e[h]].y;t[e[h]].options.mass>0&&(o>d&&(o=d),d>r&&(r=d),n>l&&(n=l),l>a&&(a=l))}var c=Math.abs(r-o)-Math.abs(a-n);c>0?(n-=.5*c,a+=.5*c):(o+=.5*c,r-=.5*c);var p=1e-5,u=Math.max(p,Math.abs(r-o)),m=.5*u,f=.5*(o+r),g=.5*(n+a),v={root:{centerOfMass:{x:0,y:0},mass:0,range:{minX:f-m,maxX:f+m,minY:g-m,maxY:g+m},size:u,calcSize:1/u,children:{data:null},maxWidth:0,level:0,childrenCount:4}};for(this._splitBranch(v.root),h=0;s>h;h++)i=t[e[h]],i.options.mass>0&&this._placeInTree(v.root,i);this.barnesHutTree=v},e._updateBranchMass=function(t,e){var i=t.mass+e.options.mass,s=1/i;t.centerOfMass.x=t.centerOfMass.x*t.mass+e.x*e.options.mass,t.centerOfMass.x*=s,t.centerOfMass.y=t.centerOfMass.y*t.mass+e.y*e.options.mass,t.centerOfMass.y*=s,t.mass=i;var o=Math.max(Math.max(e.height,e.radius),e.width);t.maxWidth=t.maxWidthe.x?t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NW"):this._placeInRegion(t,e,"SW"):t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NE"):this._placeInRegion(t,e,"SE")},e._placeInRegion=function(t,e,i){switch(t.children[i].childrenCount){case 0:t.children[i].children.data=e,t.children[i].childrenCount=1,this._updateBranchMass(t.children[i],e);break;case 1:t.children[i].children.data.x==e.x&&t.children[i].children.data.y==e.y?(e.x+=Math.random(),e.y+=Math.random()):(this._splitBranch(t.children[i]),this._placeInTree(t.children[i],e));break;case 4:this._placeInTree(t.children[i],e)}},e._splitBranch=function(t){var e=null;1==t.childrenCount&&(e=t.children.data,t.mass=0,t.centerOfMass.x=0,t.centerOfMass.y=0),t.childrenCount=4,t.children.data=null,this._insertRegion(t,"NW"),this._insertRegion(t,"NE"),this._insertRegion(t,"SW"),this._insertRegion(t,"SE"),null!=e&&this._placeInTree(t,e)},e._insertRegion=function(t,e){var i,s,o,n,r=.5*t.size;switch(e){case"NW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY,n=t.range.minY+r;break;case"NE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY,n=t.range.minY+r;break;case"SW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY+r,n=t.range.maxY;break;case"SE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY+r,n=t.range.maxY}t.children[e]={centerOfMass:{x:0,y:0},mass:0,range:{minX:i,maxX:s,minY:o,maxY:n},size:.5*t.size,calcSize:2*t.calcSize,children:{data:null},maxWidth:0,level:t.level+1,childrenCount:0}},e._drawTree=function(t,e){void 0!==this.barnesHutTree&&(t.lineWidth=1,this._drawBranch(this.barnesHutTree.root,t,e))},e._drawBranch=function(t,e,i){void 0===i&&(i="#FF0000"),4==t.childrenCount&&(this._drawBranch(t.children.NW,e),this._drawBranch(t.children.NE,e),this._drawBranch(t.children.SE,e),this._drawBranch(t.children.SW,e)),e.strokeStyle=i,e.beginPath(),e.moveTo(t.range.minX,t.range.minY),e.lineTo(t.range.maxX,t.range.minY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.minY),e.lineTo(t.range.maxX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.maxY),e.lineTo(t.range.minX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.minX,t.range.maxY),e.lineTo(t.range.minX,t.range.minY),e.stroke()}},function(t){t.exports=function(t){return t.webpackPolyfill||(t.deprecate=function(){},t.paths=[],t.children=[],t.webpackPolyfill=1),t}},function(t,e){(function(e){t.exports=e}).call(e,{})}])}); +"use strict";!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define([],e):"object"==typeof exports?exports.vis=e():t.vis=e()}(this,function(){return function(t){function e(o){if(i[o])return i[o].exports;var n=i[o]={exports:{},id:o,loaded:!1};return t[o].call(n.exports,n,n.exports,e),n.loaded=!0,n.exports}var i={};return e.m=t,e.c=i,e.p="",e(0)}([function(t,e,i){var o=i(1);o.extend(e,i(7)),o.extend(e,i(24)),o.extend(e,i(60))},function(t,e,i){var o="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},n=i(2),s=i(6);e.isNumber=function(t){return t instanceof Number||"number"==typeof t},e.recursiveDOMDelete=function(t){if(t)for(;t.hasChildNodes()===!0;)e.recursiveDOMDelete(t.firstChild),t.removeChild(t.firstChild)},e.giveRange=function(t,e,i,o){if(e==t)return.5;var n=1/(e-t);return Math.max(0,(o-t)*n)},e.isString=function(t){return t instanceof String||"string"==typeof t},e.isDate=function(t){if(t instanceof Date)return!0;if(e.isString(t)){var i=r.exec(t);if(i)return!0;if(!isNaN(Date.parse(t)))return!0}return!1},e.randomUUID=function(){return s.v4()},e.assignAllKeys=function(t,e){for(var i in t)t.hasOwnProperty(i)&&"object"!==o(t[i])&&(t[i]=e)},e.fillIfDefined=function(t,i){var n=arguments.length<=2||void 0===arguments[2]?!1:arguments[2];for(var s in t)void 0!==i[s]&&("object"!==o(i[s])?void 0!==i[s]&&null!==i[s]||void 0===t[s]||n!==!0?t[s]=i[s]:delete t[s]:"object"===o(t[s])&&e.fillIfDefined(t[s],i[s],n))},e.protoExtend=function(t,e){for(var i=1;ii;i++)if(t[i]!=e[i])return!1;return!0},e.convert=function(t,i){var o;if(void 0!==t){if(null===t)return null;if(!i)return t;if("string"!=typeof i&&!(i instanceof String))throw new Error("Type must be a string");switch(i){case"boolean":case"Boolean":return Boolean(t);case"number":case"Number":return Number(t.valueOf());case"string":case"String":return String(t);case"Date":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return new Date(t.valueOf());if(n.isMoment(t))return new Date(t.valueOf());if(e.isString(t))return o=r.exec(t),o?new Date(Number(o[1])):n(t).toDate();throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"Moment":if(e.isNumber(t))return n(t);if(t instanceof Date)return n(t.valueOf());if(n.isMoment(t))return n(t);if(e.isString(t))return o=r.exec(t),n(o?Number(o[1]):t);throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"ISODate":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return t.toISOString();if(n.isMoment(t))return t.toDate().toISOString();if(e.isString(t))return o=r.exec(t),o?new Date(Number(o[1])).toISOString():new Date(t).toISOString();throw new Error("Cannot convert object of type "+e.getType(t)+" to type ISODate");case"ASPDate":if(e.isNumber(t))return"/Date("+t+")/";if(t instanceof Date)return"/Date("+t.valueOf()+")/";if(e.isString(t)){o=r.exec(t);var s;return s=o?new Date(Number(o[1])).valueOf():new Date(t).valueOf(),"/Date("+s+")/"}throw new Error("Cannot convert object of type "+e.getType(t)+" to type ASPDate");default:throw new Error('Unknown type "'+i+'"')}}};var r=/^\/?Date\((\-?\d+)/i;e.getType=function(t){var e="undefined"==typeof t?"undefined":o(t);return"object"==e?null===t?"null":t instanceof Boolean?"Boolean":t instanceof Number?"Number":t instanceof String?"String":Array.isArray(t)?"Array":t instanceof Date?"Date":"Object":"number"==e?"Number":"boolean"==e?"Boolean":"string"==e?"String":void 0===e?"undefined":e},e.copyAndExtendArray=function(t,e){for(var i=[],o=0;oi;i++)e(t[i],i,t);else for(i in t)t.hasOwnProperty(i)&&e(t[i],i,t)},e.toArray=function(t){var e=[];for(var i in t)t.hasOwnProperty(i)&&e.push(t[i]);return e},e.updateProperty=function(t,e,i){return t[e]!==i?(t[e]=i,!0):!1},e.throttle=function(t,e){var i=null,o=!1;return function n(){i?o=!0:(o=!1,t(),i=setTimeout(function(){i=null,o&&n()},e))}},e.addEventListener=function(t,e,i,o){t.addEventListener?(void 0===o&&(o=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.addEventListener(e,i,o)):t.attachEvent("on"+e,i)},e.removeEventListener=function(t,e,i,o){t.removeEventListener?(void 0===o&&(o=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.removeEventListener(e,i,o)):t.detachEvent("on"+e,i)},e.preventDefault=function(t){t||(t=window.event),t.preventDefault?t.preventDefault():t.returnValue=!1},e.getTarget=function(t){t||(t=window.event);var e;return t.target?e=t.target:t.srcElement&&(e=t.srcElement),void 0!=e.nodeType&&3==e.nodeType&&(e=e.parentNode),e},e.hasParent=function(t,e){for(var i=t;i;){if(i===e)return!0;i=i.parentNode}return!1},e.option={},e.option.asBoolean=function(t,e){return"function"==typeof t&&(t=t()),null!=t?0!=t:e||null},e.option.asNumber=function(t,e){return"function"==typeof t&&(t=t()),null!=t?Number(t)||e||null:e||null},e.option.asString=function(t,e){return"function"==typeof t&&(t=t()),null!=t?String(t):e||null},e.option.asSize=function(t,i){return"function"==typeof t&&(t=t()),e.isString(t)?t:e.isNumber(t)?t+"px":i||null},e.option.asElement=function(t,e){return"function"==typeof t&&(t=t()),t||e||null},e.hexToRGB=function(t){var e=/^#?([a-f\d])([a-f\d])([a-f\d])$/i;t=t.replace(e,function(t,e,i,o){return e+e+i+i+o+o});var i=/^#?([a-f\d]{2})([a-f\d]{2})([a-f\d]{2})$/i.exec(t);return i?{r:parseInt(i[1],16),g:parseInt(i[2],16),b:parseInt(i[3],16)}:null},e.overrideOpacity=function(t,i){if(-1!=t.indexOf("rgba"))return t;if(-1!=t.indexOf("rgb")){var o=t.substr(t.indexOf("(")+1).replace(")","").split(",");return"rgba("+o[0]+","+o[1]+","+o[2]+","+i+")"}var o=e.hexToRGB(t);return null==o?t:"rgba("+o.r+","+o.g+","+o.b+","+i+")"},e.RGBToHex=function(t,e,i){return"#"+((1<<24)+(t<<16)+(e<<8)+i).toString(16).slice(1)},e.parseColor=function(t){var i;if(e.isString(t)===!0){if(e.isValidRGB(t)===!0){var o=t.substr(4).substr(0,t.length-5).split(",").map(function(t){return parseInt(t)});t=e.RGBToHex(o[0],o[1],o[2])}if(e.isValidHex(t)===!0){var n=e.hexToHSV(t),s={h:n.h,s:.8*n.s,v:Math.min(1,1.02*n.v)},r={h:n.h,s:Math.min(1,1.25*n.s),v:.8*n.v},a=e.HSVToHex(r.h,r.s,r.v),h=e.HSVToHex(s.h,s.s,s.v);i={background:t,border:a,highlight:{background:h,border:a},hover:{background:h,border:a}}}else i={background:t,border:t,highlight:{background:t,border:t},hover:{background:t,border:t}}}else i={},i.background=t.background||void 0,i.border=t.border||void 0,e.isString(t.highlight)?i.highlight={border:t.highlight,background:t.highlight}:(i.highlight={},i.highlight.background=t.highlight&&t.highlight.background||void 0,i.highlight.border=t.highlight&&t.highlight.border||void 0),e.isString(t.hover)?i.hover={border:t.hover,background:t.hover}:(i.hover={},i.hover.background=t.hover&&t.hover.background||void 0,i.hover.border=t.hover&&t.hover.border||void 0);return i},e.RGBToHSV=function(t,e,i){t/=255,e/=255,i/=255;var o=Math.min(t,Math.min(e,i)),n=Math.max(t,Math.max(e,i));if(o==n)return{h:0,s:0,v:o};var s=t==o?e-i:i==o?t-e:i-t,r=t==o?3:i==o?1:5,a=60*(r-s/(n-o))/360,h=(n-o)/n,d=n;return{h:a,s:h,v:d}};var a={split:function(t){var e={};return t.split(";").forEach(function(t){if(""!=t.trim()){var i=t.split(":"),o=i[0].trim(),n=i[1].trim();e[o]=n}}),e},join:function(t){return Object.keys(t).map(function(e){return e+": "+t[e]}).join("; ")}};e.addCssText=function(t,i){var o=a.split(t.style.cssText),n=a.split(i),s=e.extend(o,n);t.style.cssText=a.join(s)},e.removeCssText=function(t,e){var i=a.split(t.style.cssText),o=a.split(e);for(var n in o)o.hasOwnProperty(n)&&delete i[n];t.style.cssText=a.join(i)},e.HSVToRGB=function(t,e,i){var o,n,s,r=Math.floor(6*t),a=6*t-r,h=i*(1-e),d=i*(1-a*e),l=i*(1-(1-a)*e);switch(r%6){case 0:o=i,n=l,s=h;break;case 1:o=d,n=i,s=h;break;case 2:o=h,n=i,s=l;break;case 3:o=h,n=d,s=i;break;case 4:o=l,n=h,s=i;break;case 5:o=i,n=h,s=d}return{r:Math.floor(255*o),g:Math.floor(255*n),b:Math.floor(255*s)}},e.HSVToHex=function(t,i,o){var n=e.HSVToRGB(t,i,o);return e.RGBToHex(n.r,n.g,n.b)},e.hexToHSV=function(t){var i=e.hexToRGB(t);return e.RGBToHSV(i.r,i.g,i.b)},e.isValidHex=function(t){var e=/(^#[0-9A-F]{6}$)|(^#[0-9A-F]{3}$)/i.test(t);return e},e.isValidRGB=function(t){t=t.replace(" ","");var e=/rgb\((\d{1,3}),(\d{1,3}),(\d{1,3})\)/i.test(t);return e},e.isValidRGBA=function(t){t=t.replace(" ","");var e=/rgba\((\d{1,3}),(\d{1,3}),(\d{1,3}),(.{1,3})\)/i.test(t);return e},e.selectiveBridgeObject=function(t,i){if("object"==("undefined"==typeof i?"undefined":o(i))){for(var n=Object.create(i),s=0;s0&&e(o,t[n-1])<0;n--)t[n]=t[n-1];t[n]=o}return t},e.mergeOptions=function(t,e,i){var o=(arguments.length<=3||void 0===arguments[3]?!1:arguments[3],arguments.length<=4||void 0===arguments[4]?{}:arguments[4]);if(null===e[i])t[i]=Object.create(o[i]);else if(void 0!==e[i])if("boolean"==typeof e[i])t[i].enabled=e[i];else{void 0===e[i].enabled&&(t[i].enabled=!0);for(var n in e[i])e[i].hasOwnProperty(n)&&(t[i][n]=e[i][n])}},e.binarySearchCustom=function(t,e,i,o){for(var n=1e4,s=0,r=0,a=t.length-1;a>=r&&n>s;){var h=Math.floor((r+a)/2),d=t[h],l=void 0===o?d[i]:d[i][o],c=e(l);if(0==c)return h;-1==c?r=h+1:a=h-1,s++}return-1},e.binarySearchValue=function(t,e,i,o,n){for(var s,r,a,h,d=1e4,l=0,c=0,u=t.length-1,n=void 0!=n?n:function(t,e){return t==e?0:e>t?-1:1};u>=c&&d>l;){if(h=Math.floor(.5*(u+c)),s=t[Math.max(0,h-1)][i],r=t[h][i],a=t[Math.min(t.length-1,h+1)][i],0==n(r,e))return h;if(n(s,e)<0&&n(r,e)>0)return"before"==o?Math.max(0,h-1):h;if(n(r,e)<0&&n(a,e)>0)return"before"==o?h:Math.min(t.length-1,h+1);n(r,e)<0?c=h+1:u=h-1,l++}return-1},e.easingFunctions={linear:function(t){return t},easeInQuad:function(t){return t*t},easeOutQuad:function(t){return t*(2-t)},easeInOutQuad:function(t){return.5>t?2*t*t:-1+(4-2*t)*t},easeInCubic:function(t){return t*t*t},easeOutCubic:function(t){return--t*t*t+1},easeInOutCubic:function(t){return.5>t?4*t*t*t:(t-1)*(2*t-2)*(2*t-2)+1},easeInQuart:function(t){return t*t*t*t},easeOutQuart:function(t){return 1- --t*t*t*t},easeInOutQuart:function(t){return.5>t?8*t*t*t*t:1-8*--t*t*t*t},easeInQuint:function(t){return t*t*t*t*t},easeOutQuint:function(t){return 1+--t*t*t*t*t},easeInOutQuint:function(t){return.5>t?16*t*t*t*t*t:1+16*--t*t*t*t*t}}},function(t,e,i){t.exports="undefined"!=typeof window&&window.moment||i(3)},function(t,e,i){(function(t){!function(e,i){t.exports=i()}(this,function(){function e(){return ro.apply(null,arguments)}function i(t){ro=t}function o(t){return t instanceof Array||"[object Array]"===Object.prototype.toString.call(t)}function n(t){return t instanceof Date||"[object Date]"===Object.prototype.toString.call(t)}function s(t,e){var i,o=[];for(i=0;i0)for(i in ho)o=ho[i],n=e[o],p(n)||(t[o]=n);return t}function m(t){f(this,t),this._d=new Date(null!=t._d?t._d.getTime():NaN),lo===!1&&(lo=!0,e.updateOffset(this),lo=!1)}function v(t){return t instanceof m||null!=t&&null!=t._isAMomentObject}function g(t){return 0>t?Math.ceil(t):Math.floor(t)}function y(t){var e=+t,i=0;return 0!==e&&isFinite(e)&&(i=g(e)),i}function b(t,e,i){var o,n=Math.min(t.length,e.length),s=Math.abs(t.length-e.length),r=0;for(o=0;n>o;o++)(i&&t[o]!==e[o]||!i&&y(t[o])!==y(e[o]))&&r++;return r+s}function w(t){e.suppressDeprecationWarnings===!1&&"undefined"!=typeof console&&console.warn&&console.warn("Deprecation warning: "+t)}function _(t,i){var o=!0;return a(function(){return null!=e.deprecationHandler&&e.deprecationHandler(null,t),o&&(w(t+"\nArguments: "+Array.prototype.slice.call(arguments).join(", ")+"\n"+(new Error).stack),o=!1),i.apply(this,arguments)},i)}function x(t,i){null!=e.deprecationHandler&&e.deprecationHandler(t,i),co[t]||(w(i),co[t]=!0)}function k(t){return t instanceof Function||"[object Function]"===Object.prototype.toString.call(t)}function O(t){return"[object Object]"===Object.prototype.toString.call(t)}function M(t){var e,i;for(i in t)e=t[i],k(e)?this[i]=e:this["_"+i]=e;this._config=t,this._ordinalParseLenient=new RegExp(this._ordinalParse.source+"|"+/\d{1,2}/.source)}function D(t,e){var i,o=a({},t);for(i in e)r(e,i)&&(O(t[i])&&O(e[i])?(o[i]={},a(o[i],t[i]),a(o[i],e[i])):null!=e[i]?o[i]=e[i]:delete o[i]);return o}function S(t){null!=t&&this.set(t)}function C(t){return t?t.toLowerCase().replace("_","-"):t}function T(t){for(var e,i,o,n,s=0;s0;){if(o=E(n.slice(0,e).join("-")))return o;if(i&&i.length>=e&&b(n,i,!0)>=e-1)break;e--}s++}return null}function E(e){var i=null;if(!mo[e]&&"undefined"!=typeof t&&t&&t.exports)try{i=po._abbr,!function(){var t=new Error('Cannot find module "./locale"');throw t.code="MODULE_NOT_FOUND",t}(),P(i)}catch(o){}return mo[e]}function P(t,e){var i;return t&&(i=p(e)?R(t):I(t,e),i&&(po=i)),po._abbr}function I(t,e){return null!==e?(e.abbr=t,null!=mo[t]?(x("defineLocaleOverride","use moment.updateLocale(localeName, config) to change an existing locale. moment.defineLocale(localeName, config) should only be used for creating a new locale"),e=D(mo[t]._config,e)):null!=e.parentLocale&&(null!=mo[e.parentLocale]?e=D(mo[e.parentLocale]._config,e):x("parentLocaleUndefined","specified parentLocale is not defined yet")),mo[t]=new S(e),P(t),mo[t]):(delete mo[t],null)}function N(t,e){if(null!=e){var i;null!=mo[t]&&(e=D(mo[t]._config,e)),i=new S(e),i.parentLocale=mo[t],mo[t]=i,P(t)}else null!=mo[t]&&(null!=mo[t].parentLocale?mo[t]=mo[t].parentLocale:null!=mo[t]&&delete mo[t]);return mo[t]}function R(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return po;if(!o(t)){if(e=E(t))return e;t=[t]}return T(t)}function z(){return uo(mo)}function L(t,e){var i=t.toLowerCase();vo[i]=vo[i+"s"]=vo[e]=t}function A(t){return"string"==typeof t?vo[t]||vo[t.toLowerCase()]:void 0}function B(t){var e,i,o={};for(i in t)r(t,i)&&(e=A(i),e&&(o[e]=t[i]));return o}function F(t,i){return function(o){return null!=o?(H(this,t,o),e.updateOffset(this,i),this):j(this,t)}}function j(t,e){return t.isValid()?t._d["get"+(t._isUTC?"UTC":"")+e]():NaN}function H(t,e,i){t.isValid()&&t._d["set"+(t._isUTC?"UTC":"")+e](i)}function W(t,e){var i;if("object"==typeof t)for(i in t)this.set(i,t[i]);else if(t=A(t),k(this[t]))return this[t](e);return this}function Y(t,e,i){var o=""+Math.abs(t),n=e-o.length,s=t>=0;return(s?i?"+":"":"-")+Math.pow(10,Math.max(0,n)).toString().substr(1)+o}function G(t,e,i,o){var n=o;"string"==typeof o&&(n=function(){return this[o]()}),t&&(wo[t]=n),e&&(wo[e[0]]=function(){return Y(n.apply(this,arguments),e[1],e[2])}),i&&(wo[i]=function(){return this.localeData().ordinal(n.apply(this,arguments),t)})}function V(t){return t.match(/\[[\s\S]/)?t.replace(/^\[|\]$/g,""):t.replace(/\\/g,"")}function U(t){var e,i,o=t.match(go);for(e=0,i=o.length;i>e;e++)wo[o[e]]?o[e]=wo[o[e]]:o[e]=V(o[e]);return function(e){var n,s="";for(n=0;i>n;n++)s+=o[n]instanceof Function?o[n].call(e,t):o[n];return s}}function q(t,e){return t.isValid()?(e=X(e,t.localeData()),bo[e]=bo[e]||U(e),bo[e](t)):t.localeData().invalidDate()}function X(t,e){function i(t){return e.longDateFormat(t)||t}var o=5;for(yo.lastIndex=0;o>=0&&yo.test(t);)t=t.replace(yo,i),yo.lastIndex=0,o-=1;return t}function Z(t,e,i){Bo[t]=k(e)?e:function(t,o){return t&&i?i:e}}function K(t,e){return r(Bo,t)?Bo[t](e._strict,e._locale):new RegExp(J(t))}function J(t){return Q(t.replace("\\","").replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,function(t,e,i,o,n){return e||i||o||n}))}function Q(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}function $(t,e){var i,o=e;for("string"==typeof t&&(t=[t]),"number"==typeof e&&(o=function(t,i){i[e]=y(t)}),i=0;io;++o)s=h([2e3,o]),this._shortMonthsParse[o]=this.monthsShort(s,"").toLocaleLowerCase(),this._longMonthsParse[o]=this.months(s,"").toLocaleLowerCase();return i?"MMM"===e?(n=fo.call(this._shortMonthsParse,r),-1!==n?n:null):(n=fo.call(this._longMonthsParse,r),-1!==n?n:null):"MMM"===e?(n=fo.call(this._shortMonthsParse,r),-1!==n?n:(n=fo.call(this._longMonthsParse,r),-1!==n?n:null)):(n=fo.call(this._longMonthsParse,r),-1!==n?n:(n=fo.call(this._shortMonthsParse,r),-1!==n?n:null))}function rt(t,e,i){var o,n,s;if(this._monthsParseExact)return st.call(this,t,e,i);for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),o=0;12>o;o++){if(n=h([2e3,o]),i&&!this._longMonthsParse[o]&&(this._longMonthsParse[o]=new RegExp("^"+this.months(n,"").replace(".","")+"$","i"),this._shortMonthsParse[o]=new RegExp("^"+this.monthsShort(n,"").replace(".","")+"$","i")),i||this._monthsParse[o]||(s="^"+this.months(n,"")+"|^"+this.monthsShort(n,""),this._monthsParse[o]=new RegExp(s.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[o].test(t))return o;if(i&&"MMM"===e&&this._shortMonthsParse[o].test(t))return o;if(!i&&this._monthsParse[o].test(t))return o}}function at(t,e){var i;if(!t.isValid())return t;if("string"==typeof e)if(/^\d+$/.test(e))e=y(e);else if(e=t.localeData().monthsParse(e),"number"!=typeof e)return t;return i=Math.min(t.date(),it(t.year(),e)),t._d["set"+(t._isUTC?"UTC":"")+"Month"](e,i),t}function ht(t){return null!=t?(at(this,t),e.updateOffset(this,!0),this):j(this,"Month")}function dt(){return it(this.year(),this.month())}function lt(t){return this._monthsParseExact?(r(this,"_monthsRegex")||ut.call(this),t?this._monthsShortStrictRegex:this._monthsShortRegex):this._monthsShortStrictRegex&&t?this._monthsShortStrictRegex:this._monthsShortRegex}function ct(t){return this._monthsParseExact?(r(this,"_monthsRegex")||ut.call(this),t?this._monthsStrictRegex:this._monthsRegex):this._monthsStrictRegex&&t?this._monthsStrictRegex:this._monthsRegex}function ut(){function t(t,e){return e.length-t.length}var e,i,o=[],n=[],s=[];for(e=0;12>e;e++)i=h([2e3,e]),o.push(this.monthsShort(i,"")),n.push(this.months(i,"")),s.push(this.months(i,"")),s.push(this.monthsShort(i,""));for(o.sort(t),n.sort(t),s.sort(t),e=0;12>e;e++)o[e]=Q(o[e]),n[e]=Q(n[e]),s[e]=Q(s[e]);this._monthsRegex=new RegExp("^("+s.join("|")+")","i"),this._monthsShortRegex=this._monthsRegex,this._monthsStrictRegex=new RegExp("^("+n.join("|")+")","i"),this._monthsShortStrictRegex=new RegExp("^("+o.join("|")+")","i")}function pt(t){var e,i=t._a;return i&&-2===l(t).overflow&&(e=i[Ho]<0||i[Ho]>11?Ho:i[Wo]<1||i[Wo]>it(i[jo],i[Ho])?Wo:i[Yo]<0||i[Yo]>24||24===i[Yo]&&(0!==i[Go]||0!==i[Vo]||0!==i[Uo])?Yo:i[Go]<0||i[Go]>59?Go:i[Vo]<0||i[Vo]>59?Vo:i[Uo]<0||i[Uo]>999?Uo:-1,l(t)._overflowDayOfYear&&(jo>e||e>Wo)&&(e=Wo),l(t)._overflowWeeks&&-1===e&&(e=qo),l(t)._overflowWeekday&&-1===e&&(e=Xo),l(t).overflow=e),t}function ft(t){var e,i,o,n,s,r,a=t._i,h=tn.exec(a)||en.exec(a);if(h){for(l(t).iso=!0,e=0,i=nn.length;i>e;e++)if(nn[e][1].exec(h[1])){n=nn[e][0],o=nn[e][2]!==!1;break}if(null==n)return void(t._isValid=!1);if(h[3]){for(e=0,i=sn.length;i>e;e++)if(sn[e][1].exec(h[3])){s=(h[2]||" ")+sn[e][0];break}if(null==s)return void(t._isValid=!1)}if(!o&&null!=s)return void(t._isValid=!1);if(h[4]){if(!on.exec(h[4]))return void(t._isValid=!1);r="Z"}t._f=n+(s||"")+(r||""),Tt(t)}else t._isValid=!1}function mt(t){var i=rn.exec(t._i);return null!==i?void(t._d=new Date(+i[1])):(ft(t),void(t._isValid===!1&&(delete t._isValid,e.createFromInputFallback(t))))}function vt(t,e,i,o,n,s,r){var a=new Date(t,e,i,o,n,s,r);return 100>t&&t>=0&&isFinite(a.getFullYear())&&a.setFullYear(t),a}function gt(t){var e=new Date(Date.UTC.apply(null,arguments));return 100>t&&t>=0&&isFinite(e.getUTCFullYear())&&e.setUTCFullYear(t),e}function yt(t){return bt(t)?366:365}function bt(t){return t%4===0&&t%100!==0||t%400===0}function wt(){return bt(this.year())}function _t(t,e,i){var o=7+e-i,n=(7+gt(t,0,o).getUTCDay()-e)%7;return-n+o-1}function xt(t,e,i,o,n){var s,r,a=(7+i-o)%7,h=_t(t,o,n),d=1+7*(e-1)+a+h;return 0>=d?(s=t-1,r=yt(s)+d):d>yt(t)?(s=t+1,r=d-yt(t)):(s=t,r=d),{year:s,dayOfYear:r}}function kt(t,e,i){var o,n,s=_t(t.year(),e,i),r=Math.floor((t.dayOfYear()-s-1)/7)+1;return 1>r?(n=t.year()-1,o=r+Ot(n,e,i)):r>Ot(t.year(),e,i)?(o=r-Ot(t.year(),e,i),n=t.year()+1):(n=t.year(),o=r),{week:o,year:n}}function Ot(t,e,i){var o=_t(t,e,i),n=_t(t+1,e,i);return(yt(t)-o+n)/7}function Mt(t,e,i){return null!=t?t:null!=e?e:i}function Dt(t){var i=new Date(e.now());return t._useUTC?[i.getUTCFullYear(),i.getUTCMonth(),i.getUTCDate()]:[i.getFullYear(),i.getMonth(),i.getDate()]}function St(t){var e,i,o,n,s=[];if(!t._d){for(o=Dt(t),t._w&&null==t._a[Wo]&&null==t._a[Ho]&&Ct(t),t._dayOfYear&&(n=Mt(t._a[jo],o[jo]),t._dayOfYear>yt(n)&&(l(t)._overflowDayOfYear=!0),i=gt(n,0,t._dayOfYear),t._a[Ho]=i.getUTCMonth(),t._a[Wo]=i.getUTCDate()),e=0;3>e&&null==t._a[e];++e)t._a[e]=s[e]=o[e];for(;7>e;e++)t._a[e]=s[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[Yo]&&0===t._a[Go]&&0===t._a[Vo]&&0===t._a[Uo]&&(t._nextDay=!0,t._a[Yo]=0),t._d=(t._useUTC?gt:vt).apply(null,s),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[Yo]=24)}}function Ct(t){var e,i,o,n,s,r,a,h;e=t._w,null!=e.GG||null!=e.W||null!=e.E?(s=1,r=4,i=Mt(e.GG,t._a[jo],kt(At(),1,4).year),o=Mt(e.W,1),n=Mt(e.E,1),(1>n||n>7)&&(h=!0)):(s=t._locale._week.dow,r=t._locale._week.doy,i=Mt(e.gg,t._a[jo],kt(At(),s,r).year),o=Mt(e.w,1),null!=e.d?(n=e.d,(0>n||n>6)&&(h=!0)):null!=e.e?(n=e.e+s,(e.e<0||e.e>6)&&(h=!0)):n=s),1>o||o>Ot(i,s,r)?l(t)._overflowWeeks=!0:null!=h?l(t)._overflowWeekday=!0:(a=xt(i,o,n,s,r),t._a[jo]=a.year,t._dayOfYear=a.dayOfYear)}function Tt(t){if(t._f===e.ISO_8601)return void ft(t);t._a=[],l(t).empty=!0;var i,o,n,s,r,a=""+t._i,h=a.length,d=0;for(n=X(t._f,t._locale).match(go)||[],i=0;i0&&l(t).unusedInput.push(r),a=a.slice(a.indexOf(o)+o.length),d+=o.length),wo[s]?(o?l(t).empty=!1:l(t).unusedTokens.push(s),et(s,o,t)):t._strict&&!o&&l(t).unusedTokens.push(s);l(t).charsLeftOver=h-d,a.length>0&&l(t).unusedInput.push(a),l(t).bigHour===!0&&t._a[Yo]<=12&&t._a[Yo]>0&&(l(t).bigHour=void 0),l(t).parsedDateParts=t._a.slice(0),l(t).meridiem=t._meridiem,t._a[Yo]=Et(t._locale,t._a[Yo],t._meridiem),St(t),pt(t)}function Et(t,e,i){var o;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?(o=t.isPM(i),o&&12>e&&(e+=12),o||12!==e||(e=0),e):e}function Pt(t){var e,i,o,n,s;if(0===t._f.length)return l(t).invalidFormat=!0,void(t._d=new Date(NaN));for(n=0;ns)&&(o=s,i=e));a(t,i||e)}function It(t){if(!t._d){var e=B(t._i);t._a=s([e.year,e.month,e.day||e.date,e.hour,e.minute,e.second,e.millisecond],function(t){return t&&parseInt(t,10)}),St(t)}}function Nt(t){var e=new m(pt(Rt(t)));return e._nextDay&&(e.add(1,"d"),e._nextDay=void 0),e}function Rt(t){var e=t._i,i=t._f;return t._locale=t._locale||R(t._l),null===e||void 0===i&&""===e?u({nullInput:!0}):("string"==typeof e&&(t._i=e=t._locale.preparse(e)),v(e)?new m(pt(e)):(o(i)?Pt(t):i?Tt(t):n(e)?t._d=e:zt(t),c(t)||(t._d=null),t))}function zt(t){var i=t._i;void 0===i?t._d=new Date(e.now()):n(i)?t._d=new Date(i.valueOf()):"string"==typeof i?mt(t):o(i)?(t._a=s(i.slice(0),function(t){return parseInt(t,10)}),St(t)):"object"==typeof i?It(t):"number"==typeof i?t._d=new Date(i):e.createFromInputFallback(t)}function Lt(t,e,i,o,n){var s={};return"boolean"==typeof i&&(o=i,i=void 0),s._isAMomentObject=!0,s._useUTC=s._isUTC=n,s._l=i,s._i=t,s._f=e,s._strict=o,Nt(s)}function At(t,e,i,o){return Lt(t,e,i,o,!1)}function Bt(t,e){var i,n;if(1===e.length&&o(e[0])&&(e=e[0]),!e.length)return At();for(i=e[0],n=1;nt&&(t=-t,i="-"),i+Y(~~(t/60),2)+e+Y(~~t%60,2)})}function Gt(t,e){var i=(e||"").match(t)||[],o=i[i.length-1]||[],n=(o+"").match(cn)||["-",0,0],s=+(60*n[1])+y(n[2]);return"+"===n[0]?s:-s}function Vt(t,i){var o,s;return i._isUTC?(o=i.clone(),s=(v(t)||n(t)?t.valueOf():At(t).valueOf())-o.valueOf(),o._d.setTime(o._d.valueOf()+s),e.updateOffset(o,!1),o):At(t).local()}function Ut(t){return 15*-Math.round(t._d.getTimezoneOffset()/15)}function qt(t,i){var o,n=this._offset||0;return this.isValid()?null!=t?("string"==typeof t?t=Gt(zo,t):Math.abs(t)<16&&(t=60*t),!this._isUTC&&i&&(o=Ut(this)),this._offset=t,this._isUTC=!0,null!=o&&this.add(o,"m"),n!==t&&(!i||this._changeInProgress?le(this,ne(t-n,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,e.updateOffset(this,!0),this._changeInProgress=null)),this):this._isUTC?n:Ut(this):null!=t?this:NaN}function Xt(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()}function Zt(t){return this.utcOffset(0,t)}function Kt(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(Ut(this),"m")),this}function Jt(){return this._tzm?this.utcOffset(this._tzm):"string"==typeof this._i&&this.utcOffset(Gt(Ro,this._i)),this}function Qt(t){return this.isValid()?(t=t?At(t).utcOffset():0,(this.utcOffset()-t)%60===0):!1}function $t(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()}function te(){if(!p(this._isDSTShifted))return this._isDSTShifted;var t={};if(f(t,this),t=Rt(t),t._a){var e=t._isUTC?h(t._a):At(t._a);this._isDSTShifted=this.isValid()&&b(t._a,e.toArray())>0}else this._isDSTShifted=!1;return this._isDSTShifted}function ee(){return this.isValid()?!this._isUTC:!1}function ie(){return this.isValid()?this._isUTC:!1}function oe(){return this.isValid()?this._isUTC&&0===this._offset:!1}function ne(t,e){var i,o,n,s=t,a=null;return Wt(t)?s={ms:t._milliseconds,d:t._days,M:t._months}:"number"==typeof t?(s={},e?s[e]=t:s.milliseconds=t):(a=un.exec(t))?(i="-"===a[1]?-1:1,s={y:0,d:y(a[Wo])*i,h:y(a[Yo])*i,m:y(a[Go])*i,s:y(a[Vo])*i,ms:y(a[Uo])*i}):(a=pn.exec(t))?(i="-"===a[1]?-1:1,s={y:se(a[2],i),M:se(a[3],i),w:se(a[4],i),d:se(a[5],i),h:se(a[6],i),m:se(a[7],i),s:se(a[8],i)}):null==s?s={}:"object"==typeof s&&("from"in s||"to"in s)&&(n=ae(At(s.from),At(s.to)),s={},s.ms=n.milliseconds,s.M=n.months),o=new Ht(s),Wt(t)&&r(t,"_locale")&&(o._locale=t._locale),o}function se(t,e){var i=t&&parseFloat(t.replace(",","."));return(isNaN(i)?0:i)*e}function re(t,e){var i={milliseconds:0,months:0};return i.months=e.month()-t.month()+12*(e.year()-t.year()),t.clone().add(i.months,"M").isAfter(e)&&--i.months, +i.milliseconds=+e-+t.clone().add(i.months,"M"),i}function ae(t,e){var i;return t.isValid()&&e.isValid()?(e=Vt(e,t),t.isBefore(e)?i=re(t,e):(i=re(e,t),i.milliseconds=-i.milliseconds,i.months=-i.months),i):{milliseconds:0,months:0}}function he(t){return 0>t?-1*Math.round(-1*t):Math.round(t)}function de(t,e){return function(i,o){var n,s;return null===o||isNaN(+o)||(x(e,"moment()."+e+"(period, number) is deprecated. Please use moment()."+e+"(number, period)."),s=i,i=o,o=s),i="string"==typeof i?+i:i,n=ne(i,o),le(this,n,t),this}}function le(t,i,o,n){var s=i._milliseconds,r=he(i._days),a=he(i._months);t.isValid()&&(n=null==n?!0:n,s&&t._d.setTime(t._d.valueOf()+s*o),r&&H(t,"Date",j(t,"Date")+r*o),a&&at(t,j(t,"Month")+a*o),n&&e.updateOffset(t,r||a))}function ce(t,e){var i=t||At(),o=Vt(i,this).startOf("day"),n=this.diff(o,"days",!0),s=-6>n?"sameElse":-1>n?"lastWeek":0>n?"lastDay":1>n?"sameDay":2>n?"nextDay":7>n?"nextWeek":"sameElse",r=e&&(k(e[s])?e[s]():e[s]);return this.format(r||this.localeData().calendar(s,this,At(i)))}function ue(){return new m(this)}function pe(t,e){var i=v(t)?t:At(t);return this.isValid()&&i.isValid()?(e=A(p(e)?"millisecond":e),"millisecond"===e?this.valueOf()>i.valueOf():i.valueOf()e-s?(i=t.clone().add(n-1,"months"),o=(e-s)/(s-i)):(i=t.clone().add(n+1,"months"),o=(e-s)/(i-s)),-(n+o)||0}function _e(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")}function xe(){var t=this.clone().utc();return 0s&&(e=s),Xe.call(this,t,e,i,o,n))}function Xe(t,e,i,o,n){var s=xt(t,e,i,o,n),r=gt(s.year,0,s.dayOfYear);return this.year(r.getUTCFullYear()),this.month(r.getUTCMonth()),this.date(r.getUTCDate()),this}function Ze(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)}function Ke(t){return kt(t,this._week.dow,this._week.doy).week}function Je(){return this._week.dow}function Qe(){return this._week.doy}function $e(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")}function ti(t){var e=kt(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")}function ei(t,e){return"string"!=typeof t?t:isNaN(t)?(t=e.weekdaysParse(t),"number"==typeof t?t:null):parseInt(t,10)}function ii(t,e){return o(this._weekdays)?this._weekdays[t.day()]:this._weekdays[this._weekdays.isFormat.test(e)?"format":"standalone"][t.day()]}function oi(t){return this._weekdaysShort[t.day()]}function ni(t){return this._weekdaysMin[t.day()]}function si(t,e,i){var o,n,s,r=t.toLocaleLowerCase();if(!this._weekdaysParse)for(this._weekdaysParse=[],this._shortWeekdaysParse=[],this._minWeekdaysParse=[],o=0;7>o;++o)s=h([2e3,1]).day(o),this._minWeekdaysParse[o]=this.weekdaysMin(s,"").toLocaleLowerCase(),this._shortWeekdaysParse[o]=this.weekdaysShort(s,"").toLocaleLowerCase(),this._weekdaysParse[o]=this.weekdays(s,"").toLocaleLowerCase();return i?"dddd"===e?(n=fo.call(this._weekdaysParse,r),-1!==n?n:null):"ddd"===e?(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:null):(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null):"dddd"===e?(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null))):"ddd"===e?(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null))):(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:null)))}function ri(t,e,i){var o,n,s;if(this._weekdaysParseExact)return si.call(this,t,e,i);for(this._weekdaysParse||(this._weekdaysParse=[],this._minWeekdaysParse=[],this._shortWeekdaysParse=[],this._fullWeekdaysParse=[]),o=0;7>o;o++){if(n=h([2e3,1]).day(o),i&&!this._fullWeekdaysParse[o]&&(this._fullWeekdaysParse[o]=new RegExp("^"+this.weekdays(n,"").replace(".",".?")+"$","i"),this._shortWeekdaysParse[o]=new RegExp("^"+this.weekdaysShort(n,"").replace(".",".?")+"$","i"),this._minWeekdaysParse[o]=new RegExp("^"+this.weekdaysMin(n,"").replace(".",".?")+"$","i")),this._weekdaysParse[o]||(s="^"+this.weekdays(n,"")+"|^"+this.weekdaysShort(n,"")+"|^"+this.weekdaysMin(n,""),this._weekdaysParse[o]=new RegExp(s.replace(".",""),"i")),i&&"dddd"===e&&this._fullWeekdaysParse[o].test(t))return o;if(i&&"ddd"===e&&this._shortWeekdaysParse[o].test(t))return o;if(i&&"dd"===e&&this._minWeekdaysParse[o].test(t))return o;if(!i&&this._weekdaysParse[o].test(t))return o}}function ai(t){if(!this.isValid())return null!=t?this:NaN;var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=ei(t,this.localeData()),this.add(t-e,"d")):e}function hi(t){if(!this.isValid())return null!=t?this:NaN;var e=(this.day()+7-this.localeData()._week.dow)%7;return null==t?e:this.add(t-e,"d")}function di(t){return this.isValid()?null==t?this.day()||7:this.day(this.day()%7?t:t-7):null!=t?this:NaN}function li(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysStrictRegex:this._weekdaysRegex):this._weekdaysStrictRegex&&t?this._weekdaysStrictRegex:this._weekdaysRegex}function ci(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex):this._weekdaysShortStrictRegex&&t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex}function ui(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex):this._weekdaysMinStrictRegex&&t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex}function pi(){function t(t,e){return e.length-t.length}var e,i,o,n,s,r=[],a=[],d=[],l=[];for(e=0;7>e;e++)i=h([2e3,1]).day(e),o=this.weekdaysMin(i,""),n=this.weekdaysShort(i,""),s=this.weekdays(i,""),r.push(o),a.push(n),d.push(s),l.push(o),l.push(n),l.push(s);for(r.sort(t),a.sort(t),d.sort(t),l.sort(t),e=0;7>e;e++)a[e]=Q(a[e]),d[e]=Q(d[e]),l[e]=Q(l[e]);this._weekdaysRegex=new RegExp("^("+l.join("|")+")","i"),this._weekdaysShortRegex=this._weekdaysRegex,this._weekdaysMinRegex=this._weekdaysRegex,this._weekdaysStrictRegex=new RegExp("^("+d.join("|")+")","i"),this._weekdaysShortStrictRegex=new RegExp("^("+a.join("|")+")","i"),this._weekdaysMinStrictRegex=new RegExp("^("+r.join("|")+")","i")}function fi(t){var e=Math.round((this.clone().startOf("day")-this.clone().startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")}function mi(){return this.hours()%12||12}function vi(){return this.hours()||24}function gi(t,e){G(t,0,0,function(){return this.localeData().meridiem(this.hours(),this.minutes(),e)})}function yi(t,e){return e._meridiemParse}function bi(t){return"p"===(t+"").toLowerCase().charAt(0)}function wi(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"}function _i(t,e){e[Uo]=y(1e3*("0."+t))}function xi(){return this._isUTC?"UTC":""}function ki(){return this._isUTC?"Coordinated Universal Time":""}function Oi(t){return At(1e3*t)}function Mi(){return At.apply(null,arguments).parseZone()}function Di(t,e,i){var o=this._calendar[t];return k(o)?o.call(e,i):o}function Si(t){var e=this._longDateFormat[t],i=this._longDateFormat[t.toUpperCase()];return e||!i?e:(this._longDateFormat[t]=i.replace(/MMMM|MM|DD|dddd/g,function(t){return t.slice(1)}),this._longDateFormat[t])}function Ci(){return this._invalidDate}function Ti(t){return this._ordinal.replace("%d",t)}function Ei(t){return t}function Pi(t,e,i,o){var n=this._relativeTime[i];return k(n)?n(t,e,i,o):n.replace(/%d/i,t)}function Ii(t,e){var i=this._relativeTime[t>0?"future":"past"];return k(i)?i(e):i.replace(/%s/i,e)}function Ni(t,e,i,o){var n=R(),s=h().set(o,e);return n[i](s,t)}function Ri(t,e,i){if("number"==typeof t&&(e=t,t=void 0),t=t||"",null!=e)return Ni(t,e,i,"month");var o,n=[];for(o=0;12>o;o++)n[o]=Ni(t,o,i,"month");return n}function zi(t,e,i,o){"boolean"==typeof t?("number"==typeof e&&(i=e,e=void 0),e=e||""):(e=t,i=e,t=!1,"number"==typeof e&&(i=e,e=void 0),e=e||"");var n=R(),s=t?n._week.dow:0;if(null!=i)return Ni(e,(i+s)%7,o,"day");var r,a=[];for(r=0;7>r;r++)a[r]=Ni(e,(r+s)%7,o,"day");return a}function Li(t,e){return Ri(t,e,"months")}function Ai(t,e){return Ri(t,e,"monthsShort")}function Bi(t,e,i){return zi(t,e,i,"weekdays")}function Fi(t,e,i){return zi(t,e,i,"weekdaysShort")}function ji(t,e,i){return zi(t,e,i,"weekdaysMin")}function Hi(){var t=this._data;return this._milliseconds=jn(this._milliseconds),this._days=jn(this._days),this._months=jn(this._months),t.milliseconds=jn(t.milliseconds),t.seconds=jn(t.seconds),t.minutes=jn(t.minutes),t.hours=jn(t.hours),t.months=jn(t.months),t.years=jn(t.years),this}function Wi(t,e,i,o){var n=ne(e,i);return t._milliseconds+=o*n._milliseconds,t._days+=o*n._days,t._months+=o*n._months,t._bubble()}function Yi(t,e){return Wi(this,t,e,1)}function Gi(t,e){return Wi(this,t,e,-1)}function Vi(t){return 0>t?Math.floor(t):Math.ceil(t)}function Ui(){var t,e,i,o,n,s=this._milliseconds,r=this._days,a=this._months,h=this._data;return s>=0&&r>=0&&a>=0||0>=s&&0>=r&&0>=a||(s+=864e5*Vi(Xi(a)+r),r=0,a=0),h.milliseconds=s%1e3,t=g(s/1e3),h.seconds=t%60,e=g(t/60),h.minutes=e%60,i=g(e/60),h.hours=i%24,r+=g(i/24),n=g(qi(r)),a+=n,r-=Vi(Xi(n)),o=g(a/12),a%=12,h.days=r,h.months=a,h.years=o,this}function qi(t){return 4800*t/146097}function Xi(t){return 146097*t/4800}function Zi(t){var e,i,o=this._milliseconds;if(t=A(t),"month"===t||"year"===t)return e=this._days+o/864e5,i=this._months+qi(e),"month"===t?i:i/12;switch(e=this._days+Math.round(Xi(this._months)),t){case"week":return e/7+o/6048e5;case"day":return e+o/864e5;case"hour":return 24*e+o/36e5;case"minute":return 1440*e+o/6e4;case"second":return 86400*e+o/1e3;case"millisecond":return Math.floor(864e5*e)+o;default:throw new Error("Unknown unit "+t)}}function Ki(){return this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*y(this._months/12)}function Ji(t){return function(){return this.as(t)}}function Qi(t){return t=A(t),this[t+"s"]()}function $i(t){return function(){return this._data[t]}}function to(){return g(this.days()/7)}function eo(t,e,i,o,n){return n.relativeTime(e||1,!!i,t,o)}function io(t,e,i){var o=ne(t).abs(),n=is(o.as("s")),s=is(o.as("m")),r=is(o.as("h")),a=is(o.as("d")),h=is(o.as("M")),d=is(o.as("y")),l=n=s&&["m"]||s=r&&["h"]||r=a&&["d"]||a=h&&["M"]||h=d&&["y"]||["yy",d];return l[2]=e,l[3]=+t>0,l[4]=i,eo.apply(null,l)}function oo(t,e){return void 0===os[t]?!1:void 0===e?os[t]:(os[t]=e,!0)}function no(t){var e=this.localeData(),i=io(this,!t,e);return t&&(i=e.pastFuture(+this,i)),e.postformat(i)}function so(){var t,e,i,o=ns(this._milliseconds)/1e3,n=ns(this._days),s=ns(this._months);t=g(o/60),e=g(t/60),o%=60,t%=60,i=g(s/12),s%=12;var r=i,a=s,h=n,d=e,l=t,c=o,u=this.asSeconds();return u?(0>u?"-":"")+"P"+(r?r+"Y":"")+(a?a+"M":"")+(h?h+"D":"")+(d||l||c?"T":"")+(d?d+"H":"")+(l?l+"M":"")+(c?c+"S":""):"P0D"}var ro,ao;ao=Array.prototype.some?Array.prototype.some:function(t){for(var e=Object(this),i=e.length>>>0,o=0;i>o;o++)if(o in e&&t.call(this,e[o],o,e))return!0;return!1};var ho=e.momentProperties=[],lo=!1,co={};e.suppressDeprecationWarnings=!1,e.deprecationHandler=null;var uo;uo=Object.keys?Object.keys:function(t){var e,i=[];for(e in t)r(t,e)&&i.push(e);return i};var po,fo,mo={},vo={},go=/(\[[^\[]*\])|(\\)?([Hh]mm(ss)?|Mo|MM?M?M?|Do|DDDo|DD?D?D?|ddd?d?|do?|w[o|w]?|W[o|W]?|Qo?|YYYYYY|YYYYY|YYYY|YY|gg(ggg?)?|GG(GGG?)?|e|E|a|A|hh?|HH?|kk?|mm?|ss?|S{1,9}|x|X|zz?|ZZ?|.)/g,yo=/(\[[^\[]*\])|(\\)?(LTS|LT|LL?L?L?|l{1,4})/g,bo={},wo={},_o=/\d/,xo=/\d\d/,ko=/\d{3}/,Oo=/\d{4}/,Mo=/[+-]?\d{6}/,Do=/\d\d?/,So=/\d\d\d\d?/,Co=/\d\d\d\d\d\d?/,To=/\d{1,3}/,Eo=/\d{1,4}/,Po=/[+-]?\d{1,6}/,Io=/\d+/,No=/[+-]?\d+/,Ro=/Z|[+-]\d\d:?\d\d/gi,zo=/Z|[+-]\d\d(?::?\d\d)?/gi,Lo=/[+-]?\d+(\.\d{1,3})?/,Ao=/[0-9]*['a-z\u00A0-\u05FF\u0700-\uD7FF\uF900-\uFDCF\uFDF0-\uFFEF]+|[\u0600-\u06FF\/]+(\s*?[\u0600-\u06FF]+){1,2}/i,Bo={},Fo={},jo=0,Ho=1,Wo=2,Yo=3,Go=4,Vo=5,Uo=6,qo=7,Xo=8;fo=Array.prototype.indexOf?Array.prototype.indexOf:function(t){var e;for(e=0;e=t?""+t:"+"+t}),G(0,["YY",2],0,function(){return this.year()%100}),G(0,["YYYY",4],0,"year"),G(0,["YYYYY",5],0,"year"),G(0,["YYYYYY",6,!0],0,"year"),L("year","y"),Z("Y",No),Z("YY",Do,xo),Z("YYYY",Eo,Oo),Z("YYYYY",Po,Mo),Z("YYYYYY",Po,Mo),$(["YYYYY","YYYYYY"],jo),$("YYYY",function(t,i){i[jo]=2===t.length?e.parseTwoDigitYear(t):y(t)}),$("YY",function(t,i){i[jo]=e.parseTwoDigitYear(t)}),$("Y",function(t,e){e[jo]=parseInt(t,10)}),e.parseTwoDigitYear=function(t){return y(t)+(y(t)>68?1900:2e3)};var an=F("FullYear",!0);e.ISO_8601=function(){};var hn=_("moment().min is deprecated, use moment.max instead. https://github.com/moment/moment/issues/1548",function(){var t=At.apply(null,arguments);return this.isValid()&&t.isValid()?this>t?this:t:u()}),dn=_("moment().max is deprecated, use moment.min instead. https://github.com/moment/moment/issues/1548",function(){var t=At.apply(null,arguments);return this.isValid()&&t.isValid()?t>this?this:t:u()}),ln=function(){return Date.now?Date.now():+new Date};Yt("Z",":"),Yt("ZZ",""),Z("Z",zo),Z("ZZ",zo),$(["Z","ZZ"],function(t,e,i){i._useUTC=!0,i._tzm=Gt(zo,t)});var cn=/([\+\-]|\d\d)/gi;e.updateOffset=function(){};var un=/^(\-)?(?:(\d*)[. ])?(\d+)\:(\d+)(?:\:(\d+)\.?(\d{3})?\d*)?$/,pn=/^(-)?P(?:(-?[0-9,.]*)Y)?(?:(-?[0-9,.]*)M)?(?:(-?[0-9,.]*)W)?(?:(-?[0-9,.]*)D)?(?:T(?:(-?[0-9,.]*)H)?(?:(-?[0-9,.]*)M)?(?:(-?[0-9,.]*)S)?)?$/;ne.fn=Ht.prototype;var fn=de(1,"add"),mn=de(-1,"subtract");e.defaultFormat="YYYY-MM-DDTHH:mm:ssZ",e.defaultFormatUtc="YYYY-MM-DDTHH:mm:ss[Z]";var vn=_("moment().lang() is deprecated. Instead, use moment().localeData() to get the language configuration. Use moment().locale() to change languages.",function(t){return void 0===t?this.localeData():this.locale(t)});G(0,["gg",2],0,function(){return this.weekYear()%100}),G(0,["GG",2],0,function(){return this.isoWeekYear()%100}),We("gggg","weekYear"),We("ggggg","weekYear"),We("GGGG","isoWeekYear"),We("GGGGG","isoWeekYear"),L("weekYear","gg"),L("isoWeekYear","GG"),Z("G",No),Z("g",No),Z("GG",Do,xo),Z("gg",Do,xo),Z("GGGG",Eo,Oo),Z("gggg",Eo,Oo),Z("GGGGG",Po,Mo),Z("ggggg",Po,Mo),tt(["gggg","ggggg","GGGG","GGGGG"],function(t,e,i,o){e[o.substr(0,2)]=y(t)}),tt(["gg","GG"],function(t,i,o,n){i[n]=e.parseTwoDigitYear(t)}),G("Q",0,"Qo","quarter"),L("quarter","Q"),Z("Q",_o),$("Q",function(t,e){e[Ho]=3*(y(t)-1)}),G("w",["ww",2],"wo","week"),G("W",["WW",2],"Wo","isoWeek"),L("week","w"),L("isoWeek","W"),Z("w",Do),Z("ww",Do,xo),Z("W",Do),Z("WW",Do,xo),tt(["w","ww","W","WW"],function(t,e,i,o){e[o.substr(0,1)]=y(t)});var gn={dow:0,doy:6};G("D",["DD",2],"Do","date"),L("date","D"),Z("D",Do),Z("DD",Do,xo),Z("Do",function(t,e){return t?e._ordinalParse:e._ordinalParseLenient}),$(["D","DD"],Wo),$("Do",function(t,e){e[Wo]=y(t.match(Do)[0],10)});var yn=F("Date",!0);G("d",0,"do","day"),G("dd",0,0,function(t){return this.localeData().weekdaysMin(this,t)}),G("ddd",0,0,function(t){return this.localeData().weekdaysShort(this,t)}),G("dddd",0,0,function(t){return this.localeData().weekdays(this,t)}),G("e",0,0,"weekday"),G("E",0,0,"isoWeekday"),L("day","d"),L("weekday","e"),L("isoWeekday","E"),Z("d",Do),Z("e",Do),Z("E",Do),Z("dd",function(t,e){return e.weekdaysMinRegex(t)}),Z("ddd",function(t,e){return e.weekdaysShortRegex(t)}),Z("dddd",function(t,e){return e.weekdaysRegex(t)}),tt(["dd","ddd","dddd"],function(t,e,i,o){var n=i._locale.weekdaysParse(t,o,i._strict);null!=n?e.d=n:l(i).invalidWeekday=t}),tt(["d","e","E"],function(t,e,i,o){e[o]=y(t)});var bn="Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),wn="Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),_n="Su_Mo_Tu_We_Th_Fr_Sa".split("_"),xn=Ao,kn=Ao,On=Ao;G("DDD",["DDDD",3],"DDDo","dayOfYear"),L("dayOfYear","DDD"),Z("DDD",To),Z("DDDD",ko),$(["DDD","DDDD"],function(t,e,i){i._dayOfYear=y(t)}),G("H",["HH",2],0,"hour"),G("h",["hh",2],0,mi),G("k",["kk",2],0,vi),G("hmm",0,0,function(){return""+mi.apply(this)+Y(this.minutes(),2)}),G("hmmss",0,0,function(){return""+mi.apply(this)+Y(this.minutes(),2)+Y(this.seconds(),2)}),G("Hmm",0,0,function(){return""+this.hours()+Y(this.minutes(),2)}),G("Hmmss",0,0,function(){return""+this.hours()+Y(this.minutes(),2)+Y(this.seconds(),2)}),gi("a",!0),gi("A",!1),L("hour","h"),Z("a",yi),Z("A",yi),Z("H",Do),Z("h",Do),Z("HH",Do,xo),Z("hh",Do,xo),Z("hmm",So),Z("hmmss",Co),Z("Hmm",So),Z("Hmmss",Co),$(["H","HH"],Yo),$(["a","A"],function(t,e,i){i._isPm=i._locale.isPM(t),i._meridiem=t}),$(["h","hh"],function(t,e,i){e[Yo]=y(t),l(i).bigHour=!0}),$("hmm",function(t,e,i){var o=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o)),l(i).bigHour=!0}),$("hmmss",function(t,e,i){var o=t.length-4,n=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o,2)),e[Vo]=y(t.substr(n)),l(i).bigHour=!0}),$("Hmm",function(t,e,i){var o=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o))}),$("Hmmss",function(t,e,i){var o=t.length-4,n=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o,2)),e[Vo]=y(t.substr(n))});var Mn=/[ap]\.?m?\.?/i,Dn=F("Hours",!0);G("m",["mm",2],0,"minute"),L("minute","m"),Z("m",Do),Z("mm",Do,xo),$(["m","mm"],Go);var Sn=F("Minutes",!1);G("s",["ss",2],0,"second"),L("second","s"),Z("s",Do),Z("ss",Do,xo),$(["s","ss"],Vo);var Cn=F("Seconds",!1);G("S",0,0,function(){return~~(this.millisecond()/100)}),G(0,["SS",2],0,function(){return~~(this.millisecond()/10)}),G(0,["SSS",3],0,"millisecond"),G(0,["SSSS",4],0,function(){return 10*this.millisecond()}),G(0,["SSSSS",5],0,function(){return 100*this.millisecond()}),G(0,["SSSSSS",6],0,function(){return 1e3*this.millisecond()}),G(0,["SSSSSSS",7],0,function(){return 1e4*this.millisecond()}),G(0,["SSSSSSSS",8],0,function(){return 1e5*this.millisecond()}),G(0,["SSSSSSSSS",9],0,function(){return 1e6*this.millisecond()}),L("millisecond","ms"),Z("S",To,_o),Z("SS",To,xo),Z("SSS",To,ko);var Tn;for(Tn="SSSS";Tn.length<=9;Tn+="S")Z(Tn,Io);for(Tn="S";Tn.length<=9;Tn+="S")$(Tn,_i);var En=F("Milliseconds",!1);G("z",0,0,"zoneAbbr"),G("zz",0,0,"zoneName");var Pn=m.prototype;Pn.add=fn,Pn.calendar=ce,Pn.clone=ue,Pn.diff=be,Pn.endOf=Pe,Pn.format=ke,Pn.from=Oe,Pn.fromNow=Me,Pn.to=De,Pn.toNow=Se,Pn.get=W,Pn.invalidAt=je,Pn.isAfter=pe,Pn.isBefore=fe,Pn.isBetween=me,Pn.isSame=ve,Pn.isSameOrAfter=ge,Pn.isSameOrBefore=ye,Pn.isValid=Be,Pn.lang=vn,Pn.locale=Ce,Pn.localeData=Te,Pn.max=dn,Pn.min=hn,Pn.parsingFlags=Fe,Pn.set=W,Pn.startOf=Ee,Pn.subtract=mn,Pn.toArray=ze,Pn.toObject=Le,Pn.toDate=Re,Pn.toISOString=xe,Pn.toJSON=Ae,Pn.toString=_e,Pn.unix=Ne,Pn.valueOf=Ie,Pn.creationData=He,Pn.year=an,Pn.isLeapYear=wt,Pn.weekYear=Ye,Pn.isoWeekYear=Ge,Pn.quarter=Pn.quarters=Ze,Pn.month=ht,Pn.daysInMonth=dt,Pn.week=Pn.weeks=$e,Pn.isoWeek=Pn.isoWeeks=ti,Pn.weeksInYear=Ue,Pn.isoWeeksInYear=Ve,Pn.date=yn,Pn.day=Pn.days=ai,Pn.weekday=hi,Pn.isoWeekday=di,Pn.dayOfYear=fi,Pn.hour=Pn.hours=Dn,Pn.minute=Pn.minutes=Sn,Pn.second=Pn.seconds=Cn,Pn.millisecond=Pn.milliseconds=En,Pn.utcOffset=qt,Pn.utc=Zt,Pn.local=Kt,Pn.parseZone=Jt,Pn.hasAlignedHourOffset=Qt,Pn.isDST=$t,Pn.isDSTShifted=te,Pn.isLocal=ee,Pn.isUtcOffset=ie,Pn.isUtc=oe,Pn.isUTC=oe,Pn.zoneAbbr=xi,Pn.zoneName=ki,Pn.dates=_("dates accessor is deprecated. Use date instead.",yn),Pn.months=_("months accessor is deprecated. Use month instead",ht),Pn.years=_("years accessor is deprecated. Use year instead",an),Pn.zone=_("moment().zone is deprecated, use moment().utcOffset instead. https://github.com/moment/moment/issues/1779",Xt);var In=Pn,Nn={sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},Rn={LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY h:mm A",LLLL:"dddd, MMMM D, YYYY h:mm A"},zn="Invalid date",Ln="%d",An=/\d{1,2}/,Bn={future:"in %s",past:"%s ago",s:"a few seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},Fn=S.prototype;Fn._calendar=Nn,Fn.calendar=Di,Fn._longDateFormat=Rn,Fn.longDateFormat=Si,Fn._invalidDate=zn,Fn.invalidDate=Ci,Fn._ordinal=Ln,Fn.ordinal=Ti,Fn._ordinalParse=An,Fn.preparse=Ei,Fn.postformat=Ei,Fn._relativeTime=Bn,Fn.relativeTime=Pi,Fn.pastFuture=Ii,Fn.set=M,Fn.months=ot,Fn._months=Ko,Fn.monthsShort=nt,Fn._monthsShort=Jo,Fn.monthsParse=rt,Fn._monthsRegex=$o,Fn.monthsRegex=ct,Fn._monthsShortRegex=Qo,Fn.monthsShortRegex=lt,Fn.week=Ke,Fn._week=gn,Fn.firstDayOfYear=Qe,Fn.firstDayOfWeek=Je,Fn.weekdays=ii,Fn._weekdays=bn,Fn.weekdaysMin=ni,Fn._weekdaysMin=_n,Fn.weekdaysShort=oi,Fn._weekdaysShort=wn,Fn.weekdaysParse=ri,Fn._weekdaysRegex=xn,Fn.weekdaysRegex=li,Fn._weekdaysShortRegex=kn,Fn.weekdaysShortRegex=ci,Fn._weekdaysMinRegex=On,Fn.weekdaysMinRegex=ui,Fn.isPM=bi,Fn._meridiemParse=Mn,Fn.meridiem=wi,P("en",{ordinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10,i=1===y(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th";return t+i}}),e.lang=_("moment.lang is deprecated. Use moment.locale instead.",P),e.langData=_("moment.langData is deprecated. Use moment.localeData instead.",R);var jn=Math.abs,Hn=Ji("ms"),Wn=Ji("s"),Yn=Ji("m"),Gn=Ji("h"),Vn=Ji("d"),Un=Ji("w"),qn=Ji("M"),Xn=Ji("y"),Zn=$i("milliseconds"),Kn=$i("seconds"),Jn=$i("minutes"),Qn=$i("hours"),$n=$i("days"),ts=$i("months"),es=$i("years"),is=Math.round,os={s:45,m:45,h:22,d:26,M:11},ns=Math.abs,ss=Ht.prototype;ss.abs=Hi,ss.add=Yi,ss.subtract=Gi,ss.as=Zi,ss.asMilliseconds=Hn,ss.asSeconds=Wn,ss.asMinutes=Yn,ss.asHours=Gn,ss.asDays=Vn,ss.asWeeks=Un,ss.asMonths=qn,ss.asYears=Xn,ss.valueOf=Ki,ss._bubble=Ui,ss.get=Qi,ss.milliseconds=Zn,ss.seconds=Kn,ss.minutes=Jn,ss.hours=Qn,ss.days=$n,ss.weeks=to,ss.months=ts,ss.years=es,ss.humanize=no,ss.toISOString=so,ss.toString=so,ss.toJSON=so,ss.locale=Ce,ss.localeData=Te,ss.toIsoString=_("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",so),ss.lang=vn,G("X",0,0,"unix"),G("x",0,0,"valueOf"),Z("x",No),Z("X",Lo),$("X",function(t,e,i){i._d=new Date(1e3*parseFloat(t,10))}),$("x",function(t,e,i){i._d=new Date(y(t))}),e.version="2.13.0",i(At),e.fn=In,e.min=Ft,e.max=jt,e.now=ln,e.utc=h,e.unix=Oi,e.months=Li,e.isDate=n,e.locale=P,e.invalid=u,e.duration=ne,e.isMoment=v,e.weekdays=Bi,e.parseZone=Mi,e.localeData=R,e.isDuration=Wt,e.monthsShort=Ai,e.weekdaysMin=ji,e.defineLocale=I,e.updateLocale=N,e.locales=z,e.weekdaysShort=Fi,e.normalizeUnits=A,e.relativeTimeThreshold=oo,e.prototype=In;var rs=e;return rs})}).call(e,i(4)(t))},function(t,e){t.exports=function(t){return t.webpackPolyfill||(t.deprecate=function(){},t.paths=[],t.children=[],t.webpackPolyfill=1),t}},function(t,e){function i(t){throw new Error("Cannot find module '"+t+"'.")}i.keys=function(){return[]},i.resolve=i,t.exports=i,i.id=5},function(t,e){(function(e){function i(t,e,i){var o=e&&i||0,n=0;for(e=e||[],t.toLowerCase().replace(/[0-9a-f]{2}/g,function(t){16>n&&(e[o+n++]=c[t])});16>n;)e[o+n++]=0;return e}function o(t,e){var i=e||0,o=l;return o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]}function n(t,e,i){var n=e&&i||0,s=e||[];t=t||{};var r=void 0!==t.clockseq?t.clockseq:m,a=void 0!==t.msecs?t.msecs:(new Date).getTime(),h=void 0!==t.nsecs?t.nsecs:g+1,d=a-v+(h-g)/1e4;if(0>d&&void 0===t.clockseq&&(r=r+1&16383),(0>d||a>v)&&void 0===t.nsecs&&(h=0),h>=1e4)throw new Error("uuid.v1(): Can't create more than 10M uuids/sec");v=a,g=h,m=r,a+=122192928e5;var l=(1e4*(268435455&a)+h)%4294967296;s[n++]=l>>>24&255,s[n++]=l>>>16&255,s[n++]=l>>>8&255,s[n++]=255&l;var c=a/4294967296*1e4&268435455;s[n++]=c>>>8&255,s[n++]=255&c,s[n++]=c>>>24&15|16,s[n++]=c>>>16&255,s[n++]=r>>>8|128,s[n++]=255&r;for(var u=t.node||f,p=0;6>p;p++)s[n+p]=u[p];return e?e:o(s)}function s(t,e,i){var n=e&&i||0;"string"==typeof t&&(e="binary"==t?new Array(16):null,t=null),t=t||{};var s=t.random||(t.rng||r)();if(s[6]=15&s[6]|64,s[8]=63&s[8]|128,e)for(var a=0;16>a;a++)e[n+a]=s[a];return e||o(s)}var r,a="undefined"!=typeof window?window:"undefined"!=typeof e?e:null;if(a&&a.crypto&&crypto.getRandomValues){var h=new Uint8Array(16);r=function(){return crypto.getRandomValues(h),h}}if(!r){var d=new Array(16);r=function(){for(var t,e=0;16>e;e++)0===(3&e)&&(t=4294967296*Math.random()),d[e]=t>>>((3&e)<<3)&255;return d}}for(var l=[],c={},u=0;256>u;u++)l[u]=(u+256).toString(16).substr(1),c[l[u]]=u;var p=r(),f=[1|p[0],p[1],p[2],p[3],p[4],p[5]],m=16383&(p[6]<<8|p[7]),v=0,g=0,y=s;y.v1=n,y.v4=s,y.parse=i,y.unparse=o,t.exports=y}).call(e,function(){return this}())},function(t,e,i){e.util=i(1),e.DOMutil=i(8),e.DataSet=i(9),e.DataView=i(11),e.Queue=i(10),e.Graph3d=i(12),e.graph3d={Camera:i(16),Filter:i(17),Point2d:i(15),Point3d:i(14),Slider:i(18),StepNumber:i(19)},e.moment=i(2),e.Hammer=i(20),e.keycharm=i(23)},function(t,e){e.prepareElements=function(t){for(var e in t)t.hasOwnProperty(e)&&(t[e].redundant=t[e].used,t[e].used=[])},e.cleanupElements=function(t){for(var e in t)if(t.hasOwnProperty(e)&&t[e].redundant){for(var i=0;i0?(o=e[t].redundant[0],e[t].redundant.shift()):(o=document.createElementNS("http://www.w3.org/2000/svg",t),i.appendChild(o)):(o=document.createElementNS("http://www.w3.org/2000/svg",t),e[t]={used:[],redundant:[]},i.appendChild(o)),e[t].used.push(o),o},e.getDOMElement=function(t,e,i,o){var n;return e.hasOwnProperty(t)?e[t].redundant.length>0?(n=e[t].redundant[0],e[t].redundant.shift()):(n=document.createElement(t),void 0!==o?i.insertBefore(n,o):i.appendChild(n)):(n=document.createElement(t),e[t]={used:[],redundant:[]},void 0!==o?i.insertBefore(n,o):i.appendChild(n)),e[t].used.push(n),n},e.drawPoint=function(t,i,o,n,s,r){var a;if("circle"==o.style?(a=e.getSVGElement("circle",n,s),a.setAttributeNS(null,"cx",t),a.setAttributeNS(null,"cy",i),a.setAttributeNS(null,"r",.5*o.size)):(a=e.getSVGElement("rect",n,s),a.setAttributeNS(null,"x",t-.5*o.size),a.setAttributeNS(null,"y",i-.5*o.size),a.setAttributeNS(null,"width",o.size),a.setAttributeNS(null,"height",o.size)),void 0!==o.styles&&a.setAttributeNS(null,"style",o.styles),a.setAttributeNS(null,"class",o.className+" vis-point"),r){var h=e.getSVGElement("text",n,s); +r.xOffset&&(t+=r.xOffset),r.yOffset&&(i+=r.yOffset),r.content&&(h.textContent=r.content),r.className&&h.setAttributeNS(null,"class",r.className+" vis-label"),h.setAttributeNS(null,"x",t),h.setAttributeNS(null,"y",i)}return a},e.drawBar=function(t,i,o,n,s,r,a,h){if(0!=n){0>n&&(n*=-1,i-=n);var d=e.getSVGElement("rect",r,a);d.setAttributeNS(null,"x",t-.5*o),d.setAttributeNS(null,"y",i),d.setAttributeNS(null,"width",o),d.setAttributeNS(null,"height",n),d.setAttributeNS(null,"class",s),h&&d.setAttributeNS(null,"style",h)}}},function(t,e,i){function o(t,e){if(t&&!Array.isArray(t)&&(e=t,t=null),this._options=e||{},this._data={},this.length=0,this._fieldId=this._options.fieldId||"id",this._type={},this._options.type)for(var i=Object.keys(this._options.type),o=0,n=i.length;n>o;o++){var s=i[o],r=this._options.type[s];"Date"==r||"ISODate"==r||"ASPDate"==r?this._type[s]="Date":this._type[s]=r}if(this._options.convert)throw new Error('Option "convert" is deprecated. Use "type" instead.');this._subscribers={},t&&this.add(t),this.setOptions(e)}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(10);o.prototype.setOptions=function(t){t&&void 0!==t.queue&&(t.queue===!1?this._queue&&(this._queue.destroy(),delete this._queue):(this._queue||(this._queue=r.extend(this,{replace:["add","update","remove"]})),"object"===n(t.queue)&&this._queue.setOptions(t.queue)))},o.prototype.on=function(t,e){var i=this._subscribers[t];i||(i=[],this._subscribers[t]=i),i.push({callback:e})},o.prototype.subscribe=function(){throw new Error("DataSet.subscribe is deprecated. Use DataSet.on instead.")},o.prototype.off=function(t,e){var i=this._subscribers[t];i&&(this._subscribers[t]=i.filter(function(t){return t.callback!=e}))},o.prototype.unsubscribe=function(){throw new Error("DataSet.unsubscribe is deprecated. Use DataSet.off instead.")},o.prototype._trigger=function(t,e,i){if("*"==t)throw new Error("Cannot trigger event *");var o=[];t in this._subscribers&&(o=o.concat(this._subscribers[t])),"*"in this._subscribers&&(o=o.concat(this._subscribers["*"]));for(var n=0,s=o.length;s>n;n++){var r=o[n];r.callback&&r.callback(t,e,i||null)}},o.prototype.add=function(t,e){var i,o=[],n=this;if(Array.isArray(t))for(var s=0,r=t.length;r>s;s++)i=n._addItem(t[s]),o.push(i);else{if(!(t instanceof Object))throw new Error("Unknown dataType");i=n._addItem(t),o.push(i)}return o.length&&this._trigger("add",{items:o},e),o},o.prototype.update=function(t,e){var i=[],o=[],n=[],r=[],a=this,h=a._fieldId,d=function(t){var e=t[h];if(a._data[e]){var d=s.extend({},a._data[e]);e=a._updateItem(t),o.push(e),r.push(t),n.push(d)}else e=a._addItem(t),i.push(e)};if(Array.isArray(t))for(var l=0,c=t.length;c>l;l++)t[l]instanceof Object?d(t[l]):console.warn("Ignoring input item, which is not an object at index "+l);else{if(!(t instanceof Object))throw new Error("Unknown dataType");d(t)}if(i.length&&this._trigger("add",{items:i},e),o.length){var u={items:o,oldData:n,data:r};this._trigger("update",u,e)}return i.concat(o)},o.prototype.get=function(t){var e,i,o,n=this,r=s.getType(arguments[0]);"String"==r||"Number"==r?(e=arguments[0],o=arguments[1]):"Array"==r?(i=arguments[0],o=arguments[1]):o=arguments[0];var a;if(o&&o.returnType){var h=["Array","Object"];a=-1==h.indexOf(o.returnType)?"Array":o.returnType}else a="Array";var d,l,c,u,p,f=o&&o.type||this._options.type,m=o&&o.filter,v=[];if(void 0!=e)d=n._getItem(e,f),d&&m&&!m(d)&&(d=null);else if(void 0!=i)for(u=0,p=i.length;p>u;u++)d=n._getItem(i[u],f),m&&!m(d)||v.push(d);else for(l=Object.keys(this._data),u=0,p=l.length;p>u;u++)c=l[u],d=n._getItem(c,f),m&&!m(d)||v.push(d);if(o&&o.order&&void 0==e&&this._sort(v,o.order),o&&o.fields){var g=o.fields;if(void 0!=e)d=this._filterFields(d,g);else for(u=0,p=v.length;p>u;u++)v[u]=this._filterFields(v[u],g)}if("Object"==a){var y,b={};for(u=0,p=v.length;p>u;u++)y=v[u],b[y.id]=y;return b}return void 0!=e?d:v},o.prototype.getIds=function(t){var e,i,o,n,s,r=this._data,a=t&&t.filter,h=t&&t.order,d=t&&t.type||this._options.type,l=Object.keys(r),c=[];if(a)if(h){for(s=[],e=0,i=l.length;i>e;e++)o=l[e],n=this._getItem(o,d),a(n)&&s.push(n);for(this._sort(s,h),e=0,i=s.length;i>e;e++)c.push(s[e][this._fieldId])}else for(e=0,i=l.length;i>e;e++)o=l[e],n=this._getItem(o,d),a(n)&&c.push(n[this._fieldId]);else if(h){for(s=[],e=0,i=l.length;i>e;e++)o=l[e],s.push(r[o]);for(this._sort(s,h),e=0,i=s.length;i>e;e++)c.push(s[e][this._fieldId])}else for(e=0,i=l.length;i>e;e++)o=l[e],n=r[o],c.push(n[this._fieldId]);return c},o.prototype.getDataSet=function(){return this},o.prototype.forEach=function(t,e){var i,o,n,s,r=e&&e.filter,a=e&&e.type||this._options.type,h=this._data,d=Object.keys(h);if(e&&e.order){var l=this.get(e);for(i=0,o=l.length;o>i;i++)n=l[i],s=n[this._fieldId],t(n,s)}else for(i=0,o=d.length;o>i;i++)s=d[i],n=this._getItem(s,a),r&&!r(n)||t(n,s)},o.prototype.map=function(t,e){var i,o,n,s,r=e&&e.filter,a=e&&e.type||this._options.type,h=[],d=this._data,l=Object.keys(d);for(i=0,o=l.length;o>i;i++)n=l[i],s=this._getItem(n,a),r&&!r(s)||h.push(t(s,n));return e&&e.order&&this._sort(h,e.order),h},o.prototype._filterFields=function(t,e){if(!t)return t;var i,o,n={},s=Object.keys(t),r=s.length;if(Array.isArray(e))for(i=0;r>i;i++)o=s[i],-1!=e.indexOf(o)&&(n[o]=t[o]);else for(i=0;r>i;i++)o=s[i],e.hasOwnProperty(o)&&(n[e[o]]=t[o]);return n},o.prototype._sort=function(t,e){if(s.isString(e)){var i=e;t.sort(function(t,e){var o=t[i],n=e[i];return o>n?1:n>o?-1:0})}else{if("function"!=typeof e)throw new TypeError("Order must be a function or a string");t.sort(e)}},o.prototype.remove=function(t,e){var i,o,n,s=[];if(Array.isArray(t))for(i=0,o=t.length;o>i;i++)n=this._remove(t[i]),null!=n&&s.push(n);else n=this._remove(t),null!=n&&s.push(n);return s.length&&this._trigger("remove",{items:s},e),s},o.prototype._remove=function(t){if(s.isNumber(t)||s.isString(t)){if(this._data[t])return delete this._data[t],this.length--,t}else if(t instanceof Object){var e=t[this._fieldId];if(void 0!==e&&this._data[e])return delete this._data[e],this.length--,e}return null},o.prototype.clear=function(t){var e=Object.keys(this._data);return this._data={},this.length=0,this._trigger("remove",{items:e},t),e},o.prototype.max=function(t){var e,i,o=this._data,n=Object.keys(o),s=null,r=null;for(e=0,i=n.length;i>e;e++){var a=n[e],h=o[a],d=h[t];null!=d&&(!s||d>r)&&(s=h,r=d)}return s},o.prototype.min=function(t){var e,i,o=this._data,n=Object.keys(o),s=null,r=null;for(e=0,i=n.length;i>e;e++){var a=n[e],h=o[a],d=h[t];null!=d&&(!s||r>d)&&(s=h,r=d)}return s},o.prototype.distinct=function(t){var e,i,o,n=this._data,r=Object.keys(n),a=[],h=this._options.type&&this._options.type[t]||null,d=0;for(e=0,o=r.length;o>e;e++){var l=r[e],c=n[l],u=c[t],p=!1;for(i=0;d>i;i++)if(a[i]==u){p=!0;break}p||void 0===u||(a[d]=u,d++)}if(h)for(e=0,o=a.length;o>e;e++)a[e]=s.convert(a[e],h);return a},o.prototype._addItem=function(t){var e=t[this._fieldId];if(void 0!=e){if(this._data[e])throw new Error("Cannot add item: item with id "+e+" already exists")}else e=s.randomUUID(),t[this._fieldId]=e;var i,o,n={},r=Object.keys(t);for(i=0,o=r.length;o>i;i++){var a=r[i],h=this._type[a];n[a]=s.convert(t[a],h)}return this._data[e]=n,this.length++,e},o.prototype._getItem=function(t,e){var i,o,n,r,a=this._data[t];if(!a)return null;var h={},d=Object.keys(a);if(e)for(n=0,r=d.length;r>n;n++)i=d[n],o=a[i],h[i]=s.convert(o,e[i]);else for(n=0,r=d.length;r>n;n++)i=d[n],o=a[i],h[i]=o;return h},o.prototype._updateItem=function(t){var e=t[this._fieldId];if(void 0==e)throw new Error("Cannot update item: item has no id (item: "+JSON.stringify(t)+")");var i=this._data[e];if(!i)throw new Error("Cannot update item: no item with id "+e+" found");for(var o=Object.keys(t),n=0,r=o.length;r>n;n++){var a=o[n],h=this._type[a];i[a]=s.convert(t[a],h)}return e},t.exports=o},function(t,e){function i(t){this.delay=null,this.max=1/0,this._queue=[],this._timeout=null,this._extended=null,this.setOptions(t)}i.prototype.setOptions=function(t){t&&"undefined"!=typeof t.delay&&(this.delay=t.delay),t&&"undefined"!=typeof t.max&&(this.max=t.max),this._flushIfNeeded()},i.extend=function(t,e){var o=new i(e);if(void 0!==t.flush)throw new Error("Target object already has a property flush");t.flush=function(){o.flush()};var n=[{name:"flush",original:void 0}];if(e&&e.replace)for(var s=0;sthis.max&&this.flush(),clearTimeout(this._timeout),this.queue.length>0&&"number"==typeof this.delay){var t=this;this._timeout=setTimeout(function(){t.flush()},this.delay)}},i.prototype.flush=function(){for(;this._queue.length>0;){var t=this._queue.shift();t.fn.apply(t.context||t.fn,t.args||[])}},t.exports=i},function(t,e,i){function o(t,e){this._data=null,this._ids={},this.length=0,this._options=e||{},this._fieldId="id",this._subscribers={};var i=this;this.listener=function(){i._onEvent.apply(i,arguments)},this.setData(t)}var n=i(1),s=i(9);o.prototype.setData=function(t){var e,i,o,n;if(this._data&&(this._data.off&&this._data.off("*",this.listener),e=Object.keys(this._ids),this._ids={},this.length=0,this._trigger("remove",{items:e})),this._data=t,this._data){for(this._fieldId=this._options.fieldId||this._data&&this._data.options&&this._data.options.fieldId||"id",e=this._data.getIds({filter:this._options&&this._options.filter}),o=0,n=e.length;n>o;o++)i=e[o],this._ids[i]=!0;this.length=e.length,this._trigger("add",{items:e}),this._data.on&&this._data.on("*",this.listener)}},o.prototype.refresh=function(){var t,e,i,o=this._data.getIds({filter:this._options&&this._options.filter}),n=Object.keys(this._ids),s={},r=[],a=[];for(e=0,i=o.length;i>e;e++)t=o[e],s[t]=!0,this._ids[t]||(r.push(t),this._ids[t]=!0);for(e=0,i=n.length;i>e;e++)t=n[e],s[t]||(a.push(t),delete this._ids[t]);this.length+=r.length-a.length,r.length&&this._trigger("add",{items:r}),a.length&&this._trigger("remove",{items:a})},o.prototype.get=function(t){var e,i,o,s=this,r=n.getType(arguments[0]);"String"==r||"Number"==r||"Array"==r?(e=arguments[0],i=arguments[1],o=arguments[2]):(i=arguments[0],o=arguments[1]);var a=n.extend({},this._options,i);this._options.filter&&i&&i.filter&&(a.filter=function(t){return s._options.filter(t)&&i.filter(t)});var h=[];return void 0!=e&&h.push(e),h.push(a),h.push(o),this._data&&this._data.get.apply(this._data,h)},o.prototype.getIds=function(t){var e;if(this._data){var i,o=this._options.filter;i=t&&t.filter?o?function(e){return o(e)&&t.filter(e)}:t.filter:o,e=this._data.getIds({filter:i,order:t&&t.order})}else e=[];return e},o.prototype.map=function(t,e){var i=[];if(this._data){var o,n=this._options.filter;o=e&&e.filter?n?function(t){return n(t)&&e.filter(t)}:e.filter:n,i=this._data.map(t,{filter:o,order:e&&e.order})}else i=[];return i},o.prototype.getDataSet=function(){for(var t=this;t instanceof o;)t=t._data;return t||null},o.prototype._onEvent=function(t,e,i){var o,n,s,r,a=e&&e.items,h=this._data,d=[],l=[],c=[],u=[];if(a&&h){switch(t){case"add":for(o=0,n=a.length;n>o;o++)s=a[o],r=this.get(s),r&&(this._ids[s]=!0,l.push(s));break;case"update":for(o=0,n=a.length;n>o;o++)s=a[o],r=this.get(s),r?this._ids[s]?(c.push(s),d.push(e.data[o])):(this._ids[s]=!0,l.push(s)):this._ids[s]&&(delete this._ids[s],u.push(s));break;case"remove":for(o=0,n=a.length;n>o;o++)s=a[o],this._ids[s]&&(delete this._ids[s],u.push(s))}this.length+=l.length-u.length,l.length&&this._trigger("add",{items:l},i),c.length&&this._trigger("update",{items:c,data:d},i),u.length&&this._trigger("remove",{items:u},i)}},o.prototype.on=s.prototype.on,o.prototype.off=s.prototype.off,o.prototype._trigger=s.prototype._trigger,o.prototype.subscribe=o.prototype.on,o.prototype.unsubscribe=o.prototype.off,t.exports=o},function(t,e,i){function o(t,e,i){if(!(this instanceof o))throw new SyntaxError("Constructor must be called with the new operator");this.containerElement=t,this.width="400px",this.height="400px",this.margin=10,this.defaultXCenter="55%",this.defaultYCenter="50%",this.xLabel="x",this.yLabel="y",this.zLabel="z";var n=function(t){return t};this.xValueLabel=n,this.yValueLabel=n,this.zValueLabel=n,this.filterLabel="time",this.legendLabel="value",this.style=o.STYLE.DOT,this.showPerspective=!0,this.showGrid=!0,this.keepAspectRatio=!0,this.showShadow=!1,this.showGrayBottom=!1,this.showTooltip=!1,this.verticalRatio=.5,this.animationInterval=1e3,this.animationPreload=!1,this.camera=new p,this.camera.setArmRotation(1,.5),this.camera.setArmLength(1.7),this.eye=new c(0,0,-1),this.dataTable=null,this.dataPoints=null,this.colX=void 0,this.colY=void 0,this.colZ=void 0,this.colValue=void 0,this.colFilter=void 0,this.xMin=0,this.xStep=void 0,this.xMax=1,this.yMin=0,this.yStep=void 0,this.yMax=1,this.zMin=0,this.zStep=void 0,this.zMax=1,this.valueMin=0,this.valueMax=1,this.xBarWidth=1,this.yBarWidth=1,this.axisColor="#4D4D4D",this.gridColor="#D3D3D3",this.dataColor={fill:"#7DC1FF",stroke:"#3267D2",strokeWidth:1},this.dotSizeRatio=.02,this.create(),this.setOptions(i),e&&this.setData(e)}function n(t){return"clientX"in t?t.clientX:t.targetTouches[0]&&t.targetTouches[0].clientX||0}function s(t){return"clientY"in t?t.clientY:t.targetTouches[0]&&t.targetTouches[0].clientY||0}var r="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},a=i(13),h=i(9),d=i(11),l=i(1),c=i(14),u=i(15),p=i(16),f=i(17),m=i(18),v=i(19);a(o.prototype),o.prototype._setScale=function(){this.scale=new c(1/(this.xMax-this.xMin),1/(this.yMax-this.yMin),1/(this.zMax-this.zMin)),this.keepAspectRatio&&(this.scale.x3&&(this.colFilter=3);else{if(this.style!==o.STYLE.DOTCOLOR&&this.style!==o.STYLE.DOTSIZE&&this.style!==o.STYLE.BARCOLOR&&this.style!==o.STYLE.BARSIZE)throw'Unknown style "'+this.style+'"';this.colX=0,this.colY=1,this.colZ=2,this.colValue=3,t.getNumberOfColumns()>4&&(this.colFilter=4)}},o.prototype.getNumberOfRows=function(t){return t.length},o.prototype.getNumberOfColumns=function(t){var e=0;for(var i in t[0])t[0].hasOwnProperty(i)&&e++;return e},o.prototype.getDistinctValues=function(t,e){for(var i=[],o=0;ot[o][e]&&(i.min=t[o][e]),i.maxt;t++){var f=(t-u)/(p-u),m=240*f,g=this._hsv2rgb(m,1,1);c.strokeStyle=g,c.beginPath(),c.moveTo(h,r+t),c.lineTo(a,r+t),c.stroke()}c.strokeStyle=this.axisColor,c.strokeRect(h,r,i,s)}if(this.style===o.STYLE.DOTSIZE&&(c.strokeStyle=this.axisColor,c.fillStyle=this.dataColor.fill,c.beginPath(),c.moveTo(h,r),c.lineTo(a,r),c.lineTo(a-i+e,d),c.lineTo(h,d),c.closePath(),c.fill(),c.stroke()),this.style===o.STYLE.DOTCOLOR||this.style===o.STYLE.DOTSIZE){var y=5,b=new v(this.valueMin,this.valueMax,(this.valueMax-this.valueMin)/5,!0);for(b.start(),b.getCurrent()0?this.yMin:this.yMax,n=this._convert3Dto2D(new c(_,r,this.zMin)),Math.cos(2*w)>0?(m.textAlign="center",m.textBaseline="top",n.y+=b):Math.sin(2*w)<0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(" "+this.xValueLabel(i.getCurrent())+" ",n.x,n.y),i.next()}for(m.lineWidth=1,o=void 0===this.defaultYStep,i=new v(this.yMin,this.yMax,this.yStep,o),i.start(),i.getCurrent()0?this.xMin:this.xMax,n=this._convert3Dto2D(new c(s,i.getCurrent(),this.zMin)),Math.cos(2*w)<0?(m.textAlign="center",m.textBaseline="top",n.y+=b):Math.sin(2*w)>0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(" "+this.yValueLabel(i.getCurrent())+" ",n.x,n.y),i.next();for(m.lineWidth=1,o=void 0===this.defaultZStep,i=new v(this.zMin,this.zMax,this.zStep,o),i.start(),i.getCurrent()0?this.xMin:this.xMax,r=Math.sin(w)<0?this.yMin:this.yMax;!i.end();)t=this._convert3Dto2D(new c(s,r,i.getCurrent())),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(t.x-b,t.y),m.stroke(),m.textAlign="right",m.textBaseline="middle",m.fillStyle=this.axisColor,m.fillText(this.zValueLabel(i.getCurrent())+" ",t.x-5,t.y),i.next();m.lineWidth=1,t=this._convert3Dto2D(new c(s,r,this.zMin)),e=this._convert3Dto2D(new c(s,r,this.zMax)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke(),m.lineWidth=1,u=this._convert3Dto2D(new c(this.xMin,this.yMin,this.zMin)),p=this._convert3Dto2D(new c(this.xMax,this.yMin,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(u.x,u.y),m.lineTo(p.x,p.y),m.stroke(),u=this._convert3Dto2D(new c(this.xMin,this.yMax,this.zMin)),p=this._convert3Dto2D(new c(this.xMax,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(u.x,u.y),m.lineTo(p.x,p.y),m.stroke(),m.lineWidth=1,t=this._convert3Dto2D(new c(this.xMin,this.yMin,this.zMin)),e=this._convert3Dto2D(new c(this.xMin,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke(),t=this._convert3Dto2D(new c(this.xMax,this.yMin,this.zMin)),e=this._convert3Dto2D(new c(this.xMax,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke();var x=this.xLabel;x.length>0&&(l=.1/this.scale.y,s=(this.xMin+this.xMax)/2,r=Math.cos(w)>0?this.yMin-l:this.yMax+l,n=this._convert3Dto2D(new c(s,r,this.zMin)),Math.cos(2*w)>0?(m.textAlign="center",m.textBaseline="top"):Math.sin(2*w)<0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(x,n.x,n.y));var k=this.yLabel;k.length>0&&(d=.1/this.scale.x,s=Math.sin(w)>0?this.xMin-d:this.xMax+d,r=(this.yMin+this.yMax)/2,n=this._convert3Dto2D(new c(s,r,this.zMin)),Math.cos(2*w)<0?(m.textAlign="center",m.textBaseline="top"):Math.sin(2*w)>0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(k,n.x,n.y));var O=this.zLabel;O.length>0&&(h=30,s=Math.cos(w)>0?this.xMin:this.xMax,r=Math.sin(w)<0?this.yMin:this.yMax,a=(this.zMin+this.zMax)/2,n=this._convert3Dto2D(new c(s,r,a)),m.textAlign="right",m.textBaseline="middle",m.fillStyle=this.axisColor,m.fillText(O,n.x-h,n.y))},o.prototype._hsv2rgb=function(t,e,i){var o,n,s,r,a,h;switch(r=i*e,a=Math.floor(t/60),h=r*(1-Math.abs(t/60%2-1)),a){case 0:o=r,n=h,s=0;break;case 1:o=h,n=r,s=0;break;case 2:o=0,n=r,s=h;break;case 3:o=0,n=h,s=r;break;case 4:o=h,n=0,s=r;break;case 5:o=r,n=0,s=h;break;default:o=0,n=0,s=0}return"RGB("+parseInt(255*o)+","+parseInt(255*n)+","+parseInt(255*s)+")"},o.prototype._redrawDataGrid=function(){var t,e,i,n,s,r,a,h,d,l,u,p,f=this.frame.canvas,m=f.getContext("2d");if(m.lineJoin="round",m.lineCap="round",!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(s=0;s0}else r=!0;r?(p=(t.point.z+e.point.z+i.point.z+n.point.z)/4,d=240*(1-(p-this.zMin)*this.scale.z/this.verticalRatio),l=1,this.showShadow?(u=Math.min(1+x.x/k/2,1),a=this._hsv2rgb(d,l,u),h=a):(u=1,a=this._hsv2rgb(d,l,u),h=this.axisColor)):(a="gray",h=this.axisColor),m.lineWidth=this._getStrokeWidth(t),m.fillStyle=a,m.strokeStyle=h,m.beginPath(),m.moveTo(t.screen.x,t.screen.y),m.lineTo(e.screen.x,e.screen.y),m.lineTo(n.screen.x,n.screen.y),m.lineTo(i.screen.x,i.screen.y),m.closePath(),m.fill(),m.stroke()}}else for(s=0;su&&(u=0);var p,f,m;this.style===o.STYLE.DOTCOLOR?(p=240*(1-(d.point.value-this.valueMin)*this.scale.value),f=this._hsv2rgb(p,1,1),m=this._hsv2rgb(p,1,.8)):this.style===o.STYLE.DOTSIZE?(f=this.dataColor.fill,m=this.dataColor.stroke):(p=240*(1-(d.point.z-this.zMin)*this.scale.z/this.verticalRatio),f=this._hsv2rgb(p,1,1),m=this._hsv2rgb(p,1,.8)),i.lineWidth=this._getStrokeWidth(d),i.strokeStyle=m,i.fillStyle=f,i.beginPath(),i.arc(d.screen.x,d.screen.y,u,0,2*Math.PI,!0),i.fill(),i.stroke()}}},o.prototype._redrawDataBar=function(){var t,e,i,n,s=this.frame.canvas,r=s.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t0){for(t=this.dataPoints[0],o.lineWidth=this._getStrokeWidth(t),o.lineJoin="round",o.lineCap="round",o.strokeStyle=this.dataColor.stroke,o.beginPath(),o.moveTo(t.screen.x,t.screen.y),e=1;e0?1:0>t?-1:0}var o=e[0],n=e[1],s=e[2],r=i((n.x-o.x)*(t.y-o.y)-(n.y-o.y)*(t.x-o.x)),a=i((s.x-n.x)*(t.y-n.y)-(s.y-n.y)*(t.x-n.x)),h=i((o.x-s.x)*(t.y-s.y)-(o.y-s.y)*(t.x-s.x));return!(0!=r&&0!=a&&r!=a||0!=a&&0!=h&&a!=h||0!=r&&0!=h&&r!=h)},o.prototype._dataPointFromXY=function(t,e){var i,n=100,s=null,r=null,a=null,h=new u(t,e);if(this.style===o.STYLE.BAR||this.style===o.STYLE.BARCOLOR||this.style===o.STYLE.BARSIZE)for(i=this.dataPoints.length-1;i>=0;i--){s=this.dataPoints[i];var d=s.surfaces;if(d)for(var l=d.length-1;l>=0;l--){var c=d[l],p=c.corners,f=[p[0].screen,p[1].screen,p[2].screen],m=[p[2].screen,p[3].screen,p[0].screen];if(this._insideTriangle(h,f)||this._insideTriangle(h,m))return s}}else for(i=0;ib)&&n>b&&(a=b,r=s)}}return r},o.prototype._showTooltip=function(t){var e,i,o;this.tooltip?(e=this.tooltip.dom.content,i=this.tooltip.dom.line,o=this.tooltip.dom.dot):(e=document.createElement("div"),e.style.position="absolute",e.style.padding="10px",e.style.border="1px solid #4d4d4d",e.style.color="#1a1a1a",e.style.background="rgba(255,255,255,0.7)",e.style.borderRadius="2px",e.style.boxShadow="5px 5px 10px rgba(128,128,128,0.5)",i=document.createElement("div"),i.style.position="absolute",i.style.height="40px",i.style.width="0",i.style.borderLeft="1px solid #4d4d4d",o=document.createElement("div"),o.style.position="absolute",o.style.height="0",o.style.width="0",o.style.border="5px solid #4d4d4d",o.style.borderRadius="5px",this.tooltip={dataPoint:null,dom:{content:e,line:i,dot:o}}),this._hideTooltip(),this.tooltip.dataPoint=t,"function"==typeof this.showTooltip?e.innerHTML=this.showTooltip(t.point):e.innerHTML="
    "+this.xLabel+":"+t.point.x+"
    "+this.yLabel+":"+t.point.y+"
    "+this.zLabel+":"+t.point.z+"
    ",e.style.left="0",e.style.top="0",this.frame.appendChild(e),this.frame.appendChild(i),this.frame.appendChild(o);var n=e.offsetWidth,s=e.offsetHeight,r=i.offsetHeight,a=o.offsetWidth,h=o.offsetHeight,d=t.screen.x-n/2;d=Math.min(Math.max(d,10),this.frame.clientWidth-10-n),i.style.left=t.screen.x+"px",i.style.top=t.screen.y-r+"px",e.style.left=d+"px",e.style.top=t.screen.y-r-s+"px",o.style.left=t.screen.x-a/2+"px",o.style.top=t.screen.y-h/2+"px"},o.prototype._hideTooltip=function(){if(this.tooltip){this.tooltip.dataPoint=null;for(var t in this.tooltip.dom)if(this.tooltip.dom.hasOwnProperty(t)){var e=this.tooltip.dom[t];e&&e.parentNode&&e.parentNode.removeChild(e)}}},t.exports=o},function(t,e){function i(t){return t?o(t):void 0}function o(t){for(var e in i.prototype)t[e]=i.prototype[e];return t}t.exports=i,i.prototype.on=i.prototype.addEventListener=function(t,e){return this._callbacks=this._callbacks||{},(this._callbacks[t]=this._callbacks[t]||[]).push(e),this},i.prototype.once=function(t,e){function i(){o.off(t,i),e.apply(this,arguments)}var o=this;return this._callbacks=this._callbacks||{},i.fn=e,this.on(t,i),this},i.prototype.off=i.prototype.removeListener=i.prototype.removeAllListeners=i.prototype.removeEventListener=function(t,e){if(this._callbacks=this._callbacks||{},0==arguments.length)return this._callbacks={},this;var i=this._callbacks[t];if(!i)return this;if(1==arguments.length)return delete this._callbacks[t],this;for(var o,n=0;no;++o)i[o].apply(this,e)}return this},i.prototype.listeners=function(t){return this._callbacks=this._callbacks||{},this._callbacks[t]||[]},i.prototype.hasListeners=function(t){return!!this.listeners(t).length}},function(t,e){function i(t,e,i){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0,this.z=void 0!==i?i:0}i.subtract=function(t,e){var o=new i;return o.x=t.x-e.x,o.y=t.y-e.y,o.z=t.z-e.z,o},i.add=function(t,e){var o=new i;return o.x=t.x+e.x,o.y=t.y+e.y,o.z=t.z+e.z,o},i.avg=function(t,e){return new i((t.x+e.x)/2,(t.y+e.y)/2,(t.z+e.z)/2)},i.crossProduct=function(t,e){var o=new i;return o.x=t.y*e.z-t.z*e.y,o.y=t.z*e.x-t.x*e.z,o.z=t.x*e.y-t.y*e.x,o},i.prototype.length=function(){return Math.sqrt(this.x*this.x+this.y*this.y+this.z*this.z)},t.exports=i},function(t,e){function i(t,e){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0}t.exports=i},function(t,e,i){function o(){this.armLocation=new n,this.armRotation={},this.armRotation.horizontal=0,this.armRotation.vertical=0,this.armLength=1.7,this.cameraLocation=new n,this.cameraRotation=new n(.5*Math.PI,0,0),this.calculateCameraOrientation()}var n=i(14);o.prototype.setArmLocation=function(t,e,i){this.armLocation.x=t,this.armLocation.y=e,this.armLocation.z=i,this.calculateCameraOrientation()},o.prototype.setArmRotation=function(t,e){void 0!==t&&(this.armRotation.horizontal=t),void 0!==e&&(this.armRotation.vertical=e,this.armRotation.vertical<0&&(this.armRotation.vertical=0),this.armRotation.vertical>.5*Math.PI&&(this.armRotation.vertical=.5*Math.PI)),void 0===t&&void 0===e||this.calculateCameraOrientation()},o.prototype.getArmRotation=function(){var t={};return t.horizontal=this.armRotation.horizontal,t.vertical=this.armRotation.vertical,t},o.prototype.setArmLength=function(t){void 0!==t&&(this.armLength=t,this.armLength<.71&&(this.armLength=.71),this.armLength>5&&(this.armLength=5),this.calculateCameraOrientation())},o.prototype.getArmLength=function(){return this.armLength},o.prototype.getCameraLocation=function(){return this.cameraLocation},o.prototype.getCameraRotation=function(){return this.cameraRotation},o.prototype.calculateCameraOrientation=function(){this.cameraLocation.x=this.armLocation.x-this.armLength*Math.sin(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.y=this.armLocation.y-this.armLength*Math.cos(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.z=this.armLocation.z+this.armLength*Math.sin(this.armRotation.vertical),this.cameraRotation.x=Math.PI/2-this.armRotation.vertical,this.cameraRotation.y=0,this.cameraRotation.z=-this.armRotation.horizontal},t.exports=o},function(t,e,i){function o(t,e,i){this.data=t,this.column=e,this.graph=i,this.index=void 0,this.value=void 0,this.values=i.getDistinctValues(t.get(),this.column),this.values.sort(function(t,e){return t>e?1:e>t?-1:0}),this.values.length>0&&this.selectValue(0),this.dataPoints=[],this.loaded=!1,this.onLoadCallback=void 0,i.animationPreload?(this.loaded=!1,this.loadInBackground()):this.loaded=!0}var n=i(11);o.prototype.isLoaded=function(){return this.loaded},o.prototype.getLoadedProgress=function(){for(var t=this.values.length,e=0;this.dataPoints[e];)e++;return Math.round(e/t*100)},o.prototype.getLabel=function(){return this.graph.filterLabel},o.prototype.getColumn=function(){return this.column},o.prototype.getSelectedValue=function(){return void 0!==this.index?this.values[this.index]:void 0},o.prototype.getValues=function(){return this.values},o.prototype.getValue=function(t){if(t>=this.values.length)throw"Error: index out of range";return this.values[t]},o.prototype._getDataPoints=function(t){if(void 0===t&&(t=this.index),void 0===t)return[];var e;if(this.dataPoints[t])e=this.dataPoints[t];else{var i={};i.column=this.column,i.value=this.values[t];var o=new n(this.data,{filter:function(t){return t[i.column]==i.value}}).get();e=this.graph._getDataPoints(o),this.dataPoints[t]=e}return e},o.prototype.setOnLoadCallback=function(t){this.onLoadCallback=t},o.prototype.selectValue=function(t){if(t>=this.values.length)throw"Error: index out of range";this.index=t,this.value=this.values[t]},o.prototype.loadInBackground=function(t){void 0===t&&(t=0);var e=this.graph.frame;if(t0&&(t--,this.setIndex(t))},o.prototype.next=function(){var t=this.getIndex();t0?this.setIndex(0):this.index=void 0},o.prototype.setIndex=function(t){if(!(to&&(o=0),o>this.values.length-1&&(o=this.values.length-1),o},o.prototype.indexToLeft=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t/(this.values.length-1)*e,o=i+3;return o},o.prototype._onMouseMove=function(t){var e=t.clientX-this.startClientX,i=this.startSlideX+e,o=this.leftToIndex(i);this.setIndex(o),n.preventDefault()},o.prototype._onMouseUp=function(t){this.frame.style.cursor="auto",n.removeEventListener(document,"mousemove",this.onmousemove),n.removeEventListener(document,"mouseup",this.onmouseup),n.preventDefault()},t.exports=o},function(t,e){function i(t,e,i,o){this._start=0,this._end=0,this._step=1,this.prettyStep=!0,this.precision=5,this._current=0,this.setRange(t,e,i,o)}i.prototype.setRange=function(t,e,i,o){this._start=t?t:0,this._end=e?e:0,this.setStep(i,o)},i.prototype.setStep=function(t,e){void 0===t||0>=t||(void 0!==e&&(this.prettyStep=e),this.prettyStep===!0?this._step=i.calculatePrettyStep(t):this._step=t)},i.calculatePrettyStep=function(t){var e=function(t){return Math.log(t)/Math.LN10},i=Math.pow(10,Math.round(e(t))),o=2*Math.pow(10,Math.round(e(t/2))),n=5*Math.pow(10,Math.round(e(t/5))),s=i;return Math.abs(o-t)<=Math.abs(s-t)&&(s=o),Math.abs(n-t)<=Math.abs(s-t)&&(s=n),0>=s&&(s=1),s},i.prototype.getCurrent=function(){return parseFloat(this._current.toPrecision(this.precision))},i.prototype.getStep=function(){return this._step},i.prototype.start=function(){this._current=this._start-this._start%this._step},i.prototype.next=function(){this._current+=this._step},i.prototype.end=function(){return this._current>this._end},t.exports=i},function(t,e,i){if("undefined"!=typeof window){var o=i(21),n=window.Hammer||i(22);t.exports=o(n,{preventDefault:"mouse"})}else t.exports=function(){throw Error("hammer.js is only available in a browser, not in node.js.")}},function(t,e,i){var o,n,s;!function(i){n=[],o=i,s="function"==typeof o?o.apply(e,n):o,!(void 0!==s&&(t.exports=s))}(function(){var t=null;return function e(i,o){function n(t){return t.match(/[^ ]+/g)}function s(e){if("hammer.input"!==e.type){if(e.srcEvent._handled||(e.srcEvent._handled={}),e.srcEvent._handled[e.type])return;e.srcEvent._handled[e.type]=!0}var i=!1;e.stopPropagation=function(){i=!0};var o=e.srcEvent.stopPropagation.bind(e.srcEvent);"function"==typeof o&&(e.srcEvent.stopPropagation=function(){o(),e.stopPropagation()}),e.firstTarget=t;for(var n=t;n&&!i;){var s=n.hammer;if(s)for(var r,a=0;a0?d._handlers[t]=o:(i.off(t,s),delete d._handlers[t]))}),d},d.emit=function(e,o){t=o.target,i.emit(e,o)},d.destroy=function(){var t=i.element.hammer,e=t.indexOf(d);-1!==e&&t.splice(e,1),t.length||delete i.element.hammer,d._handlers={},i.destroy()},d}})},function(t,e,i){var o;!function(n,s,r,a){function h(t,e,i){return setTimeout(p(t,i),e)}function d(t,e,i){return Array.isArray(t)?(l(t,i[e],i),!0):!1}function l(t,e,i){var o;if(t)if(t.forEach)t.forEach(e,i);else if(t.length!==a)for(o=0;o\s*\(/gm,"{anonymous}()@"):"Unknown Stack Trace",s=n.console&&(n.console.warn||n.console.log);return s&&s.call(n.console,o,i),t.apply(this,arguments)}}function u(t,e,i){var o,n=e.prototype;o=t.prototype=Object.create(n),o.constructor=t,o._super=n,i&&ct(o,i)}function p(t,e){return function(){return t.apply(e,arguments)}}function f(t,e){return typeof t==ft?t.apply(e?e[0]||a:a,e):t}function m(t,e){return t===a?e:t}function v(t,e,i){l(w(e),function(e){t.addEventListener(e,i,!1)})}function g(t,e,i){l(w(e),function(e){t.removeEventListener(e,i,!1)})}function y(t,e){for(;t;){if(t==e)return!0;t=t.parentNode}return!1}function b(t,e){return t.indexOf(e)>-1}function w(t){return t.trim().split(/\s+/g)}function _(t,e,i){if(t.indexOf&&!i)return t.indexOf(e);for(var o=0;oi[e]}):o.sort()),o}function O(t,e){for(var i,o,n=e[0].toUpperCase()+e.slice(1),s=0;s1&&!i.firstMultiple?i.firstMultiple=N(e):1===n&&(i.firstMultiple=!1);var s=i.firstInput,r=i.firstMultiple,a=r?r.center:s.center,h=e.center=R(o);e.timeStamp=gt(),e.deltaTime=e.timeStamp-s.timeStamp,e.angle=B(a,h),e.distance=A(a,h),P(i,e),e.offsetDirection=L(e.deltaX,e.deltaY);var d=z(e.deltaTime,e.deltaX,e.deltaY);e.overallVelocityX=d.x,e.overallVelocityY=d.y,e.overallVelocity=vt(d.x)>vt(d.y)?d.x:d.y,e.scale=r?j(r.pointers,o):1,e.rotation=r?F(r.pointers,o):0,e.maxPointers=i.prevInput?e.pointers.length>i.prevInput.maxPointers?e.pointers.length:i.prevInput.maxPointers:e.pointers.length,I(i,e);var l=t.element;y(e.srcEvent.target,l)&&(l=e.srcEvent.target),e.target=l}function P(t,e){var i=e.center,o=t.offsetDelta||{},n=t.prevDelta||{},s=t.prevInput||{};e.eventType!==Et&&s.eventType!==It||(n=t.prevDelta={x:s.deltaX||0,y:s.deltaY||0},o=t.offsetDelta={x:i.x,y:i.y}),e.deltaX=n.x+(i.x-o.x),e.deltaY=n.y+(i.y-o.y)}function I(t,e){var i,o,n,s,r=t.lastInterval||e,h=e.timeStamp-r.timeStamp;if(e.eventType!=Nt&&(h>Tt||r.velocity===a)){var d=e.deltaX-r.deltaX,l=e.deltaY-r.deltaY,c=z(h,d,l);o=c.x,n=c.y,i=vt(c.x)>vt(c.y)?c.x:c.y,s=L(d,l),t.lastInterval=e}else i=r.velocity,o=r.velocityX,n=r.velocityY,s=r.direction;e.velocity=i,e.velocityX=o,e.velocityY=n,e.direction=s}function N(t){for(var e=[],i=0;in;)i+=t[n].clientX,o+=t[n].clientY,n++;return{x:mt(i/e),y:mt(o/e)}}function z(t,e,i){return{x:e/t||0,y:i/t||0}}function L(t,e){return t===e?Rt:vt(t)>=vt(e)?0>t?zt:Lt:0>e?At:Bt}function A(t,e,i){i||(i=Wt);var o=e[i[0]]-t[i[0]],n=e[i[1]]-t[i[1]];return Math.sqrt(o*o+n*n)}function B(t,e,i){i||(i=Wt);var o=e[i[0]]-t[i[0]],n=e[i[1]]-t[i[1]];return 180*Math.atan2(n,o)/Math.PI}function F(t,e){return B(e[1],e[0],Yt)+B(t[1],t[0],Yt)}function j(t,e){return A(e[0],e[1],Yt)/A(t[0],t[1],Yt)}function H(){this.evEl=Vt,this.evWin=Ut,this.allow=!0,this.pressed=!1,S.apply(this,arguments)}function W(){this.evEl=Zt,this.evWin=Kt,S.apply(this,arguments),this.store=this.manager.session.pointerEvents=[]}function Y(){this.evTarget=Qt,this.evWin=$t,this.started=!1,S.apply(this,arguments)}function G(t,e){var i=x(t.touches),o=x(t.changedTouches);return e&(It|Nt)&&(i=k(i.concat(o),"identifier",!0)),[i,o]}function V(){this.evTarget=ee,this.targetIds={},S.apply(this,arguments)}function U(t,e){var i=x(t.touches),o=this.targetIds;if(e&(Et|Pt)&&1===i.length)return o[i[0].identifier]=!0,[i,i];var n,s,r=x(t.changedTouches),a=[],h=this.target;if(s=i.filter(function(t){return y(t.target,h)}),e===Et)for(n=0;na&&(e.push(t),a=e.length-1):n&(It|Nt)&&(i=!0),0>a||(e[a]=t,this.callback(this.manager,n,{pointers:e,changedPointers:[t],pointerType:s,srcEvent:t}),i&&e.splice(a,1))}});var Jt={touchstart:Et,touchmove:Pt,touchend:It,touchcancel:Nt},Qt="touchstart",$t="touchstart touchmove touchend touchcancel";u(Y,S,{handler:function(t){var e=Jt[t.type];if(e===Et&&(this.started=!0),this.started){var i=G.call(this,t,e);e&(It|Nt)&&i[0].length-i[1].length===0&&(this.started=!1),this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Mt,srcEvent:t})}}});var te={touchstart:Et,touchmove:Pt,touchend:It,touchcancel:Nt},ee="touchstart touchmove touchend touchcancel";u(V,S,{handler:function(t){var e=te[t.type],i=U.call(this,t,e);i&&this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Mt,srcEvent:t})}}),u(q,S,{handler:function(t,e,i){var o=i.pointerType==Mt,n=i.pointerType==St;if(o)this.mouse.allow=!1;else if(n&&!this.mouse.allow)return;e&(It|Nt)&&(this.mouse.allow=!0),this.callback(t,e,i)},destroy:function(){this.touch.destroy(),this.mouse.destroy()}});var ie=O(pt.style,"touchAction"),oe=ie!==a,ne="compute",se="auto",re="manipulation",ae="none",he="pan-x",de="pan-y";X.prototype={set:function(t){t==ne&&(t=this.compute()),oe&&this.manager.element.style&&(this.manager.element.style[ie]=t),this.actions=t.toLowerCase().trim()},update:function(){this.set(this.manager.options.touchAction)},compute:function(){var t=[];return l(this.manager.recognizers,function(e){f(e.options.enable,[e])&&(t=t.concat(e.getTouchAction()))}),Z(t.join(" "))},preventDefaults:function(t){if(!oe){var e=t.srcEvent,i=t.offsetDirection;if(this.manager.session.prevented)return void e.preventDefault();var o=this.actions,n=b(o,ae),s=b(o,de),r=b(o,he);if(n){var a=1===t.pointers.length,h=t.distance<2,d=t.deltaTime<250;if(a&&h&&d)return}if(!r||!s)return n||s&&i&Ft||r&&i&jt?this.preventSrc(e):void 0}},preventSrc:function(t){this.manager.session.prevented=!0,t.preventDefault()}};var le=1,ce=2,ue=4,pe=8,fe=pe,me=16,ve=32;K.prototype={defaults:{},set:function(t){return ct(this.options,t),this.manager&&this.manager.touchAction.update(),this},recognizeWith:function(t){if(d(t,"recognizeWith",this))return this;var e=this.simultaneous;return t=$(t,this),e[t.id]||(e[t.id]=t,t.recognizeWith(this)),this},dropRecognizeWith:function(t){return d(t,"dropRecognizeWith",this)?this:(t=$(t,this),delete this.simultaneous[t.id],this)},requireFailure:function(t){if(d(t,"requireFailure",this))return this;var e=this.requireFail;return t=$(t,this),-1===_(e,t)&&(e.push(t),t.requireFailure(this)),this},dropRequireFailure:function(t){if(d(t,"dropRequireFailure",this))return this;t=$(t,this);var e=_(this.requireFail,t);return e>-1&&this.requireFail.splice(e,1),this},hasRequireFailures:function(){return this.requireFail.length>0},canRecognizeWith:function(t){return!!this.simultaneous[t.id]},emit:function(t){function e(e){i.manager.emit(e,t)}var i=this,o=this.state;pe>o&&e(i.options.event+J(o)),e(i.options.event),t.additionalEvent&&e(t.additionalEvent),o>=pe&&e(i.options.event+J(o))},tryEmit:function(t){return this.canEmit()?this.emit(t):void(this.state=ve)},canEmit:function(){for(var t=0;ts?zt:Lt,i=s!=this.pX,o=Math.abs(t.deltaX)):(n=0===r?Rt:0>r?At:Bt,i=r!=this.pY,o=Math.abs(t.deltaY))),t.direction=n,i&&o>e.threshold&&n&e.direction},attrTest:function(t){return tt.prototype.attrTest.call(this,t)&&(this.state&ce||!(this.state&ce)&&this.directionTest(t))},emit:function(t){this.pX=t.deltaX,this.pY=t.deltaY;var e=Q(t.direction);e&&(t.additionalEvent=this.options.event+e),this._super.emit.call(this,t)}}),u(it,tt,{defaults:{event:"pinch",threshold:0,pointers:2},getTouchAction:function(){return[ae]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.scale-1)>this.options.threshold||this.state&ce)},emit:function(t){if(1!==t.scale){var e=t.scale<1?"in":"out";t.additionalEvent=this.options.event+e}this._super.emit.call(this,t)}}),u(ot,K,{defaults:{event:"press",pointers:1,time:251,threshold:9},getTouchAction:function(){return[se]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,o=t.distancee.time;if(this._input=t,!o||!i||t.eventType&(It|Nt)&&!n)this.reset();else if(t.eventType&Et)this.reset(),this._timer=h(function(){this.state=fe,this.tryEmit()},e.time,this);else if(t.eventType&It)return fe;return ve},reset:function(){clearTimeout(this._timer)},emit:function(t){this.state===fe&&(t&&t.eventType&It?this.manager.emit(this.options.event+"up",t):(this._input.timeStamp=gt(),this.manager.emit(this.options.event,this._input)))}}),u(nt,tt,{defaults:{event:"rotate",threshold:0,pointers:2},getTouchAction:function(){return[ae]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.rotation)>this.options.threshold||this.state&ce)}}),u(st,tt,{defaults:{event:"swipe",threshold:10,velocity:.3,direction:Ft|jt,pointers:1},getTouchAction:function(){return et.prototype.getTouchAction.call(this)},attrTest:function(t){var e,i=this.options.direction;return i&(Ft|jt)?e=t.overallVelocity:i&Ft?e=t.overallVelocityX:i&jt&&(e=t.overallVelocityY),this._super.attrTest.call(this,t)&&i&t.offsetDirection&&t.distance>this.options.threshold&&t.maxPointers==this.options.pointers&&vt(e)>this.options.velocity&&t.eventType&It},emit:function(t){var e=Q(t.offsetDirection);e&&this.manager.emit(this.options.event+e,t),this.manager.emit(this.options.event,t)}}),u(rt,K,{defaults:{event:"tap",pointers:1,taps:1,interval:300,time:250,threshold:9,posThreshold:10},getTouchAction:function(){return[re]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,o=t.distance=e;e++)r[String.fromCharCode(e)]={code:65+(e-97),shift:!1};for(e=65;90>=e;e++)r[String.fromCharCode(e)]={code:e,shift:!0};for(e=0;9>=e;e++)r[""+e]={code:48+e,shift:!1};for(e=1;12>=e;e++)r["F"+e]={code:111+e,shift:!1};for(e=0;9>=e;e++)r["num"+e]={code:96+e,shift:!1};r["num*"]={code:106,shift:!1},r["num+"]={code:107,shift:!1},r["num-"]={code:109,shift:!1},r["num/"]={code:111,shift:!1},r["num."]={code:110,shift:!1},r.left={code:37,shift:!1},r.up={code:38,shift:!1},r.right={code:39,shift:!1},r.down={code:40,shift:!1},r.space={code:32,shift:!1},r.enter={code:13,shift:!1},r.shift={code:16,shift:void 0},r.esc={code:27,shift:!1},r.backspace={code:8,shift:!1},r.tab={code:9,shift:!1},r.ctrl={code:17,shift:!1},r.alt={code:18,shift:!1},r["delete"]={code:46,shift:!1},r.pageup={code:33,shift:!1},r.pagedown={code:34,shift:!1},r["="]={code:187,shift:!1},r["-"]={code:189,shift:!1},r["]"]={code:221,shift:!1},r["["]={code:219,shift:!1};var a=function(t){d(t,"keydown")},h=function(t){d(t,"keyup")},d=function(t,e){if(void 0!==s[e][t.keyCode]){for(var o=s[e][t.keyCode],n=0;ne)&&(n=e),(null===s||i>s)&&(s=i)}),null!==n&&null!==s){var r=(n+s)/2,a=Math.max(this.range.end-this.range.start,1.1*(s-n)),h=e&&void 0!==e.animation?e.animation:!0;this.range.setRange(r-a/2,r+a/2,h)}}},n.prototype.fit=function(t){var e,i=t&&void 0!==t.animation?t.animation:!0,o=this.itemsData&&this.itemsData.getDataSet();1===o.length&&void 0===o.get()[0].end?(e=this.getDataRange(),this.moveTo(e.min.valueOf(),{animation:i})):(e=this.getItemRange(),this.range.setRange(e.min,e.max,i))},n.prototype.getItemRange=function(){var t=this,e=this.getDataRange(),i=null!==e.min?e.min.valueOf():null,o=null!==e.max?e.max.valueOf():null,n=null,s=null;if(null!=i&&null!=o){var r,a,h,d,c;!function(){var e=function(t){return l.convert(t.data.start,"Date").valueOf()},u=function(t){var e=void 0!=t.data.end?t.data.end:t.data.start;return l.convert(e,"Date").valueOf()};r=o-i,0>=r&&(r=10),a=r/t.props.center.width,l.forEach(t.itemSet.items,function(t){t.show(),t.repositionX();var r=e(t),h=u(t);if(this.options.rtl)var d=r-(t.getWidthRight()+10)*a,l=h+(t.getWidthLeft()+10)*a;else var d=r-(t.getWidthLeft()+10)*a,l=h+(t.getWidthRight()+10)*a;i>d&&(i=d,n=t),l>o&&(o=l,s=t)}.bind(t)),n&&s&&(h=n.getWidthLeft()+10,d=s.getWidthRight()+10,c=t.props.center.width-h-d,c>0&&(t.options.rtl?(i=e(n)-d*r/c,o=u(s)+h*r/c):(i=e(n)-h*r/c,o=u(s)+d*r/c)))}()}return{min:null!=i?new Date(i):null,max:null!=o?new Date(o):null}},n.prototype.getDataRange=function(){var t=null,e=null,i=this.itemsData&&this.itemsData.getDataSet();return i&&i.forEach(function(i){var o=l.convert(i.start,"Date").valueOf(),n=l.convert(void 0!=i.end?i.end:i.start,"Date").valueOf();(null===t||t>o)&&(t=o),(null===e||n>e)&&(e=n)}),{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},n.prototype.getEventProperties=function(t){var e=t.center?t.center.x:t.clientX,i=t.center?t.center.y:t.clientY;if(this.options.rtl)var o=l.getAbsoluteRight(this.dom.centerContainer)-e;else var o=e-l.getAbsoluteLeft(this.dom.centerContainer);var n=i-l.getAbsoluteTop(this.dom.centerContainer),s=this.itemSet.itemFromTarget(t),r=this.itemSet.groupFromTarget(t),a=g.customTimeFromTarget(t),h=this.itemSet.options.snap||null,d=this.body.util.getScale(),c=this.body.util.getStep(),u=this._toTime(o),p=h?h(u,d,c):u,f=l.getTarget(t),m=null;return null!=s?m="item":null!=a?m="custom-time":l.hasParent(f,this.timeAxis.dom.foreground)?m="axis":this.timeAxis2&&l.hasParent(f,this.timeAxis2.dom.foreground)?m="axis":l.hasParent(f,this.itemSet.dom.labelSet)?m="group-label":l.hasParent(f,this.currentTime.bar)?m="current-time":l.hasParent(f,this.dom.center)&&(m="background"),{event:t,item:s?s.id:null,group:r?r.groupId:null,what:m,pageX:t.srcEvent?t.srcEvent.pageX:t.pageX,pageY:t.srcEvent?t.srcEvent.pageY:t.pageY,x:o,y:n,time:u,snappedTime:p}},t.exports=n},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i0&&this._makeItem([]),this._makeHeader(n),this._handleObject(this.configureOptions[n],[n])),i++);this.options.showButton===!0&&!function(){var e=document.createElement("div");e.className="vis-configuration vis-config-button",e.innerHTML="generate options",e.onclick=function(){t._printOptions()},e.onmouseover=function(){e.className="vis-configuration vis-config-button hover"},e.onmouseout=function(){e.className="vis-configuration vis-config-button"},t.optionsContainer=document.createElement("div"),t.optionsContainer.className="vis-configuration vis-config-option-container",t.domElements.push(t.optionsContainer),t.domElements.push(e)}(),this._push()}},{key:"_push",value:function(){this.wrapper=document.createElement("div"),this.wrapper.className="vis-configuration-wrapper",this.container.appendChild(this.wrapper);for(var t=0;t1?o-1:0),r=1;o>r;r++)n[r-1]=e[r];return n.forEach(function(t){s.appendChild(t)}),i.domElements.push(s),{v:i.domElements.length}}();if("object"===("undefined"==typeof a?"undefined":s(a)))return a.v}return 0}},{key:"_makeHeader",value:function(t){var e=document.createElement("div");e.className="vis-configuration vis-config-header",e.innerHTML=t,this._makeItem([],e)}},{key:"_makeLabel",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=document.createElement("div");return o.className="vis-configuration vis-config-label vis-config-s"+e.length,i===!0?o.innerHTML=""+t+":":o.innerHTML=t+":",o}},{key:"_makeDropdown",value:function(t,e,i){var o=document.createElement("select");o.className="vis-configuration vis-config-select";var n=0;void 0!==e&&-1!==t.indexOf(e)&&(n=t.indexOf(e));for(var s=0;se&&n>e*c?(a.min=Math.ceil(e*c),l=a.min,d="range increased"):n>e/c&&(a.min=Math.ceil(e/c),l=a.min,d="range increased"),e*c>s&&1!==s&&(a.max=Math.ceil(e*c),l=a.max,d="range increased"),a.value=e}else a.value=o;var u=document.createElement("input");u.className="vis-configuration vis-config-rangeinput",u.value=a.value;var p=this;a.onchange=function(){u.value=this.value,p._update(Number(this.value),i)},a.oninput=function(){u.value=this.value};var f=this._makeLabel(i[i.length-1],i),m=this._makeItem(i,f,a,u);""!==d&&this.popupHistory[m]!==l&&(this.popupHistory[m]=l,this._setupPopup(d,m))}},{key:"_setupPopup",value:function(t,e){var i=this;if(this.initialized===!0&&this.allowCreation===!0&&this.popupCountervar options = "+JSON.stringify(t,null,2)+""}},{key:"getOptions",value:function(){for(var t={},e=0;es;s++)for(r=0;rp?p+1:p;var f=l/this.r,m=a.RGBToHSV(this.color.r,this.color.g,this.color.b);m.h=p,m.s=f;var v=a.HSVToRGB(m.h,m.s,m.v);v.a=this.color.a,this.color=v,this.initialColorDiv.style.backgroundColor="rgba("+this.initialColor.r+","+this.initialColor.g+","+this.initialColor.b+","+this.initialColor.a+")",this.newColorDiv.style.backgroundColor="rgba("+this.color.r+","+this.color.g+","+this.color.b+","+this.color.a+")"}}]),t}();e["default"]=h},function(t,e,i){i(20);e.onTouch=function(t,e){e.inputHandler=function(t){t.isFirst&&e(t)},t.on("hammer.input",e.inputHandler)},e.onRelease=function(t,e){return e.inputHandler=function(t){t.isFinal&&e(t)},t.on("hammer.input",e.inputHandler)},e.offTouch=function(t,e){t.off("hammer.input",e.inputHandler)},e.offRelease=e.offTouch,e.disablePreventDefaultVertically=function(t){var e="pan-y";return t.getTouchAction=function(){return[e]},t}},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=function(){function t(t,e){for(var i=0;is.distance?console.log('%cUnknown option detected: "'+e+'" in '+t.printLocation(n.path,e,"")+"Perhaps it was misplaced? Matching option found at: "+t.printLocation(s.path,s.closestMatch,""),d):n.distance<=r?console.log('%cUnknown option detected: "'+e+'". Did you mean "'+n.closestMatch+'"?'+t.printLocation(n.path,e),d):console.log('%cUnknown option detected: "'+e+'". Did you mean one of these: '+t.print(Object.keys(i))+t.printLocation(o,e),d),a=!0}},{key:"findInOptions",value:function(e,i,o){var n=arguments.length<=3||void 0===arguments[3]?!1:arguments[3],s=1e9,a="",h=[],d=e.toLowerCase(),l=void 0;for(var c in i){var u=void 0;if(void 0!==i[c].__type__&&n===!0){var p=t.findInOptions(e,i[c],r.copyAndExtendArray(o,c));s>p.distance&&(a=p.closestMatch,h=p.path,s=p.distance,l=p.indexMatch)}else-1!==c.toLowerCase().indexOf(d)&&(l=c),u=t.levenshteinDistance(e,c),s>u&&(a=c,h=r.copyArray(o),s=u)}return{closestMatch:a,path:h,distance:s,indexMatch:l}}},{key:"printLocation",value:function(t,e){for(var i=arguments.length<=2||void 0===arguments[2]?"Problem value found at: \n":arguments[2],o="\n\n"+i+"options = {\n",n=0;ns;s++)o+=" ";o+=t[n]+": {\n"}for(var r=0;ru,r=s||null===n?n:l+(n-l)*i,p=s||null===a?a:c+(a-c)*i;y=h._applyRange(r,p),d.updateHiddenDates(h.options.moment,h.body,h.options.hiddenDates),v=v||y,y&&h.body.emitter.emit("rangechange",{start:new Date(h.start),end:new Date(h.end),byUser:o}),s?v&&h.body.emitter.emit("rangechanged",{start:new Date(h.start),end:new Date(h.end),byUser:o}):h.animationTimer=setTimeout(w,20)}};return g()}var y=this._applyRange(n,a);if(d.updateHiddenDates(this.options.moment,this.body,this.options.hiddenDates),y){var b={start:new Date(this.start),end:new Date(this.end),byUser:o};this.body.emitter.emit("rangechange",b),this.body.emitter.emit("rangechanged",b)}},o.prototype._cancelAnimation=function(){this.animationTimer&&(clearTimeout(this.animationTimer),this.animationTimer=null)},o.prototype._applyRange=function(t,e){var i,o=null!=t?r.convert(t,"Date").valueOf():this.start,n=null!=e?r.convert(e,"Date").valueOf():this.end,s=null!=this.options.max?r.convert(this.options.max,"Date").valueOf():null,a=null!=this.options.min?r.convert(this.options.min,"Date").valueOf():null;if(isNaN(o)||null===o)throw new Error('Invalid start "'+t+'"');if(isNaN(n)||null===n)throw new Error('Invalid end "'+e+'"');if(o>n&&(n=o),null!==a&&a>o&&(i=a-o,o+=i,n+=i,null!=s&&n>s&&(n=s)),null!==s&&n>s&&(i=n-s,o-=i,n-=i,null!=a&&a>o&&(o=a)),null!==this.options.zoomMin){var h=parseFloat(this.options.zoomMin);0>h&&(h=0),h>n-o&&(this.end-this.start===h&&o>this.start&&nd&&(d=0),n-o>d&&(this.end-this.start===d&&othis.end?(o=this.start,n=this.end):(i=n-o-d,o+=i/2,n-=i/2))}var l=this.start!=o||this.end!=n;return o>=this.start&&o<=this.end||n>=this.start&&n<=this.end||this.start>=o&&this.start<=n||this.end>=o&&this.end<=n||this.body.emitter.emit("checkRangedItems"),this.start=o,this.end=n,l},o.prototype.getRange=function(){return{start:this.start,end:this.end}},o.prototype.conversion=function(t,e){return o.conversion(this.start,this.end,t,e)},o.conversion=function(t,e,i,o){return void 0===o&&(o=0),0!=i&&e-t!=0?{offset:t,scale:i/(e-t-o)}:{offset:0,scale:1}},o.prototype._onDragStart=function(t){this.deltaDifference=0,this.previousDelta=0,this.options.moveable&&this._isInsideRange(t)&&this.props.touch.allowDragging&&(this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.dragging=!0,this.body.dom.root&&(this.body.dom.root.style.cursor="move"))},o.prototype._onDrag=function(t){if(this.props.touch.dragging&&this.options.moveable&&this.props.touch.allowDragging){var e=this.options.direction;n(e);var i="horizontal"==e?t.deltaX:t.deltaY;i-=this.deltaDifference;var o=this.props.touch.end-this.props.touch.start,s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end);o-=s;var r="horizontal"==e?this.body.domProps.center.width:this.body.domProps.center.height;if(this.options.rtl)var a=i/r*o;else var a=-i/r*o;var h=this.props.touch.start+a,l=this.props.touch.end+a,c=d.snapAwayFromHidden(this.body.hiddenDates,h,this.previousDelta-i,!0),u=d.snapAwayFromHidden(this.body.hiddenDates,l,this.previousDelta-i,!0);if(c!=h||u!=l)return this.deltaDifference+=i,this.props.touch.start=c,this.props.touch.end=u,void this._onDrag(t);this.previousDelta=i,this._applyRange(h,l);var p=new Date(this.start),f=new Date(this.end);this.body.emitter.emit("rangechange",{start:p,end:f,byUser:!0})}},o.prototype._onDragEnd=function(t){this.props.touch.dragging&&this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.dragging=!1,this.body.dom.root&&(this.body.dom.root.style.cursor="auto"),this.body.emitter.emit("rangechanged",{start:new Date(this.start),end:new Date(this.end),byUser:!0}))},o.prototype._onMouseWheel=function(t){if(this.options.zoomable&&this.options.moveable&&this._isInsideRange(t)&&(!this.options.zoomKey||t[this.options.zoomKey])){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i;i=0>e?1-e/5:1/(1+e/5);var o=this.getPointer({x:t.clientX,y:t.clientY},this.body.dom.center),n=this._pointerToDate(o);this.zoom(i,n,e)}t.preventDefault()}},o.prototype._onTouch=function(t){this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.allowDragging=!0,this.props.touch.center=null,this.scaleOffset=0,this.deltaDifference=0},o.prototype._onPinch=function(t){if(this.options.zoomable&&this.options.moveable){this.props.touch.allowDragging=!1,this.props.touch.center||(this.props.touch.center=this.getPointer(t.center,this.body.dom.center));var e=1/(t.scale+this.scaleOffset),i=this._pointerToDate(this.props.touch.center),o=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),n=d.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this,i),s=o-n,r=i-n+(this.props.touch.start-(i-n))*e,a=i+s+(this.props.touch.end-(i+s))*e; +this.startToFront=0>=1-e,this.endToFront=0>=e-1;var h=d.snapAwayFromHidden(this.body.hiddenDates,r,1-e,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,e-1,!0);h==r&&l==a||(this.props.touch.start=h,this.props.touch.end=l,this.scaleOffset=1-t.scale,r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0}},o.prototype._isInsideRange=function(t){var e=t.center?t.center.x:t.clientX;if(this.options.rtl)var i=e-r.getAbsoluteLeft(this.body.dom.centerContainer);else var i=r.getAbsoluteRight(this.body.dom.centerContainer)-e;var o=this.body.util.toTime(i);return o>=this.start&&o<=this.end},o.prototype._pointerToDate=function(t){var e,i=this.options.direction;if(n(i),"horizontal"==i)return this.body.util.toTime(t.x).valueOf();var o=this.body.domProps.center.height;return e=this.conversion(o),t.y/e.scale+e.offset},o.prototype.getPointer=function(t,e){return this.options.rtl?{x:r.getAbsoluteRight(e)-t.x,y:t.y-r.getAbsoluteTop(e)}:{x:t.x-r.getAbsoluteLeft(e),y:t.y-r.getAbsoluteTop(e)}},o.prototype.zoom=function(t,e,i){null==e&&(e=(this.start+this.end)/2);var o=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),n=d.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this,e),s=o-n,r=e-n+(this.start-(e-n))*t,a=e+s+(this.end-(e+s))*t;this.startToFront=!(i>0),this.endToFront=!(-i>0);var h=d.snapAwayFromHidden(this.body.hiddenDates,r,i,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,-i,!0);h==r&&l==a||(r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0},o.prototype.move=function(t){var e=this.end-this.start,i=this.start+e*t,o=this.end+e*t;this.start=i,this.end=o},o.prototype.moveTo=function(t){var e=(this.start+this.end)/2,i=e-t,o=this.start-i,n=this.end-i;this.setRange(o,n)},t.exports=o},function(t,e){function i(t,e){this.options=null,this.props=null}i.prototype.setOptions=function(t){t&&util.extend(this.options,t)},i.prototype.redraw=function(){return!1},i.prototype.destroy=function(){},i.prototype._isResized=function(){var t=this.props._previousWidth!==this.props.width||this.props._previousHeight!==this.props.height;return this.props._previousWidth=this.props.width,this.props._previousHeight=this.props.height,t},t.exports=i},function(t,e){e.convertHiddenOptions=function(t,i,o){if(o&&!Array.isArray(o))return e.convertHiddenOptions(t,i,[o]);if(i.hiddenDates=[],o&&1==Array.isArray(o)){for(var n=0;n=4*a){var u=0,p=s.clone();switch(o[h].repeat){case"daily":d.day()!=l.day()&&(u=1),d.dayOfYear(n.dayOfYear()),d.year(n.year()),d.subtract(7,"days"),l.dayOfYear(n.dayOfYear()),l.year(n.year()),l.subtract(7-u,"days"),p.add(1,"weeks");break;case"weekly":var f=l.diff(d,"days"),m=d.day();d.date(n.date()),d.month(n.month()),d.year(n.year()),l=d.clone(),d.day(m),l.day(m),l.add(f,"days"),d.subtract(1,"weeks"),l.subtract(1,"weeks"),p.add(1,"weeks");break;case"monthly":d.month()!=l.month()&&(u=1),d.month(n.month()),d.year(n.year()),d.subtract(1,"months"),l.month(n.month()),l.year(n.year()),l.subtract(1,"months"),l.add(u,"months"),p.add(1,"months");break;case"yearly":d.year()!=l.year()&&(u=1),d.year(n.year()),d.subtract(1,"years"),l.year(n.year()),l.subtract(1,"years"),l.add(u,"years"),p.add(1,"years");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",o[h].repeat)}for(;p>d;)switch(i.hiddenDates.push({start:d.valueOf(),end:l.valueOf()}),o[h].repeat){case"daily":d.add(1,"days"),l.add(1,"days");break;case"weekly":d.add(1,"weeks"),l.add(1,"weeks");break;case"monthly":d.add(1,"months"),l.add(1,"months");break;case"yearly":d.add(1,"y"),l.add(1,"y");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",o[h].repeat)}i.hiddenDates.push({start:d.valueOf(),end:l.valueOf()})}}e.removeDuplicates(i);var v=e.isHidden(i.range.start,i.hiddenDates),g=e.isHidden(i.range.end,i.hiddenDates),y=i.range.start,b=i.range.end;1==v.hidden&&(y=1==i.range.startToFront?v.startDate-1:v.endDate+1),1==g.hidden&&(b=1==i.range.endToFront?g.startDate-1:g.endDate+1),1!=v.hidden&&1!=g.hidden||i.range._applyRange(y,b)}},e.removeDuplicates=function(t){for(var e=t.hiddenDates,i=[],o=0;o=e[o].start&&e[n].end<=e[o].end?e[n].remove=!0:e[n].start>=e[o].start&&e[n].start<=e[o].end?(e[o].end=e[n].end,e[n].remove=!0):e[n].end>=e[o].start&&e[n].end<=e[o].end&&(e[o].start=e[n].start,e[n].remove=!0));for(var o=0;o=r&&a>n){o=!0;break}}if(1==o&&n=e&&i>r&&(o+=r-s)}return o},e.correctTimeForHidden=function(t,i,o,n){return n=t(n).toDate().valueOf(),n-=e.getHiddenDurationBefore(t,i,o,n)},e.getHiddenDurationBefore=function(t,e,i,o){var n=0;o=t(o).toDate().valueOf();for(var s=0;s=i.start&&a=a&&(n+=a-r)}return n},e.getAccumulatedHiddenDuration=function(t,e,i){for(var o=0,n=0,s=e.start,r=0;r=e.start&&h=i)break;o+=h-a}}return o},e.snapAwayFromHidden=function(t,i,o,n){var s=e.isHidden(i,t);return 1==s.hidden?0>o?1==n?s.startDate-(s.endDate-i)-1:s.startDate-1:1==n?s.endDate+(i-s.startDate)+1:s.endDate+1:i},e.isHidden=function(t,e){for(var i=0;i=o&&n>t)return{hidden:!0,startDate:o,endDate:n}}return{hidden:!1,startDate:o,endDate:n}}},function(t,e,i){function o(){}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(13),r=i(20),a=i(28),h=i(1),d=(i(9),i(11),i(30),i(34),i(44)),l=i(45),c=i(32),u=i(46);s(o.prototype),o.prototype._create=function(t){function e(t){i.isActive()&&i.emit("mousewheel",t)}this.dom={},this.dom.container=t,this.dom.root=document.createElement("div"),this.dom.background=document.createElement("div"),this.dom.backgroundVertical=document.createElement("div"),this.dom.backgroundHorizontal=document.createElement("div"),this.dom.centerContainer=document.createElement("div"),this.dom.leftContainer=document.createElement("div"),this.dom.rightContainer=document.createElement("div"),this.dom.center=document.createElement("div"),this.dom.left=document.createElement("div"),this.dom.right=document.createElement("div"),this.dom.top=document.createElement("div"),this.dom.bottom=document.createElement("div"),this.dom.shadowTop=document.createElement("div"),this.dom.shadowBottom=document.createElement("div"),this.dom.shadowTopLeft=document.createElement("div"),this.dom.shadowBottomLeft=document.createElement("div"),this.dom.shadowTopRight=document.createElement("div"),this.dom.shadowBottomRight=document.createElement("div"),this.dom.root.className="vis-timeline",this.dom.background.className="vis-panel vis-background",this.dom.backgroundVertical.className="vis-panel vis-background vis-vertical",this.dom.backgroundHorizontal.className="vis-panel vis-background vis-horizontal",this.dom.centerContainer.className="vis-panel vis-center",this.dom.leftContainer.className="vis-panel vis-left",this.dom.rightContainer.className="vis-panel vis-right",this.dom.top.className="vis-panel vis-top",this.dom.bottom.className="vis-panel vis-bottom",this.dom.left.className="vis-content",this.dom.center.className="vis-content",this.dom.right.className="vis-content",this.dom.shadowTop.className="vis-shadow vis-top",this.dom.shadowBottom.className="vis-shadow vis-bottom",this.dom.shadowTopLeft.className="vis-shadow vis-top",this.dom.shadowBottomLeft.className="vis-shadow vis-bottom",this.dom.shadowTopRight.className="vis-shadow vis-top",this.dom.shadowBottomRight.className="vis-shadow vis-bottom",this.dom.root.appendChild(this.dom.background),this.dom.root.appendChild(this.dom.backgroundVertical),this.dom.root.appendChild(this.dom.backgroundHorizontal),this.dom.root.appendChild(this.dom.centerContainer),this.dom.root.appendChild(this.dom.leftContainer),this.dom.root.appendChild(this.dom.rightContainer),this.dom.root.appendChild(this.dom.top),this.dom.root.appendChild(this.dom.bottom),this.dom.centerContainer.appendChild(this.dom.center),this.dom.leftContainer.appendChild(this.dom.left),this.dom.rightContainer.appendChild(this.dom.right),this.dom.centerContainer.appendChild(this.dom.shadowTop),this.dom.centerContainer.appendChild(this.dom.shadowBottom),this.dom.leftContainer.appendChild(this.dom.shadowTopLeft),this.dom.leftContainer.appendChild(this.dom.shadowBottomLeft),this.dom.rightContainer.appendChild(this.dom.shadowTopRight),this.dom.rightContainer.appendChild(this.dom.shadowBottomRight),this.on("rangechange",function(){this.initialDrawDone===!0&&this._redraw()}.bind(this)),this.on("touch",this._onTouch.bind(this)),this.on("pan",this._onDrag.bind(this));var i=this;this.on("_change",function(t){t&&1==t.queue?i._redrawTimer||(i._redrawTimer=setTimeout(function(){i._redrawTimer=null,i._redraw()},0)):i._redraw()}),this.hammer=new r(this.dom.root);var o=this.hammer.get("pinch").set({enable:!0});a.disablePreventDefaultVertically(o),this.hammer.get("pan").set({threshold:5,direction:r.DIRECTION_HORIZONTAL}),this.listeners={};var n=["tap","doubletap","press","pinch","pan","panstart","panmove","panend"];if(n.forEach(function(t){var e=function(e){i.isActive()&&i.emit(t,e)};i.hammer.on(t,e),i.listeners[t]=e}),a.onTouch(this.hammer,function(t){i.emit("touch",t)}.bind(this)),a.onRelease(this.hammer,function(t){i.emit("release",t)}.bind(this)),this.dom.root.addEventListener("mousewheel",e),this.dom.root.addEventListener("DOMMouseScroll",e),this.props={root:{},background:{},centerContainer:{},leftContainer:{},rightContainer:{},center:{},left:{},right:{},top:{},bottom:{},border:{},scrollTop:0,scrollTopMin:0},this.customTimes=[],this.touch={},this.redrawCount=0,this.initialDrawDone=!1,!t)throw new Error("No container provided");t.appendChild(this.dom.root)},o.prototype.setOptions=function(t){if(t){var e=["width","height","minHeight","maxHeight","autoResize","start","end","clickToUse","dataAttributes","hiddenDates","locale","locales","moment","rtl","throttleRedraw"];if(h.selectiveExtend(e,this.options,t),this.options.rtl){var i=this.dom.leftContainer;this.dom.leftContainer=this.dom.rightContainer,this.dom.rightContainer=i,this.dom.container.style.direction="rtl",this.dom.backgroundVertical.className="vis-panel vis-background vis-vertical-rtl"}if(this.options.orientation={item:void 0,axis:void 0},"orientation"in t&&("string"==typeof t.orientation?this.options.orientation={item:t.orientation,axis:t.orientation}:"object"===n(t.orientation)&&("item"in t.orientation&&(this.options.orientation.item=t.orientation.item),"axis"in t.orientation&&(this.options.orientation.axis=t.orientation.axis))),"both"===this.options.orientation.axis){if(!this.timeAxis2){var o=this.timeAxis2=new d(this.body);o.setOptions=function(t){var e=t?h.extend({},t):{};e.orientation="top",d.prototype.setOptions.call(o,e)},this.components.push(o)}}else if(this.timeAxis2){var s=this.components.indexOf(this.timeAxis2);-1!==s&&this.components.splice(s,1),this.timeAxis2.destroy(),this.timeAxis2=null}if("function"==typeof t.drawPoints&&(t.drawPoints={onRender:t.drawPoints}),"hiddenDates"in this.options&&c.convertHiddenOptions(this.options.moment,this.body,this.options.hiddenDates),"clickToUse"in t&&(t.clickToUse?this.activator||(this.activator=new l(this.dom.root)):this.activator&&(this.activator.destroy(),delete this.activator)),"showCustomTime"in t)throw new Error("Option `showCustomTime` is deprecated. Create a custom time bar via timeline.addCustomTime(time [, id])");this._initAutoResize()}if(this.components.forEach(function(e){return e.setOptions(t)}),"configure"in t){this.configurator||(this.configurator=this._createConfigurator()),this.configurator.setOptions(t.configure);var r=h.deepExtend({},this.options);this.components.forEach(function(t){h.deepExtend(r,t.options)}),this.configurator.setModuleOptions({global:r})}this._origRedraw?this._redraw():(this._origRedraw=this._redraw.bind(this),this._redraw=h.throttle(this._origRedraw,this.options.throttleRedraw))},o.prototype.isActive=function(){return!this.activator||this.activator.active},o.prototype.destroy=function(){this.setItems(null),this.setGroups(null),this.off(),this._stopAutoResize(),this.dom.root.parentNode&&this.dom.root.parentNode.removeChild(this.dom.root),this.dom=null,this.activator&&(this.activator.destroy(),delete this.activator);for(var t in this.listeners)this.listeners.hasOwnProperty(t)&&delete this.listeners[t];this.listeners=null,this.hammer=null,this.components.forEach(function(t){return t.destroy()}),this.body=null},o.prototype.setCustomTime=function(t,e){var i=this.customTimes.filter(function(t){return e===t.options.id});if(0===i.length)throw new Error("No custom time bar found with id "+JSON.stringify(e));i.length>0&&i[0].setCustomTime(t)},o.prototype.getCustomTime=function(t){var e=this.customTimes.filter(function(e){return e.options.id===t});if(0===e.length)throw new Error("No custom time bar found with id "+JSON.stringify(t));return e[0].getCustomTime()},o.prototype.setCustomTimeTitle=function(t,e){var i=this.customTimes.filter(function(t){return t.options.id===e});if(0===i.length)throw new Error("No custom time bar found with id "+JSON.stringify(e));return i.length>0?i[0].setCustomTitle(t):void 0},o.prototype.getEventProperties=function(t){return{event:t}},o.prototype.addCustomTime=function(t,e){var i=void 0!==t?h.convert(t,"Date").valueOf():new Date,o=this.customTimes.some(function(t){return t.options.id===e});if(o)throw new Error("A custom time with id "+JSON.stringify(e)+" already exists");var n=new u(this.body,h.extend({},this.options,{time:i,id:e}));return this.customTimes.push(n),this.components.push(n),this._redraw(),e},o.prototype.removeCustomTime=function(t){var e=this.customTimes.filter(function(e){return e.options.id===t});if(0===e.length)throw new Error("No custom time bar found with id "+JSON.stringify(t));e.forEach(function(t){this.customTimes.splice(this.customTimes.indexOf(t),1),this.components.splice(this.components.indexOf(t),1),t.destroy()}.bind(this))},o.prototype.getVisibleItems=function(){return this.itemSet&&this.itemSet.getVisibleItems()||[]},o.prototype.fit=function(t){var e=this.getDataRange();if(null!==e.min||null!==e.max){var i=e.max-e.min,o=new Date(e.min.valueOf()-.01*i),n=new Date(e.max.valueOf()+.01*i),s=t&&void 0!==t.animation?t.animation:!0;this.range.setRange(o,n,s)}},o.prototype.getDataRange=function(){throw new Error("Cannot invoke abstract method getDataRange")},o.prototype.setWindow=function(t,e,i){var o;if(1==arguments.length){var n=arguments[0];o=void 0!==n.animation?n.animation:!0,this.range.setRange(n.start,n.end,o)}else o=i&&void 0!==i.animation?i.animation:!0,this.range.setRange(t,e,o)},o.prototype.moveTo=function(t,e){var i=this.range.end-this.range.start,o=h.convert(t,"Date").valueOf(),n=o-i/2,s=o+i/2,r=e&&void 0!==e.animation?e.animation:!0;this.range.setRange(n,s,r)},o.prototype.getWindow=function(){var t=this.range.getRange();return{start:new Date(t.start),end:new Date(t.end)}},o.prototype.redraw=function(){this._redraw()},o.prototype._redraw=function(){this.redrawCount++;var t=!1,e=this.options,i=this.props,o=this.dom;if(o&&o.container&&0!=o.root.offsetWidth){c.updateHiddenDates(this.options.moment,this.body,this.options.hiddenDates),"top"==e.orientation?(h.addClassName(o.root,"vis-top"),h.removeClassName(o.root,"vis-bottom")):(h.removeClassName(o.root,"vis-top"),h.addClassName(o.root,"vis-bottom")),o.root.style.maxHeight=h.option.asSize(e.maxHeight,""),o.root.style.minHeight=h.option.asSize(e.minHeight,""),o.root.style.width=h.option.asSize(e.width,""),i.border.left=(o.centerContainer.offsetWidth-o.centerContainer.clientWidth)/2,i.border.right=i.border.left,i.border.top=(o.centerContainer.offsetHeight-o.centerContainer.clientHeight)/2,i.border.bottom=i.border.top;var n=o.root.offsetHeight-o.root.clientHeight,s=o.root.offsetWidth-o.root.clientWidth;0===o.centerContainer.clientHeight&&(i.border.left=i.border.top,i.border.right=i.border.left),0===o.root.clientHeight&&(s=n),i.center.height=o.center.offsetHeight,i.left.height=o.left.offsetHeight,i.right.height=o.right.offsetHeight,i.top.height=o.top.clientHeight||-i.border.top,i.bottom.height=o.bottom.clientHeight||-i.border.bottom;var a=Math.max(i.left.height,i.center.height,i.right.height),d=i.top.height+a+i.bottom.height+n+i.border.top+i.border.bottom;o.root.style.height=h.option.asSize(e.height,d+"px"),i.root.height=o.root.offsetHeight,i.background.height=i.root.height-n;var l=i.root.height-i.top.height-i.bottom.height-n;i.centerContainer.height=l,i.leftContainer.height=l,i.rightContainer.height=i.leftContainer.height,i.root.width=o.root.offsetWidth,i.background.width=i.root.width-s,i.left.width=o.leftContainer.clientWidth||-i.border.left,i.leftContainer.width=i.left.width,i.right.width=o.rightContainer.clientWidth||-i.border.right,i.rightContainer.width=i.right.width;var u=i.root.width-i.left.width-i.right.width-s;i.center.width=u,i.centerContainer.width=u,i.top.width=u,i.bottom.width=u,o.background.style.height=i.background.height+"px",o.backgroundVertical.style.height=i.background.height+"px",o.backgroundHorizontal.style.height=i.centerContainer.height+"px",o.centerContainer.style.height=i.centerContainer.height+"px",o.leftContainer.style.height=i.leftContainer.height+"px",o.rightContainer.style.height=i.rightContainer.height+"px",o.background.style.width=i.background.width+"px",o.backgroundVertical.style.width=i.centerContainer.width+"px",o.backgroundHorizontal.style.width=i.background.width+"px",o.centerContainer.style.width=i.center.width+"px",o.top.style.width=i.top.width+"px",o.bottom.style.width=i.bottom.width+"px",o.background.style.left="0",o.background.style.top="0",o.backgroundVertical.style.left=i.left.width+i.border.left+"px",o.backgroundVertical.style.top="0",o.backgroundHorizontal.style.left="0",o.backgroundHorizontal.style.top=i.top.height+"px",o.centerContainer.style.left=i.left.width+"px",o.centerContainer.style.top=i.top.height+"px",o.leftContainer.style.left="0",o.leftContainer.style.top=i.top.height+"px",o.rightContainer.style.left=i.left.width+i.center.width+"px",o.rightContainer.style.top=i.top.height+"px",o.top.style.left=i.left.width+"px",o.top.style.top="0",o.bottom.style.left=i.left.width+"px",o.bottom.style.top=i.top.height+i.centerContainer.height+"px",this._updateScrollTop();var p=this.props.scrollTop;"top"!=e.orientation.item&&(p+=Math.max(this.props.centerContainer.height-this.props.center.height-this.props.border.top-this.props.border.bottom,0)),o.center.style.left="0",o.center.style.top=p+"px",o.left.style.left="0",o.left.style.top=p+"px",o.right.style.left="0",o.right.style.top=p+"px";var f=0==this.props.scrollTop?"hidden":"",m=this.props.scrollTop==this.props.scrollTopMin?"hidden":"";o.shadowTop.style.visibility=f,o.shadowBottom.style.visibility=m,o.shadowTopLeft.style.visibility=f,o.shadowBottomLeft.style.visibility=m,o.shadowTopRight.style.visibility=f,o.shadowBottomRight.style.visibility=m;var v=this.props.center.height>this.props.centerContainer.height;this.hammer.get("pan").set({direction:v?r.DIRECTION_ALL:r.DIRECTION_HORIZONTAL}),this.components.forEach(function(e){t=e.redraw()||t});var g=5;if(t){if(this.redrawCount0&&(this.props.scrollTop=0),this.props.scrollTope;e++)o=this.selection[e],n=this.items[o],n&&n.unselect();for(this.selection=[],e=0,i=t.length;i>e;e++)o=t[e],n=this.items[o],n&&(this.selection.push(o),n.select())},o.prototype.getSelection=function(){return this.selection.concat([])},o.prototype.getVisibleItems=function(){var t=this.body.range.getRange();if(this.options.rtl)var e=this.body.util.toScreen(t.start),i=this.body.util.toScreen(t.end);else var i=this.body.util.toScreen(t.start),e=this.body.util.toScreen(t.end);var o=[];for(var n in this.groups)if(this.groups.hasOwnProperty(n))for(var s=this.groups[n],r=s.visibleItems,a=0;ae&&o.push(h.id):h.lefti&&o.push(h.id)}return o},o.prototype._deselect=function(t){for(var e=this.selection,i=0,o=e.length;o>i;i++)if(e[i]==t){e.splice(i,1);break}},o.prototype.redraw=function(){var t=this.options.margin,e=this.body.range,i=r.option.asSize,o=this.options,n=o.orientation.item,s=!1,a=this.dom.frame;this.props.top=this.body.domProps.top.height+this.body.domProps.border.top,this.options.rtl?this.props.right=this.body.domProps.right.width+this.body.domProps.border.right:this.props.left=this.body.domProps.left.width+this.body.domProps.border.left,a.className="vis-itemset",s=this._orderGroups()||s;var h=e.end-e.start,d=h!=this.lastVisibleInterval||this.props.width!=this.props.lastWidth;d&&(this.stackDirty=!0), +this.lastVisibleInterval=h,this.props.lastWidth=this.props.width;var l=this.stackDirty,c=this._firstGroup(),u={item:t.item,axis:t.axis},p={item:t.item,axis:t.item.vertical/2},f=0,m=t.axis+t.item.vertical;return this.groups[y].redraw(e,p,l),r.forEach(this.groups,function(t){var i=t==c?u:p,o=t.redraw(e,i,l);s=o||s,f+=t.height}),f=Math.max(f,m),this.stackDirty=!1,a.style.height=i(f),this.props.width=a.offsetWidth,this.props.height=f,this.dom.axis.style.top=i("top"==n?this.body.domProps.top.height+this.body.domProps.border.top:this.body.domProps.top.height+this.body.domProps.centerContainer.height),this.options.rtl?this.dom.axis.style.right="0":this.dom.axis.style.left="0",s=this._isResized()||s},o.prototype._firstGroup=function(){var t="top"==this.options.orientation.item?0:this.groupIds.length-1,e=this.groupIds[t],i=this.groups[e]||this.groups[g];return i||null},o.prototype._updateUngrouped=function(){var t,e,i=this.groups[g];this.groups[y];if(this.groupsData){if(i){i.hide(),delete this.groups[g];for(e in this.items)if(this.items.hasOwnProperty(e)){t=this.items[e],t.parent&&t.parent.remove(t);var o=this._getGroupId(t.data),n=this.groups[o];n&&n.add(t)||t.hide()}}}else if(!i){var s=null,r=null;i=new c(s,r,this),this.groups[g]=i;for(e in this.items)this.items.hasOwnProperty(e)&&(t=this.items[e],i.add(t));i.show()}},o.prototype.getLabelSet=function(){return this.dom.labelSet},o.prototype.setItems=function(t){var e,i=this,o=this.itemsData;if(t){if(!(t instanceof a||t instanceof h))throw new TypeError("Data must be an instance of DataSet or DataView");this.itemsData=t}else this.itemsData=null;if(o&&(r.forEach(this.itemListeners,function(t,e){o.off(e,t)}),e=o.getIds(),this._onRemove(e)),this.itemsData){var n=this.id;r.forEach(this.itemListeners,function(t,e){i.itemsData.on(e,t,n)}),e=this.itemsData.getIds(),this._onAdd(e),this._updateUngrouped()}this.body.emitter.emit("_change",{queue:!0})},o.prototype.getItems=function(){return this.itemsData},o.prototype.setGroups=function(t){var e,i=this;if(this.groupsData&&(r.forEach(this.groupListeners,function(t,e){i.groupsData.off(e,t)}),e=this.groupsData.getIds(),this.groupsData=null,this._onRemoveGroups(e)),t){if(!(t instanceof a||t instanceof h))throw new TypeError("Data must be an instance of DataSet or DataView");this.groupsData=t}else this.groupsData=null;if(this.groupsData){var o=this.id;r.forEach(this.groupListeners,function(t,e){i.groupsData.on(e,t,o)}),e=this.groupsData.getIds(),this._onAddGroups(e)}this._updateUngrouped(),this._order(),this.body.emitter.emit("_change",{queue:!0})},o.prototype.getGroups=function(){return this.groupsData},o.prototype.removeItem=function(t){var e=this.itemsData.get(t),i=this.itemsData.getDataSet();e&&this.options.onRemove(e,function(e){e&&i.remove(t)})},o.prototype._getType=function(t){return t.type||this.options.type||(t.end?"range":"box")},o.prototype._getGroupId=function(t){var e=this._getType(t);return"background"==e&&void 0==t.group?y:this.groupsData?t.group:g},o.prototype._onUpdate=function(t){var e=this;t.forEach(function(t){var i,n=e.itemsData.get(t,e.itemOptions),s=e.items[t],r=e._getType(n),a=o.types[r];if(s&&(a&&s instanceof a?e._updateItem(s,n):(i=s.selected,e._removeItem(s),s=null)),!s){if(!a)throw"rangeoverflow"==r?new TypeError('Item type "rangeoverflow" is deprecated. Use css styling instead: .vis-item.vis-range .vis-item-content {overflow: visible;}'):new TypeError('Unknown item type "'+r+'"');s=new a(n,e.conversion,e.options),s.id=t,e._addItem(s),i&&(this.selection.push(t),s.select())}}.bind(this)),this._order(),this.stackDirty=!0,this.body.emitter.emit("_change",{queue:!0})},o.prototype._onAdd=o.prototype._onUpdate,o.prototype._onRemove=function(t){var e=0,i=this;t.forEach(function(t){var o=i.items[t];o&&(e++,i._removeItem(o))}),e&&(this._order(),this.stackDirty=!0,this.body.emitter.emit("_change",{queue:!0}))},o.prototype._order=function(){r.forEach(this.groups,function(t){t.order()})},o.prototype._onUpdateGroups=function(t){this._onAddGroups(t)},o.prototype._onAddGroups=function(t){var e=this;t.forEach(function(t){var i=e.groupsData.get(t),o=e.groups[t];if(o)o.setData(i);else{if(t==g||t==y)throw new Error("Illegal group id. "+t+" is a reserved id.");var n=Object.create(e.options);r.extend(n,{height:null}),o=new c(t,i,e),e.groups[t]=o;for(var s in e.items)if(e.items.hasOwnProperty(s)){var a=e.items[s];a.data.group==t&&o.add(a)}o.order(),o.show()}}),this.body.emitter.emit("_change",{queue:!0})},o.prototype._onRemoveGroups=function(t){var e=this.groups;t.forEach(function(t){var i=e[t];i&&(i.hide(),delete e[t])}),this.markDirty(),this.body.emitter.emit("_change",{queue:!0})},o.prototype._orderGroups=function(){if(this.groupsData){var t=this.groupsData.getIds({order:this.options.groupOrder}),e=!r.equalArray(t,this.groupIds);if(e){var i=this.groups;t.forEach(function(t){i[t].hide()}),t.forEach(function(t){i[t].show()}),this.groupIds=t}return e}return!1},o.prototype._addItem=function(t){this.items[t.id]=t;var e=this._getGroupId(t.data),i=this.groups[e];i&&i.add(t)},o.prototype._updateItem=function(t,e){var i=t.data.group,o=t.data.subgroup;if(t.setData(e),i!=t.data.group||o!=t.data.subgroup){var n=this.groups[i];n&&n.remove(t);var s=this._getGroupId(t.data),r=this.groups[s];r&&r.add(t)}},o.prototype._removeItem=function(t){t.hide(),delete this.items[t.id];var e=this.selection.indexOf(t.id);-1!=e&&this.selection.splice(e,1),t.parent&&t.parent.remove(t)},o.prototype._constructByEndArray=function(t){for(var e=[],i=0;in+s)return}else{var a=e.height;if(n+a-s>o)return}}if(e&&e!=this.groupTouchParams.group){var h=this.groupsData,d=h.get(e.groupId),l=h.get(this.groupTouchParams.group.groupId);l&&d&&(this.options.groupOrderSwap(l,d,this.groupsData),this.groupsData.update(l),this.groupsData.update(d));var c=this.groupsData.getIds({order:this.options.groupOrder});if(!r.equalArray(c,this.groupTouchParams.originalOrder))for(var h=this.groupsData,u=this.groupTouchParams.originalOrder,p=this.groupTouchParams.group.groupId,f=Math.min(u.length,c.length),m=0,v=0,g=0;f>m;){for(;f>m+v&&f>m+g&&c[m+v]==u[m+g];)m++;if(m+v>=f)break;if(c[m+v]!=p)if(u[m+g]!=p){var y=c.indexOf(u[m+g]),b=h.get(c[m+v]),w=h.get(u[m+g]);this.options.groupOrderSwap(b,w,h),h.update(b),h.update(w);var _=c[m+v];c[m+v]=u[m+g],c[y]=_,m++}else g=1;else v=1}}}},o.prototype._onGroupDragEnd=function(t){if(this.options.groupEditable.order&&this.groupTouchParams.group){t.stopPropagation();var e=this,i=e.groupTouchParams.group.groupId,o=e.groupsData.getDataSet(),n=r.extend({},o.get(i));e.options.onMoveGroup(n,function(t){if(t)t[o._fieldId]=i,o.update(t);else{var n=o.getIds({order:e.options.groupOrder});if(!r.equalArray(n,e.groupTouchParams.originalOrder))for(var s=e.groupTouchParams.originalOrder,a=Math.min(s.length,n.length),h=0;a>h;){for(;a>h&&n[h]==s[h];)h++;if(h>=a)break;var d=n.indexOf(s[h]),l=o.get(n[h]),c=o.get(s[h]);e.options.groupOrderSwap(l,c,o),groupsData.update(l),groupsData.update(c);var u=n[h];n[h]=s[h],n[d]=u,h++}}}),e.body.emitter.emit("groupDragged",{groupId:i})}},o.prototype._onSelectItem=function(t){if(this.options.selectable){var e=t.srcEvent&&(t.srcEvent.ctrlKey||t.srcEvent.metaKey),i=t.srcEvent&&t.srcEvent.shiftKey;if(e||i)return void this._onMultiSelectItem(t);var o=this.getSelection(),n=this.itemFromTarget(t),s=n?[n.id]:[];this.setSelection(s);var r=this.getSelection();(r.length>0||o.length>0)&&this.body.emitter.emit("select",{items:r,event:t})}},o.prototype._onAddItem=function(t){if(this.options.selectable&&this.options.editable.add){var e=this,i=this.options.snap||null,o=this.itemFromTarget(t);if(o){var n=e.itemsData.get(o.id);this.options.onUpdate(n,function(t){t&&e.itemsData.getDataSet().update(t)})}else{if(this.options.rtl)var s=r.getAbsoluteRight(this.dom.frame),a=s-t.center.x;else var s=r.getAbsoluteLeft(this.dom.frame),a=t.center.x-s;var h=this.body.util.toTime(a),d=this.body.util.getScale(),l=this.body.util.getStep(),c={start:i?i(h,d,l):h,content:"new item"};if("range"===this.options.type){var u=this.body.util.toTime(a+this.props.width/5);c.end=i?i(u,d,l):u}c[this.itemsData._fieldId]=r.randomUUID();var p=this.groupFromTarget(t);p&&(c.group=p.groupId),c=this._cloneItemData(c),this.options.onAdd(c,function(t){t&&e.itemsData.getDataSet().add(t)})}}},o.prototype._onMultiSelectItem=function(t){if(this.options.selectable){var e=this.itemFromTarget(t);if(e){var i=this.options.multiselect?this.getSelection():[],n=t.srcEvent&&t.srcEvent.shiftKey||!1;if(n&&this.options.multiselect){var s=this.itemsData.get(e.id).group,r=void 0;this.options.multiselectPerGroup&&i.length>0&&(r=this.itemsData.get(i[0]).group),this.options.multiselectPerGroup&&void 0!=r&&r!=s||i.push(e.id);var a=o._getItemRange(this.itemsData.get(i,this.itemOptions));if(!this.options.multiselectPerGroup||r==s){i=[];for(var h in this.items)if(this.items.hasOwnProperty(h)){var d=this.items[h],l=d.data.start,c=void 0!==d.data.end?d.data.end:l;!(l>=a.min&&c<=a.max)||this.options.multiselectPerGroup&&r!=this.itemsData.get(d.id).group||d instanceof v||i.push(d.id)}}}else{var u=i.indexOf(e.id);-1==u?i.push(e.id):i.splice(u,1)}this.setSelection(i),this.body.emitter.emit("select",{items:this.getSelection(),event:t})}}},o._getItemRange=function(t){var e=null,i=null;return t.forEach(function(t){(null==i||t.starte)&&(e=t.end):(null==e||t.start>e)&&(e=t.start)}),{min:i,max:e}},o.prototype.itemFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-item"))return e["timeline-item"];e=e.parentNode}return null},o.prototype.groupFromTarget=function(t){for(var e=t.center?t.center.y:t.clientY,i=0;ia&&ea)return n}else if(0===i&&e0?t.step:1,this.autoScale=!1)},o.prototype.setAutoScale=function(t){this.autoScale=t},o.prototype.setMinimumStep=function(t){if(void 0!=t){var e=31104e6,i=2592e6,o=864e5,n=36e5,s=6e4,r=1e3,a=1;1e3*e>t&&(this.scale="year",this.step=1e3),500*e>t&&(this.scale="year",this.step=500),100*e>t&&(this.scale="year",this.step=100),50*e>t&&(this.scale="year",this.step=50),10*e>t&&(this.scale="year",this.step=10),5*e>t&&(this.scale="year",this.step=5),e>t&&(this.scale="year",this.step=1),3*i>t&&(this.scale="month",this.step=3),i>t&&(this.scale="month",this.step=1),5*o>t&&(this.scale="day",this.step=5),2*o>t&&(this.scale="day",this.step=2),o>t&&(this.scale="day",this.step=1),o/2>t&&(this.scale="weekday",this.step=1),4*n>t&&(this.scale="hour",this.step=4),n>t&&(this.scale="hour",this.step=1),15*s>t&&(this.scale="minute",this.step=15),10*s>t&&(this.scale="minute",this.step=10),5*s>t&&(this.scale="minute",this.step=5),s>t&&(this.scale="minute",this.step=1),15*r>t&&(this.scale="second",this.step=15),10*r>t&&(this.scale="second",this.step=10),5*r>t&&(this.scale="second",this.step=5),r>t&&(this.scale="second",this.step=1),200*a>t&&(this.scale="millisecond",this.step=200),100*a>t&&(this.scale="millisecond",this.step=100),50*a>t&&(this.scale="millisecond",this.step=50),10*a>t&&(this.scale="millisecond",this.step=10),5*a>t&&(this.scale="millisecond",this.step=5),a>t&&(this.scale="millisecond",this.step=1)}},o.snap=function(t,e,i){var o=n(t);if("year"==e){var s=o.year()+Math.round(o.month()/12);o.year(Math.round(s/i)*i),o.month(0),o.date(0),o.hours(0),o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("month"==e)o.date()>15?(o.date(1),o.add(1,"month")):o.date(1),o.hours(0),o.minutes(0),o.seconds(0),o.milliseconds(0);else if("day"==e){switch(i){case 5:case 2:o.hours(24*Math.round(o.hours()/24));break;default:o.hours(12*Math.round(o.hours()/12))}o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("weekday"==e){switch(i){case 5:case 2:o.hours(12*Math.round(o.hours()/12));break;default:o.hours(6*Math.round(o.hours()/6))}o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("hour"==e){switch(i){case 4:o.minutes(60*Math.round(o.minutes()/60));break;default:o.minutes(30*Math.round(o.minutes()/30))}o.seconds(0),o.milliseconds(0)}else if("minute"==e){switch(i){case 15:case 10:o.minutes(5*Math.round(o.minutes()/5)),o.seconds(0);break;case 5:o.seconds(60*Math.round(o.seconds()/60));break;default:o.seconds(30*Math.round(o.seconds()/30))}o.milliseconds(0)}else if("second"==e)switch(i){case 15:case 10:o.seconds(5*Math.round(o.seconds()/5)),o.milliseconds(0);break;case 5:o.milliseconds(1e3*Math.round(o.milliseconds()/1e3));break;default:o.milliseconds(500*Math.round(o.milliseconds()/500))}else if("millisecond"==e){var r=i>5?i/2:1;o.milliseconds(Math.round(o.milliseconds()/r)*r)}return o},o.prototype.isMajor=function(){if(1==this.switchedYear)switch(this.switchedYear=!1,this.scale){case"year":case"month":case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedMonth)switch(this.switchedMonth=!1,this.scale){case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedDay)switch(this.switchedDay=!1,this.scale){case"millisecond":case"second":case"minute":case"hour":return!0;default:return!1}var t=this.moment(this.current);switch(this.scale){case"millisecond":return 0==t.milliseconds();case"second":return 0==t.seconds();case"minute":return 0==t.hours()&&0==t.minutes();case"hour":return 0==t.hours();case"weekday":case"day":return 1==t.date();case"month":return 0==t.month();case"year":return!1;default:return!1}},o.prototype.getLabelMinor=function(t){void 0==t&&(t=this.current);var e=this.format.minorLabels[this.scale];return e&&e.length>0?this.moment(t).format(e):""},o.prototype.getLabelMajor=function(t){void 0==t&&(t=this.current);var e=this.format.majorLabels[this.scale];return e&&e.length>0?this.moment(t).format(e):""},o.prototype.getClassName=function(){function t(t){return t/h%2==0?" vis-even":" vis-odd"}function e(t){return t.isSame(new Date,"day")?" vis-today":t.isSame(s().add(1,"day"),"day")?" vis-tomorrow":t.isSame(s().add(-1,"day"),"day")?" vis-yesterday":""}function i(t){return t.isSame(new Date,"week")?" vis-current-week":""}function o(t){return t.isSame(new Date,"month")?" vis-current-month":""}function n(t){return t.isSame(new Date,"year")?" vis-current-year":""}var s=this.moment,r=this.moment(this.current),a=r.locale?r.locale("en"):r.lang("en"),h=this.step;switch(this.scale){case"millisecond":return t(a.milliseconds()).trim();case"second":return t(a.seconds()).trim();case"minute":return t(a.minutes()).trim();case"hour":var d=a.hours();return 4==this.step&&(d=d+"-h"+(d+4)),"vis-h"+d+e(a)+t(a.hours());case"weekday":return"vis-"+a.format("dddd").toLowerCase()+e(a)+i(a)+t(a.date());case"day":var l=a.date(),c=a.format("MMMM").toLowerCase();return"vis-day"+l+" vis-"+c+o(a)+t(l-1);case"month":return"vis-"+a.format("MMMM").toLowerCase()+o(a)+t(a.month());case"year":var u=a.year();return"vis-year"+u+n(a)+t(u);default:return""}},t.exports=o},function(t,e,i){function o(t,e,i){this.groupId=t,this.subgroups={},this.subgroupIndex=0,this.subgroupOrderer=e&&e.subgroupOrder,this.itemSet=i,this.dom={},this.props={label:{width:0,height:0}},this.className=null,this.items={},this.visibleItems=[],this.orderedItems={byStart:[],byEnd:[]},this.checkRangedItems=!1;var o=this;this.itemSet.body.emitter.on("checkRangedItems",function(){o.checkRangedItems=!0}),this._create(),this.setData(e)}var n=i(1),s=i(37);i(38);o.prototype._create=function(){var t=document.createElement("div");this.itemSet.options.groupEditable.order?t.className="vis-label draggable":t.className="vis-label",this.dom.label=t;var e=document.createElement("div");e.className="vis-inner",t.appendChild(e),this.dom.inner=e;var i=document.createElement("div");i.className="vis-group",i["timeline-group"]=this,this.dom.foreground=i,this.dom.background=document.createElement("div"),this.dom.background.className="vis-group",this.dom.axis=document.createElement("div"),this.dom.axis.className="vis-group",this.dom.marker=document.createElement("div"),this.dom.marker.style.visibility="hidden",this.dom.marker.innerHTML="?",this.dom.background.appendChild(this.dom.marker)},o.prototype.setData=function(t){var e;if(e=this.itemSet.options&&this.itemSet.options.groupTemplate?this.itemSet.options.groupTemplate(t):t&&t.content,e instanceof Element){for(this.dom.inner.appendChild(e);this.dom.inner.firstChild;)this.dom.inner.removeChild(this.dom.inner.firstChild);this.dom.inner.appendChild(e)}else void 0!==e&&null!==e?this.dom.inner.innerHTML=e:this.dom.inner.innerHTML=this.groupId||"";this.dom.label.title=t&&t.title||"",this.dom.inner.firstChild?n.removeClassName(this.dom.inner,"vis-hidden"):n.addClassName(this.dom.inner,"vis-hidden");var i=t&&t.className||null;i!=this.className&&(this.className&&(n.removeClassName(this.dom.label,this.className),n.removeClassName(this.dom.foreground,this.className),n.removeClassName(this.dom.background,this.className),n.removeClassName(this.dom.axis,this.className)),n.addClassName(this.dom.label,i),n.addClassName(this.dom.foreground,i),n.addClassName(this.dom.background,i),n.addClassName(this.dom.axis,i),this.className=i),this.style&&(n.removeCssText(this.dom.label,this.style),this.style=null),t&&t.style&&(n.addCssText(this.dom.label,t.style),this.style=t.style)},o.prototype.getLabelWidth=function(){return this.props.label.width},o.prototype.redraw=function(t,e,i){var o=!1,r=this.dom.marker.clientHeight;if(r!=this.lastMarkerHeight&&(this.lastMarkerHeight=r,n.forEach(this.items,function(t){t.dirty=!0,t.displayed&&t.redraw()}),i=!0),this._calculateSubGroupHeights(),"function"==typeof this.itemSet.options.order){if(i){var a=this,h=!1;n.forEach(this.items,function(t){t.displayed||(t.redraw(),a.visibleItems.push(t)),t.repositionX(h)});var d=this.orderedItems.byStart.slice().sort(function(t,e){return a.itemSet.options.order(t.data,e.data)});s.stack(d,e,!0)}this.visibleItems=this._updateVisibleItems(this.orderedItems,this.visibleItems,t)}else this.visibleItems=this._updateVisibleItems(this.orderedItems,this.visibleItems,t),this.itemSet.options.stack?s.stack(this.visibleItems,e,i):s.nostack(this.visibleItems,e,this.subgroups);var l=this._calculateHeight(e),c=this.dom.foreground;this.top=c.offsetTop,this.right=c.offsetLeft,this.width=c.offsetWidth,o=n.updateProperty(this,"height",l)||o,o=n.updateProperty(this.props.label,"width",this.dom.inner.clientWidth)||o,o=n.updateProperty(this.props.label,"height",this.dom.inner.clientHeight)||o,this.dom.background.style.height=l+"px",this.dom.foreground.style.height=l+"px",this.dom.label.style.height=l+"px";for(var u=0,p=this.visibleItems.length;p>u;u++){var f=this.visibleItems[u];f.repositionY(e)}return o},o.prototype._calculateSubGroupHeights=function(){if(Object.keys(this.subgroups).length>0){var t=this;this.resetSubgroups(),n.forEach(this.visibleItems,function(e){void 0!==e.data.subgroup&&(t.subgroups[e.data.subgroup].height=Math.max(t.subgroups[e.data.subgroup].height,e.height),t.subgroups[e.data.subgroup].visible=!0)})}},o.prototype._calculateHeight=function(t){var e,i=this.visibleItems;if(i.length>0){var o=i[0].top,s=i[0].top+i[0].height;if(n.forEach(i,function(t){o=Math.min(o,t.top),s=Math.max(s,t.top+t.height)}),o>t.axis){var r=o-t.axis;s-=r,n.forEach(i,function(t){t.top-=r})}e=s+t.item.vertical/2}else e=0;return e=Math.max(e,this.props.label.height)},o.prototype.show=function(){this.dom.label.parentNode||this.itemSet.dom.labelSet.appendChild(this.dom.label),this.dom.foreground.parentNode||this.itemSet.dom.foreground.appendChild(this.dom.foreground),this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background),this.dom.axis.parentNode||this.itemSet.dom.axis.appendChild(this.dom.axis)},o.prototype.hide=function(){var t=this.dom.label;t.parentNode&&t.parentNode.removeChild(t);var e=this.dom.foreground;e.parentNode&&e.parentNode.removeChild(e);var i=this.dom.background;i.parentNode&&i.parentNode.removeChild(i);var o=this.dom.axis;o.parentNode&&o.parentNode.removeChild(o)},o.prototype.add=function(t){if(this.items[t.id]=t,t.setParent(this),void 0!==t.data.subgroup&&(void 0===this.subgroups[t.data.subgroup]&&(this.subgroups[t.data.subgroup]={height:0,visible:!1,index:this.subgroupIndex,items:[]},this.subgroupIndex++),this.subgroups[t.data.subgroup].items.push(t)),this.orderSubgroups(),-1==this.visibleItems.indexOf(t)){var e=this.itemSet.body.range;this._checkIfVisible(t,this.visibleItems,e)}},o.prototype.orderSubgroups=function(){if(void 0!==this.subgroupOrderer){var t=[];if("string"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push({subgroup:e,sortField:this.subgroups[e].items[0].data[this.subgroupOrderer]});t.sort(function(t,e){return t.sortField-e.sortField})}else if("function"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push(this.subgroups[e].items[0].data);t.sort(this.subgroupOrderer)}if(t.length>0)for(var i=0;it?-1:l>=t?0:1};if(e.length>0)for(s=0;sl}),1==this.checkRangedItems)for(this.checkRangedItems=!1,s=0;sl})}for(s=0;s=0&&(s=e[r],!n(s));r--)void 0===o[s.id]&&(o[s.id]=!0,i.push(s));for(r=t+1;rn;n++)t[n].top=null;for(n=0,s=t.length;s>n;n++){var r=t[n];if(r.stack&&null===r.top){r.top=i.axis;do{for(var a=null,h=0,d=t.length;d>h;h++){var l=t[h];if(null!==l.top&&l!==r&&l.stack&&e.collision(r,l,i.item,l.options.rtl)){a=l;break}}null!=a&&(r.top=a.top+a.height+i.item.vertical)}while(a)}}},e.nostack=function(t,e,i){var o,n,s;for(o=0,n=t.length;n>o;o++)if(void 0!==t[o].data.subgroup){s=e.axis;for(var r in i)i.hasOwnProperty(r)&&1==i[r].visible&&i[r].indexe.right&&t.top-o.vertical+ie.top:t.left-o.horizontal+ie.left&&t.top-o.vertical+ie.top}},function(t,e,i){function o(t,e,i){if(this.props={content:{width:0}},this.overflow=!1,this.options=i,t){if(void 0==t.start)throw new Error('Property "start" missing in item '+t.id);if(void 0==t.end)throw new Error('Property "end" missing in item '+t.id)}n.call(this,t,e,i)}var n=(i(20),i(39));o.prototype=new n(null,null,null),o.prototype.baseClassName="vis-item vis-range",o.prototype.isVisible=function(t){return this.data.startt.start},o.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.frame=document.createElement("div"),t.frame.className="vis-item-overflow",t.box.appendChild(t.frame),t.content=document.createElement("div"),t.content.className="vis-item-content",t.frame.appendChild(t.content),t.box["timeline-item"]=this,this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.foreground;if(!e)throw new Error("Cannot redraw item: parent has no foreground container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.box),this._updateDataAttributes(this.dom.box),this._updateStyle(this.dom.box);var i=(this.options.editable.updateTime||this.options.editable.updateGroup||this.editable===!0)&&this.editable!==!1,o=(this.data.className?" "+this.data.className:"")+(this.selected?" vis-selected":"")+(i?" vis-editable":" vis-readonly");t.box.className=this.baseClassName+o,this.overflow="hidden"!==window.getComputedStyle(t.frame).overflow,this.dom.content.style.maxWidth="none",this.props.content.width=this.dom.content.offsetWidth,this.height=this.dom.box.offsetHeight,this.dom.content.style.maxWidth="",this.dirty=!1}this._repaintDeleteButton(t.box),this._repaintDragLeft(),this._repaintDragRight()},o.prototype.show=function(){this.displayed||this.redraw()},o.prototype.hide=function(){if(this.displayed){var t=this.dom.box;t.parentNode&&t.parentNode.removeChild(t),this.displayed=!1}},o.prototype.repositionX=function(t){var e,i,o=this.parent.width,n=this.conversion.toScreen(this.data.start),s=this.conversion.toScreen(this.data.end);void 0!==t&&t!==!0||(-o>n&&(n=-o),s>2*o&&(s=2*o));var r=Math.max(s-n,1);switch(this.overflow?(this.options.rtl?this.right=n:this.left=n,this.width=r+this.props.content.width,i=this.props.content.width):(this.options.rtl?this.right=n:this.left=n,this.width=r,i=Math.min(s-n,this.props.content.width)),this.options.rtl?this.dom.box.style.right=this.right+"px":this.dom.box.style.left=this.left+"px",this.dom.box.style.width=r+"px",this.options.align){case"left":this.options.rtl?this.dom.content.style.right="0":this.dom.content.style.left="0";break;case"right":this.options.rtl?this.dom.content.style.right=Math.max(r-i,0)+"px":this.dom.content.style.left=Math.max(r-i,0)+"px";break;case"center":this.options.rtl?this.dom.content.style.right=Math.max((r-i)/2,0)+"px":this.dom.content.style.left=Math.max((r-i)/2,0)+"px";break;default:e=this.overflow?s>0?Math.max(-n,0):-i:0>n?-n:0,this.options.rtl?this.dom.content.style.right=e+"px":this.dom.content.style.left=e+"px"}},o.prototype.repositionY=function(){var t=this.options.orientation.item,e=this.dom.box;"top"==t?e.style.top=this.top+"px":e.style.top=this.parent.height-this.top-this.height+"px"},o.prototype._repaintDragLeft=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragLeft){var t=document.createElement("div");t.className="vis-drag-left",t.dragLeftItem=this,this.dom.box.appendChild(t),this.dom.dragLeft=t}else!this.selected&&this.dom.dragLeft&&(this.dom.dragLeft.parentNode&&this.dom.dragLeft.parentNode.removeChild(this.dom.dragLeft),this.dom.dragLeft=null)},o.prototype._repaintDragRight=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragRight){var t=document.createElement("div");t.className="vis-drag-right",t.dragRightItem=this,this.dom.box.appendChild(t),this.dom.dragRight=t}else!this.selected&&this.dom.dragRight&&(this.dom.dragRight.parentNode&&this.dom.dragRight.parentNode.removeChild(this.dom.dragRight),this.dom.dragRight=null)},t.exports=o},function(t,e,i){function o(t,e,i){this.id=null,this.parent=null,this.data=t,this.dom=null,this.conversion=e||{},this.options=i||{},this.selected=!1,this.displayed=!1,this.dirty=!0,this.top=null,this.right=null,this.left=null,this.width=null,this.height=null,this.editable=null,this.data&&this.data.hasOwnProperty("editable")&&"boolean"==typeof this.data.editable&&(this.editable=t.editable)}var n=i(20),s=i(1);o.prototype.stack=!0,o.prototype.select=function(){this.selected=!0,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.unselect=function(){this.selected=!1,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.setData=function(t){var e=void 0!=t.group&&this.data.group!=t.group;e&&this.parent.itemSet._moveToGroup(this,t.group),t.hasOwnProperty("editable")&&"boolean"==typeof t.editable&&(this.editable=t.editable),this.data=t,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.setParent=function(t){this.displayed?(this.hide(),this.parent=t,this.parent&&this.show()):this.parent=t},o.prototype.isVisible=function(t){return!1},o.prototype.show=function(){return!1},o.prototype.hide=function(){return!1},o.prototype.redraw=function(){},o.prototype.repositionX=function(){},o.prototype.repositionY=function(){},o.prototype._repaintDeleteButton=function(t){var e=(this.options.editable.remove||this.data.editable===!0)&&this.data.editable!==!1;if(this.selected&&e&&!this.dom.deleteButton){var i=this,o=document.createElement("div");this.options.rtl?o.className="vis-delete-rtl":o.className="vis-delete",o.title="Delete this item",new n(o).on("tap",function(t){t.stopPropagation(),i.parent.removeFromDataSet(i)}),t.appendChild(o),this.dom.deleteButton=o}else!this.selected&&this.dom.deleteButton&&(this.dom.deleteButton.parentNode&&this.dom.deleteButton.parentNode.removeChild(this.dom.deleteButton),this.dom.deleteButton=null)},o.prototype._updateContents=function(t){var e;if(this.options.template){var i=this.parent.itemSet.itemsData.get(this.id);e=this.options.template(i)}else e=this.data.content;var o=this._contentToString(this.content)!==this._contentToString(e);if(o){if(e instanceof Element)t.innerHTML="",t.appendChild(e);else if(void 0!=e)t.innerHTML=e;else if("background"!=this.data.type||void 0!==this.data.content)throw new Error('Property "content" missing in item '+this.id);this.content=e}},o.prototype._updateTitle=function(t){null!=this.data.title?t.title=this.data.title||"":t.removeAttribute("vis-title")},o.prototype._updateDataAttributes=function(t){if(this.options.dataAttributes&&this.options.dataAttributes.length>0){var e=[];if(Array.isArray(this.options.dataAttributes))e=this.options.dataAttributes;else{if("all"!=this.options.dataAttributes)return;e=Object.keys(this.data)}for(var i=0;in;n++){var r=this.visibleItems[n];r.repositionY(e)}return o},o.prototype.show=function(){this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background)},t.exports=o},function(t,e,i){function o(t,e,i){if(this.props={dot:{width:0,height:0},line:{width:0,height:0}},this.options=i,t&&void 0==t.start)throw new Error('Property "start" missing in item '+t);n.call(this,t,e,i)}var n=i(39);i(1);o.prototype=new n(null,null,null),o.prototype.isVisible=function(t){var e=(t.end-t.start)/4;return this.data.start>t.start-e&&this.data.startt.start-e&&this.data.startt.start},o.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.frame=document.createElement("div"),t.frame.className="vis-item-overflow",t.box.appendChild(t.frame),t.content=document.createElement("div"),t.content.className="vis-item-content",t.frame.appendChild(t.content),this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.background;if(!e)throw new Error("Cannot redraw item: parent has no background container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.content),this._updateDataAttributes(this.dom.content),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" vis-selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.props.content.width=this.dom.content.offsetWidth,this.height=0,this.dirty=!1}},o.prototype.show=r.prototype.show,o.prototype.hide=r.prototype.hide,o.prototype.repositionX=r.prototype.repositionX,o.prototype.repositionY=function(t){var e="top"===this.options.orientation.item;this.dom.content.style.top=e?"":"0",this.dom.content.style.bottom=e?"0":"";var i;if(void 0!==this.data.subgroup){var o=this.data.subgroup,n=this.parent.subgroups,r=n[o].index;if(1==e){i=this.parent.subgroups[o].height+t.item.vertical,i+=0==r?t.axis-.5*t.item.vertical:0;var a=this.parent.top;for(var h in n)n.hasOwnProperty(h)&&1==n[h].visible&&n[h].indexr&&(a+=l)}i=this.parent.subgroups[o].height+t.item.vertical,this.dom.box.style.top=this.parent.height-d+a+"px",this.dom.box.style.bottom=""}}else this.parent instanceof s?(i=Math.max(this.parent.height,this.parent.itemSet.body.domProps.center.height,this.parent.itemSet.body.domProps.centerContainer.height),this.dom.box.style.top=e?"0":"",this.dom.box.style.bottom=e?"":"0"):(i=this.parent.height,this.dom.box.style.top=this.parent.top+"px",this.dom.box.style.bottom="");this.dom.box.style.height=i+"px"},t.exports=o},function(t,e,i){function o(t,e){this.dom={foreground:null,lines:[],majorTexts:[],minorTexts:[],redundant:{lines:[],majorTexts:[],minorTexts:[]}},this.props={range:{start:0,end:0,minimumStep:0},lineTop:0},this.defaultOptions={orientation:{axis:"bottom"},showMinorLabels:!0,showMajorLabels:!0,maxMinorChars:7,format:a.FORMAT,moment:d,timeAxis:null},this.options=s.extend({},this.defaultOptions),this.body=t,this._create(),this.setOptions(e)}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(31),a=i(35),h=i(32),d=i(2);o.prototype=new r,o.prototype.setOptions=function(t){t&&(s.selectiveExtend(["showMinorLabels","showMajorLabels","maxMinorChars","hiddenDates","timeAxis","moment","rtl"],this.options,t),s.selectiveDeepExtend(["format"],this.options,t),"orientation"in t&&("string"==typeof t.orientation?this.options.orientation.axis=t.orientation:"object"===n(t.orientation)&&"axis"in t.orientation&&(this.options.orientation.axis=t.orientation.axis)),"locale"in t&&("function"==typeof d.locale?d.locale(t.locale):d.lang(t.locale)))},o.prototype._create=function(){this.dom.foreground=document.createElement("div"),this.dom.background=document.createElement("div"),this.dom.foreground.className="vis-time-axis vis-foreground",this.dom.background.className="vis-time-axis vis-background"},o.prototype.destroy=function(){this.dom.foreground.parentNode&&this.dom.foreground.parentNode.removeChild(this.dom.foreground),this.dom.background.parentNode&&this.dom.background.parentNode.removeChild(this.dom.background),this.body=null},o.prototype.redraw=function(){var t=this.props,e=this.dom.foreground,i=this.dom.background,o="top"==this.options.orientation.axis?this.body.dom.top:this.body.dom.bottom,n=e.parentNode!==o;this._calculateCharSize();var s=this.options.showMinorLabels&&"none"!==this.options.orientation.axis,r=this.options.showMajorLabels&&"none"!==this.options.orientation.axis;t.minorLabelHeight=s?t.minorCharHeight:0,t.majorLabelHeight=r?t.majorCharHeight:0,t.height=t.minorLabelHeight+t.majorLabelHeight,t.width=e.offsetWidth,t.minorLineHeight=this.body.domProps.root.height-t.majorLabelHeight-("top"==this.options.orientation.axis?this.body.domProps.bottom.height:this.body.domProps.top.height),t.minorLineWidth=1,t.majorLineHeight=t.minorLineHeight+t.majorLabelHeight,t.majorLineWidth=1;var a=e.nextSibling,h=i.nextSibling;return e.parentNode&&e.parentNode.removeChild(e),i.parentNode&&i.parentNode.removeChild(i),e.style.height=this.props.height+"px",this._repaintLabels(),a?o.insertBefore(e,a):o.appendChild(e),h?this.body.dom.backgroundVertical.insertBefore(i,h):this.body.dom.backgroundVertical.appendChild(i),this._isResized()||n},o.prototype._repaintLabels=function(){var t=this.options.orientation.axis,e=s.convert(this.body.range.start,"Number"),i=s.convert(this.body.range.end,"Number"),o=this.body.util.toTime((this.props.minorCharWidth||10)*this.options.maxMinorChars).valueOf(),n=o-h.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this.body.range,o);n-=this.body.util.toTime(0).valueOf();var r=new a(new Date(e),new Date(i),n,this.body.hiddenDates);r.setMoment(this.options.moment),this.options.format&&r.setFormat(this.options.format),this.options.timeAxis&&r.setScale(this.options.timeAxis),this.step=r;var d=this.dom;d.redundant.lines=d.lines,d.redundant.majorTexts=d.majorTexts,d.redundant.minorTexts=d.minorTexts,d.lines=[],d.majorTexts=[],d.minorTexts=[];var c,u,p,f,m,v,g,y,b,w,_=0,x=void 0,k=0,O=1e3;for(r.start(),u=r.getCurrent(),f=this.body.util.toScreen(u);r.hasNext()&&O>k;){k++,m=r.isMajor(),w=r.getClassName(),b=r.getLabelMinor(),c=u,p=f,r.next(),u=r.getCurrent(),v=r.isMajor(),f=this.body.util.toScreen(u),g=_,_=f-p;var M=_>=.4*g;if(this.options.showMinorLabels&&M){var D=this._repaintMinorText(p,b,t,w);D.style.width=_+"px"}m&&this.options.showMajorLabels?(p>0&&(void 0==x&&(x=p),D=this._repaintMajorText(p,r.getLabelMajor(),t,w)),y=this._repaintMajorLine(p,_,t,w)):M?y=this._repaintMinorLine(p,_,t,w):y&&(y.style.width=parseInt(y.style.width)+_+"px")}if(k!==O||l||(console.warn("Something is wrong with the Timeline scale. Limited drawing of grid lines to "+O+" lines."),l=!0),this.options.showMajorLabels){var S=this.body.util.toTime(0),C=r.getLabelMajor(S),T=C.length*(this.props.majorCharWidth||10)+10;(void 0==x||x>T)&&this._repaintMajorText(0,C,t,w)}s.forEach(this.dom.redundant,function(t){for(;t.length;){var e=t.pop();e&&e.parentNode&&e.parentNode.removeChild(e)}})},o.prototype._repaintMinorText=function(t,e,i,o){var n=this.dom.redundant.minorTexts.shift();if(!n){var s=document.createTextNode("");n=document.createElement("div"),n.appendChild(s),this.dom.foreground.appendChild(n)}return this.dom.minorTexts.push(n),n.childNodes[0].nodeValue=e,n.style.top="top"==i?this.props.majorLabelHeight+"px":"0",this.options.rtl?(n.style.left="",n.style.right=t+"px"):n.style.left=t+"px",n.className="vis-text vis-minor "+o,n},o.prototype._repaintMajorText=function(t,e,i,o){var n=this.dom.redundant.majorTexts.shift();if(!n){var s=document.createTextNode(e);n=document.createElement("div"),n.appendChild(s),this.dom.foreground.appendChild(n)}return this.dom.majorTexts.push(n),n.childNodes[0].nodeValue=e,n.className="vis-text vis-major "+o,n.style.top="top"==i?"0":this.props.minorLabelHeight+"px",this.options.rtl?(n.style.left="",n.style.right=t+"px"):n.style.left=t+"px",n},o.prototype._repaintMinorLine=function(t,e,i,o){var n=this.dom.redundant.lines.shift();n||(n=document.createElement("div"),this.dom.background.appendChild(n)),this.dom.lines.push(n);var s=this.props;return"top"==i?n.style.top=s.majorLabelHeight+"px":n.style.top=this.body.domProps.top.height+"px",n.style.height=s.minorLineHeight+"px",this.options.rtl?(n.style.left="",n.style.right=t-s.minorLineWidth/2+"px",n.className="vis-grid vis-vertical-rtl vis-minor "+o):(n.style.left=t-s.minorLineWidth/2+"px",n.className="vis-grid vis-vertical vis-minor "+o),n.style.width=e+"px",n},o.prototype._repaintMajorLine=function(t,e,i,o){var n=this.dom.redundant.lines.shift();n||(n=document.createElement("div"),this.dom.background.appendChild(n)),this.dom.lines.push(n);var s=this.props;return"top"==i?n.style.top="0":n.style.top=this.body.domProps.top.height+"px",this.options.rtl?(n.style.left="",n.style.right=t-s.majorLineWidth/2+"px",n.className="vis-grid vis-vertical-rtl vis-major "+o):(n.style.left=t-s.majorLineWidth/2+"px",n.className="vis-grid vis-vertical vis-major "+o),n.style.height=s.majorLineHeight+"px",n.style.width=e+"px",n},o.prototype._calculateCharSize=function(){this.dom.measureCharMinor||(this.dom.measureCharMinor=document.createElement("DIV"),this.dom.measureCharMinor.className="vis-text vis-minor vis-measure",this.dom.measureCharMinor.style.position="absolute",this.dom.measureCharMinor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMinor)),this.props.minorCharHeight=this.dom.measureCharMinor.clientHeight,this.props.minorCharWidth=this.dom.measureCharMinor.clientWidth,this.dom.measureCharMajor||(this.dom.measureCharMajor=document.createElement("DIV"),this.dom.measureCharMajor.className="vis-text vis-major vis-measure",this.dom.measureCharMajor.style.position="absolute",this.dom.measureCharMajor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMajor)),this.props.majorCharHeight=this.dom.measureCharMajor.clientHeight,this.props.majorCharWidth=this.dom.measureCharMajor.clientWidth};var l=!1;t.exports=o},function(t,e,i){function o(t){this.active=!1,this.dom={container:t},this.dom.overlay=document.createElement("div"),this.dom.overlay.className="vis-overlay",this.dom.container.appendChild(this.dom.overlay),this.hammer=a(this.dom.overlay),this.hammer.on("tap",this._onTapOverlay.bind(this));var e=this,i=["tap","doubletap","press","pinch","pan","panstart","panmove","panend"];i.forEach(function(t){e.hammer.on(t,function(t){t.stopPropagation()})}),document&&document.body&&(this.onClick=function(i){n(i.target,t)||e.deactivate()},document.body.addEventListener("click",this.onClick)),void 0!==this.keycharm&&this.keycharm.destroy(),this.keycharm=s(),this.escListener=this.deactivate.bind(this)}function n(t,e){for(;t;){if(t===e)return!0;t=t.parentNode}return!1}var s=i(23),r=i(13),a=i(20),h=i(1);r(o.prototype),o.current=null,o.prototype.destroy=function(){this.deactivate(),this.dom.overlay.parentNode.removeChild(this.dom.overlay),this.onClick&&document.body.removeEventListener("click",this.onClick),this.hammer.destroy(),this.hammer=null},o.prototype.activate=function(){o.current&&o.current.deactivate(),o.current=this,this.active=!0,this.dom.overlay.style.display="none",h.addClassName(this.dom.container,"vis-active"),this.emit("change"),this.emit("activate"),this.keycharm.bind("esc",this.escListener)},o.prototype.deactivate=function(){this.active=!1,this.dom.overlay.style.display="",h.removeClassName(this.dom.container,"vis-active"),this.keycharm.unbind("esc",this.escListener),this.emit("change"),this.emit("deactivate")},o.prototype._onTapOverlay=function(t){this.activate(),t.stopPropagation()},t.exports=o},function(t,e,i){function o(t,e){this.body=t,this.defaultOptions={moment:a,locales:h,locale:"en",id:void 0,title:void 0},this.options=s.extend({},this.defaultOptions),e&&e.time?this.customTime=e.time:this.customTime=new Date,this.eventParams={},this.setOptions(e),this._create()}var n=i(20),s=i(1),r=i(31),a=i(2),h=i(47);o.prototype=new r,o.prototype.setOptions=function(t){t&&s.selectiveExtend(["moment","locale","locales","id"],this.options,t)},o.prototype._create=function(){var t=document.createElement("div");t["custom-time"]=this,t.className="vis-custom-time "+(this.options.id||""),t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t;var e=document.createElement("div");e.style.position="relative",e.style.top="0px",e.style.left="-10px",e.style.height="100%",e.style.width="20px",t.appendChild(e),this.hammer=new n(e),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.get("pan").set({threshold:5,direction:n.DIRECTION_HORIZONTAL})},o.prototype.destroy=function(){this.hide(),this.hammer.destroy(),this.hammer=null,this.body=null},o.prototype.redraw=function(){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar));var e=this.body.util.toScreen(this.customTime),i=this.options.locales[this.options.locale];i||(this.warned||(console.log("WARNING: options.locales['"+this.options.locale+"'] not found. See http://visjs.org/docs/timeline.html#Localization"),this.warned=!0),i=this.options.locales.en);var o=this.options.title;return void 0===o&&(o=i.time+": "+this.options.moment(this.customTime).format("dddd, MMMM Do YYYY, H:mm:ss"),o=o.charAt(0).toUpperCase()+o.substring(1)),this.bar.style.left=e+"px",this.bar.title=o,!1},o.prototype.hide=function(){this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar)},o.prototype.setCustomTime=function(t){this.customTime=s.convert(t,"Date"),this.redraw()},o.prototype.getCustomTime=function(){return new Date(this.customTime.valueOf())},o.prototype.setCustomTitle=function(t){this.options.title=t},o.prototype._onDragStart=function(t){this.eventParams.dragging=!0,this.eventParams.customTime=this.customTime,t.stopPropagation()},o.prototype._onDrag=function(t){if(this.eventParams.dragging){var e=this.body.util.toScreen(this.eventParams.customTime)+t.deltaX,i=this.body.util.toTime(e);this.setCustomTime(i),this.body.emitter.emit("timechange",{id:this.options.id,time:new Date(this.customTime.valueOf()) +}),t.stopPropagation()}},o.prototype._onDragEnd=function(t){this.eventParams.dragging&&(this.body.emitter.emit("timechanged",{id:this.options.id,time:new Date(this.customTime.valueOf())}),t.stopPropagation())},o.customTimeFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("custom-time"))return e["custom-time"];e=e.parentNode}return null},t.exports=o},function(t,e){e.en={current:"current",time:"time"},e.en_EN=e.en,e.en_US=e.en,e.nl={current:"huidige",time:"tijd"},e.nl_NL=e.nl,e.nl_BE=e.nl},function(t,e,i){function o(t,e){this.body=t,this.defaultOptions={rtl:!1,showCurrentTime:!0,moment:r,locales:a,locale:"en"},this.options=n.extend({},this.defaultOptions),this.offset=0,this._create(),this.setOptions(e)}var n=i(1),s=i(31),r=i(2),a=i(47);o.prototype=new s,o.prototype._create=function(){var t=document.createElement("div");t.className="vis-current-time",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t},o.prototype.destroy=function(){this.options.showCurrentTime=!1,this.redraw(),this.body=null},o.prototype.setOptions=function(t){t&&n.selectiveExtend(["rtl","showCurrentTime","moment","locale","locales"],this.options,t)},o.prototype.redraw=function(){if(this.options.showCurrentTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar),this.start());var e=this.options.moment((new Date).valueOf()+this.offset),i=this.body.util.toScreen(e),o=this.options.locales[this.options.locale];o||(this.warned||(console.log("WARNING: options.locales['"+this.options.locale+"'] not found. See http://visjs.org/docs/timeline/#Localization"),this.warned=!0),o=this.options.locales.en);var n=o.current+" "+o.time+": "+e.format("dddd, MMMM Do YYYY, H:mm:ss");n=n.charAt(0).toUpperCase()+n.substring(1),this.options.rtl?this.bar.style.right=i+"px":this.bar.style.left=i+"px",this.bar.title=n}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),this.stop();return!1},o.prototype.start=function(){function t(){e.stop();var i=e.body.range.conversion(e.body.domProps.center.width).scale,o=1/i/10;30>o&&(o=30),o>1e3&&(o=1e3),e.redraw(),e.body.emitter.emit("currentTimeTick"),e.currentTimeTimer=setTimeout(t,o)}var e=this;t()},o.prototype.stop=function(){void 0!==this.currentTimeTimer&&(clearTimeout(this.currentTimeTimer),delete this.currentTimeTimer)},o.prototype.setCurrentTime=function(t){var e=n.convert(t,"Date").valueOf(),i=(new Date).valueOf();this.offset=e-i,this.redraw()},o.prototype.getCurrentTime=function(){return new Date((new Date).valueOf()+this.offset)},t.exports=o},function(t,e){Object.defineProperty(e,"__esModule",{value:!0});var i="string",o="boolean",n="number",s="array",r="date",a="object",h="dom",d="moment",l="any",c={configure:{enabled:{"boolean":o},filter:{"boolean":o,"function":"function"},container:{dom:h},__type__:{object:a,"boolean":o,"function":"function"}},align:{string:i},rtl:{"boolean":o,undefined:"undefined"},autoResize:{"boolean":o},throttleRedraw:{number:n},clickToUse:{"boolean":o},dataAttributes:{string:i,array:s},editable:{add:{"boolean":o,undefined:"undefined"},remove:{"boolean":o,undefined:"undefined"},updateGroup:{"boolean":o,undefined:"undefined"},updateTime:{"boolean":o,undefined:"undefined"},__type__:{"boolean":o,object:a}},end:{number:n,date:r,string:i,moment:d},format:{minorLabels:{millisecond:{string:i,undefined:"undefined"},second:{string:i,undefined:"undefined"},minute:{string:i,undefined:"undefined"},hour:{string:i,undefined:"undefined"},weekday:{string:i,undefined:"undefined"},day:{string:i,undefined:"undefined"},month:{string:i,undefined:"undefined"},year:{string:i,undefined:"undefined"},__type__:{object:a}},majorLabels:{millisecond:{string:i,undefined:"undefined"},second:{string:i,undefined:"undefined"},minute:{string:i,undefined:"undefined"},hour:{string:i,undefined:"undefined"},weekday:{string:i,undefined:"undefined"},day:{string:i,undefined:"undefined"},month:{string:i,undefined:"undefined"},year:{string:i,undefined:"undefined"},__type__:{object:a}},__type__:{object:a}},moment:{"function":"function"},groupOrder:{string:i,"function":"function"},groupEditable:{add:{"boolean":o,undefined:"undefined"},remove:{"boolean":o,undefined:"undefined"},order:{"boolean":o,undefined:"undefined"},__type__:{"boolean":o,object:a}},groupOrderSwap:{"function":"function"},height:{string:i,number:n},hiddenDates:{start:{date:r,number:n,string:i,moment:d},end:{date:r,number:n,string:i,moment:d},repeat:{string:i},__type__:{object:a,array:s}},itemsAlwaysDraggable:{"boolean":o},locale:{string:i},locales:{__any__:{any:l},__type__:{object:a}},margin:{axis:{number:n},item:{horizontal:{number:n,undefined:"undefined"},vertical:{number:n,undefined:"undefined"},__type__:{object:a,number:n}},__type__:{object:a,number:n}},max:{date:r,number:n,string:i,moment:d},maxHeight:{number:n,string:i},maxMinorChars:{number:n},min:{date:r,number:n,string:i,moment:d},minHeight:{number:n,string:i},moveable:{"boolean":o},multiselect:{"boolean":o},multiselectPerGroup:{"boolean":o},onAdd:{"function":"function"},onUpdate:{"function":"function"},onMove:{"function":"function"},onMoving:{"function":"function"},onRemove:{"function":"function"},onAddGroup:{"function":"function"},onMoveGroup:{"function":"function"},onRemoveGroup:{"function":"function"},order:{"function":"function"},orientation:{axis:{string:i,undefined:"undefined"},item:{string:i,undefined:"undefined"},__type__:{string:i,object:a}},selectable:{"boolean":o},showCurrentTime:{"boolean":o},showMajorLabels:{"boolean":o},showMinorLabels:{"boolean":o},stack:{"boolean":o},snap:{"function":"function","null":"null"},start:{date:r,number:n,string:i,moment:d},template:{"function":"function"},groupTemplate:{"function":"function"},timeAxis:{scale:{string:i,undefined:"undefined"},step:{number:n,undefined:"undefined"},__type__:{object:a}},type:{string:i},width:{string:i,number:n},zoomable:{"boolean":o},zoomKey:{string:["ctrlKey","altKey","metaKey",""]},zoomMax:{number:n},zoomMin:{number:n},__type__:{object:a}},u={global:{align:["center","left","right"],direction:!1,autoResize:!0,throttleRedraw:[10,0,1e3,10],clickToUse:!1,editable:{add:!1,remove:!1,updateGroup:!1,updateTime:!1},end:"",format:{minorLabels:{millisecond:"SSS",second:"s",minute:"HH:mm",hour:"HH:mm",weekday:"ddd D",day:"D",month:"MMM",year:"YYYY"},majorLabels:{millisecond:"HH:mm:ss",second:"D MMMM HH:mm",minute:"ddd D MMMM",hour:"ddd D MMMM",weekday:"MMMM YYYY",day:"MMMM YYYY",month:"YYYY",year:""}},groupsDraggable:!1,height:"",locale:"",margin:{axis:[20,0,100,1],item:{horizontal:[10,0,100,1],vertical:[10,0,100,1]}},max:"",maxHeight:"",maxMinorChars:[7,0,20,1],min:"",minHeight:"",moveable:!1,multiselect:!1,multiselectPerGroup:!1,orientation:{axis:["both","bottom","top"],item:["bottom","top"]},selectable:!0,showCurrentTime:!1,showMajorLabels:!0,showMinorLabels:!0,stack:!0,start:"",type:["box","point","range","background"],width:"100%",zoomable:!0,zoomKey:["ctrlKey","altKey","metaKey",""],zoomMax:[31536e10,10,31536e10,1],zoomMin:[10,10,31536e10,1]}};e.allOptions=c,e.configureOptions=u},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e,i,o){if(!(Array.isArray(i)||i instanceof c||i instanceof u)&&i instanceof Object){var n=o;o=i,i=n}var s=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:{axis:"bottom",item:"bottom"},moment:d,width:null,height:null,maxHeight:null,minHeight:null},this.options=l.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{toScreen:s._toScreen.bind(s),toGlobalScreen:s._toGlobalScreen.bind(s),toTime:s._toTime.bind(s),toGlobalTime:s._toGlobalTime.bind(s)}},this.range=new p(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new m(this.body),this.components.push(this.timeAxis),this.currentTime=new v(this.body),this.components.push(this.currentTime),this.linegraph=new y(this.body),this.components.push(this.linegraph),this.itemsData=null,this.groupsData=null,this.on("tap",function(t){s.emit("click",s.getEventProperties(t))}),this.on("doubletap",function(t){s.emit("doubleClick",s.getEventProperties(t))}),this.dom.root.oncontextmenu=function(t){s.emit("contextmenu",s.getEventProperties(t))},o&&this.setOptions(o),i&&this.setGroups(i),e&&this.setItems(e),this._redraw()}var s=i(26),r=o(s),a=i(29),h=o(a),d=(i(13),i(20),i(2)),l=i(1),c=i(9),u=i(11),p=i(30),f=i(33),m=i(44),v=i(48),g=i(46),y=i(51),b=i(29).printStyle,w=i(59).allOptions,_=i(59).configureOptions;n.prototype=new f,n.prototype.setOptions=function(t){var e=h["default"].validate(t,w);e===!0&&console.log("%cErrors have been found in the supplied options object.",b),f.prototype.setOptions.call(this,t)},n.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof c||t instanceof u?t:new c(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.linegraph&&this.linegraph.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){var o=void 0!=this.options.start?this.options.start:null,n=void 0!=this.options.end?this.options.end:null;this.setWindow(o,n,{animation:!1})}else this.fit({animation:!1})},n.prototype.setGroups=function(t){var e;e=t?t instanceof c||t instanceof u?t:new c(t):null,this.groupsData=e,this.linegraph.setGroups(e)},n.prototype.getLegend=function(t,e,i){return void 0===e&&(e=15),void 0===i&&(i=15),void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].getLegend(e,i):"cannot find group:'"+t+"'"},n.prototype.isGroupVisible=function(t){return void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].visible&&(void 0===this.linegraph.options.groups.visibility[t]||1==this.linegraph.options.groups.visibility[t]):!1},n.prototype.getDataRange=function(){var t=null,e=null;for(var i in this.linegraph.groups)if(this.linegraph.groups.hasOwnProperty(i)&&1==this.linegraph.groups[i].visible)for(var o=0;os?s:t,e=null==e?s:s>e?s:e}return{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},n.prototype.getEventProperties=function(t){var e=t.center?t.center.x:t.clientX,i=t.center?t.center.y:t.clientY,o=e-l.getAbsoluteLeft(this.dom.centerContainer),n=i-l.getAbsoluteTop(this.dom.centerContainer),s=this._toTime(o),r=g.customTimeFromTarget(t),a=l.getTarget(t),h=null;l.hasParent(a,this.timeAxis.dom.foreground)?h="axis":this.timeAxis2&&l.hasParent(a,this.timeAxis2.dom.foreground)?h="axis":l.hasParent(a,this.linegraph.yAxisLeft.dom.frame)?h="data-axis":l.hasParent(a,this.linegraph.yAxisRight.dom.frame)?h="data-axis":l.hasParent(a,this.linegraph.legendLeft.dom.frame)?h="legend":l.hasParent(a,this.linegraph.legendRight.dom.frame)?h="legend":null!=r?h="custom-time":l.hasParent(a,this.currentTime.bar)?h="current-time":l.hasParent(a,this.dom.center)&&(h="background");var d=[],c=this.linegraph.yAxisLeft,u=this.linegraph.yAxisRight;return c.hidden||d.push(c.screenToValue(n)),u.hidden||d.push(u.screenToValue(n)),{event:t,what:h,pageX:t.srcEvent?t.srcEvent.pageX:t.pageX,pageY:t.srcEvent?t.srcEvent.pageY:t.pageY,x:o,y:n,time:s,value:d}},n.prototype._createConfigurator=function(){return new r["default"](this,this.dom.container,_)},t.exports=n},function(t,e,i){function o(t,e){this.id=s.randomUUID(),this.body=t,this.defaultOptions={yAxisOrientation:"left",defaultGroup:"default",sort:!0,sampling:!0,stack:!1,graphHeight:"400px",shaded:{enabled:!1,orientation:"bottom"},style:"line",barChart:{width:50,sideBySide:!1,align:"center"},interpolation:{enabled:!0,parametrization:"centripetal",alpha:.5},drawPoints:{enabled:!0,size:6,style:"square"},dataAxis:{},legend:{},groups:{visibility:{}}},this.options=s.extend({},this.defaultOptions),this.dom={},this.props={},this.hammer=null,this.groups={},this.abortedGraphUpdate=!1,this.updateSVGheight=!1,this.updateSVGheightOnResize=!1,this.forceGraphUpdate=!0;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e,o){i._onAdd(e.items)},update:function(t,e,o){i._onUpdate(e.items)},remove:function(t,e,o){i._onRemove(e.items)}},this.groupListeners={add:function(t,e,o){i._onAddGroups(e.items)},update:function(t,e,o){i._onUpdateGroups(e.items)},remove:function(t,e,o){i._onRemoveGroups(e.items)}},this.items={},this.selection=[],this.lastStart=this.body.range.start,this.touchParams={},this.svgElements={},this.setOptions(e),this.groupsUsingDefaultStyles=[0],this.body.emitter.on("rangechanged",function(){i.lastStart=i.body.range.start,i.svg.style.left=s.option.asSize(-i.props.width),i.forceGraphUpdate=!0,i.redraw.call(i)}),this._create(),this.framework={svg:this.svg,svgElements:this.svgElements,options:this.options,groups:this.groups}}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(8),a=i(9),h=i(11),d=i(31),l=i(52),c=i(54),u=i(58),p=i(55),f=i(57),m=i(56),v="__ungrouped__";o.prototype=new d,o.prototype._create=function(){var t=document.createElement("div");t.className="vis-line-graph",this.dom.frame=t,this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="relative",this.svg.style.height=(""+this.options.graphHeight).replace("px","")+"px",this.svg.style.display="block",t.appendChild(this.svg),this.options.dataAxis.orientation="left",this.yAxisLeft=new l(this.body,this.options.dataAxis,this.svg,this.options.groups),this.options.dataAxis.orientation="right",this.yAxisRight=new l(this.body,this.options.dataAxis,this.svg,this.options.groups),delete this.options.dataAxis.orientation,this.legendLeft=new u(this.body,this.options.legend,"left",this.options.groups),this.legendRight=new u(this.body,this.options.legend,"right",this.options.groups),this.show()},o.prototype.setOptions=function(t){if(t){var e=["sampling","defaultGroup","stack","height","graphHeight","yAxisOrientation","style","barChart","dataAxis","sort","groups"];void 0===t.graphHeight&&void 0!==t.height?(this.updateSVGheight=!0,this.updateSVGheightOnResize=!0):void 0!==this.body.domProps.centerContainer.height&&void 0!==t.graphHeight&&parseInt((t.graphHeight+"").replace("px",""))i?-1:1});for(var o=new Array(t.length),n=0;n0){var h={};for(this._getRelevantData(a,h,n,s),this._applySampling(a,h),e=0;e0)switch(t.options.style){case"line":l.hasOwnProperty(a[e])||(l[a[e]]=f.calcPath(h[a[e]],t)),f.draw(l[a[e]],t,this.framework);case"point":case"points":"point"!=t.options.style&&"points"!=t.options.style&&1!=t.options.drawPoints.enabled||m.draw(h[a[e]],t,this.framework);break;case"bar":}}}return r.cleanupElements(this.svgElements),!1},o.prototype._stack=function(t,e){var i,o,n,s,r;i=0;for(var a=0;at[a].x){r=e[h],s=0==h?r:e[h-1],i=h;break}}void 0===r&&(s=e[e.length-1],r=e[e.length-1]),o=r.x-s.x,n=r.y-s.y,0==o?t[a].y=t[a].orginalY+r.y:t[a].y=t[a].orginalY+n/o*(t[a].x-s.x)+s.y}},o.prototype._getRelevantData=function(t,e,i,o){var n,r,a,h;if(t.length>0)for(r=0;rt?-1:1},c=Math.max(0,s.binarySearchValue(d,i,"x","before",l)),u=Math.min(d.length,s.binarySearchValue(d,o,"x","after",l)+1);0>=u&&(u=d.length);var p=new Array(u-c);for(a=c;u>a;a++)h=n.itemsData[a],p[a-c]=h;e[t[r]]=p}else e[t[r]]=n.itemsData}},o.prototype._applySampling=function(t,e){var i;if(t.length>0)for(var o=0;o0){var s=1,r=n.length,a=this.body.util.toGlobalScreen(n[n.length-1].x)-this.body.util.toGlobalScreen(n[0].x),h=r/a;s=Math.min(Math.ceil(.2*r),Math.max(1,Math.round(h)));for(var d=new Array(r),l=0;r>l;l+=s){var c=Math.round(l/s);d[c]=n[l]}e[t[o]]=d.splice(0,Math.round(r/s))}}},o.prototype._getYRanges=function(t,e,i){var o,n,s,r,a=[],h=[];if(t.length>0){for(s=0;s0&&(n=this.groups[t[s]],r.stack===!0&&"bar"===r.style?"left"===r.yAxisOrientation?a=a.concat(n.getItems()):h=h.concat(n.getItems()):i[t[s]]=n.getYRange(o,t[s]));p.getStackedYRange(a,i,t,"__barStackLeft","left"),p.getStackedYRange(h,i,t,"__barStackRight","right")}},o.prototype._updateYAxis=function(t,e){var i,o,n=!1,s=!1,r=!1,a=1e9,h=1e9,d=-1e9,l=-1e9;if(t.length>0){for(var c=0;ci?i:a,d=o>d?o:d):(r=!0,h=h>i?i:h,l=o>l?o:l));1==s&&this.yAxisLeft.setRange(a,d),1==r&&this.yAxisRight.setRange(h,l)}n=this._toggleAxisVisiblity(s,this.yAxisLeft)||n,n=this._toggleAxisVisiblity(r,this.yAxisRight)||n,1==r&&1==s?(this.yAxisLeft.drawIcons=!0,this.yAxisRight.drawIcons=!0):(this.yAxisLeft.drawIcons=!1,this.yAxisRight.drawIcons=!1),this.yAxisRight.master=!s,this.yAxisRight.masterAxis=this.yAxisLeft,0==this.yAxisRight.master?(1==r?this.yAxisLeft.lineOffset=this.yAxisRight.width:this.yAxisLeft.lineOffset=0,n=this.yAxisLeft.redraw()||n,n=this.yAxisRight.redraw()||n):n=this.yAxisRight.redraw()||n;for(var p=["__barStackLeft","__barStackRight","__lineStackLeft","__lineStackRight"],c=0;ct?-1:1});for(var a=0;a=0&&t._redrawLabel(o-2,e.val,i,"vis-y-axis vis-major",t.props.majorCharHeight),t.master===!0&&(n?t._redrawLine(o,i,"vis-grid vis-horizontal vis-major",t.options.majorLinesOffset,t.props.majorLineWidth):t._redrawLine(o,i,"vis-grid vis-horizontal vis-minor",t.options.minorLinesOffset,t.props.minorLineWidth))});var d=0;void 0!==this.options[i].title&&void 0!==this.options[i].title.text&&(d=this.props.titleCharHeight);var l=this.options.icons===!0?Math.max(this.options.iconWidth,d)+this.options.labelOffsetX+15:d+this.options.labelOffsetX+15;return this.maxLabelSize>this.width-l&&this.options.visible===!0?(this.width=this.maxLabelSize+l,this.options.width=this.width+"px",s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),this.redraw(),e=!0):this.maxLabelSizethis.minWidth?(this.width=Math.max(this.minWidth,this.maxLabelSize+l),this.options.width=this.width+"px",s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),this.redraw(),e=!0):(s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),e=!1),e},o.prototype.convertValue=function(t){return this.scale.convertValue(t)},o.prototype.screenToValue=function(t){return this.scale.screenToValue(t)},o.prototype._redrawLabel=function(t,e,i,o,n){var r=s.getDOMElement("div",this.DOMelements.labels,this.dom.frame);r.className=o,r.innerHTML=e,"left"===i?(r.style.left="-"+this.options.labelOffsetX+"px",r.style.textAlign="right"):(r.style.right="-"+this.options.labelOffsetX+"px",r.style.textAlign="left"),r.style.top=t-.5*n+this.options.labelOffsetY+"px",e+="";var a=Math.max(this.props.majorCharWidth,this.props.minorCharWidth);this.maxLabelSize.5*(h.magnitudefactor*h.minorSteps[h.minorStepIdx])?e+h.magnitudefactor*h.minorSteps[h.minorStepIdx]:e};i&&(this._start-=2*this.magnitudefactor*this.minorSteps[this.minorStepIdx],this._start=d(this._start)),o&&(this._end+=this.magnitudefactor*this.minorSteps[this.minorStepIdx],this._end=d(this._end)),this.determineScale()}}i.prototype.setCharHeight=function(t){this.majorCharHeight=t},i.prototype.setHeight=function(t){this.containerHeight=t},i.prototype.determineScale=function(){var t=this._end-this._start;this.scale=this.containerHeight/t;var e=this.majorCharHeight/this.scale,i=t>0?Math.round(Math.log(t)/Math.LN10):0;this.minorStepIdx=-1,this.magnitudefactor=Math.pow(10,i);var o=0;0>i&&(o=i);for(var n=!1,s=o;Math.abs(s)<=Math.abs(i);s++){this.magnitudefactor=Math.pow(10,s);for(var r=0;r=e){n=!0,this.minorStepIdx=r;break}}if(n===!0)break}},i.prototype.is_major=function(t){return t%(this.magnitudefactor*this.majorSteps[this.minorStepIdx])===0},i.prototype.getStep=function(){return this.magnitudefactor*this.minorSteps[this.minorStepIdx]},i.prototype.getFirstMajor=function(){var t=this.magnitudefactor*this.majorSteps[this.minorStepIdx];return this.convertValue(this._start+(t-this._start%t)%t)},i.prototype.formatValue=function(t){var e=t.toPrecision(5);return"function"==typeof this.formattingFunction&&(e=this.formattingFunction(t)),"number"==typeof e?""+e:"string"==typeof e?e:t.toPrecision(5)},i.prototype.getLines=function(){for(var t=[],e=this.getStep(),i=(e-this._start%e)%e,o=this._start+i;this._end-o>1e-5;o+=e)o!=this._start&&t.push({major:this.is_major(o),y:this.convertValue(o),val:this.formatValue(o)});return t},i.prototype.followScale=function(t){var e=this.minorStepIdx,i=this._start,o=this._end,n=this,s=function(){n.magnitudefactor*=2},r=function(){n.magnitudefactor/=2};t.minorStepIdx<=1&&this.minorStepIdx<=1||t.minorStepIdx>1&&this.minorStepIdx>1||(t.minorStepIdxo+1e-5)r(),d=!1;else{if(!this.autoScaleStart&&this._start=0)){r(),d=!1;continue}console.warn("Can't adhere to given 'min' range, due to zeroalign")}this.autoScaleStart&&this.autoScaleEnd&&o-i>c?(s(),d=!1):d=!0}}},i.prototype.convertValue=function(t){return this.containerHeight-(t-this._start)*this.scale},i.prototype.screenToValue=function(t){return(this.containerHeight-t)/this.scale+this._start},t.exports=i},function(t,e,i){function o(t,e,i,o){this.id=e;var n=["sampling","style","sort","yAxisOrientation","barChart","drawPoints","shaded","interpolation","zIndex","excludeFromStacking","excludeFromLegend"];this.options=s.selectiveBridgeObject(n,i),this.usingDefaultStyle=void 0===t.className,this.groupsUsingDefaultStyles=o,this.zeroPosition=0,this.update(t),1==this.usingDefaultStyle&&(this.groupsUsingDefaultStyles[0]+=1),this.itemsData=[],this.visible=void 0===t.visible?!0:t.visible}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=(i(8),i(55)),a=i(57),h=i(56);o.prototype.setItems=function(t){null!=t?(this.itemsData=t,1==this.options.sort&&s.insertSort(this.itemsData,function(t,e){return t.x>e.x?1:-1})):this.itemsData=[]},o.prototype.getItems=function(){return this.itemsData},o.prototype.setZeroPosition=function(t){this.zeroPosition=t},o.prototype.setOptions=function(t){if(void 0!==t){var e=["sampling","style","sort","yAxisOrientation","barChart","zIndex","excludeFromStacking","excludeFromLegend"];s.selectiveDeepExtend(e,this.options,t),"function"==typeof t.drawPoints&&(t.drawPoints={onRender:t.drawPoints}),s.mergeOptions(this.options,t,"interpolation"),s.mergeOptions(this.options,t,"drawPoints"),s.mergeOptions(this.options,t,"shaded"),t.interpolation&&"object"==n(t.interpolation)&&t.interpolation.parametrization&&("uniform"==t.interpolation.parametrization?this.options.interpolation.alpha=0:"chordal"==t.interpolation.parametrization?this.options.interpolation.alpha=1:(this.options.interpolation.parametrization="centripetal",this.options.interpolation.alpha=.5))}},o.prototype.update=function(t){this.group=t,this.content=t.content||"graph",this.className=t.className||this.className||"vis-graph-group"+this.groupsUsingDefaultStyles[0]%10,this.visible=void 0===t.visible?!0:t.visible,this.style=t.style,this.setOptions(t.options)},o.prototype.getLegend=function(t,e,i,o,n){if(void 0==i||null==i){var s=document.createElementNS("http://www.w3.org/2000/svg","svg");i={svg:s,svgElements:{},options:this.options,groups:[this]}}switch(void 0!=o&&null!=o||(o=0),void 0!=n&&null!=n||(n=.5*e),this.options.style){case"line":a.drawIcon(this,o,n,t,e,i);break;case"points":case"point":h.drawIcon(this,o,n,t,e,i);break;case"bar":r.drawIcon(this,o,n,t,e,i)}return{icon:i.svg,label:this.content,orientation:this.options.yAxisOrientation}},o.prototype.getYRange=function(t){for(var e=t[0].y,i=t[0].y,o=0;ot[o].y?t[o].y:e,i=i0&&(i=Math.min(i,Math.abs(e[o-1].screen_x-e[o].screen_x))),0===i&&(void 0===t[e[o].screen_x]&&(t[e[o].screen_x]={amount:0,resolved:0,accumulatedPositive:0,accumulatedNegative:0}),t[e[o].screen_x].amount+=1)},o._getSafeDrawData=function(t,e,i){var o,n;return t0?(o=i>t?i:t,n=0,"left"===e.options.barChart.align?n-=.5*t:"right"===e.options.barChart.align&&(n+=.5*t)):(o=e.options.barChart.width,n=0,"left"===e.options.barChart.align?n-=.5*e.options.barChart.width:"right"===e.options.barChart.align&&(n+=.5*e.options.barChart.width)),{width:o,offset:n}},o.getStackedYRange=function(t,e,i,n,s){if(t.length>0){t.sort(function(t,e){return t.screen_x===e.screen_x?t.groupIde[s].screen_y?e[s].screen_y:o,n=nt[r].accumulatedNegative?t[r].accumulatedNegative:o,o=o>t[r].accumulatedPositive?t[r].accumulatedPositive:o,n=n0){var i=[];return i=1==e.options.interpolation.enabled?o._catmullRom(t,e):o._linear(t)}},o.drawIcon=function(t,e,i,o,s,r){var a,h,d=.5*s,l=n.getSVGElement("rect",r.svgElements,r.svg);if(l.setAttributeNS(null,"x",e),l.setAttributeNS(null,"y",i-d),l.setAttributeNS(null,"width",o),l.setAttributeNS(null,"height",2*d),l.setAttributeNS(null,"class","vis-outline"),a=n.getSVGElement("path",r.svgElements,r.svg),a.setAttributeNS(null,"class",t.className),void 0!==t.style&&a.setAttributeNS(null,"style",t.style),a.setAttributeNS(null,"d","M"+e+","+i+" L"+(e+o)+","+i),1==t.options.shaded.enabled&&(h=n.getSVGElement("path",r.svgElements,r.svg),"top"==t.options.shaded.orientation?h.setAttributeNS(null,"d","M"+e+", "+(i-d)+"L"+e+","+i+" L"+(e+o)+","+i+" L"+(e+o)+","+(i-d)):h.setAttributeNS(null,"d","M"+e+","+i+" L"+e+","+(i+d)+" L"+(e+o)+","+(i+d)+"L"+(e+o)+","+i),h.setAttributeNS(null,"class",t.className+" vis-icon-fill"),void 0!==t.options.shaded.style&&""!==t.options.shaded.style&&h.setAttributeNS(null,"style",t.options.shaded.style)),1==t.options.drawPoints.enabled){var c={style:t.options.drawPoints.style,styles:t.options.drawPoints.styles,size:t.options.drawPoints.size,className:t.className};n.drawPoint(e+.5*o,i,c,r.svgElements,r.svg)}},o.drawShading=function(t,e,i,o){if(1==e.options.shaded.enabled){var s=Number(o.svg.style.height.replace("px","")),r=n.getSVGElement("path",o.svgElements,o.svg),a="L";1==e.options.interpolation.enabled&&(a="C");var h,d=0;d="top"==e.options.shaded.orientation?0:"bottom"==e.options.shaded.orientation?s:Math.min(Math.max(0,e.zeroPosition),s),h="group"==e.options.shaded.orientation&&null!=i&&void 0!=i?"M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,a,!1)+" L"+i[i.length-1][0]+","+i[i.length-1][1]+" "+this.serializePath(i,a,!0)+i[0][0]+","+i[0][1]+" Z":"M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,a,!1)+" V"+d+" H"+t[0][0]+" Z",r.setAttributeNS(null,"class",e.className+" vis-fill"),void 0!==e.options.shaded.style&&r.setAttributeNS(null,"style",e.options.shaded.style),r.setAttributeNS(null,"d",h)}},o.draw=function(t,e,i){if(null!=t&&void 0!=t){var o=n.getSVGElement("path",i.svgElements,i.svg);o.setAttributeNS(null,"class",e.className),void 0!==e.style&&o.setAttributeNS(null,"style",e.style);var s="L";1==e.options.interpolation.enabled&&(s="C"),o.setAttributeNS(null,"d","M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,s,!1))}},o.serializePath=function(t,e,i){if(t.length<2)return"";var o=e;if(i)for(var n=t.length-2;n>0;n--)o+=t[n][0]+","+t[n][1]+" ";else for(var n=1;nl;l++)e=0==l?t[0]:t[l-1],i=t[l],o=t[l+1],n=d>l+2?t[l+2]:o,s={screen_x:(-e.screen_x+6*i.screen_x+o.screen_x)*h,screen_y:(-e.screen_y+6*i.screen_y+o.screen_y)*h},r={screen_x:(i.screen_x+6*o.screen_x-n.screen_x)*h,screen_y:(i.screen_y+6*o.screen_y-n.screen_y)*h},a.push([s.screen_x,s.screen_y]),a.push([r.screen_x,r.screen_y]),a.push([o.screen_x,o.screen_y]);return a},o._catmullRom=function(t,e){var i=e.options.interpolation.alpha;if(0==i||void 0===i)return this._catmullRomUniform(t);var o,n,s,r,a,h,d,l,c,u,p,f,m,v,g,y,b,w,_,x=[];x.push([Math.round(t[0].screen_x),Math.round(t[0].screen_y)]);for(var k=t.length,O=0;k-1>O;O++)o=0==O?t[0]:t[O-1],n=t[O],s=t[O+1],r=k>O+2?t[O+2]:s,d=Math.sqrt(Math.pow(o.screen_x-n.screen_x,2)+Math.pow(o.screen_y-n.screen_y,2)),l=Math.sqrt(Math.pow(n.screen_x-s.screen_x,2)+Math.pow(n.screen_y-s.screen_y,2)),c=Math.sqrt(Math.pow(s.screen_x-r.screen_x,2)+Math.pow(s.screen_y-r.screen_y,2)),v=Math.pow(c,i),y=Math.pow(c,2*i),g=Math.pow(l,i),b=Math.pow(l,2*i),_=Math.pow(d,i),w=Math.pow(d,2*i),u=2*w+3*_*g+b,p=2*y+3*v*g+b,f=3*_*(_+g),f>0&&(f=1/f),m=3*v*(v+g),m>0&&(m=1/m),a={screen_x:(-b*o.screen_x+u*n.screen_x+w*s.screen_x)*f,screen_y:(-b*o.screen_y+u*n.screen_y+w*s.screen_y)*f},h={screen_x:(y*n.screen_x+p*s.screen_x-b*r.screen_x)*m,screen_y:(y*n.screen_y+p*s.screen_y-b*r.screen_y)*m},0==a.screen_x&&0==a.screen_y&&(a=n),0==h.screen_x&&0==h.screen_y&&(h=s),x.push([a.screen_x,a.screen_y]),x.push([h.screen_x,h.screen_y]),x.push([s.screen_x,s.screen_y]);return x},o._linear=function(t){for(var e=[],i=0;it?-1:1});for(var i=0;i")}this.dom.textArea.innerHTML=s,this.dom.textArea.style.lineHeight=.75*this.options.iconSize+this.options.iconSpacing+"px"}},o.prototype.drawLegendIcons=function(){if(this.dom.frame.parentNode){var t=Object.keys(this.groups);t.sort(function(t,e){return e>t?-1:1}),s.resetElements(this.svgElements);var e=window.getComputedStyle(this.dom.frame).paddingTop,i=Number(e.replace("px","")),o=i,n=this.options.iconSize,r=.75*this.options.iconSize,a=i+.5*r+3;this.svg.style.width=n+5+i+"px";for(var h=0;h0){var i=this.groupIndex%this.groupsArray.length;this.groupIndex++,e={},e.color=this.groups[this.groupsArray[i]],this.groups[t]=e}else{var o=this.defaultIndex%this.defaultGroups.length;this.defaultIndex++,e={},e.color=this.defaultGroups[o],this.groups[t]=e}return e}},{key:"add",value:function(t,e){return this.groups[t]=e,this.groupsArray.push(t),e}}]),t}();e["default"]=r},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;it.left&&this.shape.topt.top}},{key:"isBoundingBoxOverlappingWith",value:function(t){return this.shape.boundingBox.leftt.left&&this.shape.boundingBox.topt.top}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=arguments.length<=3||void 0===arguments[3]?{}:arguments[3],n=["color","font","fixed","shadow"];if(A.selectiveNotDeepExtend(n,t,e,i),A.mergeOptions(t,e,"shadow",i,o),void 0!==e.color&&null!==e.color){var s=A.parseColor(e.color);A.fillIfDefined(t.color,s)}else i===!0&&null===e.color&&(t.color=A.bridgeObject(o.color));void 0!==e.fixed&&null!==e.fixed&&("boolean"==typeof e.fixed?(t.fixed.x=e.fixed,t.fixed.y=e.fixed):(void 0!==e.fixed.x&&"boolean"==typeof e.fixed.x&&(t.fixed.x=e.fixed.x),void 0!==e.fixed.y&&"boolean"==typeof e.fixed.y&&(t.fixed.y=e.fixed.y))),void 0!==e.font&&null!==e.font?a["default"].parseOptions(t.font,e):i===!0&&null===e.font&&(t.font=A.bridgeObject(o.font)),void 0!==e.scaling&&A.mergeOptions(t.scaling,e.scaling,"label",i,o.scaling)}}]),t}();e["default"]=B},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i=this.nodeOptions.scaling.label.maxVisible&&(r=Number(this.nodeOptions.scaling.label.maxVisible)/this.body.view.scale);var h=this.size.yLine,d=this._getColor(a),l=n(d,2),c=l[0],u=l[1],p=this._setAlignment(t,i,h,s),f=n(p,2);i=f[0],h=f[1],t.font=(e&&this.nodeOptions.labelHighlightBold?"bold ":"")+r+"px "+this.fontOptions.face,t.fillStyle=c,this.isEdgeLabel||"left"!==this.fontOptions.align?t.textAlign="center":(t.textAlign=this.fontOptions.align,i-=.5*this.size.width),this.fontOptions.strokeWidth>0&&(t.lineWidth=this.fontOptions.strokeWidth,t.strokeStyle=u,t.lineJoin="round");for(var m=0;m0&&t.strokeText(this.lines[m],i,h),t.fillText(this.lines[m],i,h),h+=r}},{key:"_setAlignment",value:function(t,e,i,o){if(this.isEdgeLabel&&"horizontal"!==this.fontOptions.align&&this.pointToSelf===!1){e=0,i=0;var n=2;"top"===this.fontOptions.align?(t.textBaseline="alphabetic",i-=2*n):"bottom"===this.fontOptions.align?(t.textBaseline="hanging",i+=2*n):t.textBaseline="middle"}else t.textBaseline=o;return[e,i]}},{key:"_getColor",value:function(t){var e=this.fontOptions.color||"#000000",i=this.fontOptions.strokeColor||"#ffffff";if(t<=this.nodeOptions.scaling.label.drawThreshold){var o=Math.max(0,Math.min(1,1-(this.nodeOptions.scaling.label.drawThreshold-t)));e=a.overrideOpacity(e,o),i=a.overrideOpacity(i,o)}return[e,i]}},{key:"getTextSize",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?!1:arguments[1],i={width:this._processLabel(t,e),height:this.fontOptions.size*this.lineCount,lineCount:this.lineCount};return i}},{key:"calculateLabelSize",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?0:arguments[2],o=arguments.length<=3||void 0===arguments[3]?0:arguments[3],n=arguments.length<=4||void 0===arguments[4]?"middle":arguments[4];this.labelDirty===!0&&(this.size.width=this._processLabel(t,e)),this.size.height=this.fontOptions.size*this.lineCount,this.size.left=i-.5*this.size.width,this.size.top=o-.5*this.size.height,this.size.yLine=o+.5*(1-this.lineCount)*this.fontOptions.size,"hanging"===n&&(this.size.top+=.5*this.fontOptions.size,this.size.top+=4,this.size.yLine+=4),this.labelDirty=!1}},{key:"_processLabel",value:function(t,e){var i=0,o=[""],n=0;if(void 0!==this.nodeOptions.label){o=String(this.nodeOptions.label).split("\n"),n=o.length,t.font=(e&&this.nodeOptions.labelHighlightBold?"bold ":"")+this.fontOptions.size+"px "+this.fontOptions.face,i=t.measureText(o[0]).width;for(var s=1;n>s;s++){var r=t.measureText(o[s]).width;i=r>i?r:i}}return this.lines=o,this.lineCount=n,i}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2];if("string"==typeof e.font){var o=e.font.split(" ");t.size=o[0].replace("px",""),t.face=o[1],t.color=o[2]}else"object"===s(e.font)&&a.fillIfDefined(t,e.font,i);t.size=Number(t.size)}}]),t}();e["default"]=h},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height;var n=this.options.shapeProperties.borderRadius;this.boundingBox.left=this.left-n,this.boundingBox.top=this.top-n,this.boundingBox.bottom=this.top+this.height+n,this.boundingBox.right=this.left+this.width+n}},{key:"distanceToBorder",value:function(t,e){this.resize(t);var i=this.options.borderWidth;return Math.min(Math.abs(this.width/2/Math.cos(e)),Math.abs(this.height/2/Math.sin(e)))+i}}]),e}(d["default"]);e["default"]=l},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ithis.imageObj.height?(o=this.imageObj.width/this.imageObj.height,e=2*this.options.size*o||this.imageObj.width,i=2*this.options.size||this.imageObj.height):(o=this.imageObj.width&&this.imageObj.height?this.imageObj.height/this.imageObj.width:1,e=2*this.options.size,i=2*this.options.size*o):(e=this.imageObj.width,i=this.imageObj.height),this.width=e,this.height=i,this.radius=.5*this.width}}},{key:"_drawRawCircle",value:function(t,e,i,o,n,s){var r=this.options.borderWidth,a=this.options.borderWidthSelected||2*this.options.borderWidth,h=(o?a:r)/this.body.view.scale;t.lineWidth=Math.min(this.width,h),t.strokeStyle=o?this.options.color.highlight.border:n?this.options.color.hover.border:this.options.color.border,t.fillStyle=o?this.options.color.highlight.background:n?this.options.color.hover.background:this.options.color.background,t.circle(e,i,s),this.enableShadow(t),t.fill(),this.disableShadow(t),t.save(),h>0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore()}},{key:"_drawImageAtPosition",value:function(t){if(0!=this.imageObj.width){t.globalAlpha=1,this.enableShadow(t);var e=this.imageObj.width/this.width/this.body.view.scale;if(e>2&&this.options.shapeProperties.interpolation===!0){var i=this.imageObj.width,o=this.imageObj.height,n=document.createElement("canvas");n.width=i,n.height=i;var s=n.getContext("2d");e*=.5,i*=.5,o*=.5,s.drawImage(this.imageObj,0,0,i,o);for(var r=0,a=1;e>2&&4>a;)s.drawImage(n,r,0,i,o,r+i,0,i/2,o/2),r+=i,e*=.5,i*=.5,o*=.5,a+=1;t.drawImage(n,r,0,i,o,this.left,this.top,this.width,this.height)}else t.drawImage(this.imageObj,this.left,this.top,this.width,this.height);this.disableShadow(t)}}},{key:"_drawImageLabel",value:function(t,e,i,o){var n,s=0;if(void 0!==this.height){s=.5*this.height;var r=this.labelModule.getTextSize(t);r.lineCount>=1&&(s+=r.height/2)}n=i+s,this.options.label&&(this.labelOffset=s),this.labelModule.draw(t,e,n,o,"hanging")}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height,this.boundingBox.left=this.left,this.boundingBox.top=this.top,this.boundingBox.bottom=this.top+this.height,this.boundingBox.right=this.left+this.width}},{key:"distanceToBorder",value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),void 0!==this.options.label){var l=n+.5*this.height+3;this.labelModule.draw(t,o,l,s,"hanging")}this.updateBoundingBox(o,n)}},{key:"updateBoundingBox",value:function(t,e){this.boundingBox.top=e-this.options.size,this.boundingBox.left=t-this.options.size,this.boundingBox.right=t+this.options.size,this.boundingBox.bottom=e+this.options.size,void 0!==this.options.label&&this.labelModule.size.width>0&&(this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelModule.size.height+3))}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height,this.boundingBox.left=this.left,this.boundingBox.top=this.top,this.boundingBox.bottom=this.top+this.height,this.boundingBox.right=this.left+this.width}},{key:"distanceToBorder",value:function(t,e){this.resize(t);var i=.5*this.width,o=.5*this.height,n=Math.sin(e)*i,s=Math.cos(e)*o;return i*o/Math.sqrt(n*n+s*s)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0){var i=5;this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelModule.size.height+i)}}},{key:"_icon",value:function(t,e,i,o){var n=Number(this.options.icon.size);void 0!==this.options.icon.code?(t.font=(o?"bold ":"")+n+"px "+this.options.icon.face,t.fillStyle=this.options.icon.color||"black",t.textAlign="center",t.textBaseline="middle",this.enableShadow(t),t.fillText(this.options.icon.code,e,i),this.disableShadow(t)):console.error("When using the icon shape, you need to define the code in the icon options object. This can be done per node or globally.")}},{key:"distanceToBorder",value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),t.closePath()}this._drawImageAtPosition(t),this._drawImageLabel(t,e,i,o||n),this.updateBoundingBox(e,i)}},{key:"updateBoundingBox",value:function(t,e){this.resize(),this.left=t-this.width/2,this.top=e-this.height/2,this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,void 0!==this.options.label&&this.labelModule.size.width>0&&(this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelOffset))}},{key:"distanceToBorder", +value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;ii.shape.height?(r=i.x+.5*i.shape.width,a=i.y-h):(r=i.x+h,a=i.y-.5*i.shape.height),s=this._pointOnCircle(r,a,h,.125),this.labelModule.draw(t,s.x,s.y,n)}}}},{key:"isOverlappingWith",value:function(t){if(this.connected){var e=10,i=this.from.x,o=this.from.y,n=this.to.x,s=this.to.y,r=t.left,a=t.top,h=this.edgeType.getDistanceToEdge(i,o,n,s,r,a);return e>h}return!1}},{key:"_rotateForLabelAlignment",value:function(t){var e=this.from.y-this.to.y,i=this.from.x-this.to.x,o=Math.atan2(e,i);(-1>o&&0>i||o>0&&0>i)&&(o+=Math.PI),t.rotate(o)}},{key:"_pointOnCircle",value:function(t,e,i,o){var n=2*o*Math.PI;return{x:t+i*Math.cos(n),y:e-i*Math.sin(n)}}},{key:"select",value:function(){this.selected=!0}},{key:"unselect",value:function(){this.selected=!1}},{key:"cleanup",value:function(){return this.edgeType.cleanup()}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=arguments.length<=3||void 0===arguments[3]?{}:arguments[3],n=["arrowStrikethrough","id","from","hidden","hoverWidth","label","labelHighlightBold","length","line","opacity","physics","scaling","selectionWidth","selfReferenceSize","to","title","value","width"];if(g.selectiveDeepExtend(n,t,e,i),g.mergeOptions(t,e,"smooth",i,o),g.mergeOptions(t,e,"shadow",i,o),void 0!==e.dashes&&null!==e.dashes?t.dashes=e.dashes:i===!0&&null===e.dashes&&(t.dashes=Object.create(o.dashes)),void 0!==e.scaling&&null!==e.scaling?(void 0!==e.scaling.min&&(t.scaling.min=e.scaling.min),void 0!==e.scaling.max&&(t.scaling.max=e.scaling.max),g.mergeOptions(t.scaling,e.scaling,"label",i,o.scaling)):i===!0&&null===e.scaling&&(t.scaling=Object.create(o.scaling)),void 0!==e.arrows&&null!==e.arrows)if("string"==typeof e.arrows){var r=e.arrows.toLowerCase();t.arrows.to.enabled=-1!=r.indexOf("to"),t.arrows.middle.enabled=-1!=r.indexOf("middle"),t.arrows.from.enabled=-1!=r.indexOf("from")}else{if("object"!==s(e.arrows))throw new Error("The arrow newOptions can only be an object or a string. Refer to the documentation. You used:"+JSON.stringify(e.arrows));g.mergeOptions(t.arrows,e.arrows,"to",i,o.arrows),g.mergeOptions(t.arrows,e.arrows,"middle",i,o.arrows),g.mergeOptions(t.arrows,e.arrows,"from",i,o.arrows)}else i===!0&&null===e.arrows&&(t.arrows=Object.create(o.arrows));if(void 0!==e.color&&null!==e.color)if(t.color=g.deepExtend({},t.color,!0),g.isString(e.color))t.color.color=e.color,t.color.highlight=e.color,t.color.hover=e.color,t.color.inherit=!1;else{var a=!1;void 0!==e.color.color&&(t.color.color=e.color.color,a=!0),void 0!==e.color.highlight&&(t.color.highlight=e.color.highlight,a=!0),void 0!==e.color.hover&&(t.color.hover=e.color.hover,a=!0),void 0!==e.color.inherit&&(t.color.inherit=e.color.inherit),void 0!==e.color.opacity&&(t.color.opacity=Math.min(1,Math.max(0,e.color.opacity))),void 0===e.color.inherit&&a===!0&&(t.color.inherit=!1)}else i===!0&&null===e.color&&(t.color=g.bridgeObject(o.color));void 0!==e.font&&null!==e.font?h["default"].parseOptions(t.font,e):i===!0&&null===e.font&&(t.font=g.bridgeObject(o.font))}}]),t}();e["default"]=y},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),h=function(){function t(t,e){for(var i=0;iMath.abs(e)||this.options.smooth.forceDirection===!0||"horizontal"===this.options.smooth.forceDirection)&&"vertical"!==this.options.smooth.forceDirection?(o=this.from.y,s=this.to.y,i=this.from.x-r*t,n=this.to.x+r*t):(o=this.from.y-r*e,s=this.to.y+r*e,i=this.from.x,n=this.to.x),[{x:i,y:o},{x:n,y:s}]}},{key:"getViaNode",value:function(){return this._getViaCoordinates()}},{key:"_findBorderPosition",value:function(t,e){return this._findBorderPositionBezier(t,e)}},{key:"_getDistanceToEdge",value:function(t,e,i,o,n,s){var r=arguments.length<=6||void 0===arguments[6]?this._getViaCoordinates():arguments[6],h=a(r,2),d=h[0],l=h[1];return this._getDistanceToBezierEdge(t,e,i,o,n,s,d,l)}},{key:"getPoint",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?this._getViaCoordinates():arguments[1],i=a(e,2),o=i[0],n=i[1],s=t,r=[];r[0]=Math.pow(1-s,3),r[1]=3*s*Math.pow(1-s,2),r[2]=3*Math.pow(s,2)*(1-s),r[3]=Math.pow(s,3);var h=r[0]*this.fromPoint.x+r[1]*o.x+r[2]*n.x+r[3]*this.toPoint.x,d=r[0]*this.fromPoint.y+r[1]*o.y+r[2]*n.y+r[3]*this.toPoint.y;return{x:h,y:d}}}]),e}(l["default"]);e["default"]=c},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;il;l++)c=.1*l,v[0]=Math.pow(1-c,3),v[1]=3*c*Math.pow(1-c,2),v[2]=3*Math.pow(c,2)*(1-c),v[3]=Math.pow(c,3),u=v[0]*t+v[1]*r.x+v[2]*a.x+v[3]*i,p=v[0]*e+v[1]*r.y+v[2]*a.y+v[3]*o,l>0&&(d=this._getDistanceToLine(f,m,u,p,n,s),h=h>d?d:h),f=u,m=p;return h}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i=l&&h>d;){var m=.5*(l+c);if(i=this.getPoint(m,a),o=Math.atan2(p.y-i.y,p.x-i.x),n=p.distanceToBorder(e,o),s=Math.sqrt(Math.pow(i.x-p.x,2)+Math.pow(i.y-p.y,2)),r=n-s,Math.abs(r)r?f===!1?l=m:c=m:f===!1?c=m:l=m,d++}return i.t=m,i}},{key:"_getDistanceToBezierEdge",value:function(t,e,i,o,n,s,r){var a=1e9,h=void 0,d=void 0,l=void 0,c=void 0,u=void 0,p=t,f=e;for(d=1;10>d;d++)l=.1*d,c=Math.pow(1-l,2)*t+2*l*(1-l)*r.x+Math.pow(l,2)*i,u=Math.pow(1-l,2)*e+2*l*(1-l)*r.y+Math.pow(l,2)*o,d>0&&(h=this._getDistanceToLine(p,f,c,u,n,s),a=a>h?h:a),p=c,f=u;return a}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),s=function(){function t(t,e){for(var i=0;io.shape.height?(e=o.x+.5*o.shape.width,i=o.y-n):(e=o.x+n,i=o.y-.5*o.shape.height),[e,i,n]}},{key:"_pointOnCircle",value:function(t,e,i,o){var n=2*o*Math.PI;return{x:t+i*Math.cos(n),y:e-i*Math.sin(n)}}},{key:"_findBorderPositionCircle",value:function(t,e,i){for(var o=i.x,n=i.y,s=i.low,r=i.high,a=i.direction,h=10,d=0,l=this.options.selfReferenceSize,c=void 0,u=void 0,p=void 0,f=void 0,m=void 0,v=.05,g=.5*(s+r);r>=s&&h>d&&(g=.5*(s+r),c=this._pointOnCircle(o,n,l,g),u=Math.atan2(t.y-c.y,t.x-c.x),p=t.distanceToBorder(e,u),f=Math.sqrt(Math.pow(c.x-t.x,2)+Math.pow(c.y-t.y,2)), +m=p-f,!(Math.abs(m)0?a>0?s=g:r=g:a>0?r=g:s=g,d++;return c.t=g,c}},{key:"getLineWidth",value:function(t,e){return t===!0?Math.max(this.selectionWidth,.3/this.body.view.scale):e===!0?Math.max(this.hoverWidth,.3/this.body.view.scale):Math.max(this.options.width,.3/this.body.view.scale)}},{key:"getColor",value:function(t,e,i){var o=this.options.color;if(o.inherit!==!1){if("both"===o.inherit&&this.from.id!==this.to.id){var n=t.createLinearGradient(this.from.x,this.from.y,this.to.x,this.to.y),s=void 0,a=void 0;return s=this.from.options.color.highlight.border,a=this.to.options.color.highlight.border,this.from.selected===!1&&this.to.selected===!1?(s=r.overrideOpacity(this.from.options.color.border,this.options.color.opacity),a=r.overrideOpacity(this.to.options.color.border,this.options.color.opacity)):this.from.selected===!0&&this.to.selected===!1?a=this.to.options.color.border:this.from.selected===!1&&this.to.selected===!0&&(s=this.from.options.color.border),n.addColorStop(0,s),n.addColorStop(1,a),n}this.colorDirty===!0&&("to"===o.inherit?(this.color.highlight=this.to.options.color.highlight.border,this.color.hover=this.to.options.color.hover.border,this.color.color=r.overrideOpacity(this.to.options.color.border,o.opacity)):(this.color.highlight=this.from.options.color.highlight.border,this.color.hover=this.from.options.color.hover.border,this.color.color=r.overrideOpacity(this.from.options.color.border,o.opacity)))}else this.colorDirty===!0&&(this.color.highlight=o.highlight,this.color.hover=o.hover,this.color.color=r.overrideOpacity(o.color,o.opacity));return this.colorDirty=!1,e===!0?this.color.highlight:i===!0?this.color.hover:this.color.color}},{key:"_circle",value:function(t,e,i,o){this.enableShadow(t),t.beginPath(),t.arc(e,i,o,0,2*Math.PI,!1),t.stroke(),this.disableShadow(t)}},{key:"getDistanceToEdge",value:function(t,e,i,o,s,r,a){var h=0;if(this.from!=this.to)h=this._getDistanceToEdge(t,e,i,o,s,r,a);else{var d=this._getCircleData(),l=n(d,3),c=l[0],u=l[1],p=l[2],f=c-s,m=u-r;h=Math.abs(Math.sqrt(f*f+m*m)-p)}return this.labelModule.size.lefts&&this.labelModule.size.topr?0:h}},{key:"_getDistanceToLine",value:function(t,e,i,o,n,s){var r=i-t,a=o-e,h=r*r+a*a,d=((n-t)*r+(s-e)*a)/h;d>1?d=1:0>d&&(d=0);var l=t+d*r,c=e+d*a,u=l-n,p=c-s;return Math.sqrt(u*u+p*p)}},{key:"getArrowData",value:function(t,e,i,o,s){var r=void 0,a=void 0,h=void 0,d=void 0,l=void 0,c=void 0,u=this.getLineWidth(o,s);if("from"===e?(h=this.from,d=this.to,l=.1,c=this.options.arrows.from.scaleFactor):"to"===e?(h=this.to,d=this.from,l=-.1,c=this.options.arrows.to.scaleFactor):(h=this.to,d=this.from,c=this.options.arrows.middle.scaleFactor),h!=d)if("middle"!==e)if(this.options.smooth.enabled===!0){a=this.findBorderPosition(h,t,{via:i});var p=this.getPoint(Math.max(0,Math.min(1,a.t+l)),i);r=Math.atan2(a.y-p.y,a.x-p.x)}else r=Math.atan2(h.y-d.y,h.x-d.x),a=this.findBorderPosition(h,t);else r=Math.atan2(h.y-d.y,h.x-d.x),a=this.getPoint(.5,i);else{var f=this._getCircleData(t),m=n(f,3),v=m[0],g=m[1],y=m[2];"from"===e?(a=this.findBorderPosition(this.from,t,{x:v,y:g,low:.25,high:.6,direction:-1}),r=-2*a.t*Math.PI+1.5*Math.PI+.1*Math.PI):"to"===e?(a=this.findBorderPosition(this.from,t,{x:v,y:g,low:.6,high:1,direction:1}),r=-2*a.t*Math.PI+1.5*Math.PI-1.1*Math.PI):(a=this._pointOnCircle(v,g,y,.175),r=3.9269908169872414)}var b=15*c+3*u,w=a.x-.9*b*Math.cos(r),_=a.y-.9*b*Math.sin(r),x={x:w,y:_};return{point:a,core:x,angle:r,length:b}}},{key:"drawArrowHead",value:function(t,e,i,o){t.strokeStyle=this.getColor(t,e,i),t.fillStyle=t.strokeStyle,t.lineWidth=this.getLineWidth(e,i),t.arrow(o.point.x,o.point.y,o.angle,o.length),this.enableShadow(t),t.fill(),this.disableShadow(t)}},{key:"enableShadow",value:function(t){this.options.shadow.enabled===!0&&(t.shadowColor=this.options.shadow.color,t.shadowBlur=this.options.shadow.size,t.shadowOffsetX=this.options.shadow.x,t.shadowOffsetY=this.options.shadow.y)}},{key:"disableShadow",value:function(t){this.options.shadow.enabled===!0&&(t.shadowColor="rgba(0,0,0,0)",t.shadowBlur=0,t.shadowOffsetX=0,t.shadowOffsetY=0)}}]),t}();e["default"]=a},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*s,e=this.from.y-i*s):this.from.x>this.to.x&&(t=this.from.x-i*s,e=this.from.y-i*s):this.from.ythis.to.x&&(t=this.from.x-i*s,e=this.from.y+i*s)),"discrete"===o&&(t=i*s>n?this.from.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*n,e=this.from.y-i*n):this.from.x>this.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n)),"discrete"===o&&(e=i*n>s?this.from.y:e));else if("straightCross"===o)Math.abs(this.from.x-this.to.x)<=Math.abs(this.from.y-this.to.y)?(t=this.from.x,e=this.from.yMath.abs(this.from.y-this.to.y)&&(t=this.from.x=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*s,e=this.from.y-i*s,t=this.to.xthis.to.x&&(t=this.from.x-i*s,e=this.from.y-i*s,t=this.to.x>t?this.to.x:t):this.from.ythis.to.x&&(t=this.from.x-i*s,e=this.from.y+i*s,t=this.to.x>t?this.to.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*n,e=this.from.y-i*n,e=this.to.y>e?this.to.y:e):this.from.x>this.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n,e=this.to.y>e?this.to.y:e):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n,e=this.to.y1||this.startedStabilization===!0)&&setTimeout(function(){t.body.emitter.emit("stabilized",{iterations:e}),t.startedStabilization=!1,t.stabilizationIterations=0},0)}},{key:"physicsTick",value:function(){if(this.startedStabilization===!1&&(this.body.emitter.emit("startStabilizing"),this.startedStabilization=!0),this.stabilized===!1){if(this.adaptiveTimestep===!0&&this.adaptiveTimestepEnabled===!0){var t=1.2;this.adaptiveCounter%this.adaptiveInterval===0?(this.timestep=2*this.timestep,this.calculateForces(),this.moveNodes(),this.revert(),this.timestep=.5*this.timestep,this.calculateForces(),this.moveNodes(),this.calculateForces(),this.moveNodes(),this._evaluateStepQuality()===!0?this.timestep=t*this.timestep:this.timestep/ts))return!1;return!0}},{key:"moveNodes",value:function(){for(var t=this.physicsBody.physicsNodeIndices,e=this.options.maxVelocity?this.options.maxVelocity:1e9,i=0,o=0,n=5,s=0;se?s[t].x>0?e:-e:s[t].x,i.x+=s[t].x*o}else n[t].x=0,s[t].x=0;if(i.options.fixed.y===!1){var h=this.modelOptions.damping*s[t].y,d=(n[t].y-h)/i.options.mass;s[t].y+=d*o,s[t].y=Math.abs(s[t].y)>e?s[t].y>0?e:-e:s[t].y,i.y+=s[t].y*o}else n[t].y=0,s[t].y=0;var l=Math.sqrt(Math.pow(s[t].x,2)+Math.pow(s[t].y,2));return l}},{key:"calculateForces",value:function(){this.gravitySolver.solve(),this.nodesSolver.solve(),this.edgesSolver.solve()}},{key:"_freezeNodes",value:function(){var t=this.body.nodes;for(var e in t)t.hasOwnProperty(e)&&t[e].x&&t[e].y&&(this.freezeCache[e]={x:t[e].options.fixed.x,y:t[e].options.fixed.y},t[e].options.fixed.x=!0,t[e].options.fixed.y=!0)}},{key:"_restoreFrozenNodes",value:function(){var t=this.body.nodes;for(var e in t)t.hasOwnProperty(e)&&void 0!==this.freezeCache[e]&&(t[e].options.fixed.x=this.freezeCache[e].x,t[e].options.fixed.y=this.freezeCache[e].y);this.freezeCache={}}},{key:"stabilize",value:function(){var t=this,e=arguments.length<=0||void 0===arguments[0]?this.options.stabilization.iterations:arguments[0];return"number"!=typeof e&&(console.log("The stabilize method needs a numeric amount of iterations. Switching to default: ",this.options.stabilization.iterations),e=this.options.stabilization.iterations),0===this.physicsBody.physicsNodeIndices.length?void(this.ready=!0):(this.adaptiveTimestep=this.options.adaptiveTimestep,this.body.emitter.emit("_resizeNodes"),this.stopSimulation(),this.stabilized=!1,this.body.emitter.emit("_blockRedraw"),this.targetIterations=e,this.options.stabilization.onlyDynamicEdges===!0&&this._freezeNodes(),this.stabilizationIterations=0,void setTimeout(function(){return t._stabilizationBatch()},0))}},{key:"_stabilizationBatch",value:function(){this.startedStabilization===!1&&(this.body.emitter.emit("startStabilizing"),this.startedStabilization=!0);for(var t=0;this.stabilized===!1&&t0){var t=void 0,e=this.body.nodes,i=this.physicsBody.physicsNodeIndices,o=i.length,n=this._formBarnesHutTree(e,i);this.barnesHutTree=n;for(var s=0;o>s;s++)t=e[i[s]],t.options.mass>0&&(this._getForceContribution(n.root.children.NW,t),this._getForceContribution(n.root.children.NE,t),this._getForceContribution(n.root.children.SW,t),this._getForceContribution(n.root.children.SE,t))}}},{key:"_getForceContribution",value:function(t,e){if(t.childrenCount>0){var i=void 0,o=void 0,n=void 0;i=t.centerOfMass.x-e.x,o=t.centerOfMass.y-e.y,n=Math.sqrt(i*i+o*o),n*t.calcSize>this.thetaInversed?this._calculateForces(n,i,o,e,t):4===t.childrenCount?(this._getForceContribution(t.children.NW,e),this._getForceContribution(t.children.NE,e),this._getForceContribution(t.children.SW,e),this._getForceContribution(t.children.SE,e)):t.children.data.id!=e.id&&this._calculateForces(n,i,o,e,t)}}},{key:"_calculateForces",value:function(t,e,i,o,n){0===t&&(t=.1,e=t),this.overlapAvoidanceFactor<1&&(t=Math.max(.1+this.overlapAvoidanceFactor*o.shape.radius,t-o.shape.radius));var s=this.options.gravitationalConstant*n.mass*o.options.mass/Math.pow(t,3),r=e*s,a=i*s;this.physicsBody.forces[o.id].x+=r,this.physicsBody.forces[o.id].y+=a}},{key:"_formBarnesHutTree",value:function(t,e){for(var i=void 0,o=e.length,n=t[e[0]].x,s=t[e[0]].y,r=t[e[0]].x,a=t[e[0]].y,h=1;o>h;h++){var d=t[e[h]].x,l=t[e[h]].y;t[e[h]].options.mass>0&&(n>d&&(n=d),d>r&&(r=d),s>l&&(s=l),l>a&&(a=l))}var c=Math.abs(r-n)-Math.abs(a-s);c>0?(s-=.5*c,a+=.5*c):(n+=.5*c,r-=.5*c);var u=1e-5,p=Math.max(u,Math.abs(r-n)),f=.5*p,m=.5*(n+r),v=.5*(s+a),g={root:{centerOfMass:{x:0,y:0},mass:0,range:{minX:m-f,maxX:m+f,minY:v-f,maxY:v+f},size:p,calcSize:1/p,children:{data:null},maxWidth:0,level:0,childrenCount:4}};this._splitBranch(g.root);for(var y=0;o>y;y++)i=t[e[y]],i.options.mass>0&&this._placeInTree(g.root,i);return g}},{key:"_updateBranchMass",value:function(t,e){var i=t.mass+e.options.mass,o=1/i;t.centerOfMass.x=t.centerOfMass.x*t.mass+e.x*e.options.mass,t.centerOfMass.x*=o,t.centerOfMass.y=t.centerOfMass.y*t.mass+e.y*e.options.mass,t.centerOfMass.y*=o,t.mass=i;var n=Math.max(Math.max(e.height,e.radius),e.width);t.maxWidth=t.maxWidthe.x?t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NW"):this._placeInRegion(t,e,"SW"):t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NE"):this._placeInRegion(t,e,"SE")}},{key:"_placeInRegion",value:function(t,e,i){switch(t.children[i].childrenCount){case 0:t.children[i].children.data=e,t.children[i].childrenCount=1,this._updateBranchMass(t.children[i],e);break;case 1:t.children[i].children.data.x===e.x&&t.children[i].children.data.y===e.y?(e.x+=this.seededRandom(),e.y+=this.seededRandom()):(this._splitBranch(t.children[i]),this._placeInTree(t.children[i],e));break;case 4:this._placeInTree(t.children[i],e)}}},{key:"_splitBranch",value:function(t){var e=null;1===t.childrenCount&&(e=t.children.data,t.mass=0,t.centerOfMass.x=0,t.centerOfMass.y=0),t.childrenCount=4,t.children.data=null,this._insertRegion(t,"NW"),this._insertRegion(t,"NE"),this._insertRegion(t,"SW"),this._insertRegion(t,"SE"),null!=e&&this._placeInTree(t,e)}},{key:"_insertRegion",value:function(t,e){var i=void 0,o=void 0,n=void 0,s=void 0,r=.5*t.size;switch(e){case"NW":i=t.range.minX,o=t.range.minX+r,n=t.range.minY,s=t.range.minY+r;break;case"NE":i=t.range.minX+r,o=t.range.maxX,n=t.range.minY,s=t.range.minY+r;break;case"SW":i=t.range.minX,o=t.range.minX+r,n=t.range.minY+r,s=t.range.maxY;break;case"SE":i=t.range.minX+r,o=t.range.maxX,n=t.range.minY+r,s=t.range.maxY}t.children[e]={centerOfMass:{x:0,y:0},mass:0,range:{minX:i,maxX:o,minY:n,maxY:s},size:.5*t.size,calcSize:2*t.calcSize,children:{data:null},maxWidth:0,level:t.level+1,childrenCount:0}}},{key:"_debug",value:function(t,e){void 0!==this.barnesHutTree&&(t.lineWidth=1,this._drawBranch(this.barnesHutTree.root,t,e))}},{key:"_drawBranch",value:function(t,e,i){void 0===i&&(i="#FF0000"),4===t.childrenCount&&(this._drawBranch(t.children.NW,e),this._drawBranch(t.children.NE,e),this._drawBranch(t.children.SE,e),this._drawBranch(t.children.SW,e)),e.strokeStyle=i,e.beginPath(),e.moveTo(t.range.minX,t.range.minY),e.lineTo(t.range.maxX,t.range.minY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.minY),e.lineTo(t.range.maxX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.maxY), +e.lineTo(t.range.minX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.minX,t.range.maxY),e.lineTo(t.range.minX,t.range.minY),e.stroke()}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ii&&(s=.5*c>i?1:u*i+p,s/=i,o=t*s,n=e*s,l[r.id].x-=o,l[r.id].y-=n,l[a.id].x+=o,l[a.id].y+=n)}}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ii?-Math.pow(f*i,2)+Math.pow(f*p,2):0,0===i?i=.01:s/=i,o=t*s,n=e*s,u[r.id].x-=o,u[r.id].y-=n,u[a.id].x+=o,u[a.id].y+=n}}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;i0){var s=n.edges.length+1,r=this.options.centralGravity*s*n.options.mass;o[n.id].x=e*r,o[n.id].y=i*r}}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i=t&&i.push(n.id)}for(var r=0;r0&&Object.keys(p).length>0&&m===!0&&o.push({nodes:u,edges:p})}}}for(var b=0;bo?r.x:o,n=r.ys?r.y:s;return{x:.5*(i+o),y:.5*(n+s)}}},{key:"openCluster",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!0:arguments[2];if(void 0===t)throw new Error("No clusterNodeId supplied to openCluster.");if(void 0===this.body.nodes[t])throw new Error("The clusterNodeId supplied to openCluster does not exist.");if(void 0===this.body.nodes[t].containedNodes)return void console.log("The node:"+t+" is not a cluster.");var o=this.body.nodes[t],n=o.containedNodes,s=o.containedEdges;if(void 0!==e&&void 0!==e.releaseFunction&&"function"==typeof e.releaseFunction){var r={},a={x:o.x,y:o.y};for(var d in n)if(n.hasOwnProperty(d)){var l=this.body.nodes[d];r[d]={x:l.x,y:l.y}}var u=e.releaseFunction(a,r);for(var p in n)if(n.hasOwnProperty(p)){var f=this.body.nodes[p];void 0!==u[p]&&(f.x=void 0===u[p].x?o.x:u[p].x,f.y=void 0===u[p].y?o.y:u[p].y)}}else for(var m in n)if(n.hasOwnProperty(m)){var v=this.body.nodes[m];v=n[m],v.options.fixed.x===!1&&(v.x=o.x),v.options.fixed.y===!1&&(v.y=o.y)}for(var g in n)if(n.hasOwnProperty(g)){var y=this.body.nodes[g];y.vx=o.vx,y.vy=o.vy,y.setOptions({hidden:!1,physics:!0}),delete this.clusteredNodes[g]}for(var b=[],w=0;wo;)e.push(this.body.nodes[t].id),t=this.clusteredNodes[t].clusterId,o++;return e.push(this.body.nodes[t].id),e.reverse(),e}},{key:"_getConnectedId",value:function(t,e){return t.toId!=e?t.toId:t.fromId!=e?t.fromId:t.fromId}},{key:"_getHubSize",value:function(){for(var t=0,e=0,i=0,o=0,n=0;no&&(o=s.edges.length),t+=s.edges.length,e+=Math.pow(s.edges.length,2),i+=1}t/=i,e/=i;var r=e-Math.pow(t,2),a=Math.sqrt(r),h=Math.floor(t+2*a);return h>o&&(h=o),h}}]),t}();e["default"]=u},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i0)for(var a=0;ae.shape.boundingBox.left&&(s=e.shape.boundingBox.left),re.shape.boundingBox.top&&(o=e.shape.boundingBox.top),n0)for(var a=0;ae.x&&(s=e.x),re.y&&(o=e.y),n0,t.renderTimer=void 0}),this.body.emitter.on("destroy",function(){t.renderRequests=0,t.allowRedraw=!1,t.renderingActive=!1,t.requiresTimeout===!0?clearTimeout(t.renderTimer):cancelAnimationFrame(t.renderTimer),t.body.emitter.off()})}},{key:"setOptions",value:function(t){if(void 0!==t){var e=["hideEdgesOnDrag","hideNodesOnDrag"];s.selectiveDeepExtend(e,this.options,t)}}},{key:"_startRendering",value:function(){this.renderingActive===!0&&void 0===this.renderTimer&&(this.requiresTimeout===!0?this.renderTimer=window.setTimeout(this._renderStep.bind(this),this.simulationInterval):this.renderTimer=window.requestAnimationFrame(this._renderStep.bind(this)))}},{key:"_renderStep",value:function(){this.renderingActive===!0&&(this.renderTimer=void 0,this.requiresTimeout===!0&&this._startRendering(),this._redraw(),this.requiresTimeout===!1&&this._startRendering())}},{key:"redraw",value:function(){this.body.emitter.emit("setSize"),this._redraw()}},{key:"_requestRedraw",value:function(){var t=this;this.redrawRequested!==!0&&this.renderingActive===!1&&this.allowRedraw===!0&&(this.redrawRequested=!0,this.requiresTimeout===!0?window.setTimeout(function(){t._redraw(!1)},0):window.requestAnimationFrame(function(){t._redraw(!1)}))}},{key:"_redraw",value:function(){var t=arguments.length<=0||void 0===arguments[0]?!1:arguments[0];if(this.allowRedraw===!0){this.body.emitter.emit("initRedraw"),this.redrawRequested=!1;var e=this.canvas.frame.canvas.getContext("2d");0!==this.canvas.frame.canvas.width&&0!==this.canvas.frame.canvas.height||this.canvas.setSize(),this.pixelRatio=(window.devicePixelRatio||1)/(e.webkitBackingStorePixelRatio||e.mozBackingStorePixelRatio||e.msBackingStorePixelRatio||e.oBackingStorePixelRatio||e.backingStorePixelRatio||1),e.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0);var i=this.canvas.frame.canvas.clientWidth,o=this.canvas.frame.canvas.clientHeight;if(e.clearRect(0,0,i,o),0===this.canvas.frame.clientWidth)return;e.save(),e.translate(this.body.view.translation.x,this.body.view.translation.y),e.scale(this.body.view.scale,this.body.view.scale),e.beginPath(),this.body.emitter.emit("beforeDrawing",e),e.closePath(),t===!1&&(this.dragging===!1||this.dragging===!0&&this.options.hideEdgesOnDrag===!1)&&this._drawEdges(e),(this.dragging===!1||this.dragging===!0&&this.options.hideNodesOnDrag===!1)&&this._drawNodes(e,t),e.beginPath(),this.body.emitter.emit("afterDrawing",e),e.closePath(),e.restore(),t===!0&&e.clearRect(0,0,i,o)}}},{key:"_resizeNodes",value:function(){var t=this.canvas.frame.canvas.getContext("2d");void 0===this.pixelRatio&&(this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1)),t.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0),t.save(),t.translate(this.body.view.translation.x,this.body.view.translation.y),t.scale(this.body.view.scale,this.body.view.scale);var e=this.body.nodes,i=void 0;for(var o in e)e.hasOwnProperty(o)&&(i=e[o],i.resize(t),i.updateBoundingBox(t,i.selected));t.restore()}},{key:"_drawNodes",value:function(t){for(var e=arguments.length<=1||void 0===arguments[1]?!1:arguments[1],i=this.body.nodes,o=this.body.nodeIndices,n=void 0,s=[],r=20,a=this.canvas.DOMtoCanvas({x:-r,y:-r}),h=this.canvas.DOMtoCanvas({x:this.canvas.frame.canvas.clientWidth+r,y:this.canvas.frame.canvas.clientHeight+r}),d={top:a.y,left:a.x,bottom:h.y,right:h.x},l=0;l0){var t=this.frame.canvas.width/this.pixelRatio/this.cameraState.previousWidth,e=this.frame.canvas.height/this.pixelRatio/this.cameraState.previousHeight,i=this.cameraState.scale;1!=t&&1!=e?i=.5*this.cameraState.scale*(t+e):1!=t?i=this.cameraState.scale*t:1!=e&&(i=this.cameraState.scale*e),this.body.view.scale=i;var o=this.DOMtoCanvas({x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight}),n={x:o.x-this.cameraState.position.x,y:o.y-this.cameraState.position.y};this.body.view.translation.x+=n.x*this.body.view.scale,this.body.view.translation.y+=n.y*this.body.view.scale}}},{key:"_prepareValue",value:function(t){if("number"==typeof t)return t+"px";if("string"==typeof t){if(-1!==t.indexOf("%")||-1!==t.indexOf("px"))return t;if(-1===t.indexOf("%"))return t+"px"}throw new Error("Could not use the value supplied for width or height:"+t)}},{key:"_create",value:function(){for(;this.body.container.hasChildNodes();)this.body.container.removeChild(this.body.container.firstChild);if(this.frame=document.createElement("div"),this.frame.className="vis-network",this.frame.style.position="relative",this.frame.style.overflow="hidden",this.frame.tabIndex=900,this.frame.canvas=document.createElement("canvas"),this.frame.canvas.style.position="relative",this.frame.appendChild(this.frame.canvas),this.frame.canvas.getContext){var t=this.frame.canvas.getContext("2d");this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1), +this.frame.canvas.getContext("2d").setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0)}else{var e=document.createElement("DIV");e.style.color="red",e.style.fontWeight="bold",e.style.padding="10px",e.innerHTML="Error: your browser does not support HTML canvas",this.frame.canvas.appendChild(e)}this.body.container.appendChild(this.frame),this.body.view.scale=1,this.body.view.translation={x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight},this._bindHammer()}},{key:"_bindHammer",value:function(){var t=this;void 0!==this.hammer&&this.hammer.destroy(),this.drag={},this.pinch={},this.hammer=new s(this.frame.canvas),this.hammer.get("pinch").set({enable:!0}),this.hammer.get("pan").set({threshold:5,direction:s.DIRECTION_ALL}),r.onTouch(this.hammer,function(e){t.body.eventListeners.onTouch(e)}),this.hammer.on("tap",function(e){t.body.eventListeners.onTap(e)}),this.hammer.on("doubletap",function(e){t.body.eventListeners.onDoubleTap(e)}),this.hammer.on("press",function(e){t.body.eventListeners.onHold(e)}),this.hammer.on("panstart",function(e){t.body.eventListeners.onDragStart(e)}),this.hammer.on("panmove",function(e){t.body.eventListeners.onDrag(e)}),this.hammer.on("panend",function(e){t.body.eventListeners.onDragEnd(e)}),this.hammer.on("pinch",function(e){t.body.eventListeners.onPinch(e)}),this.frame.canvas.addEventListener("mousewheel",function(e){t.body.eventListeners.onMouseWheel(e)}),this.frame.canvas.addEventListener("DOMMouseScroll",function(e){t.body.eventListeners.onMouseWheel(e)}),this.frame.canvas.addEventListener("mousemove",function(e){t.body.eventListeners.onMouseMove(e)}),this.frame.canvas.addEventListener("contextmenu",function(e){t.body.eventListeners.onContext(e)}),this.hammerFrame=new s(this.frame),r.onRelease(this.hammerFrame,function(e){t.body.eventListeners.onRelease(e)})}},{key:"setSize",value:function(){var t=arguments.length<=0||void 0===arguments[0]?this.options.width:arguments[0],e=arguments.length<=1||void 0===arguments[1]?this.options.height:arguments[1];t=this._prepareValue(t),e=this._prepareValue(e);var i=!1,o=this.frame.canvas.width,n=this.frame.canvas.height,s=this.frame.canvas.getContext("2d"),r=this.pixelRatio;return this.pixelRatio=(window.devicePixelRatio||1)/(s.webkitBackingStorePixelRatio||s.mozBackingStorePixelRatio||s.msBackingStorePixelRatio||s.oBackingStorePixelRatio||s.backingStorePixelRatio||1),t!=this.options.width||e!=this.options.height||this.frame.style.width!=t||this.frame.style.height!=e?(this._getCameraState(r),this.frame.style.width=t,this.frame.style.height=e,this.frame.canvas.style.width="100%",this.frame.canvas.style.height="100%",this.frame.canvas.width=Math.round(this.frame.canvas.clientWidth*this.pixelRatio),this.frame.canvas.height=Math.round(this.frame.canvas.clientHeight*this.pixelRatio),this.options.width=t,this.options.height=e,i=!0):(this.frame.canvas.width==Math.round(this.frame.canvas.clientWidth*this.pixelRatio)&&this.frame.canvas.height==Math.round(this.frame.canvas.clientHeight*this.pixelRatio)||this._getCameraState(r),this.frame.canvas.width!=Math.round(this.frame.canvas.clientWidth*this.pixelRatio)&&(this.frame.canvas.width=Math.round(this.frame.canvas.clientWidth*this.pixelRatio),i=!0),this.frame.canvas.height!=Math.round(this.frame.canvas.clientHeight*this.pixelRatio)&&(this.frame.canvas.height=Math.round(this.frame.canvas.clientHeight*this.pixelRatio),i=!0)),i===!0&&(this.body.emitter.emit("resize",{width:Math.round(this.frame.canvas.width/this.pixelRatio),height:Math.round(this.frame.canvas.height/this.pixelRatio),oldWidth:Math.round(o/this.pixelRatio),oldHeight:Math.round(n/this.pixelRatio)}),this._setCameraState()),this.initialized=!0,i}},{key:"_XconvertDOMtoCanvas",value:function(t){return(t-this.body.view.translation.x)/this.body.view.scale}},{key:"_XconvertCanvasToDOM",value:function(t){return t*this.body.view.scale+this.body.view.translation.x}},{key:"_YconvertDOMtoCanvas",value:function(t){return(t-this.body.view.translation.y)/this.body.view.scale}},{key:"_YconvertCanvasToDOM",value:function(t){return t*this.body.view.scale+this.body.view.translation.y}},{key:"canvasToDOM",value:function(t){return{x:this._XconvertCanvasToDOM(t.x),y:this._YconvertCanvasToDOM(t.y)}}},{key:"DOMtoCanvas",value:function(t){return{x:this._XconvertDOMtoCanvas(t.x),y:this._YconvertDOMtoCanvas(t.y)}}}]),t}();e["default"]=h},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;i.5*this.body.nodeIndices.length)return void this.fit(t,!1);i=a["default"].getRange(this.body.nodes,t.nodes);var h=this.body.nodeIndices.length;o=12.662/(h+7.4147)+.0964822;var d=Math.min(this.canvas.frame.canvas.clientWidth/600,this.canvas.frame.canvas.clientHeight/600);o*=d}else{this.body.emitter.emit("_resizeNodes"),i=a["default"].getRange(this.body.nodes,t.nodes);var l=1.1*Math.abs(i.maxX-i.minX),c=1.1*Math.abs(i.maxY-i.minY),u=this.canvas.frame.canvas.clientWidth/l,p=this.canvas.frame.canvas.clientHeight/c;o=p>=u?u:p}o>1?o=1:0===o&&(o=1);var f=a["default"].findCenter(i),m={position:f,scale:o,animation:t.animation};this.moveTo(m)}},{key:"focus",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?{}:arguments[1];if(void 0!==this.body.nodes[t]){var i={x:this.body.nodes[t].x,y:this.body.nodes[t].y};e.position=i,e.lockedOnNode=t,this.moveTo(e)}else console.log("Node: "+t+" cannot be found.")}},{key:"moveTo",value:function(t){return void 0===t?void(t={}):(void 0===t.offset&&(t.offset={x:0,y:0}),void 0===t.offset.x&&(t.offset.x=0),void 0===t.offset.y&&(t.offset.y=0),void 0===t.scale&&(t.scale=this.body.view.scale),void 0===t.position&&(t.position=this.getViewPosition()),void 0===t.animation&&(t.animation={duration:0}),t.animation===!1&&(t.animation={duration:0}),t.animation===!0&&(t.animation={}),void 0===t.animation.duration&&(t.animation.duration=1e3),void 0===t.animation.easingFunction&&(t.animation.easingFunction="easeInOutQuad"),void this.animateView(t))}},{key:"animateView",value:function(t){if(void 0!==t){this.animationEasingFunction=t.animation.easingFunction,this.releaseNode(),t.locked===!0&&(this.lockedOnNodeId=t.lockedOnNode,this.lockedOnNodeOffset=t.offset),0!=this.easingTime&&this._transitionRedraw(!0),this.sourceScale=this.body.view.scale,this.sourceTranslation=this.body.view.translation,this.targetScale=t.scale,this.body.view.scale=this.targetScale;var e=this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight}),i={x:e.x-t.position.x,y:e.y-t.position.y};this.targetTranslation={x:this.sourceTranslation.x+i.x*this.targetScale+t.offset.x,y:this.sourceTranslation.y+i.y*this.targetScale+t.offset.y},0===t.animation.duration?void 0!=this.lockedOnNodeId?(this.viewFunction=this._lockedRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction)):(this.body.view.scale=this.targetScale,this.body.view.translation=this.targetTranslation,this.body.emitter.emit("_requestRedraw")):(this.animationSpeed=1/(60*t.animation.duration*.001)||1/60,this.animationEasingFunction=t.animation.easingFunction,this.viewFunction=this._transitionRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction),this.body.emitter.emit("_startRendering"))}}},{key:"_lockedRedraw",value:function(){var t={x:this.body.nodes[this.lockedOnNodeId].x,y:this.body.nodes[this.lockedOnNodeId].y},e=this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight}),i={x:e.x-t.x,y:e.y-t.y},o=this.body.view.translation,n={x:o.x+i.x*this.body.view.scale+this.lockedOnNodeOffset.x,y:o.y+i.y*this.body.view.scale+this.lockedOnNodeOffset.y};this.body.view.translation=n}},{key:"releaseNode",value:function(){void 0!==this.lockedOnNodeId&&void 0!==this.viewFunction&&(this.body.emitter.off("initRedraw",this.viewFunction),this.lockedOnNodeId=void 0,this.lockedOnNodeOffset=void 0)}},{key:"_transitionRedraw",value:function(){var t=arguments.length<=0||void 0===arguments[0]?!1:arguments[0];this.easingTime+=this.animationSpeed,this.easingTime=t===!0?1:this.easingTime;var e=h.easingFunctions[this.animationEasingFunction](this.easingTime);this.body.view.scale=this.sourceScale+(this.targetScale-this.sourceScale)*e,this.body.view.translation={x:this.sourceTranslation.x+(this.targetTranslation.x-this.sourceTranslation.x)*e,y:this.sourceTranslation.y+(this.targetTranslation.y-this.sourceTranslation.y)*e},this.easingTime>=1&&(this.body.emitter.off("initRedraw",this.viewFunction),this.easingTime=0,void 0!=this.lockedOnNodeId&&(this.viewFunction=this._lockedRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction)),this.body.emitter.emit("animationFinished"))}},{key:"getScale",value:function(){return this.body.view.scale}},{key:"getViewPosition",value:function(){return this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight})}}]),t}();e["default"]=d},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;i50&&(this.drag.pointer=this.getPointer(t.center),this.drag.pinched=!1,this.pinch.scale=this.body.view.scale,this.touchTime=(new Date).valueOf())}},{key:"onTap",value:function(t){var e=this.getPointer(t.center),i=this.selectionHandler.options.multiselect&&(t.changedPointers[0].ctrlKey||t.changedPointers[0].metaKey);this.checkSelectionChanges(e,t,i),this.selectionHandler._generateClickEvent("click",t,e)}},{key:"onDoubleTap",value:function(t){var e=this.getPointer(t.center);this.selectionHandler._generateClickEvent("doubleClick",t,e)}},{key:"onHold",value:function(t){var e=this.getPointer(t.center),i=this.selectionHandler.options.multiselect;this.checkSelectionChanges(e,t,i),this.selectionHandler._generateClickEvent("click",t,e),this.selectionHandler._generateClickEvent("hold",t,e)}},{key:"onRelease",value:function(t){if((new Date).valueOf()-this.touchTime>10){var e=this.getPointer(t.center);this.selectionHandler._generateClickEvent("release",t,e),this.touchTime=(new Date).valueOf()}}},{key:"onContext",value:function(t){var e=this.getPointer({x:t.clientX,y:t.clientY});this.selectionHandler._generateClickEvent("oncontext",t,e)}},{key:"checkSelectionChanges",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=this.selectionHandler._getSelectedEdgeCount(),n=this.selectionHandler._getSelectedNodeCount(),s=this.selectionHandler.getSelection(),r=void 0;r=i===!0?this.selectionHandler.selectAdditionalOnPoint(t):this.selectionHandler.selectOnPoint(t);var a=this.selectionHandler._getSelectedEdgeCount(),h=this.selectionHandler._getSelectedNodeCount(),d=this.selectionHandler.getSelection(),l=this._determineIfDifferent(s,d),c=l.nodesChanged,u=l.edgesChanged,p=!1;h-n>0?(this.selectionHandler._generateClickEvent("selectNode",e,t),r=!0,p=!0):c===!0&&h>0?(this.selectionHandler._generateClickEvent("deselectNode",e,t,s),this.selectionHandler._generateClickEvent("selectNode",e,t),p=!0,r=!0):0>h-n&&(this.selectionHandler._generateClickEvent("deselectNode",e,t,s),r=!0),a-o>0&&p===!1?(this.selectionHandler._generateClickEvent("selectEdge",e,t),r=!0):a>0&&u===!0?(this.selectionHandler._generateClickEvent("deselectEdge",e,t,s),this.selectionHandler._generateClickEvent("selectEdge",e,t),r=!0):0>a-o&&(this.selectionHandler._generateClickEvent("deselectEdge",e,t,s),r=!0),r===!0&&this.selectionHandler._generateClickEvent("select",e,t)}},{key:"_determineIfDifferent",value:function(t,e){for(var i=!1,o=!1,n=0;nt&&(t=1e-5),t>10&&(t=10);var o=void 0;void 0!==this.drag&&this.drag.dragging===!0&&(o=this.canvas.DOMtoCanvas(this.drag.pointer));var n=this.body.view.translation,s=t/i,r=(1-s)*e.x+n.x*s,a=(1-s)*e.y+n.y*s;if(this.body.view.scale=t,this.body.view.translation={x:r,y:a},void 0!=o){var h=this.canvas.canvasToDOM(o);this.drag.pointer.x=h.x,this.drag.pointer.y=h.y}this.body.emitter.emit("_requestRedraw"),t>i?this.body.emitter.emit("zoom",{direction:"+",scale:this.body.view.scale}):this.body.emitter.emit("zoom",{direction:"-",scale:this.body.view.scale})}}},{key:"onMouseWheel",value:function(t){if(this.options.zoomView===!0){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),0!==e){var i=this.body.view.scale,o=e/10;0>e&&(o/=1-o),i*=1+o;var n=this.getPointer({x:t.clientX,y:t.clientY});this.zoom(i,n)}t.preventDefault()}}},{key:"onMouseMove",value:function(t){var e=this,i=this.getPointer({x:t.clientX,y:t.clientY}),o=!1;if(void 0!==this.popup&&(this.popup.hidden===!1&&this._checkHidePopup(i),this.popup.hidden===!1&&(o=!0,this.popup.setPosition(i.x+3,i.y-5),this.popup.show())),this.options.keyboard.bindToWindow===!1&&this.options.keyboard.enabled===!0&&this.canvas.frame.focus(),o===!1&&(void 0!==this.popupTimer&&(clearInterval(this.popupTimer),this.popupTimer=void 0),this.drag.dragging||(this.popupTimer=setTimeout(function(){return e._checkShowPopup(i)},this.options.tooltipDelay))),this.options.hover===!0){var n=this.selectionHandler.getNodeAt(i);void 0===n&&(n=this.selectionHandler.getEdgeAt(i)),this.selectionHandler.hoverObject(n)}}},{key:"_checkShowPopup",value:function(t){var e=this.canvas._XconvertDOMtoCanvas(t.x),i=this.canvas._YconvertDOMtoCanvas(t.y),o={left:e,top:i,right:e,bottom:i},n=void 0===this.popupObj?void 0:this.popupObj.id,s=!1,r="node";if(void 0===this.popupObj){for(var a=this.body.nodeIndices,h=this.body.nodes,l=void 0,c=[],u=0;u0&&(this.popupObj=h[c[c.length-1]],s=!0)}if(void 0===this.popupObj&&s===!1){for(var p=this.body.edgeIndices,f=this.body.edges,m=void 0,v=[],g=0;g0&&(this.popupObj=f[v[v.length-1]],r="edge")}void 0!==this.popupObj?this.popupObj.id!==n&&(void 0===this.popup&&(this.popup=new d["default"](this.canvas.frame)),this.popup.popupTargetType=r,this.popup.popupTargetId=this.popupObj.id,this.popup.setPosition(t.x+3,t.y-5),this.popup.setText(this.popupObj.getTitle()),this.popup.show(),this.body.emitter.emit("showPopup",this.popupObj.id)):void 0!==this.popup&&(this.popup.hide(),this.body.emitter.emit("hidePopup"))}},{key:"_checkHidePopup",value:function(t){var e=this.selectionHandler._pointerToPositionObject(t),i=!1;if("node"===this.popup.popupTargetType){if(void 0!==this.body.nodes[this.popup.popupTargetId]&&(i=this.body.nodes[this.popup.popupTargetId].isOverlappingWith(e),i===!0)){var o=this.selectionHandler.getNodeAt(t);i=o.id===this.popup.popupTargetId}}else void 0===this.selectionHandler.getNodeAt(t)&&void 0!==this.body.edges[this.popup.popupTargetId]&&(i=this.body.edges[this.popup.popupTargetId].isOverlappingWith(e));i===!1&&(this.popupObj=void 0,this.popup.hide(),this.body.emitter.emit("hidePopup"))}}]),t}();e["default"]=c},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i700&&(this.body.emitter.emit("fit",{duration:700}),this.touchTime=(new Date).valueOf())}},{key:"_stopMovement",value:function(){for(var t in this.boundFunctions)this.boundFunctions.hasOwnProperty(t)&&(this.body.emitter.off("initRedraw",this.boundFunctions[t]),this.body.emitter.emit("_stopRendering"));this.boundFunctions={}}},{key:"_moveUp",value:function(){this.body.view.translation.y+=this.options.keyboard.speed.y}},{key:"_moveDown",value:function(){this.body.view.translation.y-=this.options.keyboard.speed.y}},{key:"_moveLeft",value:function(){this.body.view.translation.x+=this.options.keyboard.speed.x}},{key:"_moveRight",value:function(){this.body.view.translation.x-=this.options.keyboard.speed.x}},{key:"_zoomIn",value:function(){this.body.view.scale*=1+this.options.keyboard.speed.zoom,this.body.emitter.emit("zoom",{direction:"+",scale:this.body.view.scale})}},{key:"_zoomOut",value:function(){this.body.view.scale/=1+this.options.keyboard.speed.zoom,this.body.emitter.emit("zoom",{direction:"-",scale:this.body.view.scale})}},{key:"configureKeyboardBindings",value:function(){var t=this;void 0!==this.keycharm&&this.keycharm.destroy(),this.options.keyboard.enabled===!0&&(this.options.keyboard.bindToWindow===!0?this.keycharm=a({container:window,preventDefault:!0}):this.keycharm=a({container:this.canvas.frame,preventDefault:!0}),this.keycharm.reset(),this.activated===!0&&(this.keycharm.bind("up",function(){t.bindToRedraw("_moveUp")},"keydown"),this.keycharm.bind("down",function(){t.bindToRedraw("_moveDown")},"keydown"),this.keycharm.bind("left",function(){t.bindToRedraw("_moveLeft")},"keydown"),this.keycharm.bind("right",function(){t.bindToRedraw("_moveRight")},"keydown"),this.keycharm.bind("=",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("num+",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("num-",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("-",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("[",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("]",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("pageup",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("pagedown",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("up",function(){t.unbindFromRedraw("_moveUp")},"keyup"),this.keycharm.bind("down",function(){t.unbindFromRedraw("_moveDown")},"keyup"),this.keycharm.bind("left",function(){t.unbindFromRedraw("_moveLeft")},"keyup"),this.keycharm.bind("right",function(){t.unbindFromRedraw("_moveRight")},"keyup"),this.keycharm.bind("=",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("num+",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("num-",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("-",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("[",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("]",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("pageup",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("pagedown",function(){t.unbindFromRedraw("_zoomOut")},"keyup")))}}]),t}();e["default"]=h},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;io&&(s=o-e-this.padding),sn&&(r=n-i-this.padding),r0?e===!0?this.body.nodes[o[o.length-1]]:o[o.length-1]:void 0}},{key:"_getEdgesOverlappingWith",value:function(t,e){for(var i=this.body.edges,o=0;o0?e===!0?this.body.edges[o[o.length-1]]:o[o.length-1]:void 0}},{key:"_addToSelection",value:function(t){t instanceof a["default"]?this.selectionObj.nodes[t.id]=t:this.selectionObj.edges[t.id]=t}},{key:"_addToHover",value:function(t){t instanceof a["default"]?this.hoverObj.nodes[t.id]=t:this.hoverObj.edges[t.id]=t}},{key:"_removeFromSelection",value:function(t){t instanceof a["default"]?(delete this.selectionObj.nodes[t.id],this._unselectConnectedEdges(t)):delete this.selectionObj.edges[t.id]}},{key:"unselectAll",value:function(){for(var t in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].unselect();for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&this.selectionObj.edges[e].unselect();this.selectionObj={nodes:{},edges:{}}}},{key:"_getSelectedNodeCount",value:function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);return t}},{key:"_getSelectedNode",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return this.selectionObj.nodes[t]}},{key:"_getSelectedEdge",value:function(){for(var t in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(t))return this.selectionObj.edges[t]}},{key:"_getSelectedEdgeCount",value:function(){var t=0;for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(t+=1);return t}},{key:"_getSelectedObjectCount",value:function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&(t+=1);return t}},{key:"_selectionIsEmpty",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return!1;for(var e in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(e))return!1;return!0}},{key:"_clusterInSelection",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].clusterSize>1)return!0;return!1}},{key:"_selectConnectedEdges",value:function(t){for(var e=0;e0&&(this.options.hierarchical.levelSeparation*=-1):this.options.hierarchical.levelSeparation<0&&(this.options.hierarchical.levelSeparation*=-1),this.body.emitter.emit("_resetHierarchicalLayout"),this.adaptAllOptionsForHierarchicalLayout(e);if(i===!0)return this.body.emitter.emit("refresh"),l.deepExtend(e,this.optionsBackup)}return e}},{key:"adaptAllOptionsForHierarchicalLayout",value:function(t){if(this.options.hierarchical.enabled===!0){void 0===t.physics||t.physics===!0?(t.physics={enabled:void 0===this.optionsBackup.physics.enabled?!0:this.optionsBackup.physics.enabled,solver:"hierarchicalRepulsion"},this.optionsBackup.physics.enabled=void 0===this.optionsBackup.physics.enabled?!0:this.optionsBackup.physics.enabled,this.optionsBackup.physics.solver=this.optionsBackup.physics.solver||"barnesHut"):"object"===r(t.physics)?(this.optionsBackup.physics.enabled=void 0===t.physics.enabled?!0:t.physics.enabled,this.optionsBackup.physics.solver=t.physics.solver||"barnesHut",t.physics.solver="hierarchicalRepulsion"):t.physics!==!1&&(this.optionsBackup.physics.solver="barnesHut",t.physics={solver:"hierarchicalRepulsion"});var e="horizontal";"RL"!==this.options.hierarchical.direction&&"LR"!==this.options.hierarchical.direction||(e="vertical"),void 0===t.edges?(this.optionsBackup.edges={smooth:{enabled:!0,type:"dynamic"}},t.edges={smooth:!1}):void 0===t.edges.smooth?(this.optionsBackup.edges={smooth:{enabled:!0,type:"dynamic"}},t.edges.smooth=!1):"boolean"==typeof t.edges.smooth?(this.optionsBackup.edges={smooth:t.edges.smooth},t.edges.smooth={enabled:t.edges.smooth,type:e}):(void 0!==t.edges.smooth.type&&"dynamic"!==t.edges.smooth.type&&(e=t.edges.smooth.type),this.optionsBackup.edges={smooth:void 0===t.edges.smooth.enabled?!0:t.edges.smooth.enabled,type:void 0===t.edges.smooth.type?"dynamic":t.edges.smooth.type,roundness:void 0===t.edges.smooth.roundness?.5:t.edges.smooth.roundness,forceDirection:void 0===t.edges.smooth.forceDirection?!1:t.edges.smooth.forceDirection},t.edges.smooth={enabled:void 0===t.edges.smooth.enabled?!0:t.edges.smooth.enabled,type:e,roundness:void 0===t.edges.smooth.roundness?.5:t.edges.smooth.roundness,forceDirection:void 0===t.edges.smooth.forceDirection?!1:t.edges.smooth.forceDirection}),this.body.emitter.emit("_forceDisableDynamicCurves",e)}return t}},{key:"seededRandom",value:function(){var t=1e4*Math.sin(this.randomSeed++);return t-Math.floor(t)}},{key:"positionInitially",value:function(t){if(this.options.hierarchical.enabled!==!0){this.randomSeed=this.initialRandomSeed;for(var e=0;es){for(var r=this.body.nodeIndices.length;this.body.nodeIndices.length>s;){n+=1;var a=this.body.nodeIndices.length;n%3===0?this.body.modules.clustering.clusterBridges():this.body.modules.clustering.clusterOutliers();var h=this.body.nodeIndices.length;if(a==h&&n%3!==0||n>o)return this._declusterAll(),this.body.emitter.emit("_layoutFailed"),void console.info("This network could not be positioned by this version of the improved layout algorithm. Please disable improvedLayout for better performance.")}this.body.modules.kamadaKawai.setOptions({springLength:Math.max(150,2*r)})}this.body.modules.kamadaKawai.solve(this.body.nodeIndices,this.body.edgeIndices,!0),this._shiftToCenter();for(var d=70,l=0;l0){var t=void 0,e=void 0,i=!1,o=!0,n=!1;this.hierarchicalLevels={},this.lastNodeOnLevel={},this.hierarchicalChildrenReference={},this.hierarchicalParentReference={},this.hierarchicalTrees={},this.treeIndex=-1,this.distributionOrdering={},this.distributionIndex={},this.distributionOrderingPresence={};for(e in this.body.nodes)this.body.nodes.hasOwnProperty(e)&&(t=this.body.nodes[e],void 0===t.options.x&&void 0===t.options.y&&(o=!1),void 0!==t.options.level?(i=!0,this.hierarchicalLevels[e]=t.options.level):n=!0);if(n===!0&&i===!0)throw new Error("To use the hierarchical layout, nodes require either no predefined levels or levels have to be defined for all nodes.");n===!0&&("hubsize"===this.options.hierarchical.sortMethod?this._determineLevelsByHubsize():"directed"===this.options.hierarchical.sortMethod?this._determineLevelsDirected():"custom"===this.options.hierarchical.sortMethod&&this._determineLevelsCustomCallback());for(var s in this.body.nodes)this.body.nodes.hasOwnProperty(s)&&void 0===this.hierarchicalLevels[s]&&(this.hierarchicalLevels[s]=0);var r=this._getDistribution();this._generateMap(),this._placeNodesByHierarchy(r),this._condenseHierarchy(),this._shiftToCenter()}}},{key:"_condenseHierarchy",value:function(){var t=this,e=!1,i={},o=function(){for(var e=a(),i=0;i0)for(var n=0;n=l&&(r=Math.min(c,r),a=Math.max(c,a))}return[r,a,o,n]},l=function _(e){var i=t.hierarchicalLevels[e];if(t.hierarchicalChildrenReference[e]){var o=t.hierarchicalChildrenReference[e];if(o.length>0)for(var n=0;n1)for(var a=0;at.options.hierarchical.nodeSpacing){var u={};u[i.id]=!0;var p={};p[o.id]=!0,h(i,u),h(o,p);var f=c(i,o),m=d(u,f),v=s(m,4),g=(v[0],v[1]),y=(v[2],v[3],d(p,f)),b=s(y,4),w=b[0],_=(b[1],b[2]),x=(b[3],Math.abs(g-w));if(x>t.options.hierarchical.nodeSpacing){var k=g-w+t.options.hierarchical.nodeSpacing;k<-_+t.options.hierarchical.nodeSpacing&&(k=-_+t.options.hierarchical.nodeSpacing),0>k&&(t._shiftBlock(o.id,k),e=!0,n===!0&&t._centerParent(o))}}},m=function(o,n){for(var r=n.id,a=n.edges,l=t.hierarchicalLevels[n.id],c=t.options.hierarchical.levelSeparation*t.options.hierarchical.levelSeparation,u={},p=[],f=0;fr;r++){var a=g(o,i),h=y(o,i),d=40,l=Math.max(-d,Math.min(d,Math.round(a/h)));if(o-=l,void 0!==s[o])break;s[o]=r}return o},w=function(o){var r=t._getPositionForHierarchy(n);if(void 0===i[n.id]){var a={};a[n.id]=!0,h(n,a),i[n.id]=a}var l=d(i[n.id]),c=s(l,4),u=(c[0],c[1],c[2]),p=c[3],f=o-r,m=0;f>0?m=Math.min(f,p-t.options.hierarchical.nodeSpacing):0>f&&(m=-Math.min(-f,u-t.options.hierarchical.nodeSpacing)),0!=m&&(t._shiftBlock(n.id,m),e=!0)},_=function(i){var o=t._getPositionForHierarchy(n),r=t._getSpaceAroundNode(n),a=s(r,2),h=a[0],d=a[1],l=i-o,c=o;l>0?c=Math.min(o+(d-t.options.hierarchical.nodeSpacing),i):0>l&&(c=Math.max(o-(h-t.options.hierarchical.nodeSpacing),i)),c!==o&&(t._setPositionForHierarchy(n,c,void 0,!0),e=!0)},x=b(o,p);w(x),x=b(o,a),_(x)},v=function(i){var o=Object.keys(t.distributionOrdering);o=o.reverse();for(var n=0;i>n;n++){e=!1;for(var s=0;sn&&(e=!1,p(f,o,!0),e===!0);n++);},y=function(){for(var e in t.body.nodes)t.body.nodes.hasOwnProperty(e)&&t._centerParent(t.body.nodes[e])},b=function(){var e=Object.keys(t.distributionOrdering);e=e.reverse();for(var i=0;i0)for(var d=0;dg&&Math.abs(g)0&&Math.abs(g)0&&(r=this._getPositionForHierarchy(i[n-1])+this.options.hierarchical.nodeSpacing),this._setPositionForHierarchy(s,r,e),this._validataPositionAndContinue(s,e,r),o++}}}}},{key:"_placeBranchNodes",value:function(t,e){if(void 0!==this.hierarchicalChildrenReference[t]){for(var i=[],o=0;oe&&void 0===this.positionedNodes[s.id]))return;var a=void 0;a=0===n?this._getPositionForHierarchy(this.body.nodes[t]):this._getPositionForHierarchy(i[n-1])+this.options.hierarchical.nodeSpacing,this._setPositionForHierarchy(s,a,r),this._validataPositionAndContinue(s,r,a)}for(var h=1e9,d=-1e9,l=0;l0&&(e=this._getHubSize(),0!==e);)for(var o in this.body.nodes)if(this.body.nodes.hasOwnProperty(o)){var n=this.body.nodes[o];n.edges.length===e&&this._crawlNetwork(i,o)}}},{key:"_determineLevelsCustomCallback",value:function(){var t=this,e=1e5,i=function(t,e,i){},o=function(o,n,s){var r=t.hierarchicalLevels[o.id];void 0===r&&(t.hierarchicalLevels[o.id]=e);var a=i(d["default"].cloneOptions(o,"node"),d["default"].cloneOptions(n,"node"),d["default"].cloneOptions(s,"edge"));t.hierarchicalLevels[n.id]=t.hierarchicalLevels[o.id]+a};this._crawlNetwork(o),this._setMinLevelToZero()}},{key:"_determineLevelsDirected",value:function(){var t=this,e=1e4,i=function(i,o,n){var s=t.hierarchicalLevels[i.id];void 0===s&&(t.hierarchicalLevels[i.id]=e),n.toId==o.id?t.hierarchicalLevels[o.id]=t.hierarchicalLevels[i.id]+1:t.hierarchicalLevels[o.id]=t.hierarchicalLevels[i.id]-1};this._crawlNetwork(i),this._setMinLevelToZero()}},{key:"_setMinLevelToZero",value:function(){var t=1e9;for(var e in this.body.nodes)this.body.nodes.hasOwnProperty(e)&&void 0!==this.hierarchicalLevels[e]&&(t=Math.min(this.hierarchicalLevels[e],t));for(var i in this.body.nodes)this.body.nodes.hasOwnProperty(i)&&void 0!==this.hierarchicalLevels[i]&&(this.hierarchicalLevels[i]-=t)}},{key:"_generateMap",value:function(){var t=this,e=function(e,i){if(t.hierarchicalLevels[i.id]>t.hierarchicalLevels[e.id]){var o=e.id,n=i.id;void 0===t.hierarchicalChildrenReference[o]&&(t.hierarchicalChildrenReference[o]=[]),t.hierarchicalChildrenReference[o].push(n),void 0===t.hierarchicalParentReference[n]&&(t.hierarchicalParentReference[n]=[]),t.hierarchicalParentReference[n].push(o)}};this._crawlNetwork(e)}},{key:"_crawlNetwork",value:function(){var t=this,e=arguments.length<=0||void 0===arguments[0]?function(){}:arguments[0],i=arguments[1],o={},n=0,s=function d(i,n){if(void 0===o[i.id]){void 0===t.hierarchicalTrees[i.id]&&(t.hierarchicalTrees[i.id]=n,t.treeIndex=Math.max(n,t.treeIndex)),o[i.id]=!0;for(var s=void 0,r=0;r1&&("UD"===this.options.hierarchical.direction||"DU"===this.options.hierarchical.direction?t.sort(function(t,e){return t.x-e.x}):t.sort(function(t,e){return t.y-e.y}))}}]),t}();e["default"]=c},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i0&&this.options.deleteNode!==!1?(n===!0&&this._createSeperator(4),this._createDeleteButton(o)):0===t&&this.options.deleteEdge!==!1&&(n===!0&&this._createSeperator(4),this._createDeleteButton(o))),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this)),this._temporaryBindEvent("select",this.showManipulatorToolbar.bind(this))}this.body.emitter.emit("_redraw")}},{key:"addNodeMode",value:function(){if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="addNode",this.guiEnabled===!0){var t=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(t),this._createSeperator(),this._createDescription(t.addDescription||this.options.locales.en.addDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this._temporaryBindEvent("click",this._performAddNode.bind(this))}},{key:"editNode",value:function(){var t=this;this.editMode!==!0&&this.enableEditMode(),this._clean();var e=this.selectionHandler._getSelectedNode();if(void 0!==e){if(this.inMode="editNode","function"!=typeof this.options.editNode)throw new Error("No function has been configured to handle the editing of nodes.");if(e.isCluster!==!0){var i=s.deepExtend({},e.options,!1); +if(i.x=e.x,i.y=e.y,2!==this.options.editNode.length)throw new Error("The function for edit does not support two arguments (data, callback)");this.options.editNode(i,function(e){null!==e&&void 0!==e&&"editNode"===t.inMode&&t.body.data.nodes.getDataSet().update(e),t.showManipulatorToolbar()})}else alert(this.options.locales[this.options.locale].editClusterError||this.options.locales.en.editClusterError)}else this.showManipulatorToolbar()}},{key:"addEdgeMode",value:function(){if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="addEdge",this.guiEnabled===!0){var t=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(t),this._createSeperator(),this._createDescription(t.edgeDescription||this.options.locales.en.edgeDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this._temporaryBindUI("onTouch",this._handleConnect.bind(this)),this._temporaryBindUI("onDragEnd",this._finishConnect.bind(this)),this._temporaryBindUI("onDrag",this._dragControlNode.bind(this)),this._temporaryBindUI("onRelease",this._finishConnect.bind(this)),this._temporaryBindUI("onDragStart",function(){}),this._temporaryBindUI("onHold",function(){})}},{key:"editEdgeMode",value:function(){var t=this;if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="editEdge",this.guiEnabled===!0){var e=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(e),this._createSeperator(),this._createDescription(e.editEdgeDescription||this.options.locales.en.editEdgeDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this.edgeBeingEditedId=this.selectionHandler.getSelectedEdges()[0],void 0!==this.edgeBeingEditedId?!function(){var e=t.body.edges[t.edgeBeingEditedId],i=t._getNewTargetNode(e.from.x,e.from.y),o=t._getNewTargetNode(e.to.x,e.to.y);t.temporaryIds.nodes.push(i.id),t.temporaryIds.nodes.push(o.id),t.body.nodes[i.id]=i,t.body.nodeIndices.push(i.id),t.body.nodes[o.id]=o,t.body.nodeIndices.push(o.id),t._temporaryBindUI("onTouch",t._controlNodeTouch.bind(t)),t._temporaryBindUI("onTap",function(){}),t._temporaryBindUI("onHold",function(){}),t._temporaryBindUI("onDragStart",t._controlNodeDragStart.bind(t)),t._temporaryBindUI("onDrag",t._controlNodeDrag.bind(t)),t._temporaryBindUI("onDragEnd",t._controlNodeDragEnd.bind(t)),t._temporaryBindUI("onMouseMove",function(){}),t._temporaryBindEvent("beforeDrawing",function(t){var n=e.edgeType.findBorderPositions(t);i.selected===!1&&(i.x=n.from.x,i.y=n.from.y),o.selected===!1&&(o.x=n.to.x,o.y=n.to.y)}),t.body.emitter.emit("_redraw")}():this.showManipulatorToolbar()}},{key:"deleteSelected",value:function(){var t=this;this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="delete";var e=this.selectionHandler.getSelectedNodes(),i=this.selectionHandler.getSelectedEdges(),o=void 0;if(e.length>0){for(var n=0;n0&&"function"==typeof this.options.deleteEdge&&(o=this.options.deleteEdge);if("function"==typeof o){var s={nodes:e,edges:i};if(2!==o.length)throw new Error("The function for delete does not support two arguments (data, callback)");o(s,function(e){null!==e&&void 0!==e&&"delete"===t.inMode?(t.body.data.edges.getDataSet().remove(e.edges),t.body.data.nodes.getDataSet().remove(e.nodes),t.body.emitter.emit("startSimulation"),t.showManipulatorToolbar()):(t.body.emitter.emit("startSimulation"),t.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().remove(i),this.body.data.nodes.getDataSet().remove(e),this.body.emitter.emit("startSimulation"),this.showManipulatorToolbar()}},{key:"_setup",value:function(){this.options.enabled===!0?(this.guiEnabled=!0,this._createWrappers(),this.editMode===!1?this._createEditButton():this.showManipulatorToolbar()):(this._removeManipulationDOM(),this.guiEnabled=!1)}},{key:"_createWrappers",value:function(){void 0===this.manipulationDiv&&(this.manipulationDiv=document.createElement("div"),this.manipulationDiv.className="vis-manipulation",this.editMode===!0?this.manipulationDiv.style.display="block":this.manipulationDiv.style.display="none",this.canvas.frame.appendChild(this.manipulationDiv)),void 0===this.editModeDiv&&(this.editModeDiv=document.createElement("div"),this.editModeDiv.className="vis-edit-mode",this.editMode===!0?this.editModeDiv.style.display="none":this.editModeDiv.style.display="block",this.canvas.frame.appendChild(this.editModeDiv)),void 0===this.closeDiv&&(this.closeDiv=document.createElement("div"),this.closeDiv.className="vis-close",this.closeDiv.style.display=this.manipulationDiv.style.display,this.canvas.frame.appendChild(this.closeDiv))}},{key:"_getNewTargetNode",value:function(t,e){var i=s.deepExtend({},this.options.controlNodeStyle);i.id="targetNode"+s.randomUUID(),i.hidden=!1,i.physics=!1,i.x=t,i.y=e;var o=this.body.functions.createNode(i);return o.shape.boundingBox={left:t,right:t,top:e,bottom:e},o}},{key:"_createEditButton",value:function(){this._clean(),this.manipulationDOM={},s.recursiveDOMDelete(this.editModeDiv);var t=this.options.locales[this.options.locale],e=this._createButton("editMode","vis-button vis-edit vis-edit-mode",t.edit||this.options.locales.en.edit);this.editModeDiv.appendChild(e),this._bindHammerToDiv(e,this.toggleEditMode.bind(this))}},{key:"_clean",value:function(){this.inMode=!1,this.guiEnabled===!0&&(s.recursiveDOMDelete(this.editModeDiv),s.recursiveDOMDelete(this.manipulationDiv),this._cleanManipulatorHammers()),this._cleanupTemporaryNodesAndEdges(),this._unbindTemporaryUIs(),this._unbindTemporaryEvents(),this.body.emitter.emit("restorePhysics")}},{key:"_cleanManipulatorHammers",value:function(){if(0!=this.manipulationHammers.length){for(var t=0;t=0;r--)if(n[r]!==this.selectedControlNode.id){s=this.body.nodes[n[r]];break}if(void 0!==s&&void 0!==this.selectedControlNode)if(s.isCluster===!0)alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError);else{var a=this.body.nodes[this.temporaryIds.nodes[0]];this.selectedControlNode.id===a.id?this._performEditEdge(s.id,o.to.id):this._performEditEdge(o.from.id,s.id)}else o.updateEdgeType(),this.body.emitter.emit("restorePhysics");this.body.emitter.emit("_redraw")}}},{key:"_handleConnect",value:function(t){if((new Date).valueOf()-this.touchTime>100){this.lastTouch=this.body.functions.getPointer(t.center),this.lastTouch.translation=s.extend({},this.body.view.translation);var e=this.lastTouch,i=this.selectionHandler.getNodeAt(e);if(void 0!==i)if(i.isCluster===!0)alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError);else{var o=this._getNewTargetNode(i.x,i.y);this.body.nodes[o.id]=o,this.body.nodeIndices.push(o.id);var n=this.body.functions.createEdge({id:"connectionEdge"+s.randomUUID(),from:i.id,to:o.id,physics:!1,smooth:{enabled:!0,type:"continuous",roundness:.5}});this.body.edges[n.id]=n,this.body.edgeIndices.push(n.id),this.temporaryIds.nodes.push(o.id),this.temporaryIds.edges.push(n.id)}this.touchTime=(new Date).valueOf()}}},{key:"_dragControlNode",value:function(t){var e=this.body.functions.getPointer(t.center);if(void 0!==this.temporaryIds.nodes[0]){var i=this.body.nodes[this.temporaryIds.nodes[0]];i.x=this.canvas._XconvertDOMtoCanvas(e.x),i.y=this.canvas._YconvertDOMtoCanvas(e.y),this.body.emitter.emit("_redraw")}else{var o=e.x-this.lastTouch.x,n=e.y-this.lastTouch.y;this.body.view.translation={x:this.lastTouch.translation.x+o,y:this.lastTouch.translation.y+n}}}},{key:"_finishConnect",value:function(t){var e=this.body.functions.getPointer(t.center),i=this.selectionHandler._pointerToPositionObject(e),o=void 0;void 0!==this.temporaryIds.edges[0]&&(o=this.body.edges[this.temporaryIds.edges[0]].fromId);for(var n=this.selectionHandler._getAllNodesOverlappingWith(i),s=void 0,r=n.length-1;r>=0;r--)if(-1===this.temporaryIds.nodes.indexOf(n[r])){s=this.body.nodes[n[r]];break}this._cleanupTemporaryNodesAndEdges(),void 0!==s&&(s.isCluster===!0?alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError):void 0!==this.body.nodes[o]&&void 0!==this.body.nodes[s.id]&&this._performAddEdge(o,s.id)),this.body.emitter.emit("_redraw")}},{key:"_performAddNode",value:function(t){var e=this,i={id:s.randomUUID(),x:t.pointer.canvas.x,y:t.pointer.canvas.y,label:"new"};if("function"==typeof this.options.addNode){if(2!==this.options.addNode.length)throw new Error("The function for add does not support two arguments (data,callback)");this.options.addNode(i,function(t){null!==t&&void 0!==t&&"addNode"===e.inMode&&(e.body.data.nodes.getDataSet().add(t),e.showManipulatorToolbar())})}else this.body.data.nodes.getDataSet().add(i),this.showManipulatorToolbar()}},{key:"_performAddEdge",value:function(t,e){var i=this,o={from:t,to:e};if("function"==typeof this.options.addEdge){if(2!==this.options.addEdge.length)throw new Error("The function for connect does not support two arguments (data,callback)");this.options.addEdge(o,function(t){null!==t&&void 0!==t&&"addEdge"===i.inMode&&(i.body.data.edges.getDataSet().add(t),i.selectionHandler.unselectAll(),i.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().add(o),this.selectionHandler.unselectAll(),this.showManipulatorToolbar()}},{key:"_performEditEdge",value:function(t,e){var i=this,o={id:this.edgeBeingEditedId,from:t,to:e};if("function"==typeof this.options.editEdge){if(2!==this.options.editEdge.length)throw new Error("The function for edit does not support two arguments (data, callback)");this.options.editEdge(o,function(t){null===t||void 0===t||"editEdge"!==i.inMode?(i.body.edges[o.id].updateEdgeType(),i.body.emitter.emit("_redraw")):(i.body.data.edges.getDataSet().update(t),i.selectionHandler.unselectAll(),i.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().update(o),this.selectionHandler.unselectAll(),this.showManipulatorToolbar()}}]),t}();e["default"]=h},function(t,e){Object.defineProperty(e,"__esModule",{value:!0});var i="string",o="boolean",n="number",s="array",r="object",a="dom",h="any",d={configure:{enabled:{"boolean":o},filter:{"boolean":o,string:i,array:s,"function":"function"},container:{dom:a},showButton:{"boolean":o},__type__:{object:r,"boolean":o,string:i,array:s,"function":"function"}},edges:{arrows:{to:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},middle:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},from:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},__type__:{string:["from","to","middle"],object:r}},arrowStrikethrough:{"boolean":o},color:{color:{string:i},highlight:{string:i},hover:{string:i},inherit:{string:["from","to","both"],"boolean":o},opacity:{number:n},__type__:{object:r,string:i}},dashes:{"boolean":o,array:s},font:{color:{string:i},size:{number:n},face:{string:i},background:{string:i},strokeWidth:{number:n},strokeColor:{string:i},align:{string:["horizontal","top","middle","bottom"]},__type__:{object:r,string:i}},hidden:{"boolean":o},hoverWidth:{"function":"function",number:n},label:{string:i,undefined:"undefined"},labelHighlightBold:{"boolean":o},length:{number:n,undefined:"undefined"},physics:{"boolean":o},scaling:{min:{number:n},max:{number:n},label:{enabled:{"boolean":o},min:{number:n},max:{number:n},maxVisible:{number:n},drawThreshold:{number:n},__type__:{object:r,"boolean":o}},customScalingFunction:{"function":"function"},__type__:{object:r}},selectionWidth:{"function":"function",number:n},selfReferenceSize:{number:n},shadow:{enabled:{"boolean":o},color:{string:i},size:{number:n},x:{number:n},y:{number:n},__type__:{object:r,"boolean":o}},smooth:{enabled:{"boolean":o},type:{string:["dynamic","continuous","discrete","diagonalCross","straightCross","horizontal","vertical","curvedCW","curvedCCW","cubicBezier"]},roundness:{number:n},forceDirection:{string:["horizontal","vertical","none"],"boolean":o},__type__:{object:r,"boolean":o}},title:{string:i,undefined:"undefined"},width:{number:n},value:{number:n,undefined:"undefined"},__type__:{object:r}},groups:{useDefaultGroups:{"boolean":o},__any__:"get from nodes, will be overwritten below",__type__:{object:r}},interaction:{dragNodes:{"boolean":o},dragView:{"boolean":o},hideEdgesOnDrag:{"boolean":o},hideNodesOnDrag:{"boolean":o},hover:{"boolean":o},keyboard:{enabled:{"boolean":o},speed:{x:{number:n},y:{number:n},zoom:{number:n},__type__:{object:r}},bindToWindow:{"boolean":o},__type__:{object:r,"boolean":o}},multiselect:{"boolean":o},navigationButtons:{"boolean":o},selectable:{"boolean":o},selectConnectedEdges:{"boolean":o},hoverConnectedEdges:{"boolean":o},tooltipDelay:{number:n},zoomView:{"boolean":o},__type__:{object:r}},layout:{randomSeed:{undefined:"undefined",number:n},improvedLayout:{"boolean":o},hierarchical:{enabled:{"boolean":o},levelSeparation:{number:n},nodeSpacing:{number:n},treeSpacing:{number:n},blockShifting:{"boolean":o},edgeMinimization:{"boolean":o},parentCentralization:{"boolean":o},direction:{string:["UD","DU","LR","RL"]},sortMethod:{string:["hubsize","directed"]},__type__:{object:r,"boolean":o}},__type__:{object:r}},manipulation:{enabled:{"boolean":o},initiallyActive:{"boolean":o},addNode:{"boolean":o,"function":"function"},addEdge:{"boolean":o,"function":"function"},editNode:{"function":"function"},editEdge:{"boolean":o,"function":"function"},deleteNode:{"boolean":o,"function":"function"},deleteEdge:{"boolean":o,"function":"function"},controlNodeStyle:"get from nodes, will be overwritten below",__type__:{object:r,"boolean":o}},nodes:{borderWidth:{number:n},borderWidthSelected:{number:n,undefined:"undefined"},brokenImage:{string:i,undefined:"undefined"},color:{border:{string:i},background:{string:i},highlight:{border:{string:i},background:{string:i},__type__:{object:r,string:i}},hover:{border:{string:i},background:{string:i},__type__:{object:r,string:i}},__type__:{object:r,string:i}},fixed:{x:{"boolean":o},y:{"boolean":o},__type__:{object:r,"boolean":o}},font:{align:{string:i},color:{string:i},size:{number:n},face:{string:i},background:{string:i},strokeWidth:{number:n},strokeColor:{string:i},__type__:{object:r,string:i}},group:{string:i,number:n,undefined:"undefined"},hidden:{"boolean":o},icon:{face:{string:i},code:{string:i},size:{number:n},color:{string:i},__type__:{object:r}},id:{string:i,number:n},image:{string:i,undefined:"undefined"},label:{string:i,undefined:"undefined"},labelHighlightBold:{"boolean":o},level:{number:n,undefined:"undefined"},mass:{number:n},physics:{"boolean":o},scaling:{min:{number:n},max:{number:n},label:{enabled:{"boolean":o},min:{number:n},max:{number:n},maxVisible:{number:n},drawThreshold:{number:n},__type__:{object:r,"boolean":o}},customScalingFunction:{"function":"function"},__type__:{object:r}},shadow:{enabled:{"boolean":o},color:{string:i},size:{number:n},x:{number:n},y:{number:n},__type__:{object:r,"boolean":o}},shape:{string:["ellipse","circle","database","box","text","image","circularImage","diamond","dot","star","triangle","triangleDown","square","icon"]},shapeProperties:{borderDashes:{"boolean":o,array:s},borderRadius:{number:n},interpolation:{"boolean":o},useImageSize:{"boolean":o},useBorderWithImage:{"boolean":o},__type__:{object:r}},size:{number:n},title:{string:i,undefined:"undefined"},value:{number:n,undefined:"undefined"},x:{number:n},y:{number:n},__type__:{object:r}},physics:{enabled:{"boolean":o},barnesHut:{gravitationalConstant:{number:n},centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},damping:{number:n},avoidOverlap:{number:n},__type__:{object:r}},forceAtlas2Based:{gravitationalConstant:{number:n},centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},damping:{number:n},avoidOverlap:{number:n},__type__:{object:r}},repulsion:{centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},nodeDistance:{number:n},damping:{number:n},__type__:{object:r}},hierarchicalRepulsion:{centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},nodeDistance:{number:n},damping:{number:n},__type__:{object:r}},maxVelocity:{number:n},minVelocity:{number:n},solver:{string:["barnesHut","repulsion","hierarchicalRepulsion","forceAtlas2Based"]},stabilization:{enabled:{"boolean":o},iterations:{number:n},updateInterval:{number:n},onlyDynamicEdges:{"boolean":o},fit:{"boolean":o},__type__:{object:r,"boolean":o}},timestep:{number:n},adaptiveTimestep:{"boolean":o},__type__:{object:r,"boolean":o}},autoResize:{"boolean":o},clickToUse:{"boolean":o},locale:{string:i},locales:{__any__:{any:h},__type__:{object:r}},height:{string:i},width:{string:i},__type__:{object:r}};d.groups.__any__=d.nodes,d.manipulation.controlNodeStyle=d.nodes;var l={nodes:{borderWidth:[1,0,10,1],borderWidthSelected:[2,0,10,1],color:{border:["color","#2B7CE9"],background:["color","#97C2FC"],highlight:{border:["color","#2B7CE9"],background:["color","#D2E5FF"]},hover:{border:["color","#2B7CE9"],background:["color","#D2E5FF"]}},fixed:{x:!1,y:!1},font:{color:["color","#343434"],size:[14,0,100,1],face:["arial","verdana","tahoma"],background:["color","none"],strokeWidth:[0,0,50,1],strokeColor:["color","#ffffff"]},hidden:!1,labelHighlightBold:!0,physics:!0,scaling:{min:[10,0,200,1],max:[30,0,200,1],label:{enabled:!1,min:[14,0,200,1],max:[30,0,200,1],maxVisible:[30,0,200,1],drawThreshold:[5,0,20,1]}},shadow:{enabled:!1,color:"rgba(0,0,0,0.5)",size:[10,0,20,1],x:[5,-30,30,1],y:[5,-30,30,1]},shape:["ellipse","box","circle","database","diamond","dot","square","star","text","triangle","triangleDown"],shapeProperties:{borderDashes:!1,borderRadius:[6,0,20,1],interpolation:!0,useImageSize:!1},size:[25,0,200,1]},edges:{arrows:{to:{enabled:!1,scaleFactor:[1,0,3,.05]},middle:{enabled:!1,scaleFactor:[1,0,3,.05]},from:{enabled:!1,scaleFactor:[1,0,3,.05]}},arrowStrikethrough:!0,color:{color:["color","#848484"],highlight:["color","#848484"],hover:["color","#848484"],inherit:["from","to","both",!0,!1],opacity:[1,0,1,.05]},dashes:!1,font:{color:["color","#343434"],size:[14,0,100,1],face:["arial","verdana","tahoma"],background:["color","none"],strokeWidth:[2,0,50,1],strokeColor:["color","#ffffff"],align:["horizontal","top","middle","bottom"]},hidden:!1,hoverWidth:[1.5,0,5,.1],labelHighlightBold:!0,physics:!0,scaling:{min:[1,0,100,1],max:[15,0,100,1],label:{enabled:!0,min:[14,0,200,1],max:[30,0,200,1],maxVisible:[30,0,200,1],drawThreshold:[5,0,20,1]}},selectionWidth:[1.5,0,5,.1],selfReferenceSize:[20,0,200,1],shadow:{enabled:!1,color:"rgba(0,0,0,0.5)",size:[10,0,20,1],x:[5,-30,30,1],y:[5,-30,30,1]},smooth:{enabled:!0,type:["dynamic","continuous","discrete","diagonalCross","straightCross","horizontal","vertical","curvedCW","curvedCCW","cubicBezier"],forceDirection:["horizontal","vertical","none"],roundness:[.5,0,1,.05]},width:[1,0,30,1]},layout:{hierarchical:{enabled:!1,levelSeparation:[150,20,500,5],nodeSpacing:[100,20,500,5],treeSpacing:[200,20,500,5],blockShifting:!0,edgeMinimization:!0,parentCentralization:!0,direction:["UD","DU","LR","RL"],sortMethod:["hubsize","directed"]}},interaction:{dragNodes:!0,dragView:!0,hideEdgesOnDrag:!1,hideNodesOnDrag:!1,hover:!1,keyboard:{enabled:!1,speed:{x:[10,0,40,1],y:[10,0,40,1],zoom:[.02,0,.1,.005]},bindToWindow:!0},multiselect:!1,navigationButtons:!1,selectable:!0,selectConnectedEdges:!0,hoverConnectedEdges:!0,tooltipDelay:[300,0,1e3,25],zoomView:!0},manipulation:{enabled:!1,initiallyActive:!1},physics:{enabled:!0,barnesHut:{gravitationalConstant:[-2e3,-3e4,0,50],centralGravity:[.3,0,10,.05],springLength:[95,0,500,5],springConstant:[.04,0,1.2,.005],damping:[.09,0,1,.01],avoidOverlap:[0,0,1,.01]},forceAtlas2Based:{gravitationalConstant:[-50,-500,0,1],centralGravity:[.01,0,1,.005],springLength:[95,0,500,5],springConstant:[.08,0,1.2,.005],damping:[.4,0,1,.01],avoidOverlap:[0,0,1,.01]},repulsion:{centralGravity:[.2,0,10,.05],springLength:[200,0,500,5],springConstant:[.05,0,1.2,.005],nodeDistance:[100,0,500,5],damping:[.09,0,1,.01]},hierarchicalRepulsion:{centralGravity:[.2,0,10,.05],springLength:[100,0,500,5],springConstant:[.01,0,1.2,.005],nodeDistance:[120,0,500,5],damping:[.09,0,1,.01]},maxVelocity:[50,0,150,1],minVelocity:[.1,.01,.5,.01],solver:["barnesHut","forceAtlas2Based","repulsion","hierarchicalRepulsion"],timestep:[.5,.01,1,.01]},global:{locale:["en","nl"]}};e.allOptions=d,e.configureOptions=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),r=function(){function t(t,e){for(var i=0;in&&h>a;){a+=1;var v=this._getHighestEnergyNode(i),g=s(v,4);for(c=g[0],l=g[1],u=g[2],p=g[3],f=l,m=0;f>r&&d>m;){m+=1,this._moveNode(c,u,p);var y=this._getEnergy(c),b=s(y,3);f=b[0],u=b[1],p=b[2]}}}},{key:"_getHighestEnergyNode",value:function(t){for(var e=this.body.nodeIndices,i=this.body.nodes,o=0,n=e[0],r=0,a=0,h=0;ho&&(o=u,n=d,r=p,a=f)}}return[n,o,r,a]}},{key:"_getEnergy",value:function(t){for(var e=this.body.nodeIndices,i=this.body.nodes,o=i[t].x,n=i[t].y,s=0,r=0,a=0;al;l++)for(var c=0;d-1>c;c++)for(var u=c+1;d>u;u++)o[e[c]][e[u]]=Math.min(o[e[c]][e[u]],o[e[c]][e[l]]+o[e[l]][e[u]]),o[e[u]][e[c]]=o[e[c]][e[u]];return o}}]),t}();e["default"]=n},function(t,e){"undefined"!=typeof CanvasRenderingContext2D&&(CanvasRenderingContext2D.prototype.circle=function(t,e,i){this.beginPath(),this.arc(t,e,i,0,2*Math.PI,!1),this.closePath()},CanvasRenderingContext2D.prototype.square=function(t,e,i){this.beginPath(),this.rect(t-i,e-i,2*i,2*i),this.closePath()},CanvasRenderingContext2D.prototype.triangle=function(t,e,i){this.beginPath(),i*=1.15,e+=.275*i;var o=2*i,n=o/2,s=Math.sqrt(3)/6*o,r=Math.sqrt(o*o-n*n);this.moveTo(t,e-(r-s)),this.lineTo(t+n,e+s),this.lineTo(t-n,e+s),this.lineTo(t,e-(r-s)),this.closePath()},CanvasRenderingContext2D.prototype.triangleDown=function(t,e,i){this.beginPath(),i*=1.15,e-=.275*i;var o=2*i,n=o/2,s=Math.sqrt(3)/6*o,r=Math.sqrt(o*o-n*n);this.moveTo(t,e+(r-s)), +this.lineTo(t+n,e-s),this.lineTo(t-n,e-s),this.lineTo(t,e+(r-s)),this.closePath()},CanvasRenderingContext2D.prototype.star=function(t,e,i){this.beginPath(),i*=.82,e+=.1*i;for(var o=0;10>o;o++){var n=o%2===0?1.3*i:.5*i;this.lineTo(t+n*Math.sin(2*o*Math.PI/10),e-n*Math.cos(2*o*Math.PI/10))}this.closePath()},CanvasRenderingContext2D.prototype.diamond=function(t,e,i){this.beginPath(),this.lineTo(t,e+i),this.lineTo(t+i,e),this.lineTo(t,e-i),this.lineTo(t-i,e),this.closePath()},CanvasRenderingContext2D.prototype.roundRect=function(t,e,i,o,n){var s=Math.PI/180;0>i-2*n&&(n=i/2),0>o-2*n&&(n=o/2),this.beginPath(),this.moveTo(t+n,e),this.lineTo(t+i-n,e),this.arc(t+i-n,e+n,n,270*s,360*s,!1),this.lineTo(t+i,e+o-n),this.arc(t+i-n,e+o-n,n,0,90*s,!1),this.lineTo(t+n,e+o),this.arc(t+n,e+o-n,n,90*s,180*s,!1),this.lineTo(t,e+n),this.arc(t+n,e+n,n,180*s,270*s,!1),this.closePath()},CanvasRenderingContext2D.prototype.ellipse=function(t,e,i,o){var n=.5522848,s=i/2*n,r=o/2*n,a=t+i,h=e+o,d=t+i/2,l=e+o/2;this.beginPath(),this.moveTo(t,l),this.bezierCurveTo(t,l-r,d-s,e,d,e),this.bezierCurveTo(d+s,e,a,l-r,a,l),this.bezierCurveTo(a,l+r,d+s,h,d,h),this.bezierCurveTo(d-s,h,t,l+r,t,l),this.closePath()},CanvasRenderingContext2D.prototype.database=function(t,e,i,o){var n=1/3,s=i,r=o*n,a=.5522848,h=s/2*a,d=r/2*a,l=t+s,c=e+r,u=t+s/2,p=e+r/2,f=e+(o-r/2),m=e+o;this.beginPath(),this.moveTo(l,p),this.bezierCurveTo(l,p+d,u+h,c,u,c),this.bezierCurveTo(u-h,c,t,p+d,t,p),this.bezierCurveTo(t,p-d,u-h,e,u,e),this.bezierCurveTo(u+h,e,l,p-d,l,p),this.lineTo(l,f),this.bezierCurveTo(l,f+d,u+h,m,u,m),this.bezierCurveTo(u-h,m,t,f+d,t,f),this.lineTo(t,p)},CanvasRenderingContext2D.prototype.arrow=function(t,e,i,o){var n=t-o*Math.cos(i),s=e-o*Math.sin(i),r=t-.9*o*Math.cos(i),a=e-.9*o*Math.sin(i),h=n+o/3*Math.cos(i+.5*Math.PI),d=s+o/3*Math.sin(i+.5*Math.PI),l=n+o/3*Math.cos(i-.5*Math.PI),c=s+o/3*Math.sin(i-.5*Math.PI);this.beginPath(),this.moveTo(t,e),this.lineTo(h,d),this.lineTo(r,a),this.lineTo(l,c),this.closePath()},CanvasRenderingContext2D.prototype.dashedLine=function(t,e,i,o,n){this.beginPath(),this.moveTo(t,e);for(var s=n.length,r=i-t,a=o-e,h=a/r,d=Math.sqrt(r*r+a*a),l=0,c=!0,u=0,p=n[0];d>=.1;)p=n[l++%s],p>d&&(p=d),u=Math.sqrt(p*p/(1+h*h)),u=0>r?-u:u,t+=u,e+=h*u,c===!0?this.lineTo(t,e):this.moveTo(t,e),d-=p,c=!c})},function(t,e){function i(t){return P=t,p()}function o(){I=0,N=P.charAt(0)}function n(){I++,N=P.charAt(I)}function s(){return P.charAt(I+1)}function r(t){return L.test(t)}function a(t,e){if(t||(t={}),e)for(var i in e)e.hasOwnProperty(i)&&(t[i]=e[i]);return t}function h(t,e,i){for(var o=e.split("."),n=t;o.length;){var s=o.shift();o.length?(n[s]||(n[s]={}),n=n[s]):n[s]=i}}function d(t,e){for(var i,o,n=null,s=[t],r=t;r.parent;)s.push(r.parent),r=r.parent;if(r.nodes)for(i=0,o=r.nodes.length;o>i;i++)if(e.id===r.nodes[i].id){n=r.nodes[i];break}for(n||(n={id:e.id},t.node&&(n.attr=a(n.attr,t.node))),i=s.length-1;i>=0;i--){var h=s[i];h.nodes||(h.nodes=[]),-1===h.nodes.indexOf(n)&&h.nodes.push(n)}e.attr&&(n.attr=a(n.attr,e.attr))}function l(t,e){if(t.edges||(t.edges=[]),t.edges.push(e),t.edge){var i=a({},t.edge);e.attr=a(i,e.attr)}}function c(t,e,i,o,n){var s={from:e,to:i,type:o};return t.edge&&(s.attr=a({},t.edge)),s.attr=a(s.attr||{},n),s}function u(){for(z=T.NULL,R="";" "===N||" "===N||"\n"===N||"\r"===N;)n();do{var t=!1;if("#"===N){for(var e=I-1;" "===P.charAt(e)||" "===P.charAt(e);)e--;if("\n"===P.charAt(e)||""===P.charAt(e)){for(;""!=N&&"\n"!=N;)n();t=!0}}if("/"===N&&"/"===s()){for(;""!=N&&"\n"!=N;)n();t=!0}if("/"===N&&"*"===s()){for(;""!=N;){if("*"===N&&"/"===s()){n(),n();break}n()}t=!0}for(;" "===N||" "===N||"\n"===N||"\r"===N;)n()}while(t);if(""===N)return void(z=T.DELIMITER);var i=N+s();if(E[i])return z=T.DELIMITER,R=i,n(),void n();if(E[N])return z=T.DELIMITER,R=N,void n();if(r(N)||"-"===N){for(R+=N,n();r(N);)R+=N,n();return"false"===R?R=!1:"true"===R?R=!0:isNaN(Number(R))||(R=Number(R)),void(z=T.IDENTIFIER)}if('"'===N){for(n();""!=N&&('"'!=N||'"'===N&&'"'===s());)R+=N,'"'===N&&n(),n();if('"'!=N)throw _('End of string " expected');return n(),void(z=T.IDENTIFIER)}for(z=T.UNKNOWN;""!=N;)R+=N,n();throw new SyntaxError('Syntax error in part "'+x(R,30)+'"')}function p(){var t={};if(o(),u(),"strict"===R&&(t.strict=!0,u()),"graph"!==R&&"digraph"!==R||(t.type=R,u()),z===T.IDENTIFIER&&(t.id=R,u()),"{"!=R)throw _("Angle bracket { expected");if(u(),f(t),"}"!=R)throw _("Angle bracket } expected");if(u(),""!==R)throw _("End of file expected");return u(),delete t.node,delete t.edge,delete t.graph,t}function f(t){for(;""!==R&&"}"!=R;)m(t),";"===R&&u()}function m(t){var e=v(t);if(e)return void b(t,e);var i=g(t);if(!i){if(z!=T.IDENTIFIER)throw _("Identifier expected");var o=R;if(u(),"="===R){if(u(),z!=T.IDENTIFIER)throw _("Identifier expected");t[o]=R,u()}else y(t,o)}}function v(t){var e=null;if("subgraph"===R&&(e={},e.type="subgraph",u(),z===T.IDENTIFIER&&(e.id=R,u())),"{"===R){if(u(),e||(e={}),e.parent=t,e.node=t.node,e.edge=t.edge,e.graph=t.graph,f(e),"}"!=R)throw _("Angle bracket } expected");u(),delete e.node,delete e.edge,delete e.graph,delete e.parent,t.subgraphs||(t.subgraphs=[]),t.subgraphs.push(e)}return e}function g(t){return"node"===R?(u(),t.node=w(),"node"):"edge"===R?(u(),t.edge=w(),"edge"):"graph"===R?(u(),t.graph=w(),"graph"):null}function y(t,e){var i={id:e},o=w();o&&(i.attr=o),d(t,i),b(t,e)}function b(t,e){for(;"->"===R||"--"===R;){var i,o=R;u();var n=v(t);if(n)i=n;else{if(z!=T.IDENTIFIER)throw _("Identifier or subgraph expected");i=R,d(t,{id:i}),u()}var s=w(),r=c(t,e,i,o,s);l(t,r),e=i}}function w(){for(var t=null;"["===R;){for(u(),t={};""!==R&&"]"!=R;){if(z!=T.IDENTIFIER)throw _("Attribute name expected");var e=R;if(u(),"="!=R)throw _("Equal sign = expected");if(u(),z!=T.IDENTIFIER)throw _("Attribute value expected");var i=R;h(t,e,i),u(),","==R&&u()}if("]"!=R)throw _("Bracket ] expected");u()}return t}function _(t){return new SyntaxError(t+', got "'+x(R,30)+'" (char '+I+")")}function x(t,e){return t.length<=e?t:t.substr(0,27)+"..."}function k(t,e,i){Array.isArray(t)?t.forEach(function(t){Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}):Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}function O(t,e,i){for(var o=e.split("."),n=o.pop(),s=t,r=0;r":!0,"--":!0},P="",I=0,N="",R="",z=T.NULL,L=/[a-zA-Z_0-9.:#]/;e.parseDOT=i,e.DOTToGraph=D},function(t,e){function i(t,e){var i=[],o=[],n={edges:{inheritColor:!1},nodes:{fixed:!1,parseColor:!1}};void 0!==e&&(void 0!==e.fixed&&(n.nodes.fixed=e.fixed),void 0!==e.parseColor&&(n.nodes.parseColor=e.parseColor),void 0!==e.inheritColor&&(n.edges.inheritColor=e.inheritColor));for(var s=t.edges,r=t.nodes,a=0;a ++ } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 22ee13b6e78c..99f2bd8bc1f2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -179,7 +179,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
    ++ } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 5d1928ac6b2c..6ddabfd8ef08 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -26,7 +26,7 @@ import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.{InternalAccumulator, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} import org.apache.spark.ui._ @@ -746,7 +746,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ++ } From 4c0af3bbd5a6e88b1fedf0d9c624bbdb82c1aa40 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 15:45:59 -0700 Subject: [PATCH 261/313] [SPARK-15392][SQL] fix default value of size estimation of logical plan ## What changes were proposed in this pull request? We use autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD. This PR change the default value to Long.MaxValue. ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #13179 from davies/fix_default_size. (cherry picked from commit fc29b896dae08b957ed15fa681b46162600a4050) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 3 +-- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 +++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7933d12e284f..a7f461381b50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -605,8 +605,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) - def defaultSizeInBytes: Long = - getConf(DEFAULT_SIZE_IN_BYTES, autoBroadcastJoinThreshold + 1L) + def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, Long.MaxValue) def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index f573abf85961..df029e44c54e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1476,4 +1476,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { getMessage() assert(e1.startsWith("Path does not exist")) } + + test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { + val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) + val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) + assert(df.queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + } } From 36acf8856c94f93f3b16f4592b6d5fb64acda39d Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Wed, 18 May 2016 16:15:09 -0700 Subject: [PATCH 262/313] [SPARK-15323][SPARK-14463][SQL] Fix reading of partitioned format=text datasets https://issues.apache.org/jira/browse/SPARK-15323 I was using partitioned text datasets in Spark 1.6.1 but it broke in Spark 2.0.0. It would be logical if you could also write those, but not entirely sure how to solve this with the new DataSet implementation. Also it doesn't work using `sqlContext.read.text`, since that method returns a `DataSet[String]`. See https://issues.apache.org/jira/browse/SPARK-14463 for that issue. Author: Jurriaan Pruis Closes #13104 from jurriaan/fix-partitioned-text-reads. (cherry picked from commit 32be51fba45f5e07a2a3520293c12dc7765a364d) Signed-off-by: Reynold Xin --- .../apache/spark/sql/DataFrameReader.scala | 3 ++- .../datasources/text/DefaultSource.scala | 14 ------------- .../text-partitioned/year=2014/data.txt | 1 + .../text-partitioned/year=2015/data.txt | 1 + .../datasources/text/TextSuite.scala | 20 +++++++++++++++++++ 5 files changed, 24 insertions(+), 15 deletions(-) create mode 100644 sql/core/src/test/resources/text-partitioned/year=2014/data.txt create mode 100644 sql/core/src/test/resources/text-partitioned/year=2015/data.txt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 011aff4ff6c2..e33fd831ab47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -457,7 +457,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { */ @scala.annotation.varargs def text(paths: String*): Dataset[String] = { - format("text").load(paths : _*).as[String](sparkSession.implicits.newStringEncoder) + format("text").load(paths : _*).select("value") + .as[String](sparkSession.implicits.newStringEncoder) } /////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index f22c0241d930..f091615a9a71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -83,19 +83,6 @@ class DefaultSource extends FileFormat with DataSourceRegister { } } - override private[sql] def buildReaderWithPartitionValues( - sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { - // Text data source doesn't support partitioning. Here we simply delegate to `buildReader`. - buildReader( - sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) - } - override def buildReader( sparkSession: SparkSession, dataSchema: StructType, @@ -152,4 +139,3 @@ class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemp recordWriter.close(context) } } - diff --git a/sql/core/src/test/resources/text-partitioned/year=2014/data.txt b/sql/core/src/test/resources/text-partitioned/year=2014/data.txt new file mode 100644 index 000000000000..e2719428bb28 --- /dev/null +++ b/sql/core/src/test/resources/text-partitioned/year=2014/data.txt @@ -0,0 +1 @@ +2014-test diff --git a/sql/core/src/test/resources/text-partitioned/year=2015/data.txt b/sql/core/src/test/resources/text-partitioned/year=2015/data.txt new file mode 100644 index 000000000000..b8c03daa8c19 --- /dev/null +++ b/sql/core/src/test/resources/text-partitioned/year=2015/data.txt @@ -0,0 +1 @@ +2015-test diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index f61fce5d4102..b5e51e963f1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -65,6 +65,26 @@ class TextSuite extends QueryTest with SharedSQLContext { } } + test("reading partitioned data using read.text()") { + val partitionedData = Thread.currentThread().getContextClassLoader + .getResource("text-partitioned").toString + val df = spark.read.text(partitionedData) + val data = df.collect() + + assert(df.schema == new StructType().add("value", StringType)) + assert(data.length == 2) + } + + test("support for partitioned reading") { + val partitionedData = Thread.currentThread().getContextClassLoader + .getResource("text-partitioned").toString + val df = spark.read.format("text").load(partitionedData) + val data = df.filter("year = '2015'").select("value").collect() + + assert(data(0) == Row("2015-test")) + assert(data.length == 1) + } + test("SPARK-13503 Support to specify the option for compression codec for TEXT") { val testDf = spark.read.text(testFile) val extensionNameMap = Map("bzip2" -> ".bz2", "deflate" -> ".deflate", "gzip" -> ".gz") From f5784459e84e84d6641725bf0d0fb31db742456c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 May 2016 18:06:38 -0700 Subject: [PATCH 263/313] [SPARK-15192][SQL] null check for SparkSession.createDataFrame ## What changes were proposed in this pull request? This PR adds null check in `SparkSession.createDataFrame`, so that we can make sure the passed in rows matches the given schema. ## How was this patch tested? new tests in `DatasetSuite` Author: Wenchen Fan Closes #13008 from cloud-fan/row-encoder. (cherry picked from commit ebfe3a1f2c77e6869c3c36ba67afb7fabe6a94f5) Signed-off-by: Yin Huai --- .../scala/org/apache/spark/mllib/fpm/FPGrowth.scala | 2 +- .../apache/spark/sql/catalyst/ScalaReflection.scala | 4 ++-- .../spark/sql/catalyst/encoders/RowEncoder.scala | 10 +++------- .../sql/catalyst/expressions/BoundAttribute.scala | 2 +- .../sql/catalyst/expressions/objects/objects.scala | 4 +++- .../scala/org/apache/spark/sql/SparkSession.scala | 4 ++-- .../scala/org/apache/spark/sql/api/r/SQLUtils.scala | 5 ++++- .../scala/org/apache/spark/sql/DatasetSuite.scala | 13 +++++++++++-- .../org/apache/spark/sql/test/SQLTestUtils.scala | 6 +----- 9 files changed, 28 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 9166faa54de5..28e4966f918a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -116,7 +116,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { StructField("freq", LongType)) val schema = StructType(fields) val rowDataRDD = model.freqItemsets.map { x => - Row(x.items, x.freq) + Row(x.items.toSeq, x.freq) } sqlContext.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) } 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 cb9a62dfd4e8..c0fa220d34bb 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 @@ -113,8 +113,8 @@ object ScalaReflection extends ScalaReflection { * Returns true if the value of this data type is same between internal and external. */ def isNativeType(dt: DataType): Boolean = dt match { - case BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | BinaryType => true + case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType | BinaryType | CalendarIntervalType => true case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index a5f39aaa2314..71b39c54fa0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -70,8 +70,7 @@ object RowEncoder { private def serializerFor( inputObject: Expression, inputType: DataType): Expression = inputType match { - case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | BinaryType | CalendarIntervalType => inputObject + case dt if ScalaReflection.isNativeType(dt) => inputObject case p: PythonUserDefinedType => serializerFor(inputObject, p.sqlType) @@ -151,7 +150,7 @@ object RowEncoder { case StructType(fields) => val convertedFields = fields.zipWithIndex.map { case (f, i) => val fieldValue = serializerFor( - GetExternalRowField(inputObject, i, externalDataTypeForInput(f.dataType)), + GetExternalRowField(inputObject, i, f.name, externalDataTypeForInput(f.dataType)), f.dataType ) if (f.nullable) { @@ -193,7 +192,6 @@ object RowEncoder { private def externalDataTypeFor(dt: DataType): DataType = dt match { case _ if ScalaReflection.isNativeType(dt) => dt - case CalendarIntervalType => dt case TimestampType => ObjectType(classOf[java.sql.Timestamp]) case DateType => ObjectType(classOf[java.sql.Date]) case _: DecimalType => ObjectType(classOf[java.math.BigDecimal]) @@ -202,7 +200,6 @@ object RowEncoder { case _: MapType => ObjectType(classOf[scala.collection.Map[_, _]]) case _: StructType => ObjectType(classOf[Row]) case udt: UserDefinedType[_] => ObjectType(udt.userClass) - case _: NullType => ObjectType(classOf[java.lang.Object]) } private def deserializerFor(schema: StructType): Expression = { @@ -222,8 +219,7 @@ object RowEncoder { } private def deserializerFor(input: Expression): Expression = input.dataType match { - case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | BinaryType | CalendarIntervalType => input + case dt if ScalaReflection.isNativeType(dt) => input case udt: UserDefinedType[_] => val annotation = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 99f156a935b5..a38f1ec09156 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) extends LeafExpression { - override def toString: String = s"input[$ordinal, ${dataType.simpleString}]" + override def toString: String = s"input[$ordinal, ${dataType.simpleString}, $nullable]" // Use special getter for primitive types (for UnsafeRow) override def eval(input: InternalRow): Any = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 7df6e06805a5..fc38369f38c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -693,6 +693,7 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) case class GetExternalRowField( child: Expression, index: Int, + fieldName: String, dataType: DataType) extends UnaryExpression with NonSQLExpression { override def nullable: Boolean = false @@ -716,7 +717,8 @@ case class GetExternalRowField( } if (${row.value}.isNullAt($index)) { - throw new RuntimeException("The ${index}th field of input row cannot be null."); + throw new RuntimeException("The ${index}th field '$fieldName' of input row " + + "cannot be null."); } final ${ctx.javaType(dataType)} ${ev.value} = $getField; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index aa974f29cc34..8ef6cd3c3557 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -478,8 +478,8 @@ class SparkSession private( // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. val catalystRows = if (needsConversion) { - val converter = CatalystTypeConverters.createToCatalystConverter(schema) - rowRDD.map(converter(_).asInstanceOf[InternalRow]) + val encoder = RowEncoder(schema) + rowRDD.map(encoder.toRow) } else { rowRDD.map{r: Row => InternalRow.fromSeq(r.toSeq)} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 36173a49250b..2021cddab440 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.api.r import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} +import scala.collection.JavaConverters._ import scala.util.matching.Regex import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} @@ -110,6 +111,8 @@ private[sql] object SQLUtils { data match { case d: java.lang.Double if dataType == FloatType => new java.lang.Float(d) + // Scala Map is the only allowed external type of map type in Row. + case m: java.util.Map[_, _] => m.asScala case _ => data } } @@ -120,7 +123,7 @@ private[sql] object SQLUtils { val num = SerDe.readInt(dis) Row.fromSeq((0 until num).map { i => doConversion(SerDe.readObject(dis), schema.fields(i).dataType) - }.toSeq) + }) } private[sql] def rowToRBytes(row: Row): Array[Byte] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index b02b714168cb..1935e41185ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -507,7 +507,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val schema = StructType(Seq( StructField("f", StructType(Seq( StructField("a", StringType, nullable = true), - StructField("b", IntegerType, nullable = false) + StructField("b", IntegerType, nullable = true) )), nullable = true) )) @@ -684,7 +684,16 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val message = intercept[Exception] { df.collect() }.getMessage - assert(message.contains("The 0th field of input row cannot be null")) + assert(message.contains("The 0th field 'i' of input row cannot be null")) + } + + test("row nullability mismatch") { + val schema = new StructType().add("a", StringType, true).add("b", StringType, false) + val rdd = sqlContext.sparkContext.parallelize(Row(null, "123") :: Row("234", null) :: Nil) + val message = intercept[Exception] { + sqlContext.createDataFrame(rdd, schema).collect() + }.getMessage + assert(message.contains("The 1th field 'b' of input row cannot be null")) } test("createTempView") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 45a9c9dc4781..51538eca644f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -217,11 +217,7 @@ private[sql] trait SQLTestUtils case FilterExec(_, child) => child } - val childRDD = withoutFilters - .execute() - .map(row => Row.fromSeq(row.copy().toSeq(schema))) - - spark.createDataFrame(childRDD, schema) + spark.internalCreateDataFrame(withoutFilters.execute(), schema) } /** From 760e7ac8161bc4a847abde6fa93ce21e14de4163 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 19 May 2016 10:05:53 +0800 Subject: [PATCH 264/313] [SPARK-15297][SQL] Fix Set -V Command #### What changes were proposed in this pull request? The command `SET -v` always outputs the default values even if we set the parameter. This behavior is incorrect. Instead, if users override it, we should output the user-specified value. In addition, the output schema of `SET -v` is wrong. We should use the column `value` instead of `default` for the parameter value. This PR is to fix the above two issues. #### How was this patch tested? Added a test case. Author: gatorsmile Closes #13081 from gatorsmile/setVcommand. (cherry picked from commit 9c2a376e413b0701097b0784bd725e4ca87cd837) Signed-off-by: Wenchen Fan --- .../sql/execution/command/SetCommand.scala | 4 +--- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../spark/sql/internal/SQLConfSuite.scala | 23 ++++++++++++++++++- 3 files changed, 24 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 2409b5d203f4..282f26ce998f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.command -import java.util.NoSuchElementException - import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute @@ -88,7 +86,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm } val schema = StructType( StructField("key", StringType, nullable = false) :: - StructField("default", StringType, nullable = false) :: + StructField("value", StringType, nullable = false) :: StructField("meaning", StringType, nullable = false) :: Nil) (schema.toAttributes, runFunc) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a7f461381b50..248c6e3335f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -751,7 +751,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { */ def getAllDefinedConfs: Seq[(String, String, String)] = sqlConfEntries.synchronized { sqlConfEntries.values.asScala.filter(_.isPublic).map { entry => - (entry.key, entry.defaultValueString, entry.doc) + (entry.key, getConfString(entry.key, entry.defaultValueString), entry.doc) }.toSeq } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 7ead97bbf693..81bc973be74a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal -import org.apache.spark.sql.{QueryTest, SparkSession, SQLContext} +import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} class SQLConfSuite extends QueryTest with SharedSQLContext { @@ -75,6 +75,27 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { spark.wrapped.conf.clear() } + test("set command for display") { + spark.wrapped.conf.clear() + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Nil) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "true")) + + sql("SET spark.sql.groupByOrdinal=false") + + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + } + test("deprecated property") { spark.wrapped.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) From 595ed8de60c2d0cfde4aaf8aafe44f734d26631a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 18 May 2016 19:16:28 -0700 Subject: [PATCH 265/313] [SPARK-14463][SQL] Document the semantics for read.text ## What changes were proposed in this pull request? This patch is a follow-up to https://github.com/apache/spark/pull/13104 and adds documentation to clarify the semantics of read.text with respect to partitioning. ## How was this patch tested? N/A Author: Reynold Xin Closes #13184 from rxin/SPARK-14463. (cherry picked from commit 4987f39ac7a694e1c8b8b82246eb4fbd863201c4) Signed-off-by: Reynold Xin --- R/pkg/R/SQLContext.R | 2 ++ python/pyspark/sql/readwriter.py | 3 +++ .../main/scala/org/apache/spark/sql/DataFrameReader.scala | 8 ++++++-- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 3824e0a99557..6b7a341bee88 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -298,6 +298,8 @@ parquetFile <- function(sqlContext, ...) { #' Create a SparkDataFrame from a text file. #' #' Loads a text file and returns a SparkDataFrame with a single string column named "value". +#' If the directory structure of the text files contains partitioning information, those are +#' ignored in the resulting DataFrame. #' Each line in the text file is a new row in the resulting SparkDataFrame. #' #' @param sqlContext SQLContext to use diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 8e6bce90010e..855c9d666f0b 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -286,6 +286,9 @@ def parquet(self, *paths): @since(1.6) def text(self, paths): """Loads a text file and returns a [[DataFrame]] with a single string column named "value". + If the directory structure of the text files contains partitioning information, + those are ignored in the resulting DataFrame. To include partitioning information as + columns, use ``read.format('text').load(...)``. Each line in the text file is a new row in the resulting DataFrame. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index e33fd831ab47..57a2091fe8c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -440,10 +440,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } /** - * Loads a text file and returns a [[Dataset]] of String. The underlying schema of the Dataset + * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset * contains a single string column named "value". * - * Each line in the text file is a new row in the resulting Dataset. For example: + * If the directory structure of the text files contains partitioning information, those are + * ignored in the resulting Dataset. To include partitioning information as columns, use + * `read.format("text").load("...")`. + * + * Each line in the text files is a new element in the resulting Dataset. For example: * {{{ * // Scala: * spark.read.text("/path/to/spark/README.md") From a1948a0af1523ab6b14b391c4208cd458bccd2c1 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 18 May 2016 20:15:00 -0700 Subject: [PATCH 266/313] [SPARK-15395][CORE] Use getHostString to create RpcAddress ## What changes were proposed in this pull request? Right now the netty RPC uses `InetSocketAddress.getHostName` to create `RpcAddress` for network events. If we use an IP address to connect, then the RpcAddress's host will be a host name (if the reverse lookup successes) instead of the IP address. However, some places need to compare the original IP address and the RpcAddress in `onDisconnect` (e.g., CoarseGrainedExecutorBackend), and this behavior will make the check incorrect. This PR uses `getHostString` to resolve the issue. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13185 from zsxwing/host-string. (cherry picked from commit 5c9117a3ed373461529f9f9306668ed4149c63fb) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 7d7b4c82fa39..89d2fb9b4797 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -574,7 +574,7 @@ private[netty] class NettyRpcHandler( private def internalReceive(client: TransportClient, message: ByteBuffer): RequestMessage = { val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) val requestMessage = nettyEnv.deserialize[RequestMessage](client, message) if (requestMessage.senderAddress == null) { // Create a new message with the socket address of the client as the sender. @@ -595,7 +595,7 @@ private[netty] class NettyRpcHandler( override def exceptionCaught(cause: Throwable, client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) dispatcher.postToAll(RemoteProcessConnectionError(cause, clientAddr)) // If the remove RpcEnv listens to some address, we should also fire a // RemoteProcessConnectionError for the remote RpcEnv listening address @@ -614,14 +614,14 @@ private[netty] class NettyRpcHandler( override def channelActive(client: TransportClient): Unit = { val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) dispatcher.postToAll(RemoteProcessConnected(clientAddr)) } override def channelInactive(client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) nettyEnv.removeOutbox(clientAddr) dispatcher.postToAll(RemoteProcessDisconnected(clientAddr)) val remoteEnvAddress = remoteAddresses.remove(clientAddr) From 34c743c4b8f009f8549d17f724a1a6b6ba8a59b4 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 May 2016 21:43:07 -0700 Subject: [PATCH 267/313] [SPARK-15381] [SQL] physical object operator should define reference correctly ## What changes were proposed in this pull request? Whole Stage Codegen depends on `SparkPlan.reference` to do some optimization. For physical object operators, they should be consistent with their logical version and set the `reference` correctly. ## How was this patch tested? new test in DatasetSuite Author: Wenchen Fan Closes #13167 from cloud-fan/bug. (cherry picked from commit 661c21049b62ebfaf788dcbc31d62a09e206265b) Signed-off-by: Davies Liu --- .../sql/catalyst/plans/logical/object.scala | 10 +- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../apache/spark/sql/execution/objects.scala | 94 +++++++++++-------- .../org/apache/spark/sql/DatasetSuite.scala | 5 + 4 files changed, 64 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 84339f439a66..98ce5dd2efd9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -94,7 +94,7 @@ case class DeserializeToObject( */ case class SerializeFromObject( serializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode with ObjectConsumer { + child: LogicalPlan) extends ObjectConsumer { override def output: Seq[Attribute] = serializer.map(_.toAttribute) } @@ -118,7 +118,7 @@ object MapPartitions { case class MapPartitions( func: Iterator[Any] => Iterator[Any], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer + child: LogicalPlan) extends ObjectConsumer with ObjectProducer object MapPartitionsInR { def apply( @@ -152,7 +152,7 @@ case class MapPartitionsInR( inputSchema: StructType, outputSchema: StructType, outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer { + child: LogicalPlan) extends ObjectConsumer with ObjectProducer { override lazy val schema = outputSchema } @@ -175,7 +175,7 @@ object MapElements { case class MapElements( func: AnyRef, outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer + child: LogicalPlan) extends ObjectConsumer with ObjectProducer /** Factory for constructing new `AppendColumn` nodes. */ object AppendColumns { @@ -215,7 +215,7 @@ case class AppendColumnsWithObject( func: Any => Any, childSerializer: Seq[NamedExpression], newColumnsSerializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode with ObjectConsumer { + child: LogicalPlan) extends ObjectConsumer { override def output: Seq[Attribute] = (childSerializer ++ newColumnsSerializer).map(_.toAttribute) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index faf359f54838..5cfb6d5363a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -303,7 +303,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { "logical except operator should have been replaced by anti-join in the optimizer") case logical.DeserializeToObject(deserializer, objAttr, child) => - execution.DeserializeToObject(deserializer, objAttr, planLater(child)) :: Nil + execution.DeserializeToObjectExec(deserializer, objAttr, planLater(child)) :: Nil case logical.SerializeFromObject(serializer, child) => execution.SerializeFromObjectExec(serializer, planLater(child)) :: Nil case logical.MapPartitions(f, objAttr, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 3ff991392dfb..5fced940b38d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -28,17 +28,41 @@ import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.{DataType, ObjectType} + +/** + * Physical version of `ObjectProducer`. + */ +trait ObjectProducerExec extends SparkPlan { + // The attribute that reference to the single object field this operator outputs. + protected def outputObjAttr: Attribute + + override def output: Seq[Attribute] = outputObjAttr :: Nil + + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + + def outputObjectType: DataType = outputObjAttr.dataType +} + +/** + * Physical version of `ObjectConsumer`. + */ +trait ObjectConsumerExec extends UnaryExecNode { + assert(child.output.length == 1) + + // This operator always need all columns of its child, even it doesn't reference to. + override def references: AttributeSet = child.outputSet + + def inputObjectType: DataType = child.output.head.dataType +} + /** * Takes the input row from child and turns it into object using the given deserializer expression. * The output of this operator is a single-field safe row containing the deserialized object. */ -case class DeserializeToObject( +case class DeserializeToObjectExec( deserializer: Expression, outputObjAttr: Attribute, - child: SparkPlan) extends UnaryExecNode with CodegenSupport { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + child: SparkPlan) extends UnaryExecNode with ObjectProducerExec with CodegenSupport { override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() @@ -70,7 +94,7 @@ case class DeserializeToObject( */ case class SerializeFromObjectExec( serializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode with CodegenSupport { + child: SparkPlan) extends ObjectConsumerExec with CodegenSupport { override def output: Seq[Attribute] = serializer.map(_.toAttribute) @@ -102,7 +126,7 @@ case class SerializeFromObjectExec( /** * Helper functions for physical operators that work with user defined objects. */ -trait ObjectOperator extends SparkPlan { +object ObjectOperator { def deserializeRowToObject( deserializer: Expression, inputSchema: Seq[Attribute]): InternalRow => Any = { @@ -141,15 +165,12 @@ case class MapPartitionsExec( func: Iterator[Any] => Iterator[Any], outputObjAttr: Attribute, child: SparkPlan) - extends UnaryExecNode with ObjectOperator { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + extends ObjectConsumerExec with ObjectProducerExec { override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getObject = unwrapObjectFromRow(child.output.head.dataType) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) func(iter.map(getObject)).map(outputObject) } } @@ -166,10 +187,7 @@ case class MapElementsExec( func: AnyRef, outputObjAttr: Attribute, child: SparkPlan) - extends UnaryExecNode with ObjectOperator with CodegenSupport { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + extends ObjectConsumerExec with ObjectProducerExec with CodegenSupport { override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() @@ -202,8 +220,8 @@ case class MapElementsExec( } child.execute().mapPartitionsInternal { iter => - val getObject = unwrapObjectFromRow(child.output.head.dataType) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) iter.map(row => outputObject(callFunc(getObject(row)))) } } @@ -218,7 +236,7 @@ case class AppendColumnsExec( func: Any => Any, deserializer: Expression, serializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode with ObjectOperator { + child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = child.output ++ serializer.map(_.toAttribute) @@ -226,9 +244,9 @@ case class AppendColumnsExec( override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getObject = deserializeRowToObject(deserializer, child.output) + val getObject = ObjectOperator.deserializeRowToObject(deserializer, child.output) val combiner = GenerateUnsafeRowJoiner.create(child.schema, newColumnSchema) - val outputObject = serializeObjectToRow(serializer) + val outputObject = ObjectOperator.serializeObjectToRow(serializer) iter.map { row => val newColumns = outputObject(func(getObject(row))) @@ -246,7 +264,7 @@ case class AppendColumnsWithObjectExec( func: Any => Any, inputSerializer: Seq[NamedExpression], newColumnsSerializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode with ObjectOperator { + child: SparkPlan) extends ObjectConsumerExec { override def output: Seq[Attribute] = (inputSerializer ++ newColumnsSerializer).map(_.toAttribute) @@ -255,9 +273,9 @@ case class AppendColumnsWithObjectExec( override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getChildObject = unwrapObjectFromRow(child.output.head.dataType) - val outputChildObject = serializeObjectToRow(inputSerializer) - val outputNewColumnOjb = serializeObjectToRow(newColumnsSerializer) + val getChildObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) + val outputChildObject = ObjectOperator.serializeObjectToRow(inputSerializer) + val outputNewColumnOjb = ObjectOperator.serializeObjectToRow(newColumnsSerializer) val combiner = GenerateUnsafeRowJoiner.create(inputSchema, newColumnSchema) iter.map { row => @@ -280,10 +298,7 @@ case class MapGroupsExec( groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: SparkPlan) extends UnaryExecNode with ObjectOperator { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(groupingAttributes) :: Nil @@ -295,9 +310,9 @@ case class MapGroupsExec( child.execute().mapPartitionsInternal { iter => val grouped = GroupedIterator(iter, groupingAttributes, child.output) - val getKey = deserializeRowToObject(keyDeserializer, groupingAttributes) - val getValue = deserializeRowToObject(valueDeserializer, dataAttributes) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) + val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) grouped.flatMap { case (key, rowIter) => val result = func( @@ -325,10 +340,7 @@ case class CoGroupExec( rightAttr: Seq[Attribute], outputObjAttr: Attribute, left: SparkPlan, - right: SparkPlan) extends BinaryExecNode with ObjectOperator { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + right: SparkPlan) extends BinaryExecNode with ObjectProducerExec { override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil @@ -341,10 +353,10 @@ case class CoGroupExec( val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) - val getKey = deserializeRowToObject(keyDeserializer, leftGroup) - val getLeft = deserializeRowToObject(leftDeserializer, leftAttr) - val getRight = deserializeRowToObject(rightDeserializer, rightAttr) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, leftGroup) + val getLeft = ObjectOperator.deserializeRowToObject(leftDeserializer, leftAttr) + val getRight = ObjectOperator.deserializeRowToObject(rightDeserializer, rightAttr) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup).flatMap { case (key, leftResult, rightResult) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 1935e41185ac..52e706285c7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -711,6 +711,11 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(e.message.contains("already exists")) dataset.sparkSession.catalog.dropTempView("tempView") } + + test("SPARK-15381: physical object operator should define `reference` correctly") { + val df = Seq(1 -> 2).toDF("a", "b") + checkAnswer(df.map(row => row)(RowEncoder(df.schema)).select("b", "a"), Row(2, 1)) + } } case class Generic[T](id: T, value: Double) From b2a4dac2d92e906460fe3ca0a38fc672a82eb6cb Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 19 May 2016 08:52:41 +0200 Subject: [PATCH 268/313] [SPARK-15031][EXAMPLES][FOLLOW-UP] Make Python param example working with SparkSession ## What changes were proposed in this pull request? It seems most of Python examples were changed to use SparkSession by https://github.com/apache/spark/pull/12809. This PR said both examples below: - `simple_params_example.py` - `aft_survival_regression.py` are not changed because it dose not work. It seems `aft_survival_regression.py` is changed by https://github.com/apache/spark/pull/13050 but `simple_params_example.py` is not yet. This PR corrects the example and make this use SparkSession. In more detail, it seems `threshold` is replaced to `thresholds` here and there by https://github.com/apache/spark/commit/5a23213c148bfe362514f9c71f5273ebda0a848a. However, when it calls `lr.fit(training, paramMap)` this overwrites the values. So, `threshold` was 5 and `thresholds` becomes 5.5 (by `1 / (1 + thresholds(0) / thresholds(1)`). According to the comment below. this is not allowed, https://github.com/apache/spark/blob/354f8f11bd4b20fa99bd67a98da3525fd3d75c81/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L58-L61. So, in this PR, it sets the equivalent value so that this does not throw an exception. ## How was this patch tested? Manully (`mvn package -DskipTests && spark-submit simple_params_example.py`) Author: hyukjinkwon Closes #13135 from HyukjinKwon/SPARK-15031. (cherry picked from commit e2ec32dab8530aa21ec95a27d60b1c22f3d1a18c) Signed-off-by: Nick Pentreath --- .../examples/ml/JavaSimpleParamsExample.java | 2 +- .../main/python/ml/simple_params_example.py | 24 +++++++++---------- .../examples/ml/SimpleParamsExample.scala | 2 +- 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index ff1eb07dc605..ca80d0d8bba5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -77,7 +77,7 @@ public static void main(String[] args) { ParamMap paramMap = new ParamMap(); paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. - double[] thresholds = {0.45, 0.55}; + double[] thresholds = {0.5, 0.5}; paramMap.put(lr.regParam().w(0.1), lr.thresholds().w(thresholds)); // Specify multiple Params. // One can also combine ParamMaps. diff --git a/examples/src/main/python/ml/simple_params_example.py b/examples/src/main/python/ml/simple_params_example.py index 2d6d115d54d0..c57e59d01b54 100644 --- a/examples/src/main/python/ml/simple_params_example.py +++ b/examples/src/main/python/ml/simple_params_example.py @@ -20,11 +20,10 @@ import pprint import sys -from pyspark import SparkContext from pyspark.ml.classification import LogisticRegression from pyspark.mllib.linalg import DenseVector from pyspark.mllib.regression import LabeledPoint -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession """ A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -33,21 +32,20 @@ """ if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: simple_params_example", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonSimpleParamsExample") - sqlContext = SQLContext(sc) + spark = SparkSession \ + .builder \ + .appName("SimpleTextClassificationPipeline") \ + .getOrCreate() # prepare training data. # We create an RDD of LabeledPoints and convert them into a DataFrame. # A LabeledPoint is an Object with two fields named label and features # and Spark SQL identifies these fields and creates the schema appropriately. - training = sc.parallelize([ + training = spark.createDataFrame([ LabeledPoint(1.0, DenseVector([0.0, 1.1, 0.1])), LabeledPoint(0.0, DenseVector([2.0, 1.0, -1.0])), LabeledPoint(0.0, DenseVector([2.0, 1.3, 1.0])), - LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]).toDF() + LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]) # Create a LogisticRegression instance with maxIter = 10. # This instance is an Estimator. @@ -70,7 +68,7 @@ # We may alternatively specify parameters using a parameter map. # paramMap overrides all lr parameters set earlier. - paramMap = {lr.maxIter: 20, lr.thresholds: [0.45, 0.55], lr.probabilityCol: "myProbability"} + paramMap = {lr.maxIter: 20, lr.thresholds: [0.5, 0.5], lr.probabilityCol: "myProbability"} # Now learn a new model using the new parameters. model2 = lr.fit(training, paramMap) @@ -78,10 +76,10 @@ pprint.pprint(model2.extractParamMap()) # prepare test data. - test = sc.parallelize([ + test = spark.createDataFrame([ LabeledPoint(1.0, DenseVector([-1.0, 1.5, 1.3])), LabeledPoint(0.0, DenseVector([3.0, 2.0, -0.1])), - LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]).toDF() + LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]) # Make predictions on test data using the Transformer.transform() method. # LogisticRegressionModel.transform will only use the 'features' column. @@ -95,4 +93,4 @@ print("features=%s,label=%s -> prob=%s, prediction=%s" % (row.features, row.label, row.myProbability, row.prediction)) - sc.stop() + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 3355c8ffa295..29f1f509608a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -70,7 +70,7 @@ object SimpleParamsExample { // which supports several methods for specifying parameters. val paramMap = ParamMap(lr.maxIter -> 20) paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. - paramMap.put(lr.regParam -> 0.1, lr.thresholds -> Array(0.45, 0.55)) // Specify multiple Params. + paramMap.put(lr.regParam -> 0.1, lr.thresholds -> Array(0.5, 0.5)) // Specify multiple Params. // One can also combine ParamMaps. val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name From ff115f515d5d0469a1d25c3cf3eaec8c9f5dd495 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 May 2016 15:57:44 +0800 Subject: [PATCH 269/313] [SPARK-14939][SQL] Add FoldablePropagation optimizer ## What changes were proposed in this pull request? This PR aims to add new **FoldablePropagation** optimizer that propagates foldable expressions by replacing all attributes with the aliases of original foldable expression. Other optimizations will take advantage of the propagated foldable expressions: e.g. `EliminateSorts` optimizer now can handle the following Case 2 and 3. (Case 1 is the previous implementation.) 1. Literals and foldable expression, e.g. "ORDER BY 1.0, 'abc', Now()" 2. Foldable ordinals, e.g. "SELECT 1.0, 'abc', Now() ORDER BY 1, 2, 3" 3. Foldable aliases, e.g. "SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, z" This PR has been generalized based on cloud-fan 's key ideas many times; he should be credited for the work he did. **Before** ``` scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain == Physical Plan == WholeStageCodegen : +- Sort [1.0#5 ASC,x#0 ASC], true, 0 : +- INPUT +- Exchange rangepartitioning(1.0#5 ASC, x#0 ASC, 200), None +- WholeStageCodegen : +- Project [1.0 AS 1.0#5,1461873043577000 AS x#0] : +- INPUT +- Scan OneRowRelation[] ``` **After** ``` scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain == Physical Plan == WholeStageCodegen : +- Project [1.0 AS 1.0#5,1461873079484000 AS x#0] : +- INPUT +- Scan OneRowRelation[] ``` ## How was this patch tested? Pass the Jenkins tests including a new test case. Author: Dongjoon Hyun Closes #12719 from dongjoon-hyun/SPARK-14939. (cherry picked from commit 5907ebfc11aac8029cfc6d5f8e91cd5f53af54fe) Signed-off-by: Wenchen Fan --- .../expressions/objects/objects.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 42 +++++- .../optimizer/AggregateOptimizeSuite.scala | 21 +-- .../optimizer/EliminateSortsSuite.scala | 15 +- .../optimizer/FoldablePropagationSuite.scala | 131 ++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 8 ++ 6 files changed, 208 insertions(+), 11 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index fc38369f38c3..5e17f8920901 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -656,7 +656,7 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) extends UnaryExpression with NonSQLExpression { override def dataType: DataType = child.dataType - + override def foldable: Boolean = false override def nullable: Boolean = false override def eval(input: InternalRow): Any = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index be9f03d4baaa..6825b65e2b28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -22,7 +22,7 @@ import scala.collection.immutable.HashSet import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} -import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, DistinctAggregationRewriter, EliminateSubqueryAliases, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -91,6 +91,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) CombineUnions, // Constant folding and strength reduction NullPropagation, + FoldablePropagation, OptimizeIn(conf), ConstantFolding, LikeSimplification, @@ -657,6 +658,45 @@ object NullPropagation extends Rule[LogicalPlan] { } } +/** + * Propagate foldable expressions: + * Replace attributes with aliases of the original foldable expressions if possible. + * Other optimizations will take advantage of the propagated foldable expressions. + * + * {{{ + * SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, 3 + * ==> SELECT 1.0 x, 'abc' y, Now() z ORDER BY 1.0, 'abc', Now() + * }}} + */ +object FoldablePropagation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val foldableMap = AttributeMap(plan.flatMap { + case Project(projectList, _) => projectList.collect { + case a: Alias if a.resolved && a.child.foldable => (a.toAttribute, a) + } + case _ => Nil + }) + + if (foldableMap.isEmpty) { + plan + } else { + var stop = false + CleanupAliases(plan.transformUp { + case u: Union => + stop = true + u + case c: Command => + stop = true + c + case p: LogicalPlan if !stop => p.transformExpressions { + case a: AttributeReference if foldableMap.contains(a) => + foldableMap(a) + } + }) + } + } +} + /** * Generate a list of additional filters from an operator's existing constraint but remove those * that are either already part of the operator's condition or are part of the operator's child diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala index c94dcb33546f..4c26c184b7b5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala @@ -34,29 +34,34 @@ class AggregateOptimizeSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Aggregate", FixedPoint(100), + FoldablePropagation, RemoveLiteralFromGroupExpressions, RemoveRepetitionFromGroupExpressions) :: Nil } + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + test("remove literals in grouping expression") { - val input = LocalRelation('a.int, 'b.int) + val query = testRelation.groupBy('a, Literal("1"), Literal(1) + Literal(2))(sum('b)) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = testRelation.groupBy('a)(sum('b)).analyze - val query = - input.groupBy('a, Literal(1), Literal(1) + Literal(2))(sum('b)) - val optimized = Optimize.execute(query) + comparePlans(optimized, correctAnswer) + } - val correctAnswer = input.groupBy('a)(sum('b)) + test("Remove aliased literals") { + val query = testRelation.select('a, Literal(1).as('y)).groupBy('a, 'y)(sum('b)) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = testRelation.select('a, Literal(1).as('y)).groupBy('a)(sum('b)).analyze comparePlans(optimized, correctAnswer) } test("remove repetition in grouping expression") { val input = LocalRelation('a.int, 'b.int, 'c.int) - val query = input.groupBy('a + 1, 'b + 2, Literal(1) + 'A, Literal(2) + 'B)(sum('c)) val optimized = Optimize.execute(analyzer.execute(query)) - - val correctAnswer = analyzer.execute(input.groupBy('a + 1, 'b + 2)(sum('c))) + val correctAnswer = input.groupBy('a + 1, 'b + 2)(sum('c)).analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala index 8c92ad82ac5b..7402918c1bbb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala @@ -34,7 +34,8 @@ class EliminateSortsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Eliminate Sorts", Once, + Batch("Eliminate Sorts", FixedPoint(10), + FoldablePropagation, EliminateSorts) :: Nil } @@ -69,4 +70,16 @@ class EliminateSortsSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("Remove no-op alias") { + val x = testRelation + + val query = x.select('a.as('x), Year(CurrentDate()).as('y), 'b) + .orderBy('x.asc, 'y.asc, 'b.desc) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = analyzer.execute( + x.select('a.as('x), Year(CurrentDate()).as('y), 'b).orderBy('x.asc, 'b.desc)) + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala new file mode 100644 index 000000000000..355b3fc4aa63 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + +class FoldablePropagationSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Foldable Propagation", FixedPoint(20), + FoldablePropagation) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int) + + test("Propagate from subquery") { + val query = OneRowRelation + .select(Literal(1).as('a), Literal(2).as('b)) + .subquery('T) + .select('a, 'b) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = OneRowRelation + .select(Literal(1).as('a), Literal(2).as('b)) + .subquery('T) + .select(Literal(1).as('a), Literal(2).as('b)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to select clause") { + val query = testRelation + .select('a.as('x), "str".as('y), 'b.as('z)) + .select('x, 'y, 'z) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), "str".as('y), 'b.as('z)) + .select('x, "str".as('y), 'z).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to where clause") { + val query = testRelation + .select("str".as('y)) + .where('y === "str" && "str" === 'y) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select("str".as('y)) + .where("str".as('y) === "str" && "str" === "str".as('y)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to orderBy clause") { + val query = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .orderBy('x.asc, 'y.asc, 'b.desc) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .orderBy('x.asc, SortOrder(Year(CurrentDate()), Ascending), 'b.desc).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to groupBy clause") { + val query = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .groupBy('x, 'y, 'b)(sum('x), avg('y).as('AVG), count('b)) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .groupBy('x, Year(CurrentDate()).as('y), 'b)(sum('x), avg(Year(CurrentDate())).as('AVG), + count('b)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate in a complex query") { + val query = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .where('x > 1 && 'y === 2016 && 'b > 1) + .groupBy('x, 'y, 'b)(sum('x), avg('y).as('AVG), count('b)) + .orderBy('x.asc, 'AVG.asc) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .where('x > 1 && Year(CurrentDate()).as('y) === 2016 && 'b > 1) + .groupBy('x, Year(CurrentDate()).as("y"), 'b)(sum('x), avg(Year(CurrentDate())).as('AVG), + count('b)) + .orderBy('x.asc, 'AVG.asc).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate in subqueries of Union queries") { + val query = Union( + Seq( + testRelation.select(Literal(1).as('x), 'a).select('x + 'a), + testRelation.select(Literal(2).as('x), 'a).select('x + 'a))) + .select('x) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = Union( + Seq( + testRelation.select(Literal(1).as('x), 'a).select((Literal(1).as('x) + 'a).as("(x + a)")), + testRelation.select(Literal(2).as('x), 'a).select((Literal(2).as('x) + 'a).as("(x + a)")))) + .select('x).analyze + + comparePlans(optimized, correctAnswer) + } +} 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 010dea5b3038..743a27aa7a21 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 @@ -2499,6 +2499,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("Eliminate noop ordinal ORDER BY") { + withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "true") { + val plan1 = sql("SELECT 1.0, 'abc', year(current_date()) ORDER BY 1, 2, 3") + val plan2 = sql("SELECT 1.0, 'abc', year(current_date())") + comparePlans(plan1.queryExecution.optimizedPlan, plan2.queryExecution.optimizedPlan) + } + } + test("check code injection is prevented") { // The end of comment (*/) should be escaped. var literal = From 282a2a79cf4cbfec5697991d52577c6419bdda6c Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 10:25:33 -0700 Subject: [PATCH 270/313] [SPARK-15362][ML] Make spark.ml KMeansModel load backwards compatible ## What changes were proposed in this pull request? [SPARK-14646](https://issues.apache.org/jira/browse/SPARK-14646) makes ```KMeansModel``` store the cluster centers one per row. ```KMeansModel.load()``` method needs to be updated in order to load models saved with Spark 1.6. ## How was this patch tested? Since ```save/load``` is ```Experimental``` for 1.6, I think offline test for backwards compatibility is enough. Author: Yanbo Liang Closes #13149 from yanboliang/spark-15362. (cherry picked from commit 1052d3644d7eb0e784eb883293ce63a352a3b123) Signed-off-by: Joseph K. Bradley --- .../apache/spark/ml/clustering/KMeans.scala | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 41c0aec0ecf9..986f7e0fb0a5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -185,6 +185,12 @@ object KMeansModel extends MLReadable[KMeansModel] { /** Helper class for storing model data */ private case class Data(clusterIdx: Int, clusterCenter: Vector) + /** + * We store all cluster centers in a single row and use this class to store model data by + * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. + */ + private case class OldData(clusterCenters: Array[OldVector]) + /** [[MLWriter]] instance for [[KMeansModel]] */ private[KMeansModel] class KMeansModelWriter(instance: KMeansModel) extends MLWriter { @@ -211,13 +217,19 @@ object KMeansModel extends MLReadable[KMeansModel] { import sqlContext.implicits._ val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] - val clusterCenters = data.collect().sortBy(_.clusterIdx).map(_.clusterCenter) - val model = new KMeansModel(metadata.uid, - new MLlibKMeansModel(clusterCenters.map(OldVectors.fromML))) + val versionRegex = "([0-9]+)\\.(.+)".r + val versionRegex(major, _) = metadata.sparkVersion + + val clusterCenters = if (major.toInt >= 2) { + val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] + data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) + } else { + // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. + sqlContext.read.parquet(dataPath).as[OldData].head().clusterCenters + } + val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) DefaultParamsReader.getAndSetParams(model, metadata) model } From 9f2730b0c7cefe83747e5dbc8a4e6ccbe5f72167 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 10:27:17 -0700 Subject: [PATCH 271/313] [SPARK-15292][ML] ML 2.0 QA: Scala APIs audit for classification ## What changes were proposed in this pull request? Audit Scala API for classification, almost all issues were related ```MultilayerPerceptronClassifier``` in this section. * Fix one wrong param getter function: ```getOptimizer``` -> ```getSolver``` * Add missing setter function for ```solver``` and ```stepSize```. * Make ```GD``` solver take effect. * Update docs, annotations and fix other minor issues. ## How was this patch tested? Existing unit tests. Author: Yanbo Liang Closes #13076 from yanboliang/spark-15292. (cherry picked from commit 8ecf7f77b2be0a178a8d94d60477876d4ab7517a) Signed-off-by: Joseph K. Bradley --- .../MultilayerPerceptronClassifier.scala | 121 +++++++++++++----- .../MultilayerPerceptronClassifierSuite.scala | 5 +- 2 files changed, 91 insertions(+), 35 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 683ae4aaf407..c4e882240ffd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -32,22 +32,22 @@ import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset /** Params for Multilayer Perceptron. */ -private[ml] trait MultilayerPerceptronParams extends PredictorParams +private[classification] trait MultilayerPerceptronParams extends PredictorParams with HasSeed with HasMaxIter with HasTol with HasStepSize { /** * Layer sizes including input size and output size. - * Default: Array(1, 1) * * @group param */ + @Since("1.5.0") final val layers: IntArrayParam = new IntArrayParam(this, "layers", - "Sizes of layers from input layer to output layer" + - " E.g., Array(780, 100, 10) means 780 inputs, " + + "Sizes of layers from input layer to output layer. " + + "E.g., Array(780, 100, 10) means 780 inputs, " + "one hidden layer with 100 neurons and output layer of 10 neurons.", - (t: Array[Int]) => t.forall(ParamValidators.gt(0)) && t.length > 1 - ) + (t: Array[Int]) => t.forall(ParamValidators.gt(0)) && t.length > 1) /** @group getParam */ + @Since("1.5.0") final def getLayers: Array[Int] = $(layers) /** @@ -59,42 +59,49 @@ private[ml] trait MultilayerPerceptronParams extends PredictorParams * * @group expertParam */ + @Since("1.5.0") final val blockSize: IntParam = new IntParam(this, "blockSize", "Block size for stacking input data in matrices. Data is stacked within partitions." + " If block size is more than remaining data in a partition then " + "it is adjusted to the size of this data. Recommended size is between 10 and 1000", ParamValidators.gt(0)) - /** @group getParam */ + /** @group expertGetParam */ + @Since("1.5.0") final def getBlockSize: Int = $(blockSize) /** - * Allows setting the solver: minibatch gradient descent (gd) or l-bfgs. - * l-bfgs is the default one. + * The solver algorithm for optimization. + * Supported options: "gd" (minibatch gradient descent) or "l-bfgs". + * Default: "l-bfgs" * * @group expertParam */ + @Since("2.0.0") final val solver: Param[String] = new Param[String](this, "solver", - " Allows setting the solver: minibatch gradient descent (gd) or l-bfgs. " + - " l-bfgs is the default one.", - ParamValidators.inArray[String](Array("gd", "l-bfgs"))) + "The solver algorithm for optimization. Supported options: " + + s"${MultilayerPerceptronClassifier.supportedSolvers.mkString(", ")}. (Default l-bfgs)", + ParamValidators.inArray[String](MultilayerPerceptronClassifier.supportedSolvers)) - /** @group getParam */ - final def getOptimizer: String = $(solver) + /** @group expertGetParam */ + @Since("2.0.0") + final def getSolver: String = $(solver) /** - * Model weights. Can be returned either after training or after explicit setting + * The initial weights of the model. * * @group expertParam */ - final val weights: Param[Vector] = new Param[Vector](this, "weights", - " Sets the weights of the model ") - - /** @group getParam */ - final def getWeights: Vector = $(weights) + @Since("2.0.0") + final val initialWeights: Param[Vector] = new Param[Vector](this, "initialWeights", + "The initial weights of the model") + /** @group expertGetParam */ + @Since("2.0.0") + final def getInitialWeights: Vector = $(initialWeights) - setDefault(maxIter -> 100, tol -> 1e-4, blockSize -> 128, solver -> "l-bfgs", stepSize -> 0.03) + setDefault(maxIter -> 100, tol -> 1e-4, blockSize -> 128, + solver -> MultilayerPerceptronClassifier.LBFGS, stepSize -> 0.03) } /** Label to vector converter. */ @@ -145,14 +152,32 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( @Since("1.5.0") def this() = this(Identifiable.randomUID("mlpc")) - /** @group setParam */ + /** + * Sets the value of param [[layers]]. + * + * @group setParam + */ @Since("1.5.0") def setLayers(value: Array[Int]): this.type = set(layers, value) - /** @group setParam */ + /** + * Sets the value of param [[blockSize]]. + * Default is 128. + * + * @group expertSetParam + */ @Since("1.5.0") def setBlockSize(value: Int): this.type = set(blockSize, value) + /** + * Sets the value of param [[solver]]. + * Default is "l-bfgs". + * + * @group expertSetParam + */ + @Since("2.0.0") + def setSolver(value: String): this.type = set(solver, value) + /** * Set the maximum number of iterations. * Default is 100. @@ -181,12 +206,21 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( def setSeed(value: Long): this.type = set(seed, value) /** - * Sets the model weights. + * Sets the value of param [[initialWeights]]. * - * @group expertParam + * @group expertSetParam + */ + @Since("2.0.0") + def setInitialWeights(value: Vector): this.type = set(initialWeights, value) + + /** + * Sets the value of param [[stepSize]] (applicable only for solver "gd"). + * Default is 0.03. + * + * @group setParam */ @Since("2.0.0") - def setWeights(value: Vector): this.type = set(weights, value) + def setStepSize(value: Double): this.type = set(stepSize, value) @Since("1.5.0") override def copy(extra: ParamMap): MultilayerPerceptronClassifier = defaultCopy(extra) @@ -204,16 +238,26 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( val labels = myLayers.last val lpData = extractLabeledPoints(dataset) val data = lpData.map(lp => LabelConverter.encodeLabeledPoint(lp, labels)) - val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, true) + val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, softmaxOnTop = true) val trainer = new FeedForwardTrainer(topology, myLayers(0), myLayers.last) - if (isDefined(weights)) { - trainer.setWeights($(weights)) + if (isDefined(initialWeights)) { + trainer.setWeights($(initialWeights)) } else { trainer.setSeed($(seed)) } - trainer.LBFGSOptimizer - .setConvergenceTol($(tol)) - .setNumIterations($(maxIter)) + if ($(solver) == MultilayerPerceptronClassifier.LBFGS) { + trainer.LBFGSOptimizer + .setConvergenceTol($(tol)) + .setNumIterations($(maxIter)) + } else if ($(solver) == MultilayerPerceptronClassifier.GD) { + trainer.SGDOptimizer + .setNumIterations($(maxIter)) + .setConvergenceTol($(tol)) + .setStepSize($(stepSize)) + } else { + throw new IllegalArgumentException( + s"The solver $solver is not supported by MultilayerPerceptronClassifier.") + } trainer.setStackSize($(blockSize)) val mlpModel = trainer.train(data) new MultilayerPerceptronClassificationModel(uid, myLayers, mlpModel.weights) @@ -224,6 +268,15 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( object MultilayerPerceptronClassifier extends DefaultParamsReadable[MultilayerPerceptronClassifier] { + /** String name for "l-bfgs" solver. */ + private[classification] val LBFGS = "l-bfgs" + + /** String name for "gd" (minibatch gradient descent) solver. */ + private[classification] val GD = "gd" + + /** Set of solvers that MultilayerPerceptronClassifier supports. */ + private[classification] val supportedSolvers = Array(LBFGS, GD) + @Since("2.0.0") override def load(path: String): MultilayerPerceptronClassifier = super.load(path) } @@ -250,7 +303,9 @@ class MultilayerPerceptronClassificationModel private[ml] ( @Since("1.6.0") override val numFeatures: Int = layers.head - private val mlpModel = FeedForwardTopology.multiLayerPerceptron(layers, true).model(weights) + private val mlpModel = FeedForwardTopology + .multiLayerPerceptron(layers, softmaxOnTop = true) + .model(weights) /** * Returns layers in a Java List. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index 85f325f0765a..e809dd4092af 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -70,6 +70,7 @@ class MultilayerPerceptronClassifierSuite .setBlockSize(1) .setSeed(123L) .setMaxIter(100) + .setSolver("l-bfgs") val model = trainer.fit(dataset) val result = model.transform(dataset) val predictionAndLabels = result.select("prediction", "label").collect() @@ -93,9 +94,9 @@ class MultilayerPerceptronClassifierSuite .setMaxIter(1) .setTol(1e-6) val initialWeights = trainer.fit(dataFrame).weights - trainer.setWeights(initialWeights.copy) + trainer.setInitialWeights(initialWeights.copy) val weights1 = trainer.fit(dataFrame).weights - trainer.setWeights(initialWeights.copy) + trainer.setInitialWeights(initialWeights.copy) val weights2 = trainer.fit(dataFrame).weights assert(weights1 ~== weights2 absTol 10e-5, "Training should produce the same weights given equal initial weights and number of steps") From bd609b0b79ea9b9aa67b16d64007c242df0f1291 Mon Sep 17 00:00:00 2001 From: Pravin Gadakh Date: Thu, 19 May 2016 10:59:07 -0700 Subject: [PATCH 272/313] [SPARK-14613][ML] Add @Since into the matrix and vector classes in spark-mllib-local ## What changes were proposed in this pull request? This PR add `Since` annotations in `Vectors.scala` and `Matrices.scala` of spark-mllib-local. ## How was this patch tested? Scala Style Checks. Author: Pravin Gadakh Closes #13191 from pravingadakh/SPARK-14613. (cherry picked from commit 31f63ac25da43746fdef2a9477f6a79ac046112f) Signed-off-by: DB Tsai --- .../org/apache/spark/ml/linalg/Matrices.scala | 71 ++++++++++++++++--- .../org/apache/spark/ml/linalg/Vectors.scala | 37 ++++++++-- 2 files changed, 94 insertions(+), 14 deletions(-) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index 8204b5af02cf..a47526d36f1a 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -24,21 +24,28 @@ import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, Has import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.annotation.Since + /** * Trait for a local matrix. */ +@Since("2.0.0") sealed trait Matrix extends Serializable { /** Number of rows. */ + @Since("2.0.0") def numRows: Int /** Number of columns. */ + @Since("2.0.0") def numCols: Int /** Flag that keeps track whether the matrix is transposed or not. False by default. */ + @Since("2.0.0") val isTransposed: Boolean = false /** Converts to a dense array in column major. */ + @Since("2.0.0") def toArray: Array[Double] = { val newArray = new Array[Double](numRows * numCols) foreachActive { (i, j, v) => @@ -51,18 +58,21 @@ sealed trait Matrix extends Serializable { * Returns an iterator of column vectors. * This operation could be expensive, depending on the underlying storage. */ + @Since("2.0.0") def colIter: Iterator[Vector] /** * Returns an iterator of row vectors. * This operation could be expensive, depending on the underlying storage. */ + @Since("2.0.0") def rowIter: Iterator[Vector] = this.transpose.colIter /** Converts to a breeze matrix. */ private[ml] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ + @Since("2.0.0") def apply(i: Int, j: Int): Double /** Return the index for the (i, j)-th element in the backing array. */ @@ -72,12 +82,15 @@ sealed trait Matrix extends Serializable { private[ml] def update(i: Int, j: Int, v: Double): Unit /** Get a deep copy of the matrix. */ + @Since("2.0.0") def copy: Matrix /** Transpose the Matrix. Returns a new `Matrix` instance sharing the same underlying data. */ + @Since("2.0.0") def transpose: Matrix /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + @Since("2.0.0") def multiply(y: DenseMatrix): DenseMatrix = { val C: DenseMatrix = DenseMatrix.zeros(numRows, y.numCols) BLAS.gemm(1.0, this, y, 0.0, C) @@ -85,11 +98,13 @@ sealed trait Matrix extends Serializable { } /** Convenience method for `Matrix`-`DenseVector` multiplication. For binary compatibility. */ + @Since("2.0.0") def multiply(y: DenseVector): DenseVector = { multiply(y.asInstanceOf[Vector]) } /** Convenience method for `Matrix`-`Vector` multiplication. */ + @Since("2.0.0") def multiply(y: Vector): DenseVector = { val output = new DenseVector(new Array[Double](numRows)) BLAS.gemv(1.0, this, y, 0.0, output) @@ -100,6 +115,7 @@ sealed trait Matrix extends Serializable { override def toString: String = toBreeze.toString() /** A human readable representation of the matrix with maximum lines and width */ + @Since("2.0.0") def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) /** @@ -129,11 +145,13 @@ sealed trait Matrix extends Serializable { /** * Find the number of non-zero active values. */ + @Since("2.0.0") def numNonzeros: Int /** * Find the number of values stored explicitly. These values can be zero as well. */ + @Since("2.0.0") def numActives: Int } @@ -154,10 +172,11 @@ sealed trait Matrix extends Serializable { * @param isTransposed whether the matrix is transposed. If true, `values` stores the matrix in * row major. */ -class DenseMatrix ( - val numRows: Int, - val numCols: Int, - val values: Array[Double], +@Since("2.0.0") +class DenseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val values: Array[Double], override val isTransposed: Boolean) extends Matrix { require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + @@ -178,6 +197,7 @@ class DenseMatrix ( * @param numCols number of columns * @param values matrix entries in column major */ + @Since("2.0.0") def this(numRows: Int, numCols: Int, values: Array[Double]) = this(numRows, numCols, values, false) @@ -266,6 +286,7 @@ class DenseMatrix ( * Generate a `SparseMatrix` from the given `DenseMatrix`. The new matrix will have isTransposed * set to false. */ + @Since("2.0.0") def toSparse: SparseMatrix = { val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble val colPtrs: Array[Int] = new Array[Int](numCols + 1) @@ -307,6 +328,7 @@ class DenseMatrix ( /** * Factory methods for [[org.apache.spark.ml.linalg.DenseMatrix]]. */ +@Since("2.0.0") object DenseMatrix { /** @@ -315,6 +337,7 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros */ + @Since("2.0.0") def zeros(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -327,6 +350,7 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones */ + @Since("2.0.0") def ones(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -338,6 +362,7 @@ object DenseMatrix { * @param n number of rows and columns of the matrix * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def eye(n: Int): DenseMatrix = { val identity = DenseMatrix.zeros(n, n) var i = 0 @@ -355,6 +380,7 @@ object DenseMatrix { * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -368,6 +394,7 @@ object DenseMatrix { * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -380,6 +407,7 @@ object DenseMatrix { * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` * on the diagonal */ + @Since("2.0.0") def diag(vector: Vector): DenseMatrix = { val n = vector.size val matrix = DenseMatrix.zeros(n, n) @@ -415,12 +443,13 @@ object DenseMatrix { * Compressed Sparse Row (CSR) format, where `colPtrs` behaves as rowPtrs, * and `rowIndices` behave as colIndices, and `values` are stored in row major. */ -class SparseMatrix ( - val numRows: Int, - val numCols: Int, - val colPtrs: Array[Int], - val rowIndices: Array[Int], - val values: Array[Double], +@Since("2.0.0") +class SparseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val colPtrs: Array[Int], + @Since("2.0.0") val rowIndices: Array[Int], + @Since("2.0.0") val values: Array[Double], override val isTransposed: Boolean) extends Matrix { require(values.length == rowIndices.length, "The number of row indices and values don't match! " + @@ -451,6 +480,7 @@ class SparseMatrix ( * order for each column * @param values non-zero matrix entries in column major */ + @Since("2.0.0") def this( numRows: Int, numCols: Int, @@ -550,6 +580,7 @@ class SparseMatrix ( * Generate a `DenseMatrix` from the given `SparseMatrix`. The new matrix will have isTransposed * set to false. */ + @Since("2.0.0") def toDense: DenseMatrix = { new DenseMatrix(numRows, numCols, toArray) } @@ -594,6 +625,7 @@ class SparseMatrix ( /** * Factory methods for [[org.apache.spark.ml.linalg.SparseMatrix]]. */ +@Since("2.0.0") object SparseMatrix { /** @@ -605,6 +637,7 @@ object SparseMatrix { * @param entries Array of (i, j, value) tuples * @return The corresponding `SparseMatrix` */ + @Since("2.0.0") def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = { val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1)) val numEntries = sortedEntries.size @@ -653,6 +686,7 @@ object SparseMatrix { * @param n number of rows and columns of the matrix * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def speye(n: Int): SparseMatrix = { new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0)) } @@ -722,6 +756,7 @@ object SparseMatrix { * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextDouble()) @@ -735,6 +770,7 @@ object SparseMatrix { * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextGaussian()) @@ -746,6 +782,7 @@ object SparseMatrix { * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero * `values` on the diagonal */ + @Since("2.0.0") def spdiag(vector: Vector): SparseMatrix = { val n = vector.size vector match { @@ -762,6 +799,7 @@ object SparseMatrix { /** * Factory methods for [[org.apache.spark.ml.linalg.Matrix]]. */ +@Since("2.0.0") object Matrices { /** @@ -771,6 +809,7 @@ object Matrices { * @param numCols number of columns * @param values matrix entries in column major */ + @Since("2.0.0") def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { new DenseMatrix(numRows, numCols, values) } @@ -784,6 +823,7 @@ object Matrices { * @param rowIndices the row index of the entry * @param values non-zero matrix entries in column major */ + @Since("2.0.0") def sparse( numRows: Int, numCols: Int, @@ -825,6 +865,7 @@ object Matrices { * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of zeros */ + @Since("2.0.0") def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols) /** @@ -833,6 +874,7 @@ object Matrices { * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of ones */ + @Since("2.0.0") def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols) /** @@ -840,6 +882,7 @@ object Matrices { * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def eye(n: Int): Matrix = DenseMatrix.eye(n) /** @@ -847,6 +890,7 @@ object Matrices { * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def speye(n: Int): Matrix = SparseMatrix.speye(n) /** @@ -856,6 +900,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def rand(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.rand(numRows, numCols, rng) @@ -867,6 +912,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprand(numRows, numCols, density, rng) @@ -877,6 +923,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def randn(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.randn(numRows, numCols, rng) @@ -888,6 +935,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprandn(numRows, numCols, density, rng) @@ -897,6 +945,7 @@ object Matrices { * @return Square `Matrix` with size `values.length` x `values.length` and `values` * on the diagonal */ + @Since("2.0.0") def diag(vector: Vector): Matrix = DenseMatrix.diag(vector) /** @@ -906,6 +955,7 @@ object Matrices { * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were horizontally concatenated */ + @Since("2.0.0") def horzcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) @@ -964,6 +1014,7 @@ object Matrices { * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were vertically concatenated */ + @Since("2.0.0") def vertcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index c0d112d2c53d..59f9c2adba5f 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -25,21 +25,26 @@ import scala.collection.JavaConverters._ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} +import org.apache.spark.annotation.Since + /** * Represents a numeric vector, whose index type is Int and value type is Double. * * Note: Users should not implement this interface. */ +@Since("2.0.0") sealed trait Vector extends Serializable { /** * Size of the vector. */ + @Since("2.0.0") def size: Int /** * Converts the instance to a double array. */ + @Since("2.0.0") def toArray: Array[Double] override def equals(other: Any): Boolean = { @@ -93,11 +98,13 @@ sealed trait Vector extends Serializable { * Gets the value of the ith element. * @param i index */ + @Since("2.0.0") def apply(i: Int): Double = toBreeze(i) /** * Makes a deep copy of this vector. */ + @Since("2.0.0") def copy: Vector = { throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") } @@ -109,32 +116,38 @@ sealed trait Vector extends Serializable { * the vector with type `Int`, and the second parameter is the corresponding value * with type `Double`. */ + @Since("2.0.0") def foreachActive(f: (Int, Double) => Unit): Unit /** * Number of active entries. An "active entry" is an element which is explicitly stored, * regardless of its value. Note that inactive entries have value 0. */ + @Since("2.0.0") def numActives: Int /** * Number of nonzero elements. This scans all active values and count nonzeros. */ + @Since("2.0.0") def numNonzeros: Int /** * Converts this vector to a sparse vector with all explicit zeros removed. */ + @Since("2.0.0") def toSparse: SparseVector /** * Converts this vector to a dense vector. */ + @Since("2.0.0") def toDense: DenseVector = new DenseVector(this.toArray) /** * Returns a vector in either dense or sparse format, whichever uses less storage. */ + @Since("2.0.0") def compressed: Vector = { val nnz = numNonzeros // A dense vector needs 8 * size + 8 bytes, while a sparse vector needs 12 * nnz + 20 bytes. @@ -149,6 +162,7 @@ sealed trait Vector extends Serializable { * Find the index of a maximal element. Returns the first maximal element in case of a tie. * Returns -1 if vector has length 0. */ + @Since("2.0.0") def argmax: Int } @@ -157,12 +171,14 @@ sealed trait Vector extends Serializable { * We don't use the name `Vector` because Scala imports * [[scala.collection.immutable.Vector]] by default. */ +@Since("2.0.0") object Vectors { /** * Creates a dense vector from its values. */ @varargs + @Since("2.0.0") def dense(firstValue: Double, otherValues: Double*): Vector = new DenseVector((firstValue +: otherValues).toArray) @@ -170,6 +186,7 @@ object Vectors { /** * Creates a dense vector from a double array. */ + @Since("2.0.0") def dense(values: Array[Double]): Vector = new DenseVector(values) /** @@ -179,6 +196,7 @@ object Vectors { * @param indices index array, must be strictly increasing. * @param values value array, must have the same length as indices. */ + @Since("2.0.0") def sparse(size: Int, indices: Array[Int], values: Array[Double]): Vector = new SparseVector(size, indices, values) @@ -188,6 +206,7 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ + @Since("2.0.0") def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { require(size > 0, "The size of the requested sparse vector must be greater than 0.") @@ -209,6 +228,7 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ + @Since("2.0.0") def sparse(size: Int, elements: JavaIterable[(JavaInteger, JavaDouble)]): Vector = { sparse(size, elements.asScala.map { case (i, x) => (i.intValue(), x.doubleValue()) @@ -221,6 +241,7 @@ object Vectors { * @param size vector size * @return a zero vector */ + @Since("2.0.0") def zeros(size: Int): Vector = { new DenseVector(new Array[Double](size)) } @@ -253,6 +274,7 @@ object Vectors { * @param p norm. * @return norm in L^p^ space. */ + @Since("2.0.0") def norm(vector: Vector, p: Double): Double = { require(p >= 1.0, "To compute the p-norm of the vector, we require that you specify a p>=1. " + s"You specified p=$p.") @@ -305,6 +327,7 @@ object Vectors { * @param v2 second Vector. * @return squared distance between two Vectors. */ + @Since("2.0.0") def sqdist(v1: Vector, v2: Vector): Double = { require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + s"=${v2.size}.") @@ -421,7 +444,8 @@ object Vectors { /** * A dense vector represented by a value array. */ -class DenseVector (val values: Array[Double]) extends Vector { +@Since("2.0.0") +class DenseVector @Since("2.0.0") ( @Since("2.0.0") val values: Array[Double]) extends Vector { override def size: Int = values.length @@ -515,9 +539,11 @@ class DenseVector (val values: Array[Double]) extends Vector { } } +@Since("2.0.0") object DenseVector { /** Extracts the value array from a dense vector. */ + @Since("2.0.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) } @@ -528,10 +554,11 @@ object DenseVector { * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ -class SparseVector ( +@Since("2.0.0") +class SparseVector @Since("2.0.0") ( override val size: Int, - val indices: Array[Int], - val values: Array[Double]) extends Vector { + @Since("2.0.0") val indices: Array[Int], + @Since("2.0.0") val values: Array[Double]) extends Vector { require(indices.length == values.length, "Sparse vectors require that the dimension of the" + s" indices match the dimension of the values. You provided ${indices.length} indices and " + @@ -693,7 +720,9 @@ class SparseVector ( } } +@Since("2.0.0") object SparseVector { + @Since("2.0.0") def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) } From 2604eadcfad20bebe6bd73fa8da36cd631e92e55 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 11:45:18 -0700 Subject: [PATCH 273/313] [SPARK-15390] fix broadcast with 100 millions rows ## What changes were proposed in this pull request? When broadcast a table with more than 100 millions rows (should not ideally), the size of needed memory will overflow. This PR fix the overflow by converting it to Long when calculating the size of memory. Also add more checking in broadcast to show reasonable messages. ## How was this patch tested? Add test. Author: Davies Liu Closes #13182 from davies/fix_broadcast. (cherry picked from commit 9308bf119204015c8733fab0c2aef70ff2e41d74) Signed-off-by: Davies Liu --- .../exchange/BroadcastExchangeExec.scala | 13 +++++++++++-- .../sql/execution/joins/HashedRelation.scala | 5 +++-- .../sql/execution/joins/HashedRelationSuite.scala | 15 +++++++++++++++ 3 files changed, 29 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index b6ecd3cb065a..d3081ba7accd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.exchange import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import org.apache.spark.broadcast +import org.apache.spark.{broadcast, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow @@ -72,9 +72,18 @@ case class BroadcastExchangeExec( val beforeCollect = System.nanoTime() // Note that we use .executeCollect() because we don't want to convert data to Scala types val input: Array[InternalRow] = child.executeCollect() + if (input.length >= 512000000) { + throw new SparkException( + s"Cannot broadcast the table with more than 512 millions rows: ${input.length} rows") + } val beforeBuild = System.nanoTime() longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 - longMetric("dataSize") += input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum + val dataSize = input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum + longMetric("dataSize") += dataSize + if (dataSize >= (8L << 30)) { + throw new SparkException( + s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") + } // Construct and broadcast the relation. val relation = mode.transform(input) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index cb41457b6653..cd6b97a85541 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -410,9 +410,10 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private def init(): Unit = { if (mm != null) { + require(capacity < 512000000, "Cannot broadcast more than 512 millions rows") var n = 1 while (n < capacity) n *= 2 - ensureAcquireMemory(n * 2 * 8 + (1 << 20)) + ensureAcquireMemory(n * 2L * 8 + (1 << 20)) array = new Array[Long](n * 2) mask = n * 2 - 2 page = new Array[Long](1 << 17) // 1M bytes @@ -788,7 +789,7 @@ private[joins] object LongHashedRelation { sizeEstimate: Int, taskMemoryManager: TaskMemoryManager): LongHashedRelation = { - val map: LongToUnsafeRowMap = new LongToUnsafeRowMap(taskMemoryManager, sizeEstimate) + val map = new LongToUnsafeRowMap(taskMemoryManager, sizeEstimate) val keyGenerator = UnsafeProjection.create(key) // Create a mapping of key -> rows diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index b7b08dc4b126..a5b56541c90f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -212,4 +212,19 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { assert(longRelation.estimatedSize > (2L << 30)) longRelation.close() } + + test("build HashedRelation with more than 100 millions rows") { + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, IntegerType, false), + BoundReference(1, StringType, true))) + val unsafeRow = unsafeProj(InternalRow(0, UTF8String.fromString(" " * 100))) + val key = Seq(BoundReference(0, IntegerType, false)) + val rows = (0 until (1 << 10)).iterator.map { i => + unsafeRow.setInt(0, i % 1000000) + unsafeRow.setInt(1, i) + unsafeRow + } + val m = LongHashedRelation(rows, key, 100 << 20, mm) + m.close() + } } From 496f6d0fc189d81c3a354df37356d5518e22f35b Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 19 May 2016 11:46:11 -0700 Subject: [PATCH 274/313] [SPARK-14603][SQL][FOLLOWUP] Verification of Metadata Operations by Session Catalog #### What changes were proposed in this pull request? This follow-up PR is to address the remaining comments in https://github.com/apache/spark/pull/12385 The major change in this PR is to issue better error messages in PySpark by using the mechanism that was proposed by davies in https://github.com/apache/spark/pull/7135 For example, in PySpark, if we input the following statement: ```python >>> l = [('Alice', 1)] >>> df = sqlContext.createDataFrame(l) >>> df.createTempView("people") >>> df.createTempView("people") ``` Before this PR, the exception we will get is like ``` Traceback (most recent call last): File "", line 1, in File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView self._jdf.createTempView(name) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o35.createTempView. : org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException: Temporary table 'people' already exists; at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTempView(SessionCatalog.scala:324) at org.apache.spark.sql.SparkSession.createTempView(SparkSession.scala:523) at org.apache.spark.sql.Dataset.createTempView(Dataset.scala:2328) 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 py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) at py4j.Gateway.invoke(Gateway.java:280) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:211) at java.lang.Thread.run(Thread.java:745) ``` After this PR, the exception we will get become cleaner: ``` Traceback (most recent call last): File "", line 1, in File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView self._jdf.createTempView(name) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 75, in deco raise AnalysisException(s.split(': ', 1)[1], stackTrace) pyspark.sql.utils.AnalysisException: u"Temporary table 'people' already exists;" ``` #### How was this patch tested? Fixed an existing PySpark test case Author: gatorsmile Closes #13126 from gatorsmile/followup-14684. --- python/pyspark/sql/dataframe.py | 3 +-- python/pyspark/sql/utils.py | 10 +++++++++- .../spark/sql/catalyst/catalog/InMemoryCatalog.scala | 4 ++-- .../spark/sql/catalyst/catalog/SessionCatalog.scala | 4 ++-- 4 files changed, 14 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index a0264ce1acf3..a68ef33d3999 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -144,8 +144,7 @@ def createTempView(self, name): >>> df.createTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... - Py4JJavaError: ... - : org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException... + AnalysisException: u"Temporary table 'people' already exists;" >>> spark.catalog.dropTempView("people") """ diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 36c93228b94a..8c8768f50bfd 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -51,6 +51,12 @@ class ContinuousQueryException(CapturedException): """ +class QueryExecutionException(CapturedException): + """ + Failed to execute a query. + """ + + def capture_sql_exception(f): def deco(*a, **kw): try: @@ -61,12 +67,14 @@ def deco(*a, **kw): e.java_exception.getStackTrace())) if s.startswith('org.apache.spark.sql.AnalysisException: '): raise AnalysisException(s.split(': ', 1)[1], stackTrace) - if s.startswith('org.apache.spark.sql.catalyst.analysis.NoSuchTableException: '): + if s.startswith('org.apache.spark.sql.catalyst.analysis'): raise AnalysisException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.catalyst.parser.ParseException: '): raise ParseException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.ContinuousQueryException: '): raise ContinuousQueryException(s.split(': ', 1)[1], stackTrace) + if s.startswith('org.apache.spark.sql.execution.QueryExecutionException: '): + raise QueryExecutionException(s.split(': ', 1)[1], stackTrace) if s.startswith('java.lang.IllegalArgumentException: '): raise IllegalArgumentException(s.split(': ', 1)[1], stackTrace) raise diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 21da55cbc375..489a1c8c3fac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -87,7 +87,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E db: String, table: String, specs: Seq[TablePartitionSpec]): Unit = { - specs foreach { s => + specs.foreach { s => if (!partitionExists(db, table, s)) { throw new NoSuchPartitionException(db = db, table = table, spec = s) } @@ -98,7 +98,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E db: String, table: String, specs: Seq[TablePartitionSpec]): Unit = { - specs foreach { s => + specs.foreach { s => if (partitionExists(db, table, s)) { throw new PartitionAlreadyExistsException(db = db, table = table, spec = s) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 54b30d389874..cf9286e6b97a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -111,13 +111,13 @@ class SessionCatalog( fs.makeQualified(hadoopPath) } - protected[this] def requireDbExists(db: String): Unit = { + private def requireDbExists(db: String): Unit = { if (!databaseExists(db)) { throw new NoSuchDatabaseException(db) } } - protected[this] def requireTableExists(name: TableIdentifier): Unit = { + private def requireTableExists(name: TableIdentifier): Unit = { if (!tableExists(name)) { val db = name.database.getOrElse(currentDb) throw new NoSuchTableException(db = db, table = name.table) From 96a473a1187b5d21c02d4af01470347a942cc854 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 11:47:17 -0700 Subject: [PATCH 275/313] [SPARK-15300] Fix writer lock conflict when remove a block ## What changes were proposed in this pull request? A writer lock could be acquired when 1) create a new block 2) remove a block 3) evict a block to disk. 1) and 3) could happen in the same time within the same task, all of them could happen in the same time outside a task. It's OK that when someone try to grab the write block for a block, but the block is acquired by another one that has the same task attempt id. This PR remove the check. ## How was this patch tested? Updated existing tests. Author: Davies Liu Closes #13082 from davies/write_lock_conflict. (cherry picked from commit ad182086cc3bd7951aaf82693d9bcb56815b43e4) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/storage/BlockInfoManager.scala | 5 +---- .../org/apache/spark/storage/BlockInfoManagerSuite.scala | 6 ++---- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index ca53534b61c4..20ffe1342e50 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -228,10 +228,7 @@ private[storage] class BlockInfoManager extends Logging { infos.get(blockId) match { case None => return None case Some(info) => - if (info.writerTask == currentTaskAttemptId) { - throw new IllegalStateException( - s"Task $currentTaskAttemptId has already locked $blockId for writing") - } else if (info.writerTask == BlockInfo.NO_WRITER && info.readerCount == 0) { + if (info.writerTask == BlockInfo.NO_WRITER && info.readerCount == 0) { info.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 9ee83b76e71d..1b325801e27f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -208,16 +208,14 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } } - test("cannot call lockForWriting while already holding a write lock") { + test("cannot grab a writer lock while already holding a write lock") { withTaskId(0) { assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) blockInfoManager.unlock("block") } withTaskId(1) { assert(blockInfoManager.lockForWriting("block").isDefined) - intercept[IllegalStateException] { - blockInfoManager.lockForWriting("block") - } + assert(blockInfoManager.lockForWriting("block", false).isEmpty) blockInfoManager.assertBlockIsLockedForWriting("block") } } From 9c817d027713859cac483b4baaaf8b53c040ad93 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 19 May 2016 11:51:59 -0700 Subject: [PATCH 276/313] [SPARK-15387][SQL] SessionCatalog in SimpleAnalyzer does not need to make database directory. ## What changes were proposed in this pull request? After #12871 is fixed, we are forced to make `/user/hive/warehouse` when SimpleAnalyzer is used but SimpleAnalyzer may not need the directory. ## How was this patch tested? Manual test. Author: Kousuke Saruta Closes #13175 from sarutak/SPARK-15387. --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1313a011c69c..2c269478ee7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogRelation, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -45,7 +45,9 @@ object SimpleAnalyzer extends Analyzer( new SessionCatalog( new InMemoryCatalog, EmptyFunctionRegistry, - new SimpleCatalystConf(caseSensitiveAnalysis = true)), + new SimpleCatalystConf(caseSensitiveAnalysis = true)) { + override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean) {} + }, new SimpleCatalystConf(caseSensitiveAnalysis = true)) /** From 554e0f30a7fd9fae4282d93ec4c0f1c6dbdffabe Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 19 May 2016 11:54:50 -0700 Subject: [PATCH 277/313] [SPARK-15322][SQL][FOLLOW-UP] Update deprecated accumulator usage into accumulatorV2 ## What changes were proposed in this pull request? This PR corrects another case that uses deprecated `accumulableCollection` to use `listAccumulator`, which seems the previous PR missed. Since `ArrayBuffer[InternalRow].asJava` is `java.util.List[InternalRow]`, it seems ok to replace the usage. ## How was this patch tested? Related existing tests `InMemoryColumnarQuerySuite` and `CachedTableSuite`. Author: hyukjinkwon Closes #13187 from HyukjinKwon/SPARK-15322. (cherry picked from commit f5065abf49dea0eac04b0ec219f2d832a0f6730a) Signed-off-by: Andrew Or --- .../columnar/InMemoryTableScanExec.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 009fbaa00657..ba61940b3d5a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.execution.columnar -import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ import org.apache.commons.lang.StringUtils -import org.apache.spark.{Accumulable, Accumulator} +import org.apache.spark.Accumulator import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.AccumulatorContext +import org.apache.spark.util.{AccumulatorContext, ListAccumulator} private[sql] object InMemoryRelation { @@ -67,14 +67,14 @@ private[sql] case class InMemoryRelation( tableName: Option[String])( @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, @transient private[sql] var _statistics: Statistics = null, - private[sql] var _batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] = null) + private[sql] var _batchStats: ListAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { override def producedAttributes: AttributeSet = outputSet - private[sql] val batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] = + private[sql] val batchStats: ListAccumulator[InternalRow] = if (_batchStats == null) { - child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[InternalRow]) + child.sqlContext.sparkContext.listAccumulator[InternalRow] } else { _batchStats } @@ -87,7 +87,7 @@ private[sql] case class InMemoryRelation( output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), partitionStatistics.schema) - batchStats.value.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum + batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum } // Statistics propagation contracts: @@ -169,7 +169,7 @@ private[sql] case class InMemoryRelation( val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) .flatMap(_.values)) - batchStats += stats + batchStats.add(stats) CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) From 97fd9a09ce1313ad7b9569fc3ca8e944d36d0ce9 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 19 May 2016 20:59:19 +0200 Subject: [PATCH 278/313] [SPARK-15316][PYSPARK][ML] Add linkPredictionCol to GeneralizedLinearRegression ## What changes were proposed in this pull request? Add linkPredictionCol to GeneralizedLinearRegression and fix the PyDoc to generate the bullet list ## How was this patch tested? doctests & built docs locally Author: Holden Karau Closes #13106 from holdenk/SPARK-15316-add-linkPredictionCol-toGeneralizedLinearRegression. (cherry picked from commit e71cd96bf733f0440f818c6efc7a04b68d7cbe45) Signed-off-by: Nick Pentreath --- python/pyspark/ml/regression.py | 46 +++++++++++++++++++++++++-------- 1 file changed, 35 insertions(+), 11 deletions(-) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index cfcbbfc98e82..25640b1f8525 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1245,10 +1245,14 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha predictor (link function) and a description of the error distribution (family). It supports "gaussian", "binomial", "poisson" and "gamma" as family. Valid link functions for each family is listed below. The first link function of each family is the default one. - - "gaussian" -> "identity", "log", "inverse" - - "binomial" -> "logit", "probit", "cloglog" - - "poisson" -> "log", "identity", "sqrt" - - "gamma" -> "inverse", "identity", "log" + + * "gaussian" -> "identity", "log", "inverse" + + * "binomial" -> "logit", "probit", "cloglog" + + * "poisson" -> "log", "identity", "sqrt" + + * "gamma" -> "inverse", "identity", "log" .. seealso:: `GLM `_ @@ -1258,9 +1262,12 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha ... (1.0, Vectors.dense(1.0, 2.0)), ... (2.0, Vectors.dense(0.0, 0.0)), ... (2.0, Vectors.dense(1.0, 1.0)),], ["label", "features"]) - >>> glr = GeneralizedLinearRegression(family="gaussian", link="identity") + >>> glr = GeneralizedLinearRegression(family="gaussian", link="identity", linkPredictionCol="p") >>> model = glr.fit(df) - >>> abs(model.transform(df).head().prediction - 1.5) < 0.001 + >>> transformed = model.transform(df) + >>> abs(transformed.head().prediction - 1.5) < 0.001 + True + >>> abs(transformed.head().p - 1.5) < 0.001 True >>> model.coefficients DenseVector([1.5..., -1.0...]) @@ -1290,20 +1297,23 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha "relationship between the linear predictor and the mean of the distribution " + "function. Supported options: identity, log, inverse, logit, probit, cloglog " + "and sqrt.", typeConverter=TypeConverters.toString) + linkPredictionCol = Param(Params._dummy(), "linkPredictionCol", "link prediction (linear " + + "predictor) column name", typeConverter=TypeConverters.toString) @keyword_only def __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, - regParam=0.0, weightCol=None, solver="irls"): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=""): """ __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ - regParam=0.0, weightCol=None, solver="irls") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol="") """ super(GeneralizedLinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.GeneralizedLinearRegression", self.uid) - self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls") + self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", + linkPredictionCol="") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -1311,11 +1321,11 @@ def __init__(self, labelCol="label", featuresCol="features", predictionCol="pred @since("2.0.0") def setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, - regParam=0.0, weightCol=None, solver="irls"): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=""): """ setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ - regParam=0.0, weightCol=None, solver="irls") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol="") Sets params for generalized linear regression. """ kwargs = self.setParams._input_kwargs @@ -1338,6 +1348,20 @@ def getFamily(self): """ return self.getOrDefault(self.family) + @since("2.0.0") + def setLinkPredictionCol(self, value): + """ + Sets the value of :py:attr:`linkPredictionCol`. + """ + return self._set(linkPredictionCol=value) + + @since("2.0.0") + def getLinkPredictionCol(self): + """ + Gets the value of linkPredictionCol or its default value. + """ + return self.getOrDefault(self.linkPredictionCol) + @since("2.0.0") def setLink(self, value): """ From 4f8639f9d4144a5e2b1eb24a98e4c752c2dc00b8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 19 May 2016 12:02:41 -0700 Subject: [PATCH 279/313] [SPARK-14346][SQL] Lists unsupported Hive features in SHOW CREATE TABLE output ## What changes were proposed in this pull request? This PR is a follow-up of #13079. It replaces `hasUnsupportedFeatures: Boolean` in `CatalogTable` with `unsupportedFeatures: Seq[String]`, which contains unsupported Hive features of the underlying Hive table. In this way, we can accurately report all unsupported Hive features in the exception message. ## How was this patch tested? Updated existing test case to check exception message. Author: Cheng Lian Closes #13173 from liancheng/spark-14346-follow-up. (cherry picked from commit 6ac1c3a040f88fae15c46acd73e7e3691f7d3619) Signed-off-by: Andrew Or --- .../sql/catalyst/catalog/interface.scala | 9 +++------ .../spark/sql/execution/command/tables.scala | 14 +++++++------- .../sql/hive/client/HiveClientImpl.scala | 19 ++++++++++++++----- .../spark/sql/hive/ShowCreateTableSuite.scala | 8 +++++--- 4 files changed, 29 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 3fdd411ac4cc..4a073d11893c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -80,11 +80,8 @@ case class CatalogTablePartition( * Note that Hive's metastore also tracks skewed columns. We should consider adding that in the * future once we have a better understanding of how we want to handle skewed columns. * - * @param hasUnsupportedFeatures is used to indicate whether all table metadata entries retrieved - * from the concrete underlying external catalog (e.g. Hive metastore) are supported by - * Spark SQL. For example, if the underlying Hive table has skewed columns, this information - * can't be mapped to [[CatalogTable]] since Spark SQL doesn't handle skewed columns for now. - * In this case `hasUnsupportedFeatures` is set to true. By default, it is false. + * @param unsupportedFeatures is a list of string descriptions of features that are used by the + * underlying table but not supported by Spark SQL yet. */ case class CatalogTable( identifier: TableIdentifier, @@ -102,7 +99,7 @@ case class CatalogTable( viewOriginalText: Option[String] = None, viewText: Option[String] = None, comment: Option[String] = None, - hasUnsupportedFeatures: Boolean = false) { + unsupportedFeatures: Seq[String] = Seq.empty) { // Verify that the provided columns are part of the schema private val colNames = schema.map(_.name).toSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 1fc02d1d4b46..a3472745371f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -633,16 +633,16 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman } private def showCreateHiveTable(metadata: CatalogTable): String = { - def reportUnsupportedError(): Unit = { - throw new UnsupportedOperationException( + def reportUnsupportedError(features: Seq[String]): Unit = { + throw new AnalysisException( s"Failed to execute SHOW CREATE TABLE against table ${metadata.identifier.quotedString}, " + - "because it contains table structure(s) (e.g. skewed columns) that Spark SQL doesn't " + - "support yet." + "which is created by Hive and uses the following unsupported feature(s)\n" + + features.map(" - " + _).mkString("\n") ) } - if (metadata.hasUnsupportedFeatures) { - reportUnsupportedError() + if (metadata.unsupportedFeatures.nonEmpty) { + reportUnsupportedError(metadata.unsupportedFeatures) } val builder = StringBuilder.newBuilder @@ -651,7 +651,7 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman case EXTERNAL => " EXTERNAL TABLE" case VIEW => " VIEW" case MANAGED => " TABLE" - case INDEX => reportUnsupportedError() + case INDEX => reportUnsupportedError(Seq("index table")) } builder ++= s"CREATE$tableTypeString ${table.quotedString}" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index af2850d4f568..0f0c1b0702fb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -337,10 +337,19 @@ private[hive] class HiveClientImpl( val schema = h.getCols.asScala.map(fromHiveColumn) ++ partCols // Skew spec, storage handler, and bucketing info can't be mapped to CatalogTable (yet) - val hasUnsupportedFeatures = - !h.getSkewedColNames.isEmpty || - h.getStorageHandler != null || - !h.getBucketCols.isEmpty + val unsupportedFeatures = ArrayBuffer.empty[String] + + if (!h.getSkewedColNames.isEmpty) { + unsupportedFeatures += "skewed columns" + } + + if (h.getStorageHandler != null) { + unsupportedFeatures += "storage handler" + } + + if (!h.getBucketCols.isEmpty) { + unsupportedFeatures += "bucketing" + } CatalogTable( identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), @@ -369,7 +378,7 @@ private[hive] class HiveClientImpl( properties = h.getParameters.asScala.toMap, viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), - hasUnsupportedFeatures = hasUnsupportedFeatures) + unsupportedFeatures = unsupportedFeatures) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala index 3b8068d3bc47..dedc8f55f01b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -247,7 +247,7 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing } } - test("hive bucketing not supported") { + test("hive bucketing is not supported") { withTable("t1") { createRawHiveTable( s"""CREATE TABLE t1 (a INT, b STRING) @@ -257,9 +257,11 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing """.stripMargin ) - intercept[UnsupportedOperationException] { + val cause = intercept[AnalysisException] { sql("SHOW CREATE TABLE t1") } + + assert(cause.getMessage.contains(" - bucketing")) } } From 62e5158f167425b9565a2b01fc494e593c57cae3 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 12:05:17 -0700 Subject: [PATCH 280/313] [SPARK-15317][CORE] Don't store accumulators for every task in listeners ## What changes were proposed in this pull request? In general, the Web UI doesn't need to store the Accumulator/AccumulableInfo for every task. It only needs the Accumulator values. In this PR, it creates new UIData classes to store the necessary fields and make `JobProgressListener` store only these new classes, so that `JobProgressListener` won't store Accumulator/AccumulableInfo and the size of `JobProgressListener` becomes pretty small. I also eliminates `AccumulableInfo` from `SQLListener` so that we don't keep any references for those unused `AccumulableInfo`s. ## How was this patch tested? I ran two tests reported in JIRA locally: The first one is: ``` val data = spark.range(0, 10000, 1, 10000) data.cache().count() ``` The retained size of JobProgressListener decreases from 60.7M to 6.9M. The second one is: ``` import org.apache.spark.ml.CC import org.apache.spark.sql.SQLContext val sqlContext = SQLContext.getOrCreate(sc) CC.runTest(sqlContext) ``` This test won't cause OOM after applying this patch. Author: Shixiong Zhu Closes #13153 from zsxwing/memory. (cherry picked from commit 4e3cb7a5d965fd490390398ecfe35f1fc05e8511) Signed-off-by: Andrew Or --- .../status/api/v1/AllStagesResource.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 12 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../org/apache/spark/ui/jobs/UIData.scala | 136 +++++++++++++++++- .../org/apache/spark/util/AccumulatorV2.scala | 3 + .../api/v1/AllStagesResourceSuite.scala | 4 +- .../ui/jobs/JobProgressListenerSuite.scala | 29 +++- .../sql/execution/metric/SQLMetrics.scala | 8 +- .../spark/sql/execution/ui/SQLListener.scala | 13 +- .../execution/metric/SQLMetricsSuite.scala | 7 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- 11 files changed, 186 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index eddc36edc961..7d63a8f734f0 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -20,10 +20,10 @@ import java.util.{Arrays, Date, List => JList} import javax.ws.rs.{GET, Produces, QueryParam} import javax.ws.rs.core.MediaType -import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} +import org.apache.spark.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} import org.apache.spark.util.Distribution @Produces(Array(MediaType.APPLICATION_JSON)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 945830c8bf24..842f42b4c98d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -332,7 +332,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { new StageUIData }) stageData.numActiveTasks += 1 - stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo, Some(metrics))) + stageData.taskData.put(taskInfo.taskId, TaskUIData(taskInfo, Some(metrics))) } for ( activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); @@ -395,9 +395,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) } - val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) - taskData.taskInfo = info - taskData.metrics = taskMetrics + val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) + taskData.updateTaskInfo(info) + taskData.updateTaskMetrics(taskMetrics) taskData.errorMessage = errorMessage for ( @@ -425,7 +425,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData: StageUIData, execId: String, taskMetrics: TaskMetrics, - oldMetrics: Option[TaskMetrics]) { + oldMetrics: Option[TaskMetricsUIData]) { val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) val shuffleWriteDelta = @@ -503,7 +503,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { if (!t.taskInfo.finished) { updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.metrics) // Overwrite task metrics - t.metrics = Some(metrics) + t.updateTaskMetrics(Some(metrics)) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 6ddabfd8ef08..d986a55959b8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -768,7 +768,7 @@ private[ui] object StagePage { } private[ui] def getSchedulerDelay( - info: TaskInfo, metrics: TaskMetrics, currentTime: Long): Long = { + info: TaskInfo, metrics: TaskMetricsUIData, currentTime: Long): Long = { if (info.finished) { val totalExecutionTime = info.finishTime - info.launchTime val executorOverhead = (metrics.executorDeserializeTime + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index b454ef1b204b..d76a0e657c28 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -21,8 +21,10 @@ import scala.collection.mutable import scala.collection.mutable.HashMap import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.util.AccumulatorContext import org.apache.spark.util.collection.OpenHashSet private[spark] object UIData { @@ -105,13 +107,137 @@ private[spark] object UIData { /** * These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation. */ - class TaskUIData( - var taskInfo: TaskInfo, - var metrics: Option[TaskMetrics] = None, - var errorMessage: Option[String] = None) + class TaskUIData private( + private var _taskInfo: TaskInfo, + private var _metrics: Option[TaskMetricsUIData]) { + + var errorMessage: Option[String] = None + + def taskInfo: TaskInfo = _taskInfo + + def metrics: Option[TaskMetricsUIData] = _metrics + + def updateTaskInfo(taskInfo: TaskInfo): Unit = { + _taskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) + } + + def updateTaskMetrics(metrics: Option[TaskMetrics]): Unit = { + _metrics = TaskUIData.toTaskMetricsUIData(metrics) + } + } + + object TaskUIData { + def apply(taskInfo: TaskInfo, metrics: Option[TaskMetrics]): TaskUIData = { + new TaskUIData(dropInternalAndSQLAccumulables(taskInfo), toTaskMetricsUIData(metrics)) + } + + private def toTaskMetricsUIData(metrics: Option[TaskMetrics]): Option[TaskMetricsUIData] = { + metrics.map { m => + TaskMetricsUIData( + executorDeserializeTime = m.executorDeserializeTime, + executorRunTime = m.executorRunTime, + resultSize = m.resultSize, + jvmGCTime = m.jvmGCTime, + resultSerializationTime = m.resultSerializationTime, + memoryBytesSpilled = m.memoryBytesSpilled, + diskBytesSpilled = m.diskBytesSpilled, + peakExecutionMemory = m.peakExecutionMemory, + updatedBlockStatuses = m.updatedBlockStatuses.toList, + inputMetrics = InputMetricsUIData(m.inputMetrics.bytesRead, m.inputMetrics.recordsRead), + outputMetrics = + OutputMetricsUIData(m.outputMetrics.bytesWritten, m.outputMetrics.recordsWritten), + shuffleReadMetrics = ShuffleReadMetricsUIData(m.shuffleReadMetrics), + shuffleWriteMetrics = ShuffleWriteMetricsUIData(m.shuffleWriteMetrics)) + } + } + + /** + * We don't need to store internal or SQL accumulables as their values will be shown in other + * places, so drop them to reduce the memory usage. + */ + private[spark] def dropInternalAndSQLAccumulables(taskInfo: TaskInfo): TaskInfo = { + val newTaskInfo = new TaskInfo( + taskId = taskInfo.taskId, + index = taskInfo.index, + attemptNumber = taskInfo.attemptNumber, + launchTime = taskInfo.launchTime, + executorId = taskInfo.executorId, + host = taskInfo.host, + taskLocality = taskInfo.taskLocality, + speculative = taskInfo.speculative + ) + newTaskInfo.gettingResultTime = taskInfo.gettingResultTime + newTaskInfo.accumulables ++= taskInfo.accumulables.filter { + accum => !accum.internal && accum.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) + } + newTaskInfo.finishTime = taskInfo.finishTime + newTaskInfo.failed = taskInfo.failed + newTaskInfo + } + } class ExecutorUIData( val startTime: Long, var finishTime: Option[Long] = None, var finishReason: Option[String] = None) + + case class TaskMetricsUIData( + executorDeserializeTime: Long, + executorRunTime: Long, + resultSize: Long, + jvmGCTime: Long, + resultSerializationTime: Long, + memoryBytesSpilled: Long, + diskBytesSpilled: Long, + peakExecutionMemory: Long, + updatedBlockStatuses: Seq[(BlockId, BlockStatus)], + inputMetrics: InputMetricsUIData, + outputMetrics: OutputMetricsUIData, + shuffleReadMetrics: ShuffleReadMetricsUIData, + shuffleWriteMetrics: ShuffleWriteMetricsUIData) + + case class InputMetricsUIData(bytesRead: Long, recordsRead: Long) + + case class OutputMetricsUIData(bytesWritten: Long, recordsWritten: Long) + + case class ShuffleReadMetricsUIData( + remoteBlocksFetched: Long, + localBlocksFetched: Long, + remoteBytesRead: Long, + localBytesRead: Long, + fetchWaitTime: Long, + recordsRead: Long, + totalBytesRead: Long, + totalBlocksFetched: Long) + + object ShuffleReadMetricsUIData { + def apply(metrics: ShuffleReadMetrics): ShuffleReadMetricsUIData = { + new ShuffleReadMetricsUIData( + remoteBlocksFetched = metrics.remoteBlocksFetched, + localBlocksFetched = metrics.localBlocksFetched, + remoteBytesRead = metrics.remoteBytesRead, + localBytesRead = metrics.localBytesRead, + fetchWaitTime = metrics.fetchWaitTime, + recordsRead = metrics.recordsRead, + totalBytesRead = metrics.totalBytesRead, + totalBlocksFetched = metrics.totalBlocksFetched + ) + } + } + + case class ShuffleWriteMetricsUIData( + bytesWritten: Long, + recordsWritten: Long, + writeTime: Long) + + object ShuffleWriteMetricsUIData { + def apply(metrics: ShuffleWriteMetrics): ShuffleWriteMetricsUIData = { + new ShuffleWriteMetricsUIData( + bytesWritten = metrics.bytesWritten, + recordsWritten = metrics.recordsWritten, + writeTime = metrics.writeTime + ) + } + } + } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 0cf9df084fdb..13cb6a28c346 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -255,6 +255,9 @@ private[spark] object AccumulatorContext { def clear(): Unit = { originals.clear() } + + // Identifier for distinguishing SQL metrics from other accumulators + private[spark] val SQL_ACCUM_IDENTIFIER = "sql" } diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala index d223af1496a4..f684e16c25f7 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala @@ -30,8 +30,8 @@ class AllStagesResourceSuite extends SparkFunSuite { def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = { val tasks = new HashMap[Long, TaskUIData] taskLaunchTimes.zipWithIndex.foreach { case (time, idx) => - tasks(idx.toLong) = new TaskUIData( - new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None) + tasks(idx.toLong) = TaskUIData( + new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None) } val stageUiData = new StageUIData() diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index ce7d51d1c371..6f7c9f282a58 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -25,7 +25,8 @@ import org.apache.spark._ import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor._ import org.apache.spark.scheduler._ -import org.apache.spark.util.Utils +import org.apache.spark.ui.jobs.UIData.TaskUIData +import org.apache.spark.util.{AccumulatorContext, Utils} class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers { @@ -359,4 +360,30 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with assert( stage1Data.taskData.get(1237L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 402) } + + test("drop internal and sql accumulators") { + val taskInfo = new TaskInfo(0, 0, 0, 0, "", "", TaskLocality.ANY, false) + val internalAccum = + AccumulableInfo(id = 1, name = Some("internal"), None, None, internal = true, false) + val sqlAccum = AccumulableInfo( + id = 2, + name = Some("sql"), + None, + None, + internal = false, + countFailedValues = false, + metadata = Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) + val userAccum = AccumulableInfo( + id = 3, + name = Some("user"), + None, + None, + internal = false, + countFailedValues = false, + metadata = None) + taskInfo.accumulables ++= Seq(internalAccum, sqlAccum, userAccum) + + val newTaskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) + assert(newTaskInfo.accumulables === Seq(userAccum)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index d6de15494fef..e63c7c581e9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -21,7 +21,7 @@ import java.text.NumberFormat import org.apache.spark.SparkContext import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.util.{AccumulatorV2, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, Utils} class SQLMetric(val metricType: String, initValue: Long = 0L) extends AccumulatorV2[Long, Long] { @@ -56,15 +56,13 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato // Provide special identifier as metadata so we can tell that this is a `SQLMetric` later private[spark] override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { - new AccumulableInfo(id, name, update, value, true, true, Some(SQLMetrics.ACCUM_IDENTIFIER)) + new AccumulableInfo( + id, name, update, value, true, true, Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) } } private[sql] object SQLMetrics { - // Identifier for distinguishing SQL metrics from other accumulators - private[sql] val ACCUM_IDENTIFIER = "sql" - private[sql] val SUM_METRIC = "sum" private[sql] val SIZE_METRIC = "size" private[sql] val TIMING_METRIC = "timing" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 510a2ee3bfaf..03b532664a0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -26,6 +26,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric._ import org.apache.spark.ui.SparkUI +import org.apache.spark.util.AccumulatorContext @DeveloperApi case class SparkListenerSQLExecutionStart( @@ -177,8 +178,10 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi taskId: Long, stageId: Int, stageAttemptID: Int, - accumulatorUpdates: Seq[AccumulableInfo], + _accumulatorUpdates: Seq[AccumulableInfo], finishTask: Boolean): Unit = { + val accumulatorUpdates = + _accumulatorUpdates.filter(_.update.isDefined).map(accum => (accum.id, accum.update.get)) _stageIdToStageMetrics.get(stageId) match { case Some(stageMetrics) => @@ -290,9 +293,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi stageMetrics <- _stageIdToStageMetrics.get(stageId).toIterable; taskMetrics <- stageMetrics.taskIdToMetricUpdates.values; accumulatorUpdate <- taskMetrics.accumulatorUpdates) yield { - assert(accumulatorUpdate.update.isDefined, s"accumulator update from " + - s"task did not have a partial value: ${accumulatorUpdate.name}") - (accumulatorUpdate.id, accumulatorUpdate.update.get) + (accumulatorUpdate._1, accumulatorUpdate._2) } }.filter { case (id, _) => executionUIData.accumulatorMetrics.contains(id) } mergeAccumulatorUpdates(accumulatorUpdates, accumulatorId => @@ -336,7 +337,7 @@ private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) taskEnd.taskInfo.accumulables.flatMap { a => // Filter out accumulators that are not SQL metrics // For now we assume all SQL metrics are Long's that have been JSON serialized as String's - if (a.metadata == Some(SQLMetrics.ACCUM_IDENTIFIER)) { + if (a.metadata == Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) { val newValue = a.update.map(_.toString.toLong).getOrElse(0L) Some(a.copy(update = Some(newValue))) } else { @@ -418,4 +419,4 @@ private[ui] class SQLStageMetrics( private[ui] class SQLTaskMetrics( val attemptId: Long, // TODO not used yet var finished: Boolean, - var accumulatorUpdates: Seq[AccumulableInfo]) + var accumulatorUpdates: Seq[(Long, Any)]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 08f596f130aa..7a89b484eb85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -29,9 +29,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.{AccumulatorContext, JsonProtocol, Utils} class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { @@ -308,7 +307,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { case Some(v) => fail(s"metric value was not a Long: ${v.getClass.getName}") case _ => fail("metric update is missing") } - assert(metricInfo.metadata === Some(SQLMetrics.ACCUM_IDENTIFIER)) + assert(metricInfo.metadata === Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) // After serializing to JSON, the original value type is lost, but we can still // identify that it's a SQL metric from the metadata val metricInfoJson = JsonProtocol.accumulableInfoToJson(metricInfo) @@ -318,7 +317,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { case Some(v) => fail(s"deserialized metric value was not a string: ${v.getClass.getName}") case _ => fail("deserialized metric update is missing") } - assert(metricInfoDeser.metadata === Some(SQLMetrics.ACCUM_IDENTIFIER)) + assert(metricInfoDeser.metadata === Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 9eff42ab2d55..1c467137baa8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -383,7 +383,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { } // Listener tracks only SQL metrics, not other accumulators assert(trackedAccums.size === 1) - assert(trackedAccums.head === sqlMetricInfo) + assert(trackedAccums.head === (sqlMetricInfo.id, sqlMetricInfo.update.get)) } } From d1b5df83d789510340e20a98bee8fd3e0e55b8f8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 12:12:42 -0700 Subject: [PATCH 281/313] [SPARK-15392][SQL] fix default value of size estimation of logical plan ## What changes were proposed in this pull request? We use autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD. This PR change the default value to Long.MaxValue. ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #13183 from davies/fix_default_size. (cherry picked from commit 5ccecc078aa757d3f1f6632aa6df5659490f602f) Signed-off-by: Andrew Or --- python/pyspark/sql/dataframe.py | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 6 ++-- .../org/apache/spark/sql/JoinSuite.scala | 2 +- .../apache/spark/sql/StatisticsSuite.scala | 34 +++++++++++++++++++ 4 files changed, 39 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index a68ef33d3999..4fa799ac55bd 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -576,7 +576,7 @@ def alias(self, alias): >>> df_as2 = df.alias("df_as2") >>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner') >>> joined_df.select("df_as1.name", "df_as2.name", "df_as2.age").collect() - [Row(name=u'Alice', name=u'Alice', age=2), Row(name=u'Bob', name=u'Bob', age=5)] + [Row(name=u'Bob', name=u'Bob', age=5), Row(name=u'Alice', name=u'Alice', age=2)] """ assert isinstance(alias, basestring), "alias should be a string" return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 248c6e3335f2..5d1868980163 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -120,8 +120,8 @@ object SQLConf { "nodes when performing a join. By setting this value to -1 broadcasting can be disabled. " + "Note that currently statistics are only supported for Hive Metastore tables where the " + "commandANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run.") - .intConf - .createWithDefault(10 * 1024 * 1024) + .longConf + .createWithDefault(10L * 1024 * 1024) val DEFAULT_SIZE_IN_BYTES = SQLConfigBuilder("spark.sql.defaultSizeInBytes") .internal() @@ -599,7 +599,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def subexpressionEliminationEnabled: Boolean = getConf(SUBEXPRESSION_ELIMINATION_ENABLED) - def autoBroadcastJoinThreshold: Int = getConf(AUTO_BROADCASTJOIN_THRESHOLD) + def autoBroadcastJoinThreshold: Long = getConf(AUTO_BROADCASTJOIN_THRESHOLD) def preferSortMergeJoin: Boolean = getConf(PREFER_SORTMERGEJOIN) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index a6b83b3d0750..a5d8cb19eadc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -438,7 +438,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { spark.cacheManager.clearCache() sql("CACHE TABLE testData") - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000000000") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString) { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[BroadcastHashJoinExec]), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala new file mode 100644 index 000000000000..9523f6f9f5bb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ + +class StatisticsSuite extends QueryTest with SharedSQLContext { + + test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { + val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) + val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) + assert(df.queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + } + +} From 4257ba37203b406fa63b978db101e578be075bfb Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 12:32:26 -0700 Subject: [PATCH 282/313] Fix the compiler error introduced by #13153 for Scala 2.10 --- .../spark/ui/jobs/JobProgressListenerSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 6f7c9f282a58..1fa9b28edf4b 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -364,20 +364,20 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with test("drop internal and sql accumulators") { val taskInfo = new TaskInfo(0, 0, 0, 0, "", "", TaskLocality.ANY, false) val internalAccum = - AccumulableInfo(id = 1, name = Some("internal"), None, None, internal = true, false) + AccumulableInfo(id = 1, name = Some("internal"), None, None, true, false, None) val sqlAccum = AccumulableInfo( id = 2, name = Some("sql"), - None, - None, + update = None, + value = None, internal = false, countFailedValues = false, metadata = Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) val userAccum = AccumulableInfo( id = 3, name = Some("user"), - None, - None, + update = None, + value = None, internal = false, countFailedValues = false, metadata = None) From 833dbf9263f90b2b61dd5148249b8ec49524a8c4 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 19 May 2016 13:10:51 -0700 Subject: [PATCH 283/313] [SPARK-15411][ML] Add @since to ml.stat.MultivariateOnlineSummarizer.scala ## What changes were proposed in this pull request? Add since to ml.stat.MultivariateOnlineSummarizer.scala ## How was this patch tested? unit tests Author: DB Tsai Closes #13197 from dbtsai/cleanup. (cherry picked from commit 5255e55c843c7b67fcb2abb4284b8b1a09bd6672) Signed-off-by: DB Tsai --- .../ml/stat/distribution/MultivariateGaussian.scala | 11 ++++++++--- .../stat/distribution/MultivariateGaussian.scala | 4 ++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala index c62a1eab2016..383d6d96e8d7 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.stat.distribution import breeze.linalg.{diag, eigSym, max, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.ml.impl.Utils import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} @@ -32,9 +33,11 @@ import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} * @param mean The mean vector of the distribution * @param cov The covariance matrix of the distribution */ -class MultivariateGaussian( - val mean: Vector, - val cov: Matrix) extends Serializable { +@Since("2.0.0") +@DeveloperApi +class MultivariateGaussian @Since("2.0.0") ( + @Since("2.0.0") val mean: Vector, + @Since("2.0.0") val cov: Matrix) extends Serializable { require(cov.numCols == cov.numRows, "Covariance matrix must be square") require(mean.size == cov.numCols, "Mean vector length must match covariance matrix size") @@ -56,6 +59,7 @@ class MultivariateGaussian( /** * Returns density of this multivariate Gaussian at given point, x */ + @Since("2.0.0") def pdf(x: Vector): Double = { pdf(x.toBreeze) } @@ -63,6 +67,7 @@ class MultivariateGaussian( /** * Returns the log-density of this multivariate Gaussian at given point, x */ + @Since("2.0.0") def logpdf(x: Vector): Double = { logpdf(x.toBreeze) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index 6c6e9fb7c6b3..c806d6bdf6b6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -64,7 +64,7 @@ class MultivariateGaussian @Since("1.3.0") ( /** * Returns density of this multivariate Gaussian at given point, x */ - @Since("1.3.0") + @Since("1.3.0") def pdf(x: Vector): Double = { pdf(x.toBreeze) } @@ -72,7 +72,7 @@ class MultivariateGaussian @Since("1.3.0") ( /** * Returns the log-density of this multivariate Gaussian at given point, x */ - @Since("1.3.0") + @Since("1.3.0") def logpdf(x: Vector): Double = { logpdf(x.toBreeze) } From ebf30eddd5f72b3785cfa0042719b2a983e3788e Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 13:26:41 -0700 Subject: [PATCH 284/313] [SPARK-15361][ML] ML 2.0 QA: Scala APIs audit for ml.clustering ## What changes were proposed in this pull request? Audit Scala API for ml.clustering. Fix some wrong API documentations and update outdated one. ## How was this patch tested? Existing unit tests. Author: Yanbo Liang Closes #13148 from yanboliang/spark-15361. (cherry picked from commit 59e6c5560d13def686091391aabe024ecb43174b) Signed-off-by: Joseph K. Bradley --- .../spark/ml/clustering/BisectingKMeans.scala | 22 +++++++++++-------- .../spark/ml/clustering/GaussianMixture.scala | 22 ++++++++++++++++--- .../apache/spark/ml/clustering/KMeans.scala | 14 +++++++----- .../org/apache/spark/ml/clustering/LDA.scala | 6 ++--- 4 files changed, 43 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 138e059f9493..afb1080b9b7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -41,23 +41,27 @@ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter with HasFeaturesCol with HasSeed with HasPredictionCol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * The desired number of leaf clusters. Must be > 1. Default: 4. + * The actual number could be smaller if there are no divisible leaf clusters. * @group param */ @Since("2.0.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "The desired number of leaf clusters. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("2.0.0") def getK: Int = $(k) - /** @group expertParam */ + /** + * The minimum number of points (if >= 1.0) or the minimum proportion + * of points (if < 1.0) of a divisible cluster (default: 1.0). + * @group expertParam + */ @Since("2.0.0") - final val minDivisibleClusterSize = new DoubleParam( - this, - "minDivisibleClusterSize", - "the minimum number of points (if >= 1.0) or the minimum proportion", - (value: Double) => value > 0) + final val minDivisibleClusterSize = new DoubleParam(this, "minDivisibleClusterSize", + "The minimum number of points (if >= 1.0) or the minimum proportion " + + "of points (if < 1.0) of a divisible cluster.", ParamValidators.gt(0.0)) /** @group expertGetParam */ @Since("2.0.0") @@ -78,7 +82,7 @@ private[clustering] trait BisectingKMeansParams extends Params * :: Experimental :: * Model fitted by BisectingKMeans. * - * @param parentModel a model trained by spark.mllib.clustering.BisectingKMeans. + * @param parentModel a model trained by [[org.apache.spark.mllib.clustering.BisectingKMeans]]. */ @Since("2.0.0") @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 63ca812609b7..d81b33760755 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -25,7 +25,7 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.impl.Utils.EPSILON import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param.{IntParam, ParamMap, Params} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.stat.distribution.MultivariateGaussian import org.apache.spark.ml.util._ @@ -45,11 +45,12 @@ private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter w with HasSeed with HasPredictionCol with HasProbabilityCol with HasTol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * Number of independent Gaussians in the mixture model. Must be > 1. Default: 2. * @group param */ @Since("2.0.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "Number of independent Gaussians in the mixture model. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("2.0.0") @@ -249,6 +250,21 @@ object GaussianMixtureModel extends MLReadable[GaussianMixtureModel] { /** * :: Experimental :: * Gaussian Mixture clustering. + * + * This class performs expectation maximization for multivariate Gaussian + * Mixture Models (GMMs). A GMM represents a composite distribution of + * independent Gaussian distributions with associated "mixing" weights + * specifying each's contribution to the composite. + * + * Given a set of sample points, this class will maximize the log-likelihood + * for a mixture of k Gaussians, iterating until the log-likelihood changes by + * less than convergenceTol, or until it has reached the max number of iterations. + * While this process is generally guaranteed to converge, it is not guaranteed + * to find a global optimum. + * + * Note: For high-dimensional data (with many features), this algorithm may perform poorly. + * This is due to high-dimensional data (a) making it difficult to cluster at all (based + * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. */ @Since("2.0.0") @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 986f7e0fb0a5..0ab370e3b476 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -23,7 +23,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{Vector, VectorUDT} -import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} @@ -41,11 +41,12 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe with HasSeed with HasPredictionCol with HasTol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * The number of clusters to create (k). Must be > 1. Default: 2. * @group param */ @Since("1.5.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "The number of clusters to create. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("1.5.0") @@ -58,7 +59,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * @group expertParam */ @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "initialization algorithm", + final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + + "Supported options: 'random' and 'k-means||'.", (value: String) => MLlibKMeans.validateInitMode(value)) /** @group expertGetParam */ @@ -71,8 +73,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * @group expertParam */ @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "number of steps for k-means||", - (value: Int) => value > 0) + final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + + "initialization mode. Must be > 0.", ParamValidators.gt(0)) /** @group expertGetParam */ @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 5a83b2870005..ec60991af64f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -50,8 +50,8 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM * @group param */ @Since("1.6.0") - final val k = new IntParam(this, "k", "number of topics (clusters) to infer", - ParamValidators.gt(1)) + final val k = new IntParam(this, "k", "The number of topics (clusters) to infer. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("1.6.0") @@ -165,7 +165,7 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM */ @Since("1.6.0") final val optimizer = new Param[String](this, "optimizer", "Optimizer or inference" + - " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "), + " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "), (o: String) => ParamValidators.inArray(supportedOptimizers).apply(o.toLowerCase)) /** @group getParam */ From 758253f7c7493df60e417ed5c3ee360015cfd166 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 17:24:42 -0700 Subject: [PATCH 285/313] [SPARK-15414][MLLIB] Make the mllib,ml linalg type conversion APIs public ## What changes were proposed in this pull request? Open up APIs for converting between new, old linear algebra types (in spark.mllib.linalg): `Sparse`/`Dense` X `Vector`/`Matrices` `.asML` and `.fromML` ## How was this patch tested? Existing Tests Author: Sandeep Singh Closes #13202 from techaddict/SPARK-15414. (cherry picked from commit ef43a5fe51614eecce2d144cc13b33004a47533a) Signed-off-by: Xiangrui Meng --- .../apache/spark/mllib/linalg/Matrices.scala | 30 +++++++++++++------ .../apache/spark/mllib/linalg/Vectors.scala | 30 +++++++++++++------ 2 files changed, 42 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 5c9a112ca69c..ee1956c2d413 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -164,7 +164,8 @@ sealed trait Matrix extends Serializable { * Convert this matrix to the new mllib-local representation. * This does NOT copy the data; it copies references. */ - private[spark] def asML: newlinalg.Matrix + @Since("2.0.0") + def asML: newlinalg.Matrix } private[spark] class MatrixUDT extends UserDefinedType[Matrix] { @@ -427,7 +428,8 @@ class DenseMatrix @Since("1.3.0") ( } } - private[spark] override def asML: newlinalg.DenseMatrix = { + @Since("2.0.0") + override def asML: newlinalg.DenseMatrix = { new newlinalg.DenseMatrix(numRows, numCols, values, isTransposed) } } @@ -527,8 +529,11 @@ object DenseMatrix { matrix } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.DenseMatrix): DenseMatrix = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.DenseMatrix): DenseMatrix = { new DenseMatrix(m.numRows, m.numCols, m.values, m.isTransposed) } } @@ -740,7 +745,8 @@ class SparseMatrix @Since("1.3.0") ( } } - private[spark] override def asML: newlinalg.SparseMatrix = { + @Since("2.0.0") + override def asML: newlinalg.SparseMatrix = { new newlinalg.SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) } } @@ -918,8 +924,11 @@ object SparseMatrix { } } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.SparseMatrix): SparseMatrix = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.SparseMatrix): SparseMatrix = { new SparseMatrix(m.numRows, m.numCols, m.colPtrs, m.rowIndices, m.values, m.isTransposed) } } @@ -1205,8 +1214,11 @@ object Matrices { } } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.Matrix): Matrix = m match { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.Matrix): Matrix = m match { case dm: newlinalg.DenseMatrix => DenseMatrix.fromML(dm) case sm: newlinalg.SparseMatrix => 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 1f1cfa0cb28b..7ebcd297bd54 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 @@ -186,7 +186,8 @@ sealed trait Vector extends Serializable { * Convert this vector to the new mllib-local representation. * This does NOT copy the data; it copies references. */ - private[spark] def asML: newlinalg.Vector + @Since("2.0.0") + def asML: newlinalg.Vector } /** @@ -581,8 +582,11 @@ object Vectors { /** Max number of nonzero entries used in computing hash code. */ private[linalg] val MAX_HASH_NNZ = 128 - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.Vector): Vector = v match { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.Vector): Vector = v match { case dv: newlinalg.DenseVector => DenseVector.fromML(dv) case sv: newlinalg.SparseVector => @@ -704,7 +708,8 @@ class DenseVector @Since("1.0.0") ( compact(render(jValue)) } - private[spark] override def asML: newlinalg.DenseVector = { + @Since("2.0.0") + override def asML: newlinalg.DenseVector = { new newlinalg.DenseVector(values) } } @@ -716,8 +721,11 @@ object DenseVector { @Since("1.3.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.DenseVector): DenseVector = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.DenseVector): DenseVector = { new DenseVector(v.values) } } @@ -911,7 +919,8 @@ class SparseVector @Since("1.0.0") ( compact(render(jValue)) } - private[spark] override def asML: newlinalg.SparseVector = { + @Since("2.0.0") + override def asML: newlinalg.SparseVector = { new newlinalg.SparseVector(size, indices, values) } } @@ -922,8 +931,11 @@ object SparseVector { def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.SparseVector): SparseVector = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.SparseVector): SparseVector = { new SparseVector(v.size, v.indices, v.values) } } From 2c939e541db67a339002204e9e0c22a611ccac85 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 19 May 2016 17:42:59 -0700 Subject: [PATCH 286/313] [SPARK-15375][SQL][STREAMING] Add ConsoleSink to structure streaming ## What changes were proposed in this pull request? Add ConsoleSink to structure streaming, user could use it to display dataframes on the console (useful for debugging and demostrating), similar to the functionality of `DStream#print`, to use it: ``` val query = result.write .format("console") .trigger(ProcessingTime("2 seconds")) .startStream() ``` ## How was this patch tested? local verified. Not sure it is suitable to add into structure streaming, please review and help to comment, thanks a lot. Author: jerryshao Closes #13162 from jerryshao/SPARK-15375. (cherry picked from commit dcf407de676ce99462a62f17cd98c9df56f057df) Signed-off-by: Reynold Xin --- ...pache.spark.sql.sources.DataSourceRegister | 1 + .../sql/execution/streaming/console.scala | 60 +++++++++++++++++++ .../DataFrameReaderWriterSuite.scala | 15 +++++ 3 files changed, 76 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 226d59d0eae8..ef9255794b6a 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -3,3 +3,4 @@ org.apache.spark.sql.execution.datasources.jdbc.DefaultSource org.apache.spark.sql.execution.datasources.json.DefaultSource org.apache.spark.sql.execution.datasources.parquet.DefaultSource org.apache.spark.sql.execution.datasources.text.DefaultSource +org.apache.spark.sql.execution.streaming.ConsoleSinkProvider diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala new file mode 100644 index 000000000000..f11a3fb969db --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} + +class ConsoleSink(options: Map[String, String]) extends Sink with Logging { + // Number of rows to display, by default 20 rows + private val numRowsToShow = options.get("numRows").map(_.toInt).getOrElse(20) + + // Truncate the displayed data if it is too long, by default it is true + private val isTruncated = options.get("truncate").map(_.toBoolean).getOrElse(true) + + // Track the batch id + private var lastBatchId = -1L + + override def addBatch(batchId: Long, data: DataFrame): Unit = synchronized { + val batchIdStr = if (batchId <= lastBatchId) { + s"Rerun batch: $batchId" + } else { + lastBatchId = batchId + s"Batch: $batchId" + } + + // scalastyle:off println + println("-------------------------------------------") + println(batchIdStr) + println("-------------------------------------------") + // scalastyle:off println + data.show(numRowsToShow, isTruncated) + } +} + +class ConsoleSinkProvider extends StreamSinkProvider with DataSourceRegister { + def createSink( + sqlContext: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String]): Sink = { + new ConsoleSink(parameters) + } + + def shortName(): String = "console" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala index cb53b2b1aac1..ff5350554933 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala @@ -524,4 +524,19 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B val e = intercept[AnalysisException](w.csv("non_exist_path")) assert(e.getMessage == "csv() can only be called on non-continuous queries;") } + + test("ConsoleSink can be correctly loaded") { + LastOptions.clear() + val df = spark.read + .format("org.apache.spark.sql.streaming.test") + .stream() + + val cq = df.write + .format("console") + .option("checkpointLocation", newMetadataDir) + .trigger(ProcessingTime(2.seconds)) + .startStream() + + cq.awaitTermination(2000L) + } } From b0aff55dd4ed4edd1dd64bc42fcacdf505a6db56 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 17:54:18 -0700 Subject: [PATCH 287/313] [SPARK-15341][DOC][ML] Add documentation for "model.write" to clarify "summary" was not saved ## What changes were proposed in this pull request? Currently in ```model.write```, we don't save ```summary```(if applicable). We should add documentation to clarify it. We fixed the incorrect link ```[[MLWriter]]``` to ```[[org.apache.spark.ml.util.MLWriter]]``` BTW. ## How was this patch tested? Documentation update, no unit test. Author: Yanbo Liang Closes #13131 from yanboliang/spark-15341. (cherry picked from commit f8107c7846c9fcabbe2579867574305c7f2028e7) Signed-off-by: Xiangrui Meng --- .../spark/ml/classification/LogisticRegression.scala | 2 +- .../org/apache/spark/ml/clustering/GaussianMixture.scala | 7 +++++++ .../main/scala/org/apache/spark/ml/clustering/KMeans.scala | 7 +++++++ .../spark/ml/regression/GeneralizedLinearRegression.scala | 7 +++++++ .../org/apache/spark/ml/regression/LinearRegression.scala | 2 +- 5 files changed, 23 insertions(+), 2 deletions(-) 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 40cff8a86f36..27380ca5c0a7 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 @@ -613,7 +613,7 @@ class LogisticRegressionModel private[spark] ( } /** - * Returns a [[MLWriter]] instance for this ML instance. + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. * * For [[LogisticRegressionModel]], this does NOT currently save the training [[summary]]. * An option to save [[summary]] may be added in the future. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index d81b33760755..88b6b27e62d9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -139,6 +139,13 @@ class GaussianMixtureModel private[ml] ( sqlContext.createDataFrame(modelGaussians).toDF("mean", "cov") } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GaussianMixtureModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ @Since("2.0.0") override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 0ab370e3b476..790ef1fe8dc9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -148,6 +148,13 @@ class KMeansModel private[ml] ( parentModel.computeCost(data) } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[KMeansModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ @Since("1.6.0") override def write: MLWriter = new KMeansModel.KMeansModelWriter(this) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 4aa7c2cc0b9b..e8474d035ee6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -776,6 +776,13 @@ class GeneralizedLinearRegressionModel private[ml] ( .setParent(parent) } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GeneralizedLinearRegressionModel]], this does NOT currently save the + * training [[summary]]. An option to save [[summary]] may be added in the future. + * + */ @Since("2.0.0") override def write: MLWriter = new GeneralizedLinearRegressionModel.GeneralizedLinearRegressionModelWriter(this) 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 3e9a3f9db5f4..a702f02c9143 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 @@ -449,7 +449,7 @@ class LinearRegressionModel private[ml] ( } /** - * Returns a [[MLWriter]] instance for this ML instance. + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. * * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. * An option to save [[summary]] may be added in the future. From e53a8f2188b853e02716d36dd4bfc01b9d7abc74 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 17:56:21 -0700 Subject: [PATCH 288/313] [MINOR][ML][PYSPARK] ml.evaluation Scala and Python API sync ## What changes were proposed in this pull request? ```ml.evaluation``` Scala and Python API sync. ## How was this patch tested? Only API docs change, no new tests. Author: Yanbo Liang Closes #13195 from yanboliang/evaluation-doc. (cherry picked from commit 664367781786df7ec52e39950dccd5a09681602c) Signed-off-by: Xiangrui Meng --- .../ml/evaluation/MulticlassClassificationEvaluator.scala | 2 +- python/pyspark/ml/evaluation.py | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 840851675110..390e9b6444c7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.DoubleType /** * :: Experimental :: - * Evaluator for multiclass classification, which expects two input columns: score and label. + * Evaluator for multiclass classification, which expects two input columns: prediction and label. */ @Since("1.5.0") @Experimental diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index fc9099b7ec17..16029dc34863 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -193,9 +193,6 @@ class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): .. versionadded:: 1.4.0 """ - # Because we will maximize evaluation value (ref: `CrossValidator`), - # when we evaluate a metric that is needed to minimize (e.g., `"rmse"`, `"mse"`, `"mae"`), - # we take and output the negative of this metric. metricName = Param(Params._dummy(), "metricName", """metric name in evaluation - one of: rmse - root mean squared error (default) @@ -270,7 +267,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio """ metricName = Param(Params._dummy(), "metricName", "metric name in evaluation " - "(f1|precision|recall|weightedPrecision|weightedRecall)", + "(f1|precision|recall|weightedPrecision|weightedRecall|accuracy)", typeConverter=TypeConverters.toString) @keyword_only From 7e25131a93f2f13ecb525179bbe4ad77def84292 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 18:31:05 -0700 Subject: [PATCH 289/313] [SPARK-15416][SQL] Display a better message for not finding classes removed in Spark 2.0 ## What changes were proposed in this pull request? If finding `NoClassDefFoundError` or `ClassNotFoundException`, check if the class name is removed in Spark 2.0. If so, the user must be using an incompatible library and we can provide a better message. ## How was this patch tested? 1. Run `bin/pyspark --packages com.databricks:spark-avro_2.10:2.0.1` 2. type `sqlContext.read.format("com.databricks.spark.avro").load("src/test/resources/episodes.avro")`. It will show `java.lang.ClassNotFoundException: org.apache.spark.sql.sources.HadoopFsRelationProvider is removed in Spark 2.0. Please check if your library is compatible with Spark 2.0` Author: Shixiong Zhu Closes #13201 from zsxwing/better-message. (cherry picked from commit 16ba71aba4e68bbb892d4ceb38d6d1d135d63fd3) Signed-off-by: Michael Armbrust --- .../execution/datasources/DataSource.scala | 61 +++++++++++++------ 1 file changed, 44 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index ce45168a1319..ccad9b3fd52f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -83,6 +83,14 @@ case class DataSource( "com.databricks.spark.csv" -> classOf[csv.DefaultSource].getCanonicalName ) + /** + * Class that were removed in Spark 2.0. Used to detect incompatibility libraries for Spark 2.0. + */ + private val spark2RemovedClasses = Set( + "org.apache.spark.sql.DataFrame", + "org.apache.spark.sql.sources.HadoopFsRelationProvider", + "org.apache.spark.Logging") + /** Given a provider name, look up the data source class definition. */ private def lookupDataSource(provider0: String): Class[_] = { val provider = backwardCompatibilityMap.getOrElse(provider0, provider0) @@ -93,26 +101,45 @@ case class DataSource( serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList match { // the provider format did not match any given registered aliases case Nil => - Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match { - case Success(dataSource) => - // Found the data source using fully qualified path - dataSource - case Failure(error) => - if (provider.startsWith("org.apache.spark.sql.hive.orc")) { - throw new ClassNotFoundException( - "The ORC data source must be used with Hive support enabled.", error) - } else { - if (provider == "avro" || provider == "com.databricks.spark.avro") { + try { + Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match { + case Success(dataSource) => + // Found the data source using fully qualified path + dataSource + case Failure(error) => + if (error.isInstanceOf[ClassNotFoundException]) { + val className = error.getMessage + if (spark2RemovedClasses.contains(className)) { + throw new ClassNotFoundException(s"$className is removed in Spark 2.0. " + + "Please check if your library is compatible with Spark 2.0") + } + } + if (provider.startsWith("org.apache.spark.sql.hive.orc")) { throw new ClassNotFoundException( - s"Failed to find data source: $provider. Please use Spark package " + - "http://spark-packages.org/package/databricks/spark-avro", - error) + "The ORC data source must be used with Hive support enabled.", error) } else { - throw new ClassNotFoundException( - s"Failed to find data source: $provider. Please find packages at " + - "http://spark-packages.org", - error) + if (provider == "avro" || provider == "com.databricks.spark.avro") { + throw new ClassNotFoundException( + s"Failed to find data source: $provider. Please use Spark package " + + "http://spark-packages.org/package/databricks/spark-avro", + error) + } else { + throw new ClassNotFoundException( + s"Failed to find data source: $provider. Please find packages at " + + "http://spark-packages.org", + error) + } } + } + } catch { + case e: NoClassDefFoundError => // This one won't be caught by Scala NonFatal + // NoClassDefFoundError's class name uses "/" rather than "." for packages + val className = e.getMessage.replaceAll("/", ".") + if (spark2RemovedClasses.contains(className)) { + throw new ClassNotFoundException(s"$className was removed in Spark 2.0. " + + "Please check if your library is compatible with Spark 2.0", e) + } else { + throw e } } case head :: Nil => From 5fa23956b23a9488c1382ee17a5b3c5113e73c87 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 20:38:44 -0700 Subject: [PATCH 290/313] [SPARK-15296][MLLIB] Refactor All Java Tests that use SparkSession ## What changes were proposed in this pull request? Refactor All Java Tests that use SparkSession, to extend SharedSparkSesion ## How was this patch tested? Existing Tests Author: Sandeep Singh Closes #13101 from techaddict/SPARK-15296. (cherry picked from commit 01cf649c4f96f64fb4bd09e0e1811cabcc5ead2e) Signed-off-by: Xiangrui Meng --- .../ml/JavaGaussianMixtureExample.java | 2 +- .../org/apache/spark/SharedSparkSession.java | 48 +++++++++++++++++++ .../apache/spark/ml/JavaPipelineSuite.java | 27 +++-------- .../JavaDecisionTreeClassifierSuite.java | 27 +---------- .../JavaGBTClassifierSuite.java | 28 +---------- .../JavaLogisticRegressionSuite.java | 28 +++-------- ...vaMultilayerPerceptronClassifierSuite.java | 23 +-------- .../classification/JavaNaiveBayesSuite.java | 23 +-------- .../ml/classification/JavaOneVsRestSuite.java | 30 +++--------- .../JavaRandomForestClassifierSuite.java | 28 +---------- .../spark/ml/clustering/JavaKMeansSuite.java | 27 +++-------- .../spark/ml/feature/JavaBucketizerSuite.java | 21 +------- .../apache/spark/ml/feature/JavaDCTSuite.java | 21 +------- .../spark/ml/feature/JavaHashingTFSuite.java | 21 +------- .../spark/ml/feature/JavaNormalizerSuite.java | 24 +--------- .../apache/spark/ml/feature/JavaPCASuite.java | 26 ++-------- .../feature/JavaPolynomialExpansionSuite.java | 24 +--------- .../ml/feature/JavaStandardScalerSuite.java | 24 +--------- .../ml/feature/JavaStopWordsRemoverSuite.java | 22 +-------- .../ml/feature/JavaStringIndexerSuite.java | 26 ++-------- .../spark/ml/feature/JavaTokenizerSuite.java | 24 +--------- .../ml/feature/JavaVectorAssemblerSuite.java | 26 ++-------- .../ml/feature/JavaVectorIndexerSuite.java | 25 +--------- .../ml/feature/JavaVectorSlicerSuite.java | 21 +------- .../spark/ml/feature/JavaWord2VecSuite.java | 21 +------- .../spark/ml/param/JavaParamsSuite.java | 23 --------- .../JavaDecisionTreeRegressorSuite.java | 26 +--------- .../ml/regression/JavaGBTRegressorSuite.java | 26 +--------- .../regression/JavaLinearRegressionSuite.java | 28 +++-------- .../JavaRandomForestRegressorSuite.java | 26 +--------- .../libsvm/JavaLibSVMRelationSuite.java | 20 +++----- .../ml/tuning/JavaCrossValidatorSuite.java | 33 ++++--------- .../ml/util/JavaDefaultReadWriteSuite.java | 31 +++--------- .../JavaLogisticRegressionSuite.java | 25 +--------- .../classification/JavaNaiveBayesSuite.java | 25 +--------- .../mllib/classification/JavaSVMSuite.java | 25 +--------- .../JavaStreamingLogisticRegressionSuite.java | 3 +- .../clustering/JavaBisectingKMeansSuite.java | 26 +--------- .../clustering/JavaGaussianMixtureSuite.java | 25 +--------- .../mllib/clustering/JavaKMeansSuite.java | 25 +--------- .../spark/mllib/clustering/JavaLDASuite.java | 29 +++-------- .../clustering/JavaStreamingKMeansSuite.java | 3 +- .../evaluation/JavaRankingMetricsSuite.java | 28 +++-------- .../spark/mllib/feature/JavaTfIdfSuite.java | 25 +--------- .../mllib/feature/JavaWord2VecSuite.java | 25 +--------- .../mllib/fpm/JavaAssociationRulesSuite.java | 25 +--------- .../spark/mllib/fpm/JavaFPGrowthSuite.java | 25 +--------- .../spark/mllib/fpm/JavaPrefixSpanSuite.java | 24 +--------- .../spark/mllib/linalg/JavaMatricesSuite.java | 3 +- .../spark/mllib/linalg/JavaVectorsSuite.java | 3 +- .../mllib/random/JavaRandomRDDsSuite.java | 24 +--------- .../mllib/recommendation/JavaALSSuite.java | 25 +--------- .../JavaIsotonicRegressionSuite.java | 25 +--------- .../mllib/regression/JavaLassoSuite.java | 25 +--------- .../regression/JavaLinearRegressionSuite.java | 25 +--------- .../regression/JavaRidgeRegressionSuite.java | 25 +--------- .../JavaStreamingLinearRegressionSuite.java | 3 +- .../spark/mllib/stat/JavaStatisticsSuite.java | 3 +- .../mllib/tree/JavaDecisionTreeSuite.java | 26 +--------- 59 files changed, 207 insertions(+), 1148 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/SharedSparkSession.java diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java index 79b99095815a..526bed93fbd2 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java @@ -37,7 +37,7 @@ public class JavaGaussianMixtureExample { public static void main(String[] args) { - // Creates a SparkSession + // Creates a SparkSession SparkSession spark = SparkSession .builder() .appName("JavaGaussianMixtureExample") diff --git a/mllib/src/test/java/org/apache/spark/SharedSparkSession.java b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java new file mode 100644 index 000000000000..43779878890d --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java @@ -0,0 +1,48 @@ +/* + * 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; + +import java.io.IOException; +import java.io.Serializable; + +import org.junit.After; +import org.junit.Before; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +public abstract class SharedSparkSession implements Serializable { + + protected transient SparkSession spark; + protected transient JavaSparkContext jsc; + + @Before + public void setUp() throws IOException { + spark = SparkSession.builder() + .master("local[2]") + .appName(getClass().getSimpleName()) + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } +} diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index a81a36d1b1c4..9b209006bc36 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -17,47 +17,34 @@ package org.apache.spark.ml; -import org.junit.After; -import org.junit.Before; +import java.io.IOException; + import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegression; import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.feature.StandardScaler; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; /** * Test Pipeline construction and fitting in Java. */ -public class JavaPipelineSuite { +public class JavaPipelineSuite extends SharedSparkSession { - private transient SparkSession spark; - private transient JavaSparkContext jsc; private transient Dataset dataset; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPipelineSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); + @Override + public void setUp() throws IOException { + super.setUp(); JavaRDD points = jsc.parallelize(generateLogisticInputAsList(1.0, 1.0, 100, 42), 2); dataset = spark.createDataFrame(points, LabeledPoint.class); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void pipeline() { StandardScaler scaler = new StandardScaler() diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java index c76a1947c64f..5aba4e8f7de0 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -17,42 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaDecisionTreeClassifierSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDecisionTreeClassifierSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDecisionTreeClassifierSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java index 4648926c3421..74bb46bd217a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java @@ -17,43 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -public class JavaGBTClassifierSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaGBTClassifierSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaGBTClassifierSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index b8da04c26a49..004102103d52 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -17,52 +17,36 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaLogisticRegressionSuite implements Serializable { +public class JavaLogisticRegressionSuite extends SharedSparkSession { - private transient SparkSession spark; - private transient JavaSparkContext jsc; private transient Dataset dataset; private transient JavaRDD datasetRDD; private double eps = 1e-5; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); dataset.createOrReplaceTempView("dataset"); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java index 48edbc838c81..6d0604d8f9a5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java @@ -17,38 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaMultilayerPerceptronClassifierSuite implements Serializable { - - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaMultilayerPerceptronClassifierSuite extends SharedSparkSession { @Test public void testMLPC() { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java index 787909821be9..c2a9e7b58b47 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java @@ -17,43 +17,24 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaNaiveBayesSuite implements Serializable { - - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaNaiveBayesSuite extends SharedSparkSession { public void validatePrediction(Dataset predictionAndLabels) { for (Row r : predictionAndLabels.collectAsList()) { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index 58bc5a448aed..6194167bda35 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -17,39 +17,29 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; +import java.io.IOException; import java.util.List; import scala.collection.JavaConverters; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; -public class JavaOneVsRestSuite implements Serializable { +public class JavaOneVsRestSuite extends SharedSparkSession { - private transient SparkSession spark; - private transient JavaSparkContext jsc; private transient Dataset dataset; private transient JavaRDD datasetRDD; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLOneVsRestSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - + @Override + public void setUp() throws IOException { + super.setUp(); int nPoints = 3; // The following coefficients and xMean/xVariance are computed from iris dataset with @@ -68,12 +58,6 @@ public void setUp() { dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void oneVsRestDefaultParams() { OneVsRest ova = new OneVsRest(); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java index 1ed20b1bfa17..dd98513f37ec 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java @@ -17,45 +17,21 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -public class JavaRandomForestClassifierSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRandomForestClassifierSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRandomForestClassifierSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java index 9d07170fa1d5..1be6f96f4c94 100644 --- a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java @@ -17,43 +17,30 @@ package org.apache.spark.ml.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; +import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaKMeansSuite implements Serializable { +public class JavaKMeansSuite extends SharedSparkSession { private transient int k = 5; private transient Dataset dataset; - private transient SparkSession spark; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaKMeansSuite") - .getOrCreate(); + @Override + public void setUp() throws IOException { + super.setUp(); dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void fitAndTransform() { KMeans kmeans = new KMeans().setK(k).setSeed(1); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java index a96b43de1577..87639380bdcf 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java @@ -20,36 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaBucketizerSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaBucketizerSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaBucketizerSuite extends SharedSparkSession { @Test public void bucketizerTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index 9d8c09b30c01..b7956b6fd3e9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -22,38 +22,21 @@ import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaDCTSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDCTSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDCTSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java index 3c37441a77be..57696d0150a8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java @@ -20,38 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaHashingTFSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaHashingTFSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaHashingTFSuite extends SharedSparkSession { @Test public void hashingTF() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java index b3e213a49700..6f877b566875 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java @@ -19,35 +19,15 @@ import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaNormalizerSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaNormalizerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaNormalizerSuite extends SharedSparkSession { @Test public void normalizer() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index a4bce2283b86..ac479c08418c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -23,13 +23,11 @@ import scala.Tuple2; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.Vectors; @@ -37,26 +35,8 @@ import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -public class JavaPCASuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaPCASuite extends SharedSparkSession { public static class VectorPair implements Serializable { private Vector features = Vectors.dense(0.0); @@ -95,7 +75,7 @@ public org.apache.spark.mllib.linalg.Vector call(Vector vector) { } } ).rdd()); - + Matrix pc = mat.computePrincipalComponents(3); mat.multiply(pc).rows().toJavaRDD(); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java index a28f73f10a0b..df5d34fbe94e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java @@ -20,41 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaPolynomialExpansionSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPolynomialExpansionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaPolynomialExpansionSuite extends SharedSparkSession { @Test public void polynomialExpansionTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java index 8415fdb84f21..dbc0b1db5c00 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java @@ -20,34 +20,14 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaStandardScalerSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaStandardScalerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaStandardScalerSuite extends SharedSparkSession { @Test public void standardScaler() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java index 2b156f3bca5b..6480b57e1f79 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java @@ -20,37 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaStopWordsRemoverSuite { - - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaStopWordsRemoverSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaStopWordsRemoverSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java index 52c0bde8f367..c1928a26b609 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java @@ -20,37 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.SparkConf; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaStringIndexerSuite { - private transient SparkSession spark; - @Before - public void setUp() { - SparkConf sparkConf = new SparkConf(); - sparkConf.setMaster("local"); - sparkConf.setAppName("JavaStringIndexerSuite"); - - spark = SparkSession.builder().config(sparkConf).getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaStringIndexerSuite extends SharedSparkSession { @Test public void testStringIndexer() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java index 0bac2839e179..27550a3d5c37 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java @@ -20,35 +20,15 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaTokenizerSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaTokenizerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaTokenizerSuite extends SharedSparkSession { @Test public void regexTokenizer() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java index fedaa7717658..583652badb8f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java @@ -19,40 +19,22 @@ import java.util.Arrays; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.SparkConf; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaVectorAssemblerSuite { - private transient SparkSession spark; - @Before - public void setUp() { - SparkConf sparkConf = new SparkConf(); - sparkConf.setMaster("local"); - sparkConf.setAppName("JavaVectorAssemblerSuite"); - - spark = SparkSession.builder().config(sparkConf).getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaVectorAssemblerSuite extends SharedSparkSession { @Test public void testVectorAssembler() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java index a8dd44608db3..ca8fae3a48b9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -17,42 +17,21 @@ package org.apache.spark.ml.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaVectorIndexerSuite implements Serializable { - private transient SparkSession spark; - private JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaVectorIndexerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaVectorIndexerSuite extends SharedSparkSession { @Test public void vectorIndexerAPI() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index a565c77af4ab..3dc2e1f89614 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -20,11 +20,10 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; @@ -33,26 +32,10 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructType; -public class JavaVectorSlicerSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaVectorSlicerSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaVectorSlicerSuite extends SharedSparkSession { @Test public void vectorSlice() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java index bef7eb0f995f..d0a849fd11c7 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java @@ -19,34 +19,17 @@ import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; -public class JavaWord2VecSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaWord2VecSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test public void testJavaWord2Vec() { diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java index a5b5dd4088ff..1077e103a3b8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java @@ -19,37 +19,14 @@ import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; - /** * Test Param and related classes in Java */ public class JavaParamsSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaParamsSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void testParams() { JavaTestParams testParams = new JavaTestParams(); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java index 4ea3f2255efa..1da85ed9dab4 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -17,43 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaDecisionTreeRegressorSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDecisionTreeRegressorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDecisionTreeRegressorSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java index 3b5edf1e15ae..7fd9b1feb7f8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java @@ -17,43 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaGBTRegressorSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaGBTRegressorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaGBTRegressorSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index 126aa6298f3d..6cdcdda1a648 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -17,48 +17,32 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaLinearRegressionSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; +public class JavaLinearRegressionSuite extends SharedSparkSession { private transient Dataset dataset; private transient JavaRDD datasetRDD; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLinearRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); dataset.createOrReplaceTempView("dataset"); } - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - @Test public void linearRegressionDefaultParams() { LinearRegression lr = new LinearRegression(); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java index d601e7c540e1..4ba13e2e06c8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java @@ -17,45 +17,23 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaRandomForestRegressorSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRandomForestRegressorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRandomForestRegressorSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java index 022dcf94bd00..fa39f4560c8a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java @@ -23,35 +23,28 @@ import com.google.common.io.Files; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.DenseVector; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; import org.apache.spark.util.Utils; /** * Test LibSVMRelation in Java. */ -public class JavaLibSVMRelationSuite { - private transient SparkSession spark; +public class JavaLibSVMRelationSuite extends SharedSparkSession { private File tempDir; private String path; - @Before + @Override public void setUp() throws IOException { - spark = SparkSession.builder() - .master("local") - .appName("JavaLibSVMRelationSuite") - .getOrCreate(); - + super.setUp(); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource"); File file = new File(tempDir, "part-00000"); String s = "1 1:1.0 3:2.0 5:3.0\n0\n0 2:4.0 4:5.0 6:6.0"; @@ -59,10 +52,9 @@ public void setUp() throws IOException { path = tempDir.toURI().toString(); } - @After + @Override public void tearDown() { - spark.stop(); - spark = null; + super.tearDown(); Utils.deleteRecursively(tempDir); } diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java index b874ccd48b3b..692d5ad591e8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java @@ -17,48 +17,33 @@ package org.apache.spark.ml.tuning; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.classification.LogisticRegression; -import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; -import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; -public class JavaCrossValidatorSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - private transient Dataset dataset; +public class JavaCrossValidatorSuite extends SharedSparkSession { - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaCrossValidatorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); + private transient Dataset dataset; + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); dataset = spark.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); } - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - @Test public void crossValidationWithLogisticRegression() { LogisticRegression lr = new LogisticRegression(); diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java index 7151e27cde83..da623d1d1570 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -20,42 +20,25 @@ import java.io.File; import java.io.IOException; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.SharedSparkSession; import org.apache.spark.util.Utils; -public class JavaDefaultReadWriteSuite { - - JavaSparkContext jsc = null; - SparkSession spark = null; +public class JavaDefaultReadWriteSuite extends SharedSparkSession { File tempDir = null; - @Before - public void setUp() { - SQLContext.clearActive(); - spark = SparkSession.builder() - .master("local[2]") - .appName("JavaDefaultReadWriteSuite") - .getOrCreate(); - SQLContext.setActive(spark.wrapped()); - + @Override + public void setUp() throws IOException { + super.setUp(); tempDir = Utils.createTempDir( System.getProperty("java.io.tmpdir"), "JavaDefaultReadWriteSuite"); } - @After + @Override public void tearDown() { - SQLContext.clearActive(); - if (spark != null) { - spark.stop(); - spark = null; - } + super.tearDown(); Utils.deleteRecursively(tempDir); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index 2f10d14da5e4..c04e2e69541b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -17,37 +17,16 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.SparkSession; -public class JavaLogisticRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaLogisticRegressionSuite extends SharedSparkSession { int validatePrediction(List validationData, LogisticRegressionModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 5e212e2fc5b3..6ded42e92825 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -17,42 +17,21 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.SparkSession; -public class JavaNaiveBayesSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaNaiveBayesSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaNaiveBayesSuite extends SharedSparkSession { private static final List POINTS = Arrays.asList( new LabeledPoint(0, Vectors.dense(1.0, 0.0, 0.0)), diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 2a090c054fe2..0f54e684e447 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -17,37 +17,16 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.SparkSession; -public class JavaSVMSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaSVMSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaSVMSuite extends SharedSparkSession { int validatePrediction(List validationData, SVMModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java index 62c6d9b7e390..8c6bced52dd7 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -37,7 +36,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLogisticRegressionSuite implements Serializable { +public class JavaStreamingLogisticRegressionSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java index 7f29b050479f..3d62b273d221 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java @@ -17,39 +17,17 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; - import com.google.common.collect.Lists; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; - -public class JavaBisectingKMeansSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaBisectingKMeansSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaBisectingKMeansSuite extends SharedSparkSession { @Test public void twoDimensionalData() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java index 20edd08a2172..bf7671993777 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java @@ -17,40 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import static org.junit.Assert.assertEquals; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; -public class JavaGaussianMixtureSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaGaussianMixture") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaGaussianMixtureSuite extends SharedSparkSession { @Test public void runGaussianMixture() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 4e5b87f588e3..270e636f8211 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -17,40 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import static org.junit.Assert.assertEquals; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; -public class JavaKMeansSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaKMeans") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaKMeansSuite extends SharedSparkSession { @Test public void runKMeansUsingStaticMethods() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index f16585aff4f5..08d6713ab2bc 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -17,39 +17,28 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import scala.Tuple2; import scala.Tuple3; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; - -public class JavaLDASuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLDASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); +public class JavaLDASuite extends SharedSparkSession { + @Override + public void setUp() throws IOException { + super.setUp(); ArrayList> tinyCorpus = new ArrayList<>(); for (int i = 0; i < LDASuite.tinyCorpus().length; i++) { tinyCorpus.add(new Tuple2<>((Long) LDASuite.tinyCorpus()[i]._1(), @@ -59,12 +48,6 @@ public void setUp() { corpus = JavaPairRDD.fromJavaRDD(tmpCorpus); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void localLDAModel() { Matrix topics = LDASuite.tinyTopics(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java index d1d618f7de2d..d41fc0e4dca9 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -36,7 +35,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingKMeansSuite implements Serializable { +public class JavaStreamingKMeansSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java index 6a096d638655..e9d7e4fdbe8c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java @@ -17,35 +17,25 @@ package org.apache.spark.mllib.evaluation; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; import scala.Tuple2; import scala.Tuple2$; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -public class JavaRankingMetricsSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; +public class JavaRankingMetricsSuite extends SharedSparkSession { private transient JavaRDD, List>> predictionAndLabels; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - + @Override + public void setUp() throws IOException { + super.setUp(); predictionAndLabels = jsc.parallelize(Arrays.asList( Tuple2$.MODULE$.apply( Arrays.asList(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Arrays.asList(1, 2, 3, 4, 5)), @@ -55,12 +45,6 @@ public void setUp() { Arrays.asList(1, 2, 3, 4, 5), Arrays.asList())), 2); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void rankingMetrics() { @SuppressWarnings("unchecked") diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index de50fb8c4fdb..05128ea34342 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -17,38 +17,17 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.SparkSession; -public class JavaTfIdfSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaTfIdfSuite extends SharedSparkSession { @Test public void tfIdf() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java index 64885cc8425d..3e3abddbee63 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -25,33 +24,13 @@ import scala.Tuple2; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -public class JavaWord2VecSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test @SuppressWarnings("unchecked") diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java index fdc19a5b3dc4..3451e0773759 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java @@ -16,36 +16,15 @@ */ package org.apache.spark.mllib.fpm; -import java.io.Serializable; import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset; -import org.apache.spark.sql.SparkSession; -public class JavaAssociationRulesSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaAssociationRulesSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaAssociationRulesSuite extends SharedSparkSession { @Test public void runAssociationRules() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java index f235251e61d4..46e9dd8b5982 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java @@ -18,39 +18,18 @@ package org.apache.spark.mllib.fpm; import java.io.File; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import static org.junit.Assert.assertEquals; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; import org.apache.spark.util.Utils; -public class JavaFPGrowthSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaFPGrowth") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaFPGrowthSuite extends SharedSparkSession { @Test public void runFPGrowth() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java index bf7f1fc71b08..75b0ec648019 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java @@ -21,35 +21,15 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.PrefixSpan.FreqSequence; -import org.apache.spark.sql.SparkSession; import org.apache.spark.util.Utils; -public class JavaPrefixSpanSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPrefixSpan") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaPrefixSpanSuite extends SharedSparkSession { @Test public void runPrefixSpan() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java index 92fc57871cdb..f427846b9ad1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.linalg; -import java.io.Serializable; import java.util.Random; import static org.junit.Assert.assertArrayEquals; @@ -25,7 +24,7 @@ import org.junit.Test; -public class JavaMatricesSuite implements Serializable { +public class JavaMatricesSuite { @Test public void randMatrixConstruction() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index 817b962c7500..f67f555e418a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.linalg; -import java.io.Serializable; import java.util.Arrays; import static org.junit.Assert.assertArrayEquals; @@ -26,7 +25,7 @@ import org.junit.Test; -public class JavaVectorsSuite implements Serializable { +public class JavaVectorsSuite { @Test public void denseArrayConstruction() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java index b449108a9b83..6d114024c31b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -20,36 +20,16 @@ import java.io.Serializable; import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.SparkSession; import static org.apache.spark.mllib.random.RandomRDDs.*; -public class JavaRandomRDDsSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRandomRDDsSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRandomRDDsSuite extends SharedSparkSession { @Test public void testUniformRDD() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index aa784054d551..363ab42546d1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -17,41 +17,20 @@ package org.apache.spark.mllib.recommendation; -import java.io.Serializable; import java.util.ArrayList; import java.util.List; import scala.Tuple2; import scala.Tuple3; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; - -public class JavaALSSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaALS") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaALSSuite extends SharedSparkSession { private void validatePrediction( MatrixFactorizationModel model, diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java index 8b05675d65a1..dbd4cbfd2b74 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java @@ -17,26 +17,20 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import scala.Tuple3; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -public class JavaIsotonicRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; +public class JavaIsotonicRegressionSuite extends SharedSparkSession { private static List> generateIsotonicInput(double[] labels) { List> input = new ArrayList<>(labels.length); @@ -55,21 +49,6 @@ private IsotonicRegressionModel runIsotonicRegression(double[] labels) { return new IsotonicRegression().run(trainRDD); } - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLinearRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void testIsotonicRegressionJavaRDD() { IsotonicRegressionModel model = diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index 098bac3bedff..1458cc72bc17 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -17,37 +17,16 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -import org.apache.spark.sql.SparkSession; -public class JavaLassoSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLassoSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaLassoSuite extends SharedSparkSession { int validatePrediction(List validationData, LassoModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 35087a5e461d..a46b1321b3ca 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -17,39 +17,18 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.util.LinearDataGenerator; -import org.apache.spark.sql.SparkSession; -public class JavaLinearRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLinearRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaLinearRegressionSuite extends SharedSparkSession { int validatePrediction(List validationData, LinearRegressionModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index b2efb2e72e37..cb0097741234 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -17,38 +17,17 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; import java.util.Random; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -import org.apache.spark.sql.SparkSession; - -public class JavaRidgeRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRidgeRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRidgeRegressionSuite extends SharedSparkSession { private static double predictionError(List validationData, RidgeRegressionModel model) { diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java index ea0ccd744898..ab554475d59a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -36,7 +35,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLinearRegressionSuite implements Serializable { +public class JavaStreamingLinearRegressionSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java index 373417d3ba7c..1abaa39eadc2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.stat; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -42,7 +41,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStatisticsSuite implements Serializable { +public class JavaStatisticsSuite { private transient SparkSession spark; private transient JavaSparkContext jsc; private transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java index 5b464a4722d9..1dcbbcaa0223 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java @@ -17,17 +17,14 @@ package org.apache.spark.mllib.tree; -import java.io.Serializable; import java.util.HashMap; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.regression.LabeledPoint; @@ -35,27 +32,8 @@ import org.apache.spark.mllib.tree.configuration.Strategy; import org.apache.spark.mllib.tree.impurity.Gini; import org.apache.spark.mllib.tree.model.DecisionTreeModel; -import org.apache.spark.sql.SparkSession; - -public class JavaDecisionTreeSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDecisionTreeSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDecisionTreeSuite extends SharedSparkSession { int validatePrediction(List validationData, DecisionTreeModel model) { int numCorrect = 0; From c21c69187652936e5c74ed7571f07a0b423e9193 Mon Sep 17 00:00:00 2001 From: Sumedh Mungee Date: Fri, 20 May 2016 12:30:04 +0800 Subject: [PATCH 291/313] [SPARK-15321] Fix bug where Array[Timestamp] cannot be encoded/decoded correctly ## What changes were proposed in this pull request? Fix `MapObjects.itemAccessorMethod` to handle `TimestampType`. Without this fix, `Array[Timestamp]` cannot be properly encoded or decoded. To reproduce this, in `ExpressionEncoderSuite`, if you add the following test case: `encodeDecodeTest(Array(Timestamp.valueOf("2016-01-29 10:00:00")), "array of timestamp") ` ... you will see that (without this fix) it fails with the following output: ``` - encode/decode for array of timestamp: [Ljava.sql.Timestamp;fd9ebde *** FAILED *** Exception thrown while decoding Converted: [0,1000000010,800000001,52a7ccdc36800] Schema: value#61615 root -- value: array (nullable = true) |-- element: timestamp (containsNull = true) Encoder: class[value[0]: array] (ExpressionEncoderSuite.scala:312) ``` ## How was this patch tested? Existing tests Author: Sumedh Mungee Closes #13108 from smungee/fix-itemAccessorMethod. (cherry picked from commit d5c47f8ff8c09ff017e896835db044661ee60909) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 177b1390b2f4..227e835e7ed5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -115,6 +115,7 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { encodeDecodeTest("hello", "string") encodeDecodeTest(Date.valueOf("2012-12-23"), "date") encodeDecodeTest(Timestamp.valueOf("2016-01-29 10:00:00"), "timestamp") + encodeDecodeTest(Array(Timestamp.valueOf("2016-01-29 10:00:00")), "array of timestamp") encodeDecodeTest(Array[Byte](13, 21, -23), "binary") encodeDecodeTest(Seq(31, -123, 4), "seq of int") From e6810e9cd74cd8e8999ea5eca622367f55e268b1 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Fri, 20 May 2016 12:41:14 +0800 Subject: [PATCH 292/313] [SPARK-11827][SQL] Adding java.math.BigInteger support in Java type inference for POJOs and Java collections Hello : Can you help check this PR? I am adding support for the java.math.BigInteger for java bean code path. I saw internally spark is converting the BigInteger to BigDecimal in ColumnType.scala and CatalystRowConverter.scala. I use the similar way and convert the BigInteger to the BigDecimal. . Author: Kevin Yu Closes #10125 from kevinyu98/working_on_spark-11827. (cherry picked from commit 17591d90e6873f30a042112f56a1686726ccbd60) Signed-off-by: Wenchen Fan --- .../sql/catalyst/CatalystTypeConverters.scala | 2 ++ .../sql/catalyst/JavaTypeInference.scala | 1 + .../spark/sql/catalyst/ScalaReflection.scala | 24 +++++++++++++++ .../org/apache/spark/sql/types/Decimal.scala | 29 ++++++++++++++++++- .../apache/spark/sql/types/DecimalType.scala | 1 + .../encoders/ExpressionEncoderSuite.scala | 4 ++- .../apache/spark/sql/JavaDataFrameSuite.java | 11 ++++++- .../sql/ScalaReflectionRelationSuite.scala | 10 +++++-- 8 files changed, 76 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 9bfc38163914..9cc7b2ac7920 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst import java.lang.{Iterable => JavaIterable} import java.math.{BigDecimal => JavaBigDecimal} +import java.math.{BigInteger => JavaBigInteger} import java.sql.{Date, Timestamp} import java.util.{Map => JavaMap} import javax.annotation.Nullable @@ -326,6 +327,7 @@ object CatalystTypeConverters { val decimal = scalaValue match { case d: BigDecimal => Decimal(d) case d: JavaBigDecimal => Decimal(d) + case d: JavaBigInteger => Decimal(d) case d: Decimal => d } if (decimal.changePrecision(dataType.precision, dataType.scale)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 690758205eff..1fe143494aba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -89,6 +89,7 @@ object JavaTypeInference { case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType.SYSTEM_DEFAULT, true) + case c: Class[_] if c == classOf[java.math.BigInteger] => (DecimalType.BigIntDecimal, true) case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) 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 c0fa220d34bb..58df651da294 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 @@ -259,6 +259,12 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[BigDecimal] => Invoke(getPath, "toBigDecimal", ObjectType(classOf[BigDecimal])) + case t if t <:< localTypeOf[java.math.BigInteger] => + Invoke(getPath, "toJavaBigInteger", ObjectType(classOf[java.math.BigInteger])) + + case t if t <:< localTypeOf[scala.math.BigInt] => + Invoke(getPath, "toScalaBigInt", ObjectType(classOf[scala.math.BigInt])) + case t if t <:< localTypeOf[Array[_]] => val TypeRef(_, _, Seq(elementType)) = t @@ -592,6 +598,20 @@ object ScalaReflection extends ScalaReflection { "apply", inputObject :: Nil) + case t if t <:< localTypeOf[java.math.BigInteger] => + StaticInvoke( + Decimal.getClass, + DecimalType.BigIntDecimal, + "apply", + inputObject :: Nil) + + case t if t <:< localTypeOf[scala.math.BigInt] => + StaticInvoke( + Decimal.getClass, + DecimalType.BigIntDecimal, + "apply", + inputObject :: Nil) + case t if t <:< localTypeOf[java.lang.Integer] => Invoke(inputObject, "intValue", IntegerType) case t if t <:< localTypeOf[java.lang.Long] => @@ -736,6 +756,10 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if t <:< localTypeOf[java.math.BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) + case t if t <:< localTypeOf[java.math.BigInteger] => + Schema(DecimalType.BigIntDecimal, nullable = true) + case t if t <:< localTypeOf[scala.math.BigInt] => + Schema(DecimalType.BigIntDecimal, nullable = true) case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.SYSTEM_DEFAULT, 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) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 2f7422b7420d..b907f6280217 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import java.math.{MathContext, RoundingMode} +import java.math.{BigInteger, MathContext, RoundingMode} import org.apache.spark.annotation.DeveloperApi @@ -128,6 +128,23 @@ final class Decimal extends Ordered[Decimal] with Serializable { this } + /** + * Set this Decimal to the given BigInteger value. Will have precision 38 and scale 0. + */ + def set(bigintval: BigInteger): Decimal = { + try { + this.decimalVal = null + this.longVal = bigintval.longValueExact() + this._precision = DecimalType.MAX_PRECISION + this._scale = 0 + this + } + catch { + case e: ArithmeticException => + throw new IllegalArgumentException(s"BigInteger ${bigintval} too large for decimal") + } + } + /** * Set this Decimal to the given Decimal value. */ @@ -155,6 +172,10 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } + def toScalaBigInt: BigInt = BigInt(toLong) + + def toJavaBigInteger: java.math.BigInteger = java.math.BigInteger.valueOf(toLong) + def toUnscaledLong: Long = { if (decimalVal.ne(null)) { decimalVal.underlying().unscaledValue().longValue() @@ -371,6 +392,10 @@ object Decimal { def apply(value: java.math.BigDecimal): Decimal = new Decimal().set(value) + def apply(value: java.math.BigInteger): Decimal = new Decimal().set(value) + + def apply(value: scala.math.BigInt): Decimal = new Decimal().set(value.bigInteger) + def apply(value: BigDecimal, precision: Int, scale: Int): Decimal = new Decimal().set(value, precision, scale) @@ -387,6 +412,8 @@ object Decimal { value match { case j: java.math.BigDecimal => apply(j) case d: BigDecimal => apply(d) + case k: scala.math.BigInt => apply(k) + case l: java.math.BigInteger => apply(l) case d: Decimal => d } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 9c1319c1c5e6..6b7e3714e0b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { private[sql] val LongDecimal = DecimalType(20, 0) private[sql] val FloatDecimal = DecimalType(14, 7) private[sql] val DoubleDecimal = DecimalType(30, 15) + private[sql] val BigIntDecimal = DecimalType(38, 0) private[sql] def forType(dataType: DataType): DecimalType = dataType match { case ByteType => ByteDecimal diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 227e835e7ed5..d4387890b403 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.encoders +import java.math.BigInteger import java.sql.{Date, Timestamp} import java.util.Arrays @@ -109,7 +110,8 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { encodeDecodeTest(BigDecimal("32131413.211321313"), "scala decimal") encodeDecodeTest(new java.math.BigDecimal("231341.23123"), "java decimal") - + encodeDecodeTest(BigInt("23134123123"), "scala biginteger") + encodeDecodeTest(new BigInteger("23134123123"), "java BigInteger") encodeDecodeTest(Decimal("32131413.211321313"), "catalyst decimal") encodeDecodeTest("hello", "string") diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 324ebbae3876..35a9f44feca6 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -21,6 +21,8 @@ import java.net.URISyntaxException; import java.net.URL; import java.util.*; +import java.math.BigInteger; +import java.math.BigDecimal; import scala.collection.JavaConverters; import scala.collection.Seq; @@ -130,6 +132,7 @@ public static class Bean implements Serializable { private Integer[] b = { 0, 1 }; private Map c = ImmutableMap.of("hello", new int[] { 1, 2 }); private List d = Arrays.asList("floppy", "disk"); + private BigInteger e = new BigInteger("1234567"); public double getA() { return a; @@ -146,6 +149,8 @@ public Map getC() { public List getD() { return d; } + + public BigInteger getE() { return e; } } void validateDataFrameWithBeans(Bean bean, Dataset df) { @@ -163,7 +168,9 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { Assert.assertEquals( new StructField("d", new ArrayType(DataTypes.StringType, true), true, Metadata.empty()), schema.apply("d")); - Row first = df.select("a", "b", "c", "d").first(); + Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, Metadata.empty()), + schema.apply("e")); + Row first = df.select("a", "b", "c", "d", "e").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); // Now Java lists and maps are converted to Scala Seq's and Map's. Once we get a Seq below, // verify that it has the expected length, and contains expected elements. @@ -182,6 +189,8 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { for (int i = 0; i < d.length(); i++) { Assert.assertEquals(bean.getD().get(i), d.apply(i)); } + // Java.math.BigInteger is equavient to Spark Decimal(38,0) + Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); } @Test diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 491bdb3ef9db..c9bd05d0e4e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -34,7 +34,9 @@ case class ReflectData( decimalField: java.math.BigDecimal, date: Date, timestampField: Timestamp, - seqInt: Seq[Int]) + seqInt: Seq[Int], + javaBigInt: java.math.BigInteger, + scalaBigInt: scala.math.BigInt) case class NullReflectData( intField: java.lang.Integer, @@ -77,13 +79,15 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3)) + new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3), + new java.math.BigInteger("1"), scala.math.BigInt(1)) Seq(data).toDF().createOrReplaceTempView("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), - new Timestamp(12345), Seq(1, 2, 3))) + new Timestamp(12345), Seq(1, 2, 3), new java.math.BigDecimal(1), + new java.math.BigDecimal(1))) } test("query case class RDD with nulls") { From 52b967fe66ad07b4eed4b5a5861e5358b7c4af1d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 21:53:26 -0700 Subject: [PATCH 293/313] [SPARK-15075][SPARK-15345][SQL] Clean up SparkSession builder and propagate config options to existing sessions if specified ## What changes were proposed in this pull request? Currently SparkSession.Builder use SQLContext.getOrCreate. It should probably the the other way around, i.e. all the core logic goes in SparkSession, and SQLContext just calls that. This patch does that. This patch also makes sure config options specified in the builder are propagated to the existing (and of course the new) SparkSession. ## How was this patch tested? Updated tests to reflect the change, and also introduced a new SparkSessionBuilderSuite that should cover all the branches. Author: Reynold Xin Closes #13200 from rxin/SPARK-15075. (cherry picked from commit f2ee0ed4b7ecb2855cc4928a9613a07d45446f4e) Signed-off-by: Reynold Xin --- .../ml/util/JavaDefaultReadWriteSuite.java | 2 +- python/pyspark/sql/context.py | 5 +- python/pyspark/sql/session.py | 17 +- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 124 ++------------ .../org/apache/spark/sql/SparkSession.scala | 155 ++++++++++++++---- .../spark/sql/execution/ExistingRDD.scala | 3 +- .../spark/sql/execution/QueryExecution.scala | 4 +- .../spark/sql/execution/SparkPlan.scala | 6 +- .../execution/datasources/DataSource.scala | 13 +- .../datasources/fileSourceInterfaces.scala | 2 +- .../datasources/jdbc/JDBCRelation.scala | 2 +- .../execution/streaming/StreamExecution.scala | 2 +- .../org/apache/spark/sql/functions.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 10 -- .../spark/sql/internal/SharedState.scala | 19 ++- .../apache/spark/sql/ListTablesSuite.scala | 13 +- .../spark/sql/MultiSQLContextsSuite.scala | 100 ----------- .../apache/spark/sql/SQLContextSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 8 +- .../apache/spark/sql/SerializationSuite.scala | 2 +- .../spark/sql/SparkSessionBuilderSuite.scala | 93 +++++++++++ .../apache/spark/sql/StatisticsSuite.scala | 4 +- .../execution/ExchangeCoordinatorSuite.scala | 16 +- .../spark/sql/execution/PlannerSuite.scala | 2 +- .../spark/sql/execution/SparkPlanTest.scala | 4 +- .../datasources/parquet/ParquetTest.scala | 2 +- .../streaming/state/StateStoreRDDSuite.scala | 14 +- .../sql/execution/ui/SQLListenerSuite.scala | 6 +- .../spark/sql/internal/SQLConfSuite.scala | 18 +- .../spark/sql/sources/DDLTestSuite.scala | 2 +- .../spark/sql/sources/FilteredScanSuite.scala | 2 +- .../spark/sql/sources/PrunedScanSuite.scala | 2 +- .../spark/sql/sources/TableScanSuite.scala | 4 +- .../DataFrameReaderWriterSuite.scala | 4 +- .../apache/spark/sql/test/SQLTestData.scala | 2 +- .../apache/spark/sql/test/SQLTestUtils.scala | 2 +- .../spark/sql/test/SharedSQLContext.scala | 4 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/MultiDatabaseSuite.scala | 42 ++--- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- .../apache/spark/sql/hive/orc/OrcTest.scala | 2 +- 43 files changed, 367 insertions(+), 357 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java index da623d1d1570..7bda219243bf 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -56,7 +56,7 @@ public void testDefaultReadWrite() throws IOException { } catch (IOException e) { // expected } - instance.write().context(spark.wrapped()).overwrite().save(outputPath); + instance.write().context(spark.sqlContext()).overwrite().save(outputPath); MyParams newInstance = MyParams.load(outputPath); Assert.assertEquals("UID should match.", instance.uid(), newInstance.uid()); Assert.assertEquals("Params should be preserved.", diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e8e60c64121b..486733a390a0 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -34,7 +34,10 @@ class SQLContext(object): - """Wrapper around :class:`SparkSession`, the main entry point to Spark SQL functionality. + """The entry point for working with structured data (rows and columns) in Spark, in Spark 1.x. + + As of Spark 2.0, this is replaced by :class:`SparkSession`. However, we are keeping the class + here for backward compatibility. A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 257a239c8d7b..0e04b88265fa 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -120,6 +120,8 @@ def master(self, master): def appName(self, name): """Sets a name for the application, which will be shown in the Spark web UI. + If no application name is set, a randomly generated name will be used. + :param name: an application name """ return self.config("spark.app.name", name) @@ -133,8 +135,17 @@ def enableHiveSupport(self): @since(2.0) def getOrCreate(self): - """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a new - one based on the options set in this builder. + """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a + new one based on the options set in this builder. + + This method first checks whether there is a valid thread-local SparkSession, + and if yes, return that one. It then checks whether there is a valid global + default SparkSession, and if yes, return that one. If no valid global default + SparkSession exists, the method creates a new SparkSession and assigns the + newly created SparkSession as the global default. + + In case an existing SparkSession is returned, the config options specified + in this builder will be applied to the existing SparkSession. """ with self._lock: from pyspark.conf import SparkConf @@ -175,7 +186,7 @@ def __init__(self, sparkContext, jsparkSession=None): if jsparkSession is None: jsparkSession = self._jvm.SparkSession(self._jsc.sc()) self._jsparkSession = jsparkSession - self._jwrapped = self._jsparkSession.wrapped() + self._jwrapped = self._jsparkSession.sqlContext() self._wrapped = SQLContext(self._sc, self, self._jwrapped) _monkey_patch_RDD(self) install_exception_handler() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 02dd6547a4ad..78a167eef2e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -213,7 +213,7 @@ class Dataset[T] private[sql]( private implicit def classTag = unresolvedTEncoder.clsTag // sqlContext must be val because a stable identifier is expected when you import implicits - @transient lazy val sqlContext: SQLContext = sparkSession.wrapped + @transient lazy val sqlContext: SQLContext = sparkSession.sqlContext protected[sql] def resolve(colName: String): NamedExpression = { queryExecution.analyzed.resolveQuoted(colName, sparkSession.sessionState.analyzer.resolver) 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 a3e2b49556e4..14d12d30bc0b 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 @@ -19,25 +19,22 @@ package org.apache.spark.sql import java.beans.BeanInfo import java.util.Properties -import java.util.concurrent.atomic.AtomicReference import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.rdd.RDD -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ShowTablesCommand -import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ @@ -46,8 +43,8 @@ import org.apache.spark.sql.util.ExecutionListenerManager /** * The entry point for working with structured data (rows and columns) in Spark, in Spark 1.x. * - * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class here - * for backward compatibility. + * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class + * here for backward compatibility. * * @groupname basic Basic Operations * @groupname ddl_ops Persistent Catalog DDL @@ -76,42 +73,21 @@ class SQLContext private[sql]( this(sparkSession, true) } + @deprecated("Use SparkSession.builder instead", "2.0.0") def this(sc: SparkContext) = { this(new SparkSession(sc)) } + @deprecated("Use SparkSession.builder instead", "2.0.0") def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) // TODO: move this logic into SparkSession - // If spark.sql.allowMultipleContexts is true, we will throw an exception if a user - // wants to create a new root SQLContext (a SQLContext that is not created by newSession). - private val allowMultipleContexts = - sparkContext.conf.getBoolean( - SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, - SQLConf.ALLOW_MULTIPLE_CONTEXTS.defaultValue.get) - - // Assert no root SQLContext is running when allowMultipleContexts is false. - { - if (!allowMultipleContexts && isRootContext) { - SQLContext.getInstantiatedContextOption() match { - case Some(rootSQLContext) => - val errMsg = "Only one SQLContext/HiveContext may be running in this JVM. " + - s"It is recommended to use SQLContext.getOrCreate to get the instantiated " + - s"SQLContext/HiveContext. To ignore this error, " + - s"set ${SQLConf.ALLOW_MULTIPLE_CONTEXTS.key} = true in SparkConf." - throw new SparkException(errMsg) - case None => // OK - } - } - } - protected[sql] def sessionState: SessionState = sparkSession.sessionState protected[sql] def sharedState: SharedState = sparkSession.sharedState protected[sql] def conf: SQLConf = sessionState.conf protected[sql] def runtimeConf: RuntimeConfig = sparkSession.conf protected[sql] def cacheManager: CacheManager = sparkSession.cacheManager - protected[sql] def listener: SQLListener = sparkSession.listener protected[sql] def externalCatalog: ExternalCatalog = sparkSession.externalCatalog def sparkContext: SparkContext = sparkSession.sparkContext @@ -123,7 +99,7 @@ class SQLContext private[sql]( * * @since 1.6.0 */ - def newSession(): SQLContext = sparkSession.newSession().wrapped + def newSession(): SQLContext = sparkSession.newSession().sqlContext /** * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s @@ -760,21 +736,6 @@ class SQLContext private[sql]( schema: StructType): DataFrame = { sparkSession.applySchemaToPythonRDD(rdd, schema) } - - // TODO: move this logic into SparkSession - - // Register a successfully instantiated context to the singleton. This should be at the end of - // the class definition so that the singleton is updated only if there is no exception in the - // construction of the instance. - sparkContext.addSparkListener(new SparkListener { - override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { - SQLContext.clearInstantiatedContext() - SQLContext.clearSqlListener() - } - }) - - sparkSession.setWrappedContext(self) - SQLContext.setInstantiatedContext(self) } /** @@ -787,19 +748,6 @@ class SQLContext private[sql]( */ object SQLContext { - /** - * The active SQLContext for the current thread. - */ - private val activeContext: InheritableThreadLocal[SQLContext] = - new InheritableThreadLocal[SQLContext] - - /** - * Reference to the created SQLContext. - */ - @transient private val instantiatedContext = new AtomicReference[SQLContext]() - - @transient private val sqlListener = new AtomicReference[SQLListener]() - /** * Get the singleton SQLContext if it exists or create a new one using the given SparkContext. * @@ -811,41 +759,9 @@ object SQLContext { * * @since 1.5.0 */ + @deprecated("Use SparkSession.builder instead", "2.0.0") def getOrCreate(sparkContext: SparkContext): SQLContext = { - val ctx = activeContext.get() - if (ctx != null && !ctx.sparkContext.isStopped) { - return ctx - } - - synchronized { - val ctx = instantiatedContext.get() - if (ctx == null || ctx.sparkContext.isStopped) { - new SQLContext(sparkContext) - } else { - ctx - } - } - } - - private[sql] def clearInstantiatedContext(): Unit = { - instantiatedContext.set(null) - } - - private[sql] def setInstantiatedContext(sqlContext: SQLContext): Unit = { - synchronized { - val ctx = instantiatedContext.get() - if (ctx == null || ctx.sparkContext.isStopped) { - instantiatedContext.set(sqlContext) - } - } - } - - private[sql] def getInstantiatedContextOption(): Option[SQLContext] = { - Option(instantiatedContext.get()) - } - - private[sql] def clearSqlListener(): Unit = { - sqlListener.set(null) + SparkSession.builder().sparkContext(sparkContext).getOrCreate().sqlContext } /** @@ -855,8 +771,9 @@ object SQLContext { * * @since 1.6.0 */ + @deprecated("Use SparkSession.setActiveSession instead", "2.0.0") def setActive(sqlContext: SQLContext): Unit = { - activeContext.set(sqlContext) + SparkSession.setActiveSession(sqlContext.sparkSession) } /** @@ -865,12 +782,9 @@ object SQLContext { * * @since 1.6.0 */ + @deprecated("Use SparkSession.clearActiveSession instead", "2.0.0") def clearActive(): Unit = { - activeContext.remove() - } - - private[sql] def getActive(): Option[SQLContext] = { - Option(activeContext.get()) + SparkSession.clearActiveSession() } /** @@ -894,20 +808,6 @@ object SQLContext { } } - /** - * Create a SQLListener then add it into SparkContext, and create an SQLTab if there is SparkUI. - */ - private[sql] def createListenerAndUI(sc: SparkContext): SQLListener = { - if (sqlListener.get() == null) { - val listener = new SQLListener(sc.conf) - if (sqlListener.compareAndSet(null, listener)) { - sc.addSparkListener(listener) - sc.ui.foreach(new SQLTab(listener, _)) - } - } - sqlListener.get() - } - /** * Extract `spark.sql.*` properties from the conf and return them as a [[Properties]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 8ef6cd3c3557..ab751f0c0411 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.beans.Introspector +import java.util.concurrent.atomic.AtomicReference import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -30,6 +31,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ @@ -98,24 +100,10 @@ class SparkSession private( } /** - * A wrapped version of this session in the form of a [[SQLContext]]. + * A wrapped version of this session in the form of a [[SQLContext]], for backward compatibility. */ @transient - private var _wrapped: SQLContext = _ - - @transient - private val _wrappedLock = new Object - - protected[sql] def wrapped: SQLContext = _wrappedLock.synchronized { - if (_wrapped == null) { - _wrapped = new SQLContext(self, isRootContext = false) - } - _wrapped - } - - protected[sql] def setWrappedContext(sqlContext: SQLContext): Unit = _wrappedLock.synchronized { - _wrapped = sqlContext - } + private[sql] val sqlContext: SQLContext = new SQLContext(this) protected[sql] def cacheManager: CacheManager = sharedState.cacheManager protected[sql] def listener: SQLListener = sharedState.listener @@ -238,7 +226,7 @@ class SparkSession private( */ @Experimental def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { - SQLContext.setActive(wrapped) + SparkSession.setActiveSession(this) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes val rowRDD = RDDConversions.productToRowRdd(rdd, schema.map(_.dataType)) @@ -254,7 +242,7 @@ class SparkSession private( */ @Experimental def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { - SQLContext.setActive(wrapped) + SparkSession.setActiveSession(this) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes Dataset.ofRows(self, LocalRelation.fromProduct(attributeSeq, data)) @@ -573,7 +561,7 @@ class SparkSession private( */ @Experimental object implicits extends SQLImplicits with Serializable { - protected override def _sqlContext: SQLContext = wrapped + protected override def _sqlContext: SQLContext = SparkSession.this.sqlContext } // scalastyle:on @@ -649,8 +637,16 @@ object SparkSession { private[this] val options = new scala.collection.mutable.HashMap[String, String] + private[this] var userSuppliedContext: Option[SparkContext] = None + + private[sql] def sparkContext(sparkContext: SparkContext): Builder = synchronized { + userSuppliedContext = Option(sparkContext) + this + } + /** * Sets a name for the application, which will be shown in the Spark web UI. + * If no application name is set, a randomly generated name will be used. * * @since 2.0.0 */ @@ -735,29 +731,130 @@ object SparkSession { } /** - * Gets an existing [[SparkSession]] or, if there is no existing one, creates a new one - * based on the options set in this builder. + * Gets an existing [[SparkSession]] or, if there is no existing one, creates a new + * one based on the options set in this builder. + * + * This method first checks whether there is a valid thread-local SparkSession, + * and if yes, return that one. It then checks whether there is a valid global + * default SparkSession, and if yes, return that one. If no valid global default + * SparkSession exists, the method creates a new SparkSession and assigns the + * newly created SparkSession as the global default. + * + * In case an existing SparkSession is returned, the config options specified in + * this builder will be applied to the existing SparkSession. * * @since 2.0.0 */ def getOrCreate(): SparkSession = synchronized { - // Step 1. Create a SparkConf - // Step 2. Get a SparkContext - // Step 3. Get a SparkSession - val sparkConf = new SparkConf() - options.foreach { case (k, v) => sparkConf.set(k, v) } - val sparkContext = SparkContext.getOrCreate(sparkConf) - - SQLContext.getOrCreate(sparkContext).sparkSession + // Get the session from current thread's active session. + var session = activeThreadSession.get() + if ((session ne null) && !session.sparkContext.isStopped) { + options.foreach { case (k, v) => session.conf.set(k, v) } + return session + } + + // Global synchronization so we will only set the default session once. + SparkSession.synchronized { + // If the current thread does not have an active session, get it from the global session. + session = defaultSession.get() + if ((session ne null) && !session.sparkContext.isStopped) { + options.foreach { case (k, v) => session.conf.set(k, v) } + return session + } + + // No active nor global default session. Create a new one. + val sparkContext = userSuppliedContext.getOrElse { + // set app name if not given + if (!options.contains("spark.app.name")) { + options += "spark.app.name" -> java.util.UUID.randomUUID().toString + } + + val sparkConf = new SparkConf() + options.foreach { case (k, v) => sparkConf.set(k, v) } + SparkContext.getOrCreate(sparkConf) + } + session = new SparkSession(sparkContext) + options.foreach { case (k, v) => session.conf.set(k, v) } + defaultSession.set(session) + + // Register a successfully instantiated context to the singleton. This should be at the + // end of the class definition so that the singleton is updated only if there is no + // exception in the construction of the instance. + sparkContext.addSparkListener(new SparkListener { + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { + defaultSession.set(null) + sqlListener.set(null) + } + }) + } + + return session } } /** * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. + * * @since 2.0.0 */ def builder(): Builder = new Builder + /** + * Changes the SparkSession that will be returned in this thread and its children when + * SparkSession.getOrCreate() is called. This can be used to ensure that a given thread receives + * a SparkSession with an isolated session, instead of the global (first created) context. + * + * @since 2.0.0 + */ + def setActiveSession(session: SparkSession): Unit = { + activeThreadSession.set(session) + } + + /** + * Clears the active SparkSession for current thread. Subsequent calls to getOrCreate will + * return the first created context instead of a thread-local override. + * + * @since 2.0.0 + */ + def clearActiveSession(): Unit = { + activeThreadSession.remove() + } + + /** + * Sets the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def setDefaultSession(session: SparkSession): Unit = { + defaultSession.set(session) + } + + /** + * Clears the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def clearDefaultSession(): Unit = { + defaultSession.set(null) + } + + private[sql] def getActiveSession: Option[SparkSession] = Option(activeThreadSession.get) + + private[sql] def getDefaultSession: Option[SparkSession] = Option(defaultSession.get) + + /** A global SQL listener used for the SQL UI. */ + private[sql] val sqlListener = new AtomicReference[SQLListener]() + + //////////////////////////////////////////////////////////////////////////////////////// + // Private methods from now on + //////////////////////////////////////////////////////////////////////////////////////// + + /** The active SparkSession for the current thread. */ + private val activeThreadSession = new InheritableThreadLocal[SparkSession] + + /** Reference to the root SparkSession. */ + private val defaultSession = new AtomicReference[SparkSession] + private val HIVE_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" private val HIVE_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 85af4faf4d09..d8911f88b000 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -157,7 +157,8 @@ private[sql] case class RowDataSourceScanExec( val outputUnsafeRows = relation match { case r: HadoopFsRelation if r.fileFormat.isInstanceOf[ParquetSource] => - !SQLContext.getActive().get.conf.getConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED) + !SparkSession.getActiveSession.get.sessionState.conf.getConf( + SQLConf.PARQUET_VECTORIZED_READER_ENABLED) case _: HadoopFsRelation => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index cb3c46a98bfb..34187b9a1ae7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -60,7 +60,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { } lazy val analyzed: LogicalPlan = { - SQLContext.setActive(sparkSession.wrapped) + SparkSession.setActiveSession(sparkSession) sparkSession.sessionState.analyzer.execute(logical) } @@ -73,7 +73,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { lazy val optimizedPlan: LogicalPlan = sparkSession.sessionState.optimizer.execute(withCachedData) lazy val sparkPlan: SparkPlan = { - SQLContext.setActive(sparkSession.wrapped) + SparkSession.setActiveSession(sparkSession) planner.plan(ReturnAnswer(optimizedPlan)).next() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index b94b84d77a50..045ccc7bd6ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -27,7 +27,7 @@ import org.apache.spark.{broadcast, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd.{RDD, RDDOperationScope} -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -50,7 +50,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ * populated by the query planning infrastructure. */ @transient - protected[spark] final val sqlContext = SQLContext.getActive().orNull + final val sqlContext = SparkSession.getActiveSession.map(_.sqlContext).orNull protected def sparkContext = sqlContext.sparkContext @@ -65,7 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** Overridden make copy also propagates sqlContext to copied plan. */ override def makeCopy(newArgs: Array[AnyRef]): SparkPlan = { - SQLContext.setActive(sqlContext) + SparkSession.setActiveSession(sqlContext.sparkSession) super.makeCopy(newArgs) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index ccad9b3fd52f..2e17b763a537 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -178,7 +178,7 @@ case class DataSource( providingClass.newInstance() match { case s: StreamSourceProvider => val (name, schema) = s.sourceSchema( - sparkSession.wrapped, userSpecifiedSchema, className, options) + sparkSession.sqlContext, userSpecifiedSchema, className, options) SourceInfo(name, schema) case format: FileFormat => @@ -198,7 +198,8 @@ case class DataSource( def createSource(metadataPath: String): Source = { providingClass.newInstance() match { case s: StreamSourceProvider => - s.createSource(sparkSession.wrapped, metadataPath, userSpecifiedSchema, className, options) + s.createSource( + sparkSession.sqlContext, metadataPath, userSpecifiedSchema, className, options) case format: FileFormat => val path = new CaseInsensitiveMap(options).getOrElse("path", { @@ -215,7 +216,7 @@ case class DataSource( /** Returns a sink that can be used to continually write data. */ def createSink(): Sink = { providingClass.newInstance() match { - case s: StreamSinkProvider => s.createSink(sparkSession.wrapped, options, partitionColumns) + case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, options, partitionColumns) case parquet: parquet.DefaultSource => val caseInsensitiveOptions = new CaseInsensitiveMap(options) @@ -265,9 +266,9 @@ case class DataSource( val relation = (providingClass.newInstance(), userSpecifiedSchema) match { // TODO: Throw when too much is given. case (dataSource: SchemaRelationProvider, Some(schema)) => - dataSource.createRelation(sparkSession.wrapped, caseInsensitiveOptions, schema) + dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions, schema) case (dataSource: RelationProvider, None) => - dataSource.createRelation(sparkSession.wrapped, caseInsensitiveOptions) + dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) case (_: SchemaRelationProvider, None) => throw new AnalysisException(s"A schema needs to be specified when using $className.") case (_: RelationProvider, Some(_)) => @@ -383,7 +384,7 @@ case class DataSource( providingClass.newInstance() match { case dataSource: CreatableRelationProvider => - dataSource.createRelation(sparkSession.wrapped, mode, options, data) + dataSource.createRelation(sparkSession.sqlContext, mode, options, data) case format: FileFormat => // Don't glob path for the write path. The contracts here are: // 1. Only one output path can be specified on the write path; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 8d332df02916..88125a2b4da7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -142,7 +142,7 @@ case class HadoopFsRelation( fileFormat: FileFormat, options: Map[String, String]) extends BaseRelation with FileRelation { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext val schema: StructType = { val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index bcf70fdc4a49..233b7891d664 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -92,7 +92,7 @@ private[sql] case class JDBCRelation( with PrunedFilteredScan with InsertableRelation { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override val needConversion: Boolean = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index df6304d85fe7..7d09bdcebdc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -173,7 +173,7 @@ class StreamExecution( startLatch.countDown() // While active, repeatedly attempt to run batches. - SQLContext.setActive(sparkSession.wrapped) + SparkSession.setActiveSession(sparkSession) triggerExecutor.execute(() => { if (isActive) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 65bc04307675..0b490fe71c52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1168,7 +1168,7 @@ object functions { * @group normal_funcs */ def expr(expr: String): Column = { - val parser = SQLContext.getActive().map(_.sessionState.sqlParser).getOrElse { + val parser = SparkSession.getActiveSession.map(_.sessionState.sqlParser).getOrElse { new SparkSqlParser(new SQLConf) } Column(parser.parseExpression(expr)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5d1868980163..35d67ca2d8c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -70,16 +70,6 @@ object SQLConf { .intConf .createWithDefault(10) - val ALLOW_MULTIPLE_CONTEXTS = SQLConfigBuilder("spark.sql.allowMultipleContexts") - .doc("When set to true, creating multiple SQLContexts/HiveContexts is allowed. " + - "When set to false, only one SQLContext/HiveContext is allowed to be created " + - "through the constructor (new SQLContexts/HiveContexts created through newSession " + - "method is allowed). Please note that this conf needs to be set in Spark Conf. Once " + - "a SQLContext/HiveContext has been created, changing the value of this conf will not " + - "have effect.") - .booleanConf - .createWithDefault(true) - val COMPRESS_CACHED = SQLConfigBuilder("spark.sql.inMemoryColumnarStorage.compressed") .internal() .doc("When set to true Spark SQL will automatically select a compression codec for each " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index eaf993aaed4d..9f6137d6e3c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.internal import org.apache.spark.SparkContext -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} import org.apache.spark.sql.execution.CacheManager -import org.apache.spark.sql.execution.ui.SQLListener +import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.util.MutableURLClassLoader @@ -38,7 +38,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) { /** * A listener for SQL-specific [[org.apache.spark.scheduler.SparkListenerEvent]]s. */ - val listener: SQLListener = SQLContext.createListenerAndUI(sparkContext) + val listener: SQLListener = createListenerAndUI(sparkContext) /** * A catalog that interacts with external systems. @@ -51,6 +51,19 @@ private[sql] class SharedState(val sparkContext: SparkContext) { val jarClassLoader = new NonClosableMutableURLClassLoader( org.apache.spark.util.Utils.getContextOrSparkClassLoader) + /** + * Create a SQLListener then add it into SparkContext, and create an SQLTab if there is SparkUI. + */ + private def createListenerAndUI(sc: SparkContext): SQLListener = { + if (SparkSession.sqlListener.get() == null) { + val listener = new SQLListener(sc.conf) + if (SparkSession.sqlListener.compareAndSet(null, listener)) { + sc.addSparkListener(listener) + sc.ui.foreach(new SQLTab(listener, _)) + } + } + SparkSession.sqlListener.get() + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index 65fe271b6917..b447006761f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -39,7 +39,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex test("get all tables") { checkAnswer( - spark.wrapped.tables().filter("tableName = 'listtablessuitetable'"), + spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'"), Row("listtablessuitetable", true)) checkAnswer( @@ -48,12 +48,12 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex spark.sessionState.catalog.dropTable( TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) - assert(spark.wrapped.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + assert(spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) } test("getting all tables with a database name has no impact on returned table names") { checkAnswer( - spark.wrapped.tables("default").filter("tableName = 'listtablessuitetable'"), + spark.sqlContext.tables("default").filter("tableName = 'listtablessuitetable'"), Row("listtablessuitetable", true)) checkAnswer( @@ -62,7 +62,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex spark.sessionState.catalog.dropTable( TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) - assert(spark.wrapped.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + assert(spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) } test("query the returned DataFrame of tables") { @@ -70,7 +70,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex StructField("tableName", StringType, false) :: StructField("isTemporary", BooleanType, false) :: Nil) - Seq(spark.wrapped.tables(), sql("SHOW TABLes")).foreach { + Seq(spark.sqlContext.tables(), sql("SHOW TABLes")).foreach { case tableDF => assert(expectedSchema === tableDF.schema) @@ -81,7 +81,8 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex Row(true, "listtablessuitetable") ) checkAnswer( - spark.wrapped.tables().filter("tableName = 'tables'").select("tableName", "isTemporary"), + spark.sqlContext.tables() + .filter("tableName = 'tables'").select("tableName", "isTemporary"), Row("tables", true)) spark.catalog.dropTempView("tables") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala deleted file mode 100644 index 0b5a92c256e5..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark._ -import org.apache.spark.sql.internal.SQLConf - -class MultiSQLContextsSuite extends SparkFunSuite with BeforeAndAfterAll { - - private var originalActiveSQLContext: Option[SQLContext] = _ - private var originalInstantiatedSQLContext: Option[SQLContext] = _ - private var sparkConf: SparkConf = _ - - override protected def beforeAll(): Unit = { - originalActiveSQLContext = SQLContext.getActive() - originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption() - - SQLContext.clearActive() - SQLContext.clearInstantiatedContext() - sparkConf = - new SparkConf(false) - .setMaster("local[*]") - .setAppName("test") - .set("spark.ui.enabled", "false") - .set("spark.driver.allowMultipleContexts", "true") - } - - override protected def afterAll(): Unit = { - // Set these states back. - originalActiveSQLContext.foreach(ctx => SQLContext.setActive(ctx)) - originalInstantiatedSQLContext.foreach(ctx => SQLContext.setInstantiatedContext(ctx)) - } - - def testNewSession(rootSQLContext: SQLContext): Unit = { - // Make sure we can successfully create new Session. - rootSQLContext.newSession() - - // Reset the state. It is always safe to clear the active context. - SQLContext.clearActive() - } - - def testCreatingNewSQLContext(allowsMultipleContexts: Boolean): Unit = { - val conf = - sparkConf - .clone - .set(SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, allowsMultipleContexts.toString) - val sparkContext = new SparkContext(conf) - - try { - if (allowsMultipleContexts) { - new SQLContext(sparkContext) - SQLContext.clearActive() - } else { - // If allowsMultipleContexts is false, make sure we can get the error. - val message = intercept[SparkException] { - new SQLContext(sparkContext) - }.getMessage - assert(message.contains("Only one SQLContext/HiveContext may be running")) - } - } finally { - sparkContext.stop() - } - } - - test("test the flag to disallow creating multiple root SQLContext") { - Seq(false, true).foreach { allowMultipleSQLContexts => - val conf = - sparkConf - .clone - .set(SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, allowMultipleSQLContexts.toString) - val sc = new SparkContext(conf) - try { - val rootSQLContext = new SQLContext(sc) - testNewSession(rootSQLContext) - testNewSession(rootSQLContext) - testCreatingNewSQLContext(allowMultipleSQLContexts) - } finally { - sc.stop() - SQLContext.clearInstantiatedContext() - } - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 38d7b6e25b82..c9594a7e9ab2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -40,7 +40,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { val newSession = sqlContext.newSession() assert(SQLContext.getOrCreate(sc).eq(sqlContext), "SQLContext.getOrCreate after explicitly created SQLContext did not return the context") - SQLContext.setActive(newSession) + SparkSession.setActiveSession(newSession.sparkSession) assert(SQLContext.getOrCreate(sc).eq(newSession), "SQLContext.getOrCreate after explicitly setActive() did not return the active context") } 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 743a27aa7a21..460e34a5ff30 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 @@ -1042,7 +1042,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SET commands semantics using sql()") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() val testKey = "test.key.0" val testVal = "test.val.0" val nonexistentKey = "nonexistent" @@ -1083,17 +1083,17 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SET $nonexistentKey"), Row(nonexistentKey, "") ) - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("SET commands with illegal or inappropriate argument") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() // Set negative mapred.reduce.tasks for automatically determining // 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")) - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("apply schema") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala index b489b74fec07..cd6b2647e0be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -25,6 +25,6 @@ class SerializationSuite extends SparkFunSuite with SharedSQLContext { test("[SPARK-5235] SQLContext should be serializable") { val spark = SparkSession.builder.getOrCreate() - new JavaSerializer(new SparkConf()).newInstance().serialize(spark.wrapped) + new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sqlContext) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala new file mode 100644 index 000000000000..ec6a2b357586 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.{SparkContext, SparkFunSuite} + +/** + * Test cases for the builder pattern of [[SparkSession]]. + */ +class SparkSessionBuilderSuite extends SparkFunSuite { + + private var initialSession: SparkSession = _ + + private lazy val sparkContext: SparkContext = { + initialSession = SparkSession.builder() + .master("local") + .config("spark.ui.enabled", value = false) + .config("some-config", "v2") + .getOrCreate() + initialSession.sparkContext + } + + test("create with config options and propagate them to SparkContext and SparkSession") { + // Creating a new session with config - this works by just calling the lazy val + sparkContext + assert(initialSession.sparkContext.conf.get("some-config") == "v2") + assert(initialSession.conf.get("some-config") == "v2") + SparkSession.clearDefaultSession() + } + + test("use global default session") { + val session = SparkSession.builder().getOrCreate() + assert(SparkSession.builder().getOrCreate() == session) + SparkSession.clearDefaultSession() + } + + test("config options are propagated to existing SparkSession") { + val session1 = SparkSession.builder().config("spark-config1", "a").getOrCreate() + assert(session1.conf.get("spark-config1") == "a") + val session2 = SparkSession.builder().config("spark-config1", "b").getOrCreate() + assert(session1 == session2) + assert(session1.conf.get("spark-config1") == "b") + SparkSession.clearDefaultSession() + } + + test("use session from active thread session and propagate config options") { + val defaultSession = SparkSession.builder().getOrCreate() + val activeSession = defaultSession.newSession() + SparkSession.setActiveSession(activeSession) + val session = SparkSession.builder().config("spark-config2", "a").getOrCreate() + + assert(activeSession != defaultSession) + assert(session == activeSession) + assert(session.conf.get("spark-config2") == "a") + SparkSession.clearActiveSession() + + assert(SparkSession.builder().getOrCreate() == defaultSession) + SparkSession.clearDefaultSession() + } + + test("create a new session if the default session has been stopped") { + val defaultSession = SparkSession.builder().getOrCreate() + SparkSession.setDefaultSession(defaultSession) + defaultSession.stop() + val newSession = SparkSession.builder().master("local").getOrCreate() + assert(newSession != defaultSession) + newSession.stop() + } + + test("create a new session if the active thread session has been stopped") { + val activeSession = SparkSession.builder().master("local").getOrCreate() + SparkSession.setActiveSession(activeSession) + activeSession.stop() + val newSession = SparkSession.builder().master("local").getOrCreate() + assert(newSession != activeSession) + newSession.stop() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index 9523f6f9f5bb..4de3cf605caa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -26,9 +26,9 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 70a00a43f7db..2f45db3925a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -27,21 +27,21 @@ import org.apache.spark.sql.internal.SQLConf class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { - private var originalActiveSQLContext: Option[SQLContext] = _ - private var originalInstantiatedSQLContext: Option[SQLContext] = _ + private var originalActiveSQLContext: Option[SparkSession] = _ + private var originalInstantiatedSQLContext: Option[SparkSession] = _ override protected def beforeAll(): Unit = { - originalActiveSQLContext = SQLContext.getActive() - originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption() + originalActiveSQLContext = SparkSession.getActiveSession + originalInstantiatedSQLContext = SparkSession.getDefaultSession - SQLContext.clearActive() - SQLContext.clearInstantiatedContext() + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() } override protected def afterAll(): Unit = { // Set these states back. - originalActiveSQLContext.foreach(ctx => SQLContext.setActive(ctx)) - originalInstantiatedSQLContext.foreach(ctx => SQLContext.setInstantiatedContext(ctx)) + originalActiveSQLContext.foreach(ctx => SparkSession.setActiveSession(ctx)) + originalInstantiatedSQLContext.foreach(ctx => SparkSession.setDefaultSession(ctx)) } private def checkEstimation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 2a5295d0d223..8243470b1933 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -155,7 +155,7 @@ class PlannerSuite extends SharedSQLContext { val path = file.getCanonicalPath testData.write.parquet(path) val df = spark.read.parquet(path) - spark.wrapped.registerDataFrameAsTable(df, "testPushed") + spark.sqlContext.registerDataFrameAsTable(df, "testPushed") withTempTable("testPushed") { val exp = sql("select * from testPushed where key = 15").queryExecution.sparkPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala index d7eae21f9f55..9fe0e9646e31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala @@ -91,7 +91,7 @@ private[sql] abstract class SparkPlanTest extends SparkFunSuite { expectedAnswer: Seq[Row], sortAnswers: Boolean = true): Unit = { SparkPlanTest - .checkAnswer(input, planFunction, expectedAnswer, sortAnswers, spark.wrapped) match { + .checkAnswer(input, planFunction, expectedAnswer, sortAnswers, spark.sqlContext) match { case Some(errorMessage) => fail(errorMessage) case None => } @@ -115,7 +115,7 @@ private[sql] abstract class SparkPlanTest extends SparkFunSuite { expectedPlanFunction: SparkPlan => SparkPlan, sortAnswers: Boolean = true): Unit = { SparkPlanTest.checkAnswer( - input, planFunction, expectedPlanFunction, sortAnswers, spark.wrapped) match { + input, planFunction, expectedPlanFunction, sortAnswers, spark.sqlContext) match { case Some(errorMessage) => fail(errorMessage) case None => } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index b5fc51603e16..1753b84ba6af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -90,7 +90,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { (data: Seq[T], tableName: String, testVectorized: Boolean = true) (f: => Unit): Unit = { withParquetDataFrame(data, testVectorized) { df => - spark.wrapped.registerDataFrameAsTable(df, tableName) + spark.sqlContext.registerDataFrameAsTable(df, tableName) withTempTable(tableName)(f) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala index 4fa1754253af..bd197be655d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala @@ -60,13 +60,13 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn val opId = 0 val rdd1 = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( - spark.wrapped, path, opId, storeVersion = 0, keySchema, valueSchema)( + spark.sqlContext, path, opId, storeVersion = 0, keySchema, valueSchema)( increment) assert(rdd1.collect().toSet === Set("a" -> 2, "b" -> 1)) // Generate next version of stores val rdd2 = makeRDD(spark.sparkContext, Seq("a", "c")).mapPartitionsWithStateStore( - spark.wrapped, path, opId, storeVersion = 1, keySchema, valueSchema)(increment) + spark.sqlContext, path, opId, storeVersion = 1, keySchema, valueSchema)(increment) assert(rdd2.collect().toSet === Set("a" -> 3, "b" -> 1, "c" -> 1)) // Make sure the previous RDD still has the same data. @@ -82,7 +82,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn spark: SparkSession, seq: Seq[String], storeVersion: Int): RDD[(String, Int)] = { - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext makeRDD(spark.sparkContext, Seq("a")).mapPartitionsWithStateStore( sqlContext, path, opId, storeVersion, keySchema, valueSchema)(increment) } @@ -102,7 +102,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn test("usage with iterators - only gets and only puts") { withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString val opId = 0 @@ -131,7 +131,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn } val rddOfGets1 = makeRDD(spark.sparkContext, Seq("a", "b", "c")).mapPartitionsWithStateStore( - spark.wrapped, path, opId, storeVersion = 0, keySchema, valueSchema)(iteratorOfGets) + spark.sqlContext, path, opId, storeVersion = 0, keySchema, valueSchema)(iteratorOfGets) assert(rddOfGets1.collect().toSet === Set("a" -> None, "b" -> None, "c" -> None)) val rddOfPuts = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( @@ -150,7 +150,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext val coordinatorRef = sqlContext.streams.stateStoreCoordinator coordinatorRef.reportActiveInstance(StateStoreId(path, opId, 0), "host1", "exec1") coordinatorRef.reportActiveInstance(StateStoreId(path, opId, 1), "host2", "exec2") @@ -183,7 +183,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn SparkSession.builder .config(sparkConf.setMaster("local-cluster[2, 1, 1024]")) .getOrCreate()) { spark => - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString val opId = 0 val rdd1 = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 1c467137baa8..2374ffaaa503 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -24,7 +24,7 @@ import org.mockito.Mockito.mock import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -400,8 +400,8 @@ class SQLListenerMemoryLeakSuite extends SparkFunSuite { .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly val sc = new SparkContext(conf) try { - SQLContext.clearSqlListener() - val spark = new SQLContext(sc) + SparkSession.sqlListener.set(null) + val spark = new SparkSession(sc) import spark.implicits._ // Run 100 successful executions and 100 failed executions. // Each execution only has one job and one stage. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 81bc973be74a..0296229100a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -35,7 +35,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // Set a conf first. spark.conf.set(testKey, testVal) // Clear the conf. - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() // After clear, only overrideConfs used by unit test should be in the SQLConf. assert(spark.conf.getAll === TestSQLContext.overrideConfs) @@ -50,11 +50,11 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { assert(spark.conf.get(testKey, testVal + "_") === testVal) assert(spark.conf.getAll.contains(testKey)) - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("parse SQL set commands") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() sql(s"set $testKey=$testVal") assert(spark.conf.get(testKey, testVal + "_") === testVal) assert(spark.conf.get(testKey, testVal + "_") === testVal) @@ -72,11 +72,11 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { sql(s"set $key=") assert(spark.conf.get(key, "0") === "") - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("set command for display") { - spark.wrapped.conf.clear() + spark.sessionState.conf.clear() checkAnswer( sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), Nil) @@ -97,7 +97,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("deprecated property") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) try{ sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") @@ -108,7 +108,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("invalid conf value") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() val e = intercept[IllegalArgumentException] { sql(s"set ${SQLConf.CASE_SENSITIVE.key}=10") } @@ -116,7 +116,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("Test SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE's method") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "100") assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === 100) @@ -144,7 +144,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-90000000000g") } - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("SparkSession can access configs set in SparkConf") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 612cfc7ec7bd..a34f70ed65b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -41,7 +41,7 @@ case class SimpleDDLScan( table: String)(@transient val sparkSession: SparkSession) extends BaseRelation with TableScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType(Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 51d04f2f4efc..f969660ddd32 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -40,7 +40,7 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sparkSession: S extends BaseRelation with PrunedFilteredScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index cd0256db43aa..9cdf7dea7663 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -37,7 +37,7 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sparkSession: Spa extends BaseRelation with PrunedScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 34b8726a922f..cddf4a1884fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -38,7 +38,7 @@ class SimpleScanSource extends RelationProvider { case class SimpleScan(from: Int, to: Int)(@transient val sparkSession: SparkSession) extends BaseRelation with TableScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType(StructField("i", IntegerType, nullable = false) :: Nil) @@ -70,7 +70,7 @@ case class AllDataTypesScan( extends BaseRelation with TableScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = userSpecifiedSchema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala index ff5350554933..e6c0ce95e7b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala @@ -355,14 +355,14 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B q.stop() verify(LastOptions.mockStreamSourceProvider).createSource( - spark.wrapped, + spark.sqlContext, checkpointLocation + "/sources/0", None, "org.apache.spark.sql.streaming.test", Map.empty) verify(LastOptions.mockStreamSourceProvider).createSource( - spark.wrapped, + spark.sqlContext, checkpointLocation + "/sources/1", None, "org.apache.spark.sql.streaming.test", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 421f6bca7f86..0cfe260e5215 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -30,7 +30,7 @@ private[sql] trait SQLTestData { self => // Helper object to import SQL implicits without a concrete SQLContext private object internalImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.spark.wrapped + protected override def _sqlContext: SQLContext = self.spark.sqlContext } import internalImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 51538eca644f..853dd0ff3f60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -66,7 +66,7 @@ private[sql] trait SQLTestUtils * but the implicits import is needed in the constructor. */ protected object testImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.spark.wrapped + protected override def _sqlContext: SQLContext = self.spark.sqlContext } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala index 620bfa995aa2..79c37faa4e9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala @@ -44,13 +44,13 @@ trait SharedSQLContext extends SQLTestUtils { /** * The [[TestSQLContext]] to use for all tests in this suite. */ - protected implicit def sqlContext: SQLContext = _spark.wrapped + protected implicit def sqlContext: SQLContext = _spark.sqlContext /** * Initialize the [[TestSparkSession]]. */ protected override def beforeAll(): Unit = { - SQLContext.clearSqlListener() + SparkSession.sqlListener.set(null) if (_spark == null) { _spark = new TestSparkSession(sparkConf) } 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 8de223f444f7..638911599aad 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 @@ -56,7 +56,7 @@ private[hive] object SparkSQLEnv extends Logging { val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() sparkContext = sparkSession.sparkContext - sqlContext = sparkSession.wrapped + sqlContext = sparkSession.sqlContext val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState] sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) 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 d2cb62c617d4..7c74a0308d48 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 @@ -30,7 +30,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd override protected def beforeEach(): Unit = { super.beforeEach() - if (spark.wrapped.tableNames().contains("src")) { + if (spark.sqlContext.tableNames().contains("src")) { spark.catalog.dropTempView("src") } Seq((1, "")).toDF("key", "value").createOrReplaceTempView("src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 6c9ce208dbd6..622b043581c5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -36,11 +36,11 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) checkAnswer(spark.table("t"), df) } - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") @@ -50,7 +50,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle test(s"saveAsTable() to non-default database - without USE - Overwrite") { withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") @@ -65,7 +65,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").mode(SaveMode.Overwrite).save(path) spark.catalog.createExternalTable("t", path, "parquet") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table("t"), df) sql( @@ -76,7 +76,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle | path '$path' |) """.stripMargin) - assert(spark.wrapped.tableNames(db).contains("t1")) + assert(spark.sqlContext.tableNames(db).contains("t1")) checkAnswer(spark.table("t1"), df) } } @@ -90,7 +90,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").mode(SaveMode.Overwrite).save(path) spark.catalog.createExternalTable(s"$db.t", path, "parquet") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df) sql( @@ -101,7 +101,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle | path '$path' |) """.stripMargin) - assert(spark.wrapped.tableNames(db).contains("t1")) + assert(spark.sqlContext.tableNames(db).contains("t1")) checkAnswer(spark.table(s"$db.t1"), df) } } @@ -112,11 +112,11 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") df.write.mode(SaveMode.Append).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) checkAnswer(spark.table("t"), df.union(df)) } - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") @@ -127,7 +127,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") df.write.mode(SaveMode.Append).saveAsTable(s"$db.t") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") @@ -138,7 +138,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) df.write.insertInto(s"$db.t") checkAnswer(spark.table(s"$db.t"), df.union(df)) @@ -150,10 +150,10 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) } - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) df.write.insertInto(s"$db.t") checkAnswer(spark.table(s"$db.t"), df.union(df)) @@ -175,21 +175,21 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { sql(s"CREATE TABLE t (key INT)") - assert(spark.wrapped.tableNames().contains("t")) - assert(!spark.wrapped.tableNames("default").contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) + assert(!spark.sqlContext.tableNames("default").contains("t")) } - assert(!spark.wrapped.tableNames().contains("t")) - assert(spark.wrapped.tableNames(db).contains("t")) + assert(!spark.sqlContext.tableNames().contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) activateDatabase(db) { sql(s"DROP TABLE t") - assert(!spark.wrapped.tableNames().contains("t")) - assert(!spark.wrapped.tableNames("default").contains("t")) + assert(!spark.sqlContext.tableNames().contains("t")) + assert(!spark.sqlContext.tableNames("default").contains("t")) } - assert(!spark.wrapped.tableNames().contains("t")) - assert(!spark.wrapped.tableNames(db).contains("t")) + assert(!spark.sqlContext.tableNames().contains("t")) + assert(!spark.sqlContext.tableNames(db).contains("t")) } } 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 81f3ea8a6e80..8a31a49d97f0 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 @@ -1417,7 +1417,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) checkAnswer( - spark.wrapped.tables().select('isTemporary).filter('tableName === "t2"), + spark.sqlContext.tables().select('isTemporary).filter('tableName === "t2"), Row(true) ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala index aba60da33fe3..bb351e20c5e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala @@ -61,7 +61,7 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton { (data: Seq[T], tableName: String) (f: => Unit): Unit = { withOrcDataFrame(data) { df => - spark.wrapped.registerDataFrameAsTable(df, tableName) + spark.sqlContext.registerDataFrameAsTable(df, tableName) withTempTable(tableName)(f) } } From c08739afb89ef67e696b2a86ef88c988ad9f4f5d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 22:14:10 -0700 Subject: [PATCH 294/313] [SPARK-14990][SQL] Fix checkForSameTypeInputExpr (ignore nullability) ## What changes were proposed in this pull request? This patch fixes a bug in TypeUtils.checkForSameTypeInputExpr. Previously the code was testing on strict equality, which does not taking nullability into account. This is based on https://github.com/apache/spark/pull/12768. This patch fixed a bug there (with empty expression) and added a test case. ## How was this patch tested? Added a new test suite and test case. Closes #12768. Author: Reynold Xin Author: Oleg Danilov Closes #13208 from rxin/SPARK-14990. (cherry picked from commit 3ba34d435c1f61435c2dddc28650cd111e7c1f33) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/util/TypeUtils.scala | 14 ++++-- .../sql/catalyst/util/TypeUtilsSuite.scala | 46 +++++++++++++++++++ 2 files changed, 56 insertions(+), 4 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala index f603cbfb0cc2..7101ca5a17de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala @@ -42,11 +42,17 @@ object TypeUtils { } def checkForSameTypeInputExpr(types: Seq[DataType], caller: String): TypeCheckResult = { - if (types.distinct.size > 1) { - TypeCheckResult.TypeCheckFailure( - s"input to $caller should all be the same type, but it's " + - types.map(_.simpleString).mkString("[", ", ", "]")) + if (types.size <= 1) { + TypeCheckResult.TypeCheckSuccess } else { + val firstType = types.head + types.foreach { t => + if (!t.sameType(firstType)) { + return TypeCheckResult.TypeCheckFailure( + s"input to $caller should all be the same type, but it's " + + types.map(_.simpleString).mkString("[", ", ", "]")) + } + } TypeCheckResult.TypeCheckSuccess } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala new file mode 100644 index 000000000000..bc6852ca7e1f --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.types._ + +class TypeUtilsSuite extends SparkFunSuite { + + private def typeCheckPass(types: Seq[DataType]): Unit = { + assert(TypeUtils.checkForSameTypeInputExpr(types, "a") == TypeCheckSuccess) + } + + private def typeCheckFail(types: Seq[DataType]): Unit = { + assert(TypeUtils.checkForSameTypeInputExpr(types, "a").isInstanceOf[TypeCheckFailure]) + } + + test("checkForSameTypeInputExpr") { + typeCheckPass(Nil) + typeCheckPass(StringType :: Nil) + typeCheckPass(StringType :: StringType :: Nil) + + typeCheckFail(StringType :: IntegerType :: Nil) + typeCheckFail(StringType :: IntegerType :: Nil) + + // Should also work on arrays. See SPARK-14990 + typeCheckPass(ArrayType(StringType, containsNull = true) :: + ArrayType(StringType, containsNull = false) :: Nil) + } +} From 7bb33352f2409493bd7e3880d98508dcb1be888d Mon Sep 17 00:00:00 2001 From: Oleg Danilov Date: Thu, 19 May 2016 22:23:28 -0700 Subject: [PATCH 295/313] [SPARK-14261][SQL] Memory leak in Spark Thrift Server Fixed memory leak (HiveConf in the CommandProcessorFactory) Author: Oleg Danilov Closes #12932 from dosoft/SPARK-14261. (cherry picked from commit e384c7fbb94cef3c18e8fa8d06159b76b88b5167) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 0f0c1b0702fb..71d5c9960a70 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -545,12 +545,14 @@ private[hive] class HiveClientImpl( // Throw an exception if there is an error in query processing. if (response.getResponseCode != 0) { driver.close() + CommandProcessorFactory.clean(conf) throw new QueryExecutionException(response.getErrorMessage) } driver.setMaxRows(maxRows) val results = shim.getDriverResults(driver) driver.close() + CommandProcessorFactory.clean(conf) results case _ => From dcf36ad54598118408c1425e81aa6552f42328c8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 May 2016 14:02:04 +0100 Subject: [PATCH 296/313] [SPARK-15057][GRAPHX] Remove stale TODO comment for making `enum` in GraphGenerators This PR removes a stale TODO comment in `GraphGenerators.scala` Just comment removed. Author: Dongjoon Hyun Closes #12839 from dongjoon-hyun/SPARK-15057. (cherry picked from commit 46965cd014fd4ba68bdec15156ec9bcc27d9b217) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 80c6b6838faf..4da1ecb2a9af 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -209,7 +209,6 @@ object GraphGenerators extends Logging { } } - // TODO(crankshaw) turn result into an enum (or case class for pattern matching} private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { if (a + b + c + d != 1.0) { throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a + b + c + d) From 1dc30f189ac30f070068ca5f60b7b4c85f2adc9e Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 19 May 2016 04:48:36 +0200 Subject: [PATCH 297/313] [DOC][MINOR] ml.feature Scala and Python API sync I reviewed Scala and Python APIs for ml.feature and corrected discrepancies. Built docs locally, ran style checks Author: Bryan Cutler Closes #13159 from BryanCutler/ml.feature-api-sync. (cherry picked from commit b1bc5ebdd52ed12aea3fdc7b8f2fa2d00ea09c6b) Signed-off-by: Reynold Xin --- .../org/apache/spark/ml/feature/IDF.scala | 4 +- .../org/apache/spark/ml/feature/PCA.scala | 5 ++- .../apache/spark/ml/feature/RFormula.scala | 4 +- .../spark/ml/feature/VectorIndexer.scala | 3 +- python/pyspark/ml/feature.py | 39 ++++++++++++------- 5 files changed, 36 insertions(+), 19 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index f85f4c65af7d..08beda6d7515 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -38,12 +38,12 @@ import org.apache.spark.sql.types.StructType private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol { /** - * The minimum of documents in which a term should appear. + * The minimum number of documents in which a term should appear. * Default: 0 * @group param */ final val minDocFreq = new IntParam( - this, "minDocFreq", "minimum of documents in which a term should appear for filtering") + this, "minDocFreq", "minimum number of documents in which a term should appear for filtering") setDefault(minDocFreq -> 0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 141d3b924b4f..dbbaa5aa46f4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -53,7 +53,8 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC /** * :: Experimental :: - * PCA trains a model to project vectors to a low-dimensional space using PCA. + * PCA trains a model to project vectors to a lower dimensional space of the top [[PCA!.k]] + * principal components. */ @Experimental class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams @@ -106,7 +107,7 @@ object PCA extends DefaultParamsReadable[PCA] { /** * :: Experimental :: - * Model fitted by [[PCA]]. + * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. * * @param pc A principal components Matrix. Each column is one principal component. * @param explainedVariance A vector of proportions of variance explained by diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index c0feaa01fc86..2916b6d9df3b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -194,7 +194,9 @@ object RFormula extends DefaultParamsReadable[RFormula] { /** * :: Experimental :: - * A fitted RFormula. Fitting is required to determine the factor levels of formula terms. + * Model fitted by [[RFormula]]. Fitting is required to determine the factor levels of + * formula terms. + * * @param resolvedFormula the fitted R formula. * @param pipelineModel the fitted feature model, including factor to index mappings. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 2bc9d225ac2f..d814528ec48d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -240,7 +240,8 @@ object VectorIndexer extends DefaultParamsReadable[VectorIndexer] { /** * :: Experimental :: - * Transform categorical features to use 0-based indices instead of their original values. + * Model fitted by [[VectorIndexer]]. Transform categorical features to use 0-based indices + * instead of their original values. * - Categorical features are mapped to indices. * - Continuous features (columns) are left unchanged. * This also appends metadata to the output column, marking features as Numeric (continuous), diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 983b6a5301ae..497f2ad68ee5 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -352,7 +352,7 @@ class CountVectorizerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by CountVectorizer. + Model fitted by :py:class:`CountVectorizer`. .. versionadded:: 1.6.0 """ @@ -609,7 +609,7 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab """ minDocFreq = Param(Params._dummy(), "minDocFreq", - "minimum of documents in which a term should appear for filtering", + "minimum number of documents in which a term should appear for filtering", typeConverter=TypeConverters.toInt) @keyword_only @@ -655,7 +655,7 @@ class IDFModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by IDF. + Model fitted by :py:class:`IDF`. .. versionadded:: 1.4.0 """ @@ -1302,7 +1302,8 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, minTokenLength = Param(Params._dummy(), "minTokenLength", "minimum token length (>= 0)", typeConverter=TypeConverters.toInt) - gaps = Param(Params._dummy(), "gaps", "whether regex splits on gaps (True) or matches tokens") + gaps = Param(Params._dummy(), "gaps", "whether regex splits on gaps (True) or matches tokens " + + "(False)") pattern = Param(Params._dummy(), "pattern", "regex pattern (Java dialect) used for tokenizing", typeConverter=TypeConverters.toString) toLowercase = Param(Params._dummy(), "toLowercase", "whether to convert all characters to " + @@ -1549,7 +1550,7 @@ class StandardScalerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by StandardScaler. + Model fitted by :py:class:`StandardScaler`. .. versionadded:: 1.4.0 """ @@ -1641,7 +1642,7 @@ class StringIndexerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by StringIndexer. + Model fitted by :py:class:`StringIndexer`. .. versionadded:: 1.4.0 """ @@ -1907,7 +1908,7 @@ class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Ja """ .. note:: Experimental - Class for indexing categorical feature columns in a dataset of [[Vector]]. + Class for indexing categorical feature columns in a dataset of `Vector`. This has 2 usage modes: - Automatically identify categorical features (default behavior) @@ -2023,7 +2024,17 @@ class VectorIndexerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by VectorIndexer. + Model fitted by :py:class:`VectorIndexer`. + + Transform categorical features to use 0-based indices instead of their original values. + - Categorical features are mapped to indices. + - Continuous features (columns) are left unchanged. + + This also appends metadata to the output column, marking features as Numeric (continuous), + Nominal (categorical), or Binary (either continuous or categorical). + Non-ML metadata is not carried over from the input to the output column. + + This maintains vector sparsity. .. versionadded:: 1.4.0 """ @@ -2296,7 +2307,7 @@ class Word2VecModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by Word2Vec. + Model fitted by :py:class:`Word2Vec`. .. versionadded:: 1.4.0 """ @@ -2327,7 +2338,8 @@ class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab """ .. note:: Experimental - PCA trains a model to project vectors to a low-dimensional space using PCA. + PCA trains a model to project vectors to a lower dimensional space of the + top :py:attr:`k` principal components. >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), @@ -2401,7 +2413,7 @@ class PCAModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by PCA. + Model fitted by :py:class:`PCA`. Transforms vectors to a lower dimensional space. .. versionadded:: 1.5.0 """ @@ -2532,7 +2544,8 @@ class RFormulaModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by :py:class:`RFormula`. + Model fitted by :py:class:`RFormula`. Fitting is required to determine the + factor levels of formula terms. .. versionadded:: 1.5.0 """ @@ -2624,7 +2637,7 @@ class ChiSqSelectorModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by ChiSqSelector. + Model fitted by :py:class:`ChiSqSelector`. .. versionadded:: 2.0.0 """ From 642f00980f1de13a0f6d1dc8bc7ed5b0547f3a9d Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sun, 15 May 2016 15:59:49 +0100 Subject: [PATCH 298/313] [MINOR] Fix Typos 1,Rename matrix args in BreezeUtil to upper to match the doc 2,Fix several typos in ML and SQL manual tests Author: Zheng RuiFeng Closes #13078 from zhengruifeng/fix_ann. (cherry picked from commit c7efc56c7b6fc99c005b35c335716ff676856c6c) Signed-off-by: Reynold Xin --- docs/ml-guide.md | 2 +- .../org/apache/spark/ml/ann/BreezeUtil.scala | 33 ++++++++--------- .../scala/org/apache/spark/ml/ann/Layer.scala | 36 ++++++++++--------- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 8 ++--- .../sql/expressions/scalalang/typed.scala | 2 +- 6 files changed, 42 insertions(+), 41 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 99167873cd02..cc353df1ecd0 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -257,7 +257,7 @@ Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/ The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) -for binary data, or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) +for binary data, or a [`MulticlassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) for multiclass problems. The default metric used to choose the best `ParamMap` can be overridden by the `setMetricName` method in each of these evaluators. diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala index 7429f9d652ac..6bbe7e1cb213 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala @@ -26,38 +26,39 @@ import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} private[ann] object BreezeUtil { // TODO: switch to MLlib BLAS interface - private def transposeString(a: BDM[Double]): String = if (a.isTranspose) "T" else "N" + private def transposeString(A: BDM[Double]): String = if (A.isTranspose) "T" else "N" /** * DGEMM: C := alpha * A * B + beta * C * @param alpha alpha - * @param a A - * @param b B + * @param A A + * @param B B * @param beta beta - * @param c C + * @param C C */ - def dgemm(alpha: Double, a: BDM[Double], b: BDM[Double], beta: Double, c: BDM[Double]): Unit = { + def dgemm(alpha: Double, A: BDM[Double], B: BDM[Double], beta: Double, C: BDM[Double]): Unit = { // TODO: add code if matrices isTranspose!!! - require(a.cols == b.rows, "A & B Dimension mismatch!") - require(a.rows == c.rows, "A & C Dimension mismatch!") - require(b.cols == c.cols, "A & C Dimension mismatch!") - NativeBLAS.dgemm(transposeString(a), transposeString(b), c.rows, c.cols, a.cols, - alpha, a.data, a.offset, a.majorStride, b.data, b.offset, b.majorStride, - beta, c.data, c.offset, c.rows) + require(A.cols == B.rows, "A & B Dimension mismatch!") + require(A.rows == C.rows, "A & C Dimension mismatch!") + require(B.cols == C.cols, "A & C Dimension mismatch!") + NativeBLAS.dgemm(transposeString(A), transposeString(B), C.rows, C.cols, A.cols, + alpha, A.data, A.offset, A.majorStride, B.data, B.offset, B.majorStride, + beta, C.data, C.offset, C.rows) } /** * DGEMV: y := alpha * A * x + beta * y * @param alpha alpha - * @param a A + * @param A A * @param x x * @param beta beta * @param y y */ - def dgemv(alpha: Double, a: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { - require(a.cols == x.length, "A & b Dimension mismatch!") - NativeBLAS.dgemv(transposeString(a), a.rows, a.cols, - alpha, a.data, a.offset, a.majorStride, x.data, x.offset, x.stride, + def dgemv(alpha: Double, A: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { + require(A.cols == x.length, "A & x Dimension mismatch!") + require(A.rows == y.length, "A & y Dimension mismatch!") + NativeBLAS.dgemv(transposeString(A), A.rows, A.cols, + alpha, A.data, A.offset, A.majorStride, x.data, x.offset, x.stride, beta, y.data, y.offset, y.stride) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 889b7f9f719c..0a569c49179a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -66,8 +66,9 @@ private[ann] trait Layer extends Serializable { * @return the layer model */ def createModel(initialWeights: BDV[Double]): LayerModel + /** - * Returns the instance of the layer with random generated weights + * Returns the instance of the layer with random generated weights. * * @param weights vector for weights initialization, must be equal to weightSize * @param random random number generator @@ -85,11 +86,11 @@ private[ann] trait LayerModel extends Serializable { val weights: BDV[Double] /** - * Evaluates the data (process the data through the layer) + * Evaluates the data (process the data through the layer). * Output is allocated based on the size provided by the - * LayerModel implementation and the stack (batch) size + * LayerModel implementation and the stack (batch) size. * Developer is responsible for checking the size of output - * when writing to it + * when writing to it. * * @param data data * @param output output (modified in place) @@ -97,11 +98,11 @@ private[ann] trait LayerModel extends Serializable { def eval(data: BDM[Double], output: BDM[Double]): Unit /** - * Computes the delta for back propagation + * Computes the delta for back propagation. * Delta is allocated based on the size provided by the - * LayerModel implementation and the stack (batch) size + * LayerModel implementation and the stack (batch) size. * Developer is responsible for checking the size of - * prevDelta when writing to it + * prevDelta when writing to it. * * @param delta delta of this layer * @param output output of this layer @@ -110,10 +111,10 @@ private[ann] trait LayerModel extends Serializable { def computePrevDelta(delta: BDM[Double], output: BDM[Double], prevDelta: BDM[Double]): Unit /** - * Computes the gradient - * cumGrad is a wrapper on the part of the weight vector - * size of cumGrad is based on weightSize provided by - * implementation of LayerModel + * Computes the gradient. + * cumGrad is a wrapper on the part of the weight vector. + * Size of cumGrad is based on weightSize provided by + * implementation of LayerModel. * * @param delta delta for this layer * @param input input data @@ -199,11 +200,11 @@ private[ann] object AffineLayerModel { } /** - * Initialize weights randomly in the interval - * Uses [Bottou-88] heuristic [-a/sqrt(in); a/sqrt(in)] - * where a is chosen in a such way that the weight variance corresponds + * Initialize weights randomly in the interval. + * Uses [Bottou-88] heuristic [-a/sqrt(in); a/sqrt(in)], + * where `a` is chosen in such a way that the weight variance corresponds * to the points to the maximal curvature of the activation function - * (which is approximately 2.38 for a standard sigmoid) + * (which is approximately 2.38 for a standard sigmoid). * * @param numIn number of inputs * @param numOut number of outputs @@ -308,7 +309,7 @@ private[ann] class FunctionalLayer (val activationFunction: ActivationFunction) /** * Functional layer model. Holds no weights. * - * @param layer functiona layer + * @param layer functional layer */ private[ann] class FunctionalLayerModel private[ann] (val layer: FunctionalLayer) extends LayerModel { @@ -354,6 +355,7 @@ private[ann] trait TopologyModel extends Serializable { * Array of layer models */ val layerModels: Array[LayerModel] + /** * Forward propagation * @@ -412,7 +414,7 @@ private[ml] object FeedForwardTopology { * Creates a multi-layer perceptron * * @param layerSizes sizes of layers including input and output size - * @param softmaxOnTop wether to use SoftMax or Sigmoid function for an output layer. + * @param softmaxOnTop whether to use SoftMax or Sigmoid function for an output layer. * Softmax is default * @return multilayer perceptron topology */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index ab751f0c0411..f697769bdcdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.ui.SQLListener -import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState, SQLConf} +import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, LongType, StructType} import org.apache.spark.sql.util.ExecutionListenerManager diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 2021cddab440..486a440b6f9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -27,9 +27,7 @@ import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext} -import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.Encoder import org.apache.spark.sql.types._ private[sql] object SQLUtils { @@ -76,7 +74,7 @@ private[sql] object SQLUtils { org.apache.spark.sql.types.MapType(getSQLDataType(keyType), getSQLDataType(valueType)) case r"\Astruct<(.+)${fieldsStr}>\Z" => if (fieldsStr(fieldsStr.length - 1) == ',') { - throw new IllegalArgumentException(s"Invaid type $dataType") + throw new IllegalArgumentException(s"Invalid type $dataType") } val fields = fieldsStr.split(",") val structFields = fields.map { field => @@ -84,11 +82,11 @@ private[sql] object SQLUtils { case r"\A(.+)${fieldName}:(.+)${fieldType}\Z" => createStructField(fieldName, fieldType, true) - case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + case _ => throw new IllegalArgumentException(s"Invalid type $dataType") } } createStructType(structFields) - case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + case _ => throw new IllegalArgumentException(s"Invalid type $dataType") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala index f46a4a787978..60d7b7d0894d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala @@ -38,7 +38,7 @@ object typed { // The reason we have separate files for Java and Scala is because in the Scala version, we can // use tighter types (primitive types) for return types, whereas in the Java version we can only // use boxed primitive types. - // For example, avg in the Scala veresion returns Scala primitive Double, whose bytecode + // For example, avg in the Scala version returns Scala primitive Double, whose bytecode // signature is just a java.lang.Object; avg in the Java version returns java.lang.Double. // TODO: This is pretty hacky. Maybe we should have an object for implicit encoders. From 2126fb0c2b2bb8ac4c5338df15182fcf8713fb2f Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 10:44:26 +0100 Subject: [PATCH 299/313] [CORE][MINOR] Remove redundant set master in OutputCommitCoordinatorIntegrationSuite Remove redundant set master in OutputCommitCoordinatorIntegrationSuite, as we are already setting it in SparkContext below on line 43. existing tests Author: Sandeep Singh Closes #13168 from techaddict/minor-1. (cherry picked from commit 3facca5152e685d9c7da96bff5102169740a4a06) Signed-off-by: Reynold Xin --- .../scheduler/OutputCommitCoordinatorIntegrationSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala index 601f1c378c41..32cdf16dd331 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala @@ -37,7 +37,6 @@ class OutputCommitCoordinatorIntegrationSuite override def beforeAll(): Unit = { super.beforeAll() val conf = new SparkConf() - .set("master", "local[2,4]") .set("spark.hadoop.outputCommitCoordination.enabled", "true") .set("spark.hadoop.mapred.output.committer.class", classOf[ThrowExceptionOnFirstAttemptOutputCommitter].getCanonicalName) From 1fc0f95eb8abbb9cc8ede2139670e493e6939317 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 22:40:03 -0700 Subject: [PATCH 300/313] [HOTFIX] Test compilation error from 52b967f --- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index df029e44c54e..9b7eac8e0f9c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1481,8 +1481,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) } } From dd0c7fb39cac44e8f0d73f9884fd1582c25e9cf4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 22:46:08 -0700 Subject: [PATCH 301/313] Revert "[HOTFIX] Test compilation error from 52b967f" This reverts commit 1fc0f95eb8abbb9cc8ede2139670e493e6939317. --- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 9b7eac8e0f9c..df029e44c54e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1481,8 +1481,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.sessionState.conf.autoBroadcastJoinThreshold) + spark.wrapped.conf.autoBroadcastJoinThreshold) assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.sessionState.conf.autoBroadcastJoinThreshold) + spark.wrapped.conf.autoBroadcastJoinThreshold) } } From f8d0177c31d43eab59a7535945f3dfa24e906273 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 16:02:52 -0700 Subject: [PATCH 302/313] Revert "[SPARK-15392][SQL] fix default value of size estimation of logical plan" This reverts commit fc29b896dae08b957ed15fa681b46162600a4050. (cherry picked from commit 84b23453ddb0a97e3d81306de0a5dcb64f88bdd0) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 --------- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 35d67ca2d8c5..d1c83f2cebfa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -595,7 +595,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) - def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, Long.MaxValue) + def defaultSizeInBytes: Long = + getConf(DEFAULT_SIZE_IN_BYTES, autoBroadcastJoinThreshold + 1L) def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index df029e44c54e..f573abf85961 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1476,13 +1476,4 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { getMessage() assert(e1.startsWith("Path does not exist")) } - - test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { - val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) - val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) - assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) - assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) - } } From 2ef645724a7f229309a87c5053b0fbdf45d06f52 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 19 May 2016 22:55:44 -0700 Subject: [PATCH 303/313] [SPARK-15313][SQL] EmbedSerializerInFilter rule should keep exprIds of output of surrounded SerializeFromObject. ## What changes were proposed in this pull request? The following code: ``` val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() ds.filter(_._1 == "b").select(expr("_1").as[String]).foreach(println(_)) ``` throws an Exception: ``` org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: _1#420 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87) ... Cause: java.lang.RuntimeException: Couldn't find _1#420 in [_1#416,_2#417] at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:94) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87) ... ``` This is because `EmbedSerializerInFilter` rule drops the `exprId`s of output of surrounded `SerializeFromObject`. The analyzed and optimized plans of the above example are as follows: ``` == Analyzed Logical Plan == _1: string Project [_1#420] +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421] +- Filter .apply +- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2 +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] == Optimized Logical Plan == !Project [_1#420] +- Filter .apply +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] ``` This PR fixes `EmbedSerializerInFilter` rule to keep `exprId`s of output of surrounded `SerializeFromObject`. The plans after this patch are as follows: ``` == Analyzed Logical Plan == _1: string Project [_1#420] +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421] +- Filter .apply +- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2 +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] == Optimized Logical Plan == Project [_1#416] +- Filter .apply +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] ``` ## How was this patch tested? Existing tests and I added a test to check if `filter and then select` works. Author: Takuya UESHIN Closes #13096 from ueshin/issues/SPARK-15313. (cherry picked from commit d5e1c5acde95158db38448526c8afad4a6d21dc2) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 12 ++++++++++-- .../optimizer/TypedFilterOptimizationSuite.scala | 2 +- .../scala/org/apache/spark/sql/DatasetSuite.scala | 7 +++++++ 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 6825b65e2b28..a6fb34cbfb11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -109,7 +109,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: Batch("Typed Filter Optimization", fixedPoint, - EmbedSerializerInFilter) :: + EmbedSerializerInFilter, + RemoveAliasOnlyProject) :: Batch("LocalRelation", fixedPoint, ConvertToLocalRelation) :: Batch("OptimizeCodegen", Once, @@ -1611,7 +1612,14 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] { val newCondition = condition transform { case a: Attribute if a == d.output.head => d.deserializer } - Filter(newCondition, d.child) + val filter = Filter(newCondition, d.child) + + // Adds an extra Project here, to preserve the output expr id of `SerializeFromObject`. + // We will remove it later in RemoveAliasOnlyProject rule. + val objAttrs = filter.output.zip(s.output).map { case (fout, sout) => + Alias(fout, fout.name)(exprId = sout.exprId) + } + Project(objAttrs, filter) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala index 1fae64e3bc6b..289c16aef47a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala @@ -67,7 +67,7 @@ class TypedFilterOptimizationSuite extends PlanTest { val deserializer = UnresolvedDeserializer(encoderFor[(Int, Int)].deserializer) val condition = callFunction(f, BooleanType, deserializer) - val expected = input.where(condition).analyze + val expected = input.where(condition).select('_1.as("_1"), '_2.as("_2")).analyze comparePlans(optimized, expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 52e706285c7a..0ffbd6db12be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -205,6 +205,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("b", 2)) } + test("filter and then select") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( + ds.filter(_._1 == "b").select(expr("_1").as[String]), + ("b")) + } + test("foreach") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.longAccumulator From 612866473503cbf4f025ae9678cef0f75a94aba8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 22:55:29 -0700 Subject: [PATCH 304/313] [HOTFIX] Add back intended change from SPARK-15392 This was accidentally reverted in f8d0177. --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d1c83f2cebfa..35d67ca2d8c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -595,8 +595,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) - def defaultSizeInBytes: Long = - getConf(DEFAULT_SIZE_IN_BYTES, autoBroadcastJoinThreshold + 1L) + def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, Long.MaxValue) def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING) From 47feebd13dca730c7769bcdc64a0ecc5b6c6c563 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 19 May 2016 23:03:59 -0700 Subject: [PATCH 305/313] [SPARK-15335][SQL] Implement TRUNCATE TABLE Command ## What changes were proposed in this pull request? Like TRUNCATE TABLE Command in Hive, TRUNCATE TABLE is also supported by Hive. See the link: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL Below is the related Hive JIRA: https://issues.apache.org/jira/browse/HIVE-446 This PR is to implement such a command for truncate table excluded column truncation(HIVE-4005). ## How was this patch tested? Added a test case. Author: Lianhui Wang Closes #13170 from lianhuiwang/truncate. (cherry picked from commit 09a00510c4759ff87abb0b2fdf1630ddf36ca12c) Signed-off-by: Reynold Xin --- .../spark/sql/execution/SparkSqlParser.scala | 19 +++++ .../spark/sql/execution/command/tables.scala | 53 +++++++++++++ .../sql/hive/execution/HiveCommandSuite.scala | 79 +++++++++++++++++++ 3 files changed, 151 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3045f3af360b..8af6d07719c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -350,6 +350,25 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ) } + /** + * Create a [[TruncateTable]] command. + * + * For example: + * {{{ + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * [COLUMNS (col1, col2)] + * }}} + */ + override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { + if (ctx.identifierList != null) { + throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) + } + TruncateTable( + visitTableIdentifier(ctx.tableIdentifier), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) + ) + } + /** * Convert a table property list into a key-value map. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index a3472745371f..d13492e55070 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -22,6 +22,9 @@ import java.net.URI import java.util.Date import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal + +import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier @@ -270,6 +273,56 @@ case class LoadData( } } +/** + * A command to truncate table. + * + * The syntax of this command is: + * {{{ + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * }}} + */ +case class TruncateTable( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + if (!catalog.tableExists(tableName)) { + logError(s"table '$tableName' in TRUNCATE TABLE does not exist.") + } else if (catalog.isTemporaryTable(tableName)) { + logError(s"table '$tableName' in TRUNCATE TABLE is a temporary table.") + } else { + val locations = if (partitionSpec.isDefined) { + catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) + } else { + val table = catalog.getTableMetadata(tableName) + if (table.partitionColumnNames.nonEmpty) { + catalog.listPartitions(tableName).map(_.storage.locationUri) + } else { + Seq(table.storage.locationUri) + } + } + val hadoopConf = sparkSession.sessionState.newHadoopConf() + locations.foreach { location => + if (location.isDefined) { + val path = new Path(location.get) + try { + val fs = path.getFileSystem(hadoopConf) + fs.delete(path, true) + fs.mkdirs(path) + } catch { + case NonFatal(e) => + throw new AnalysisException( + s"Failed to truncate table '$tableName' when removing data of the path: $path " + + s"because of ${e.toString}") + } + } + } + } + Seq.empty[Row] + } +} + /** * Command that looks like * {{{ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 8225bd69c1c7..df62ba08b801 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -269,6 +270,84 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } + test("Truncate Table") { + withTable("non_part_table", "part_table") { + sql( + """ + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + val testData = hiveContext.getHiveFile("data/files/employee.dat").getCanonicalPath + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE non_part_table""") + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Nil) + + val testResults = sql("SELECT * FROM non_part_table").collect() + + intercept[ParseException] { + sql("TRUNCATE TABLE non_part_table COLUMNS (employeeID)") + } + + sql("TRUNCATE TABLE non_part_table") + checkAnswer(sql("SELECT * FROM non_part_table"), Seq.empty[Row]) + + sql( + """ + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="1", d="1")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '1'"), + testResults) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="1", d="2")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + testResults) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="2", d="2")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '2'"), + testResults) + + intercept[ParseException] { + sql("TRUNCATE TABLE part_table PARTITION(c='1', d='1') COLUMNS (employeeID)") + } + + sql("TRUNCATE TABLE part_table PARTITION(c='1', d='1')") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '1'"), + Seq.empty[Row]) + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + testResults) + + intercept[ParseException] { + sql("TRUNCATE TABLE part_table PARTITION(c='1') COLUMNS (employeeID)") + } + + sql("TRUNCATE TABLE part_table PARTITION(c='1')") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1'"), + Seq.empty[Row]) + + sql("TRUNCATE TABLE part_table") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table"), + Seq.empty[Row]) + } + } + test("show columns") { checkAnswer( sql("SHOW COLUMNS IN parquet_tab3"), From 8fb087772d7e226a188e2f3298abb603fd3909ed Mon Sep 17 00:00:00 2001 From: dding3 Date: Mon, 9 May 2016 09:43:07 +0100 Subject: [PATCH 306/313] [SPARK-15172][ML] Explicitly tell user initial coefficients is ignored when size mismatch happened in LogisticRegression ## What changes were proposed in this pull request? Explicitly tell user initial coefficients is ignored if its size doesn't match expected size in LogisticRegression ## How was this patch tested? local build Author: dding3 Closes #12948 from dding3/master. (cherry picked from commit a78fbfa619a13421b294328b80c82510ca7efed0) Signed-off-by: Xiangrui Meng --- .../apache/spark/ml/classification/LogisticRegression.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 27380ca5c0a7..0ab4459bdb9d 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 @@ -367,9 +367,10 @@ class LogisticRegression @Since("1.2.0") ( Vectors.zeros(if ($(fitIntercept)) numFeatures + 1 else numFeatures) if (optInitialModel.isDefined && optInitialModel.get.coefficients.size != numFeatures) { - val vec = optInitialModel.get.coefficients + val vecSize = optInitialModel.get.coefficients.size logWarning( - s"Initial coefficients provided $vec did not match the expected size $numFeatures") + s"Initial coefficients will be ignored!! As its size $vecSize did not match the " + + s"expected size $numFeatures") } if (optInitialModel.isDefined && optInitialModel.get.coefficients.size == numFeatures) { From e4e3e9867e3aba6f3c32bc2c2d060bc681d829c9 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Thu, 19 May 2016 23:21:17 -0700 Subject: [PATCH 307/313] [SPARK-15363][ML][EXAMPLE] Example code shouldn't use VectorImplicits._, asML/fromML ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) In this DataFrame example, we use VectorImplicits._, which is private API. Since Vectors object has public API, we use Vectors.fromML instead of implicts. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually run the example. Author: wm624@hotmail.com Closes #13213 from wangmiao1981/ml. (cherry picked from commit 4c7a6b385c79f4de07a89495afce4f8e73b06086) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/examples/ml/DataFrameExample.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 8ed636778720..c69027babba8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -25,7 +25,7 @@ import scopt.OptionParser import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.linalg.Vector -import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.sql.{DataFrame, Row, SparkSession} @@ -81,7 +81,7 @@ object DataFrameExample { // Convert features column to an RDD of vectors. val features = df.select("features").rdd.map { case Row(v: Vector) => v } val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())( - (summary, feat) => summary.add(feat), + (summary, feat) => summary.add(Vectors.fromML(feat)), (sum1, sum2) => sum1.merge(sum2)) println(s"Selected features column with average values:\n ${featureSummary.mean.toString}") From 539dfa205dacea72188642f15773a30a99f8e8ac Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Thu, 19 May 2016 23:26:11 -0700 Subject: [PATCH 308/313] [SPARK-15398][ML] Update the warning message to recommend ML usage ## What changes were proposed in this pull request? MLlib are not recommended to use, and some methods are even deprecated. Update the warning message to recommend ML usage. ``` def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS |for more conventional use. """.stripMargin) } ``` To ``` def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } ``` ## How was this patch tested? local build Author: Zheng RuiFeng Closes #13190 from zhengruifeng/update_recd. (cherry picked from commit 47a2940da97caa55bbb8bb8ec1d51c9f6d5041c6) Signed-off-by: Xiangrui Meng --- .../main/java/org/apache/spark/examples/JavaHdfsLR.java | 6 ++---- examples/src/main/python/als.py | 4 ++-- examples/src/main/python/kmeans.py | 8 ++++---- examples/src/main/python/logistic_regression.py | 7 ++++--- .../main/scala/org/apache/spark/examples/LocalALS.scala | 4 ++-- .../scala/org/apache/spark/examples/LocalFileLR.scala | 6 ++---- .../scala/org/apache/spark/examples/LocalKMeans.scala | 4 ++-- .../main/scala/org/apache/spark/examples/LocalLR.scala | 6 ++---- .../main/scala/org/apache/spark/examples/SparkALS.scala | 4 ++-- .../scala/org/apache/spark/examples/SparkHdfsLR.scala | 6 ++---- .../scala/org/apache/spark/examples/SparkKMeans.scala | 4 ++-- .../main/scala/org/apache/spark/examples/SparkLR.scala | 6 ++---- 12 files changed, 28 insertions(+), 37 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 31a79ddd3fff..f64155ce3c0c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -32,8 +32,7 @@ * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ public final class JavaHdfsLR { @@ -43,8 +42,7 @@ public final class JavaHdfsLR { static void showWarning() { String warning = "WARN: This is a naive implementation of Logistic Regression " + "and is given as an example!\n" + - "Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD " + - "or org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS " + + "Please use org.apache.spark.ml.classification.LogisticRegression " + "for more conventional use."; System.err.println(warning); } diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 205ca02962be..f07020b50397 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -17,7 +17,7 @@ """ This is an example implementation of ALS for learning how to use Spark. Please refer to -ALS in pyspark.mllib.recommendation for more conventional use. +pyspark.ml.recommendation.ALS for more conventional use. This example requires numpy (http://www.numpy.org/) """ @@ -59,7 +59,7 @@ def update(i, vec, mat, ratings): """ print("""WARN: This is a naive implementation of ALS and is given as an - example. Please use the ALS method found in pyspark.mllib.recommendation for more + example. Please use pyspark.ml.recommendation.ALS for more conventional use.""", file=sys.stderr) sc = SparkContext(appName="PythonALS") diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 0ea7cfb7025a..3426e491dc74 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -17,8 +17,8 @@ """ The K-means algorithm written from scratch against PySpark. In practice, -one may prefer to use the KMeans algorithm in MLlib, as shown in -examples/src/main/python/mllib/kmeans.py. +one may prefer to use the KMeans algorithm in ML, as shown in +examples/src/main/python/ml/kmeans_example.py. This example requires NumPy (http://www.numpy.org/). """ @@ -52,8 +52,8 @@ def closestPoint(p, centers): exit(-1) print("""WARN: This is a naive implementation of KMeans Clustering and is given - as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on - how to use MLlib's KMeans implementation.""", file=sys.stderr) + as an example! Please refer to examples/src/main/python/ml/kmeans_example.py for an + example on how to use ML's KMeans implementation.""", file=sys.stderr) sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index b318b7d87bfd..7d33be7e81d7 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -20,7 +20,7 @@ to act on batches of input data using efficient matrix operations. In practice, one may prefer to use the LogisticRegression algorithm in -MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. +ML, as shown in examples/src/main/python/ml/logistic_regression_with_elastic_net.py. """ from __future__ import print_function @@ -51,8 +51,9 @@ def readPointBatch(iterator): exit(-1) print("""WARN: This is a naive implementation of Logistic Regression and is - given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py - to see how MLlib's implementation is used.""", file=sys.stderr) + given as an example! + Please refer to examples/src/main/python/ml/logistic_regression_with_elastic_net.py + to see how ML's implementation is used.""", file=sys.stderr) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index fa1010195551..97aefac025e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -24,7 +24,7 @@ import org.apache.commons.math3.linear._ * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object LocalALS { @@ -96,7 +96,7 @@ object LocalALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index bec89f7c3dff..3d02ce05619a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -26,8 +26,7 @@ import breeze.linalg.{DenseVector, Vector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalFileLR { val D = 10 // Number of dimensions @@ -43,8 +42,7 @@ object LocalFileLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index f8961847f3df..fca585c2a362 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -29,7 +29,7 @@ import breeze.linalg.{squaredDistance, DenseVector, Vector} * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object LocalKMeans { val N = 1000 @@ -66,7 +66,7 @@ object LocalKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 0baf6db607ad..13ccc2ae7c3d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -26,8 +26,7 @@ import breeze.linalg.{DenseVector, Vector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalLR { val N = 10000 // Number of data points @@ -50,8 +49,7 @@ object LocalLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 4263680c6fde..b06c62980200 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -26,7 +26,7 @@ import org.apache.spark._ * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object SparkALS { @@ -81,7 +81,7 @@ object SparkALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 7463b868ff19..c514eb0fa51a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -31,8 +31,7 @@ import org.apache.spark._ * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkHdfsLR { val D = 10 // Number of dimensions @@ -54,8 +53,7 @@ object SparkHdfsLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index d9f94a42b1a0..676164806e19 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkConf, SparkContext} * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object SparkKMeans { @@ -52,7 +52,7 @@ object SparkKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index acd8656b65a6..718f84f6450e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -31,8 +31,7 @@ import org.apache.spark._ * Usage: SparkLR [slices] * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkLR { val N = 10000 // Number of data points @@ -55,8 +54,7 @@ object SparkLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } From 5f73f627f966926ac477663642903f175cad54d0 Mon Sep 17 00:00:00 2001 From: sethah Date: Thu, 19 May 2016 23:29:37 -0700 Subject: [PATCH 309/313] [SPARK-15394][ML][DOCS] User guide typos and grammar audit ## What changes were proposed in this pull request? Correct some typos and incorrectly worded sentences. ## How was this patch tested? Doc changes only. Note that many of these changes were identified by whomfire01 Author: sethah Closes #13180 from sethah/ml_guide_audit. (cherry picked from commit 5e203505f1a092e5849ebd01d9ff9e4fc6cdc34a) Signed-off-by: Xiangrui Meng --- docs/ml-classification-regression.md | 28 ++++++++--------- docs/ml-clustering.md | 2 +- docs/ml-collaborative-filtering.md | 6 ++-- docs/ml-features.md | 47 ++++++++++++++-------------- docs/ml-guide.md | 8 ++--- 5 files changed, 45 insertions(+), 46 deletions(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index f6a6937e29f0..f1a21f436f13 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -236,9 +236,9 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). MLPC consists of multiple layers of nodes. -Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes maps inputs to the outputs -by performing linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. -It can be written in matrix form for MLPC with `$K+1$` layers as follows: +Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes map inputs to outputs +by a linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. +This can be written in matrix form for MLPC with `$K+1$` layers as follows: `\[ \mathrm{y}(\x) = \mathrm{f_K}(...\mathrm{f_2}(\wv_2^T\mathrm{f_1}(\wv_1^T \x+b_1)+b_2)...+b_K) \]` @@ -252,7 +252,7 @@ Nodes in the output layer use softmax function: \]` The number of nodes `$N$` in the output layer corresponds to the number of classes. -MLPC employs backpropagation for learning the model. We use logistic loss function for optimization and L-BFGS as optimization routine. +MLPC employs backpropagation for learning the model. We use the logistic loss function for optimization and L-BFGS as an optimization routine. **Example** @@ -311,9 +311,9 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat ## Naive Bayes -[Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple +[Naive Bayes classifiers](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple probabilistic classifiers based on applying Bayes' theorem with strong (naive) independence -assumptions between the features. The spark.ml implementation currently supports both [multinomial +assumptions between the features. The `spark.ml` implementation currently supports both [multinomial naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). More information can be found in the section on [Naive Bayes in MLlib](mllib-naive-bayes.html#naive-bayes-sparkmllib). @@ -482,11 +482,11 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression. In `spark.ml`, we implement the [Accelerated failure time (AFT)](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) model which is a parametric survival regression model for censored data. -It describes a model for the log of survival time, so it's often called -log-linear model for survival analysis. Different from +It describes a model for the log of survival time, so it's often called a +log-linear model for survival analysis. Different from a [Proportional hazards](https://en.wikipedia.org/wiki/Proportional_hazards_model) model -designed for the same purpose, the AFT model is more easily to parallelize -because each instance contribute to the objective function independently. +designed for the same purpose, the AFT model is easier to parallelize +because each instance contributes to the objective function independently. Given the values of the covariates $x^{'}$, for random lifetime $t_{i}$ of subjects i = 1, ..., n, with possible right-censoring, @@ -501,10 +501,10 @@ assumes the form: \iota(\beta,\sigma)=\sum_{i=1}^{n}[-\delta_{i}\log\sigma+\delta_{i}\log{f_{0}}(\epsilon_{i})+(1-\delta_{i})\log{S_{0}(\epsilon_{i})}] \]` Where $S_{0}(\epsilon_{i})$ is the baseline survivor function, -and $f_{0}(\epsilon_{i})$ is corresponding density function. +and $f_{0}(\epsilon_{i})$ is the corresponding density function. The most commonly used AFT model is based on the Weibull distribution of the survival time. -The Weibull distribution for lifetime corresponding to extreme value distribution for +The Weibull distribution for lifetime corresponds to the extreme value distribution for the log of the lifetime, and the $S_{0}(\epsilon)$ function is: `\[ S_{0}(\epsilon_{i})=\exp(-e^{\epsilon_{i}}) @@ -513,7 +513,7 @@ the $f_{0}(\epsilon_{i})$ function is: `\[ f_{0}(\epsilon_{i})=e^{\epsilon_{i}}\exp(-e^{\epsilon_{i}}) \]` -The log-likelihood function for AFT model with Weibull distribution of lifetime is: +The log-likelihood function for AFT model with a Weibull distribution of lifetime is: `\[ \iota(\beta,\sigma)= -\sum_{i=1}^n[\delta_{i}\log\sigma-\delta_{i}\epsilon_{i}+e^{\epsilon_{i}}] \]` @@ -529,7 +529,7 @@ The gradient functions for $\beta$ and $\log\sigma$ respectively are: The AFT model can be formulated as a convex optimization problem, i.e. the task of finding a minimizer of a convex function $-\iota(\beta,\sigma)$ -that depends coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. +that depends on the coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. The optimization algorithm underlying the implementation is L-BFGS. The implementation matches the result from R's survival function [survreg](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html) diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 33e4b7b0d2cc..8656eb4001f4 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -89,7 +89,7 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. ## Latent Dirichlet allocation (LDA) `LDA` is implemented as an `Estimator` that supports both `EMLDAOptimizer` and `OnlineLDAOptimizer`, -and generates a `LDAModel` as the base models. Expert users may cast a `LDAModel` generated by +and generates a `LDAModel` as the base model. Expert users may cast a `LDAModel` generated by `EMLDAOptimizer` to a `DistributedLDAModel` if needed.
    diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md index 4514a358e12f..bd3d527d9a0e 100644 --- a/docs/ml-collaborative-filtering.md +++ b/docs/ml-collaborative-filtering.md @@ -60,7 +60,7 @@ best parameter learned from a sampled subset to the full dataset and expect simi
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are @@ -91,7 +91,7 @@ val als = new ALS()
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are @@ -122,7 +122,7 @@ ALS als = new ALS()
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are diff --git a/docs/ml-features.md b/docs/ml-features.md index c44ace91f23f..3db24a384059 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -26,7 +26,7 @@ to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and th Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, while document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. If we only use term frequency to measure the importance, it is very easy to over-emphasize terms that appear very -often but carry little information about the document, e.g., "a", "the", and "of". If a term appears +often but carry little information about the document, e.g. "a", "the", and "of". If a term appears very often across the corpus, it means it doesn't carry special information about a particular document. Inverse document frequency is a numerical measure of how much information a term provides: `\[ @@ -50,7 +50,7 @@ A raw feature is mapped into an index (term) by applying a hash function. Then t are calculated based on the mapped indices. This approach avoids the need to compute a global term-to-index map, which can be expensive for a large corpus, but it suffers from potential hash collisions, where different raw features may become the same term after hashing. To reduce the -chance of collision, we can increase the target feature dimension, i.e., the number of buckets +chance of collision, we can increase the target feature dimension, i.e. the number of buckets of the hash table. Since a simple modulo is used to transform the hash function to a column index, it is advisable to use a power of two as the feature dimension, otherwise the features will not be mapped evenly to the columns. The default feature dimension is `$2^{18} = 262,144$`. @@ -104,7 +104,7 @@ the [IDF Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.IDF) for mor `Word2Vec` is an `Estimator` which takes sequences of words representing documents and trains a `Word2VecModel`. The model maps each word to a unique fixed-size vector. The `Word2VecModel` transforms each document into a vector using the average of all words in the document; this vector -can then be used for as features for prediction, document similarity calculations, etc. +can then be used as features for prediction, document similarity calculations, etc. Please refer to the [MLlib user guide on Word2Vec](mllib-feature-extraction.html#word2vec) for more details. @@ -140,12 +140,12 @@ for more details on the API. `CountVectorizer` and `CountVectorizerModel` aim to help convert a collection of text documents to vectors of token counts. When an a-priori dictionary is not available, `CountVectorizer` can - be used as an `Estimator` to extract the vocabulary and generates a `CountVectorizerModel`. The + be used as an `Estimator` to extract the vocabulary, and generates a `CountVectorizerModel`. The model produces sparse representations for the documents over the vocabulary, which can then be passed to other algorithms like LDA. During the fitting process, `CountVectorizer` will select the top `vocabSize` words ordered by - term frequency across the corpus. An optional parameter "minDF" also affect the fitting process + term frequency across the corpus. An optional parameter "minDF" also affects the fitting process by specifying the minimum number (or fraction if < 1.0) of documents a term must appear in to be included in the vocabulary. @@ -161,8 +161,8 @@ Assume that we have the following DataFrame with columns `id` and `texts`: ~~~~ each row in `texts` is a document of type Array[String]. -Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c), -then the output column "vector" after transformation contains: +Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c). +Then the output column "vector" after transformation contains: ~~~~ id | texts | vector @@ -171,7 +171,7 @@ then the output column "vector" after transformation contains: 1 | Array("a", "b", "b", "c", "a") | (3,[0,1,2],[2.0,2.0,1.0]) ~~~~ -each vector represents the token counts of the document over the vocabulary. +Each vector represents the token counts of the document over the vocabulary.
    @@ -477,8 +477,7 @@ for more details on the API. ## StringIndexer `StringIndexer` encodes a string column of labels to a column of label indices. -The indices are in `[0, numLabels)`, ordered by label frequencies. -So the most frequent label gets index `0`. +The indices are in `[0, numLabels)`, ordered by label frequencies, so the most frequent label gets index `0`. If the input column is numeric, we cast it to string and index the string values. When downstream pipeline components such as `Estimator` or `Transformer` make use of this string-indexed label, you must set the input @@ -585,7 +584,7 @@ for more details on the API. ## IndexToString Symmetrically to `StringIndexer`, `IndexToString` maps a column of label indices -back to a column containing the original labels as strings. The common use case +back to a column containing the original labels as strings. A common use case is to produce indices from labels with `StringIndexer`, train a model with those indices and retrieve the original labels from the column of predicted indices with `IndexToString`. However, you are free to supply your own labels. @@ -652,7 +651,7 @@ for more details on the API. ## OneHotEncoder -[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features +[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features.
    @@ -888,7 +887,7 @@ for more details on the API. * `splits`: Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which also includes y. Splits should be strictly increasing. Values at -inf, inf must be explicitly provided to cover all Double values; Otherwise, values outside the splits specified will be treated as errors. Two examples of `splits` are `Array(Double.NegativeInfinity, 0.0, 1.0, Double.PositiveInfinity)` and `Array(0.0, 1.0, 2.0)`. -Note that if you have no idea of the upper bound and lower bound of the targeted column, you would better add the `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potential out of Bucketizer bounds exception. +Note that if you have no idea of the upper and lower bounds of the targeted column, you should add `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potential out of Bucketizer bounds exception. Note also that the splits that you provided have to be in strictly increasing order, i.e. `s0 < s1 < s2 < ... < sn`. @@ -976,7 +975,7 @@ for more details on the API. Currently we only support SQL syntax like `"SELECT ... FROM __THIS__ ..."` where `"__THIS__"` represents the underlying table of the input dataset. The select clause specifies the fields, constants, and expressions to display in -the output, it can be any select clause that Spark SQL supports. Users can also +the output, and can be any select clause that Spark SQL supports. Users can also use Spark SQL built-in function and UDFs to operate on these selected columns. For example, `SQLTransformer` supports statements like: @@ -1121,7 +1120,7 @@ Assume that we have a DataFrame with the columns `id`, `hour`: ~~~ `hour` is a continuous feature with `Double` type. We want to turn the continuous feature into -categorical one. Given `numBuckets = 3`, we should get the following DataFrame: +a categorical one. Given `numBuckets = 3`, we should get the following DataFrame: ~~~ id | hour | result @@ -1171,19 +1170,19 @@ for more details on the API. `VectorSlicer` is a transformer that takes a feature vector and outputs a new feature vector with a sub-array of the original features. It is useful for extracting features from a vector column. -`VectorSlicer` accepts a vector column with a specified indices, then outputs a new vector column +`VectorSlicer` accepts a vector column with specified indices, then outputs a new vector column whose values are selected via those indices. There are two types of indices, - 1. Integer indices that represents the indices into the vector, `setIndices()`; + 1. Integer indices that represent the indices into the vector, `setIndices()`. - 2. String indices that represents the names of features into the vector, `setNames()`. + 2. String indices that represent the names of features into the vector, `setNames()`. *This requires the vector column to have an `AttributeGroup` since the implementation matches on the name field of an `Attribute`.* Specification by integer and string are both acceptable. Moreover, you can use integer index and string name simultaneously. At least one feature must be selected. Duplicate features are not allowed, so there can be no overlap between selected indices and names. Note that if names of -features are selected, an exception will be threw out when encountering with empty input attributes. +features are selected, an exception will be thrown if empty input attributes are encountered. The output vector will order features with the selected indices first (in the order given), followed by the selected names (in the order given). @@ -1198,8 +1197,8 @@ Suppose that we have a DataFrame with the column `userFeatures`: [0.0, 10.0, 0.5] ~~~ -`userFeatures` is a vector column that contains three user features. Assuming that the first column -of `userFeatures` are all zeros, so we want to remove it and only the last two columns are selected. +`userFeatures` is a vector column that contains three user features. Assume that the first column +of `userFeatures` are all zeros, so we want to remove it and select only the last two columns. The `VectorSlicer` selects the last two elements with `setIndices(1, 2)` then produces a new vector column named `features`: @@ -1209,7 +1208,7 @@ column named `features`: [0.0, 10.0, 0.5] | [10.0, 0.5] ~~~ -Suppose also that we have a potential input attributes for the `userFeatures`, i.e. +Suppose also that we have potential input attributes for the `userFeatures`, i.e. `["f1", "f2", "f3"]`, then we can use `setNames("f2", "f3")` to select them. ~~~ @@ -1337,8 +1336,8 @@ id | features | clicked 9 | [1.0, 0.0, 15.0, 0.1] | 0.0 ~~~ -If we use `ChiSqSelector` with a `numTopFeatures = 1`, then according to our label `clicked` the -last column in our `features` chosen as the most useful feature: +If we use `ChiSqSelector` with `numTopFeatures = 1`, then according to our label `clicked` the +last column in our `features` is chosen as the most useful feature: ~~~ id | features | clicked | selectedFeatures diff --git a/docs/ml-guide.md b/docs/ml-guide.md index cc353df1ecd0..dae86d84804d 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -47,7 +47,7 @@ mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. * **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. -E.g., an ML model is a `Transformer` which transforms `DataFrame` with features into a `DataFrame` with predictions. +E.g., an ML model is a `Transformer` which transforms a `DataFrame` with features into a `DataFrame` with predictions. * **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. @@ -292,13 +292,13 @@ However, it is also a well-established method for choosing parameters which is m ## Example: model selection via train validation split In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. -`TrainValidationSplit` only evaluates each combination of parameters once as opposed to k times in - case of `CrossValidator`. It is therefore less expensive, +`TrainValidationSplit` only evaluates each combination of parameters once, as opposed to k times in + the case of `CrossValidator`. It is therefore less expensive, but will not produce as reliable results when the training dataset is not sufficiently large. `TrainValidationSplit` takes an `Estimator`, a set of `ParamMap`s provided in the `estimatorParamMaps` parameter, and an `Evaluator`. -It begins by splitting the dataset into two parts using `trainRatio` parameter +It begins by splitting the dataset into two parts using the `trainRatio` parameter which are used as separate training and test datasets. For example with `$trainRatio=0.75$` (default), `TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. Similar to `CrossValidator`, `TrainValidationSplit` also iterates through the set of `ParamMap`s. From 9963fd4398d7ef6c632fc9851ef64bd71a87aa12 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 23:35:20 -0700 Subject: [PATCH 310/313] [SPARK-15339][ML] ML 2.0 QA: Scala APIs and code audit for regression ## What changes were proposed in this pull request? * ```GeneralizedLinearRegression``` API docs enhancement. * The default value of ```GeneralizedLinearRegression``` ```linkPredictionCol``` is not set rather than empty. This will consistent with other similar params such as ```weightCol``` * Make some methods more private. * Fix a minor bug of LinearRegression. * Fix some other issues. ## How was this patch tested? Existing tests. Author: Yanbo Liang Closes #13129 from yanboliang/spark-15339. (cherry picked from commit c94b34ebbf4c6ce353c899c571beb34e8db98917) Signed-off-by: Xiangrui Meng --- .../ml/regression/AFTSurvivalRegression.scala | 4 +- .../GeneralizedLinearRegression.scala | 74 ++++++++++--------- .../ml/regression/IsotonicRegression.scala | 4 +- .../ml/regression/LinearRegression.scala | 8 +- .../ml/regression/LinearRegressionSuite.scala | 15 +++- 5 files changed, 58 insertions(+), 47 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index cc16c2f038be..e63eb7108060 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -89,8 +89,8 @@ private[regression] trait AFTSurvivalRegressionParams extends Params def getQuantilesCol: String = $(quantilesCol) /** Checks whether the input has quantiles column name. */ - protected[regression] def hasQuantilesCol: Boolean = { - isDefined(quantilesCol) && $(quantilesCol) != "" + private[regression] def hasQuantilesCol: Boolean = { + isDefined(quantilesCol) && $(quantilesCol).nonEmpty } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index e8474d035ee6..adbdd345e92e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -43,6 +43,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam with HasFitIntercept with HasMaxIter with HasTol with HasRegParam with HasWeightCol with HasSolver with Logging { + import GeneralizedLinearRegression._ + /** * Param for the name of family which is a description of the error distribution * to be used in the model. @@ -54,8 +56,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") final val family: Param[String] = new Param(this, "family", "The name of family which is a description of the error distribution to be used in the " + - "model. Supported options: gaussian(default), binomial, poisson and gamma.", - ParamValidators.inArray[String](GeneralizedLinearRegression.supportedFamilyNames.toArray)) + s"model. Supported options: ${supportedFamilyNames.mkString(", ")}.", + ParamValidators.inArray[String](supportedFamilyNames.toArray)) /** @group getParam */ @Since("2.0.0") @@ -71,9 +73,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") final val link: Param[String] = new Param(this, "link", "The name of link function " + "which provides the relationship between the linear predictor and the mean of the " + - "distribution function. Supported options: identity, log, inverse, logit, probit, " + - "cloglog and sqrt.", - ParamValidators.inArray[String](GeneralizedLinearRegression.supportedLinkNames.toArray)) + s"distribution function. Supported options: ${supportedLinkNames.mkString(", ")}", + ParamValidators.inArray[String](supportedLinkNames.toArray)) /** @group getParam */ @Since("2.0.0") @@ -81,19 +82,23 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam /** * Param for link prediction (linear predictor) column name. - * Default is empty, which means we do not output link prediction. + * Default is not set, which means we do not output link prediction. * * @group param */ @Since("2.0.0") final val linkPredictionCol: Param[String] = new Param[String](this, "linkPredictionCol", "link prediction (linear predictor) column name") - setDefault(linkPredictionCol, "") /** @group getParam */ @Since("2.0.0") def getLinkPredictionCol: String = $(linkPredictionCol) + /** Checks whether we should output link prediction. */ + private[regression] def hasLinkPredictionCol: Boolean = { + isDefined(linkPredictionCol) && $(linkPredictionCol).nonEmpty + } + import GeneralizedLinearRegression._ @Since("2.0.0") @@ -107,7 +112,7 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam s"with ${$(family)} family does not support ${$(link)} link function.") } val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) - if ($(linkPredictionCol).nonEmpty) { + if (hasLinkPredictionCol) { SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) } else { newSchema @@ -205,7 +210,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val /** * Sets the value of param [[weightCol]]. * If this is not set or empty, we treat all instance weights as 1.0. - * Default is empty, so all instances have weight one. + * Default is not set, so all instances have weight one. * * @group setParam */ @@ -214,7 +219,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val /** * Sets the solver algorithm used for optimization. - * Currently only support "irls" which is also the default solver. + * Currently only supports "irls" which is also the default solver. * * @group setParam */ @@ -239,10 +244,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val } val familyAndLink = new FamilyAndLink(familyObj, linkObj) - val numFeatures = dataset.select(col($(featuresCol))).limit(1).rdd - .map { case Row(features: Vector) => - features.size - }.first() + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size if (numFeatures > WeightedLeastSquares.MAX_NUM_FEATURES) { val msg = "Currently, GeneralizedLinearRegression only supports number of features" + s" <= ${WeightedLeastSquares.MAX_NUM_FEATURES}. Found $numFeatures in the input dataset." @@ -294,7 +296,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def load(path: String): GeneralizedLinearRegression = super.load(path) /** Set of family and link pairs that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedFamilyAndLinkPairs = Set( + private[regression] lazy val supportedFamilyAndLinkPairs = Set( Gaussian -> Identity, Gaussian -> Log, Gaussian -> Inverse, Binomial -> Logit, Binomial -> Probit, Binomial -> CLogLog, Poisson -> Log, Poisson -> Identity, Poisson -> Sqrt, @@ -302,17 +304,17 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine ) /** Set of family names that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name) + private[regression] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name) /** Set of link names that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedLinkNames = supportedFamilyAndLinkPairs.map(_._2.name) + private[regression] lazy val supportedLinkNames = supportedFamilyAndLinkPairs.map(_._2.name) - private[ml] val epsilon: Double = 1E-16 + private[regression] val epsilon: Double = 1E-16 /** * Wrapper of family and link combination used in the model. */ - private[ml] class FamilyAndLink(val family: Family, val link: Link) extends Serializable { + private[regression] class FamilyAndLink(val family: Family, val link: Link) extends Serializable { /** Linear predictor based on given mu. */ def predict(mu: Double): Double = link.link(family.project(mu)) @@ -359,7 +361,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * * @param name the name of the family. */ - private[ml] abstract class Family(val name: String) extends Serializable { + private[regression] abstract class Family(val name: String) extends Serializable { /** The default link instance of this family. */ val defaultLink: Link @@ -391,7 +393,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine def project(mu: Double): Double = mu } - private[ml] object Family { + private[regression] object Family { /** * Gets the [[Family]] object from its name. @@ -412,7 +414,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Gaussian exponential family distribution. * The default link for the Gaussian family is the identity link. */ - private[ml] object Gaussian extends Family("gaussian") { + private[regression] object Gaussian extends Family("gaussian") { val defaultLink: Link = Identity @@ -448,7 +450,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Binomial exponential family distribution. * The default link for the Binomial family is the logit link. */ - private[ml] object Binomial extends Family("binomial") { + private[regression] object Binomial extends Family("binomial") { val defaultLink: Link = Logit @@ -492,7 +494,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Poisson exponential family distribution. * The default link for the Poisson family is the log link. */ - private[ml] object Poisson extends Family("poisson") { + private[regression] object Poisson extends Family("poisson") { val defaultLink: Link = Log @@ -533,7 +535,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Gamma exponential family distribution. * The default link for the Gamma family is the inverse link. */ - private[ml] object Gamma extends Family("gamma") { + private[regression] object Gamma extends Family("gamma") { val defaultLink: Link = Inverse @@ -578,7 +580,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * * @param name the name of link function. */ - private[ml] abstract class Link(val name: String) extends Serializable { + private[regression] abstract class Link(val name: String) extends Serializable { /** The link function. */ def link(mu: Double): Double @@ -590,7 +592,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine def unlink(eta: Double): Double } - private[ml] object Link { + private[regression] object Link { /** * Gets the [[Link]] object from its name. @@ -611,7 +613,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine } } - private[ml] object Identity extends Link("identity") { + private[regression] object Identity extends Link("identity") { override def link(mu: Double): Double = mu @@ -620,7 +622,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = eta } - private[ml] object Logit extends Link("logit") { + private[regression] object Logit extends Link("logit") { override def link(mu: Double): Double = math.log(mu / (1.0 - mu)) @@ -629,7 +631,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 / (1.0 + math.exp(-1.0 * eta)) } - private[ml] object Log extends Link("log") { + private[regression] object Log extends Link("log") { override def link(mu: Double): Double = math.log(mu) @@ -638,7 +640,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = math.exp(eta) } - private[ml] object Inverse extends Link("inverse") { + private[regression] object Inverse extends Link("inverse") { override def link(mu: Double): Double = 1.0 / mu @@ -647,7 +649,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 / eta } - private[ml] object Probit extends Link("probit") { + private[regression] object Probit extends Link("probit") { override def link(mu: Double): Double = dist.Gaussian(0.0, 1.0).icdf(mu) @@ -658,7 +660,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = dist.Gaussian(0.0, 1.0).cdf(eta) } - private[ml] object CLogLog extends Link("cloglog") { + private[regression] object CLogLog extends Link("cloglog") { override def link(mu: Double): Double = math.log(-1.0 * math.log(1 - mu)) @@ -667,7 +669,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 - math.exp(-1.0 * math.exp(eta)) } - private[ml] object Sqrt extends Link("sqrt") { + private[regression] object Sqrt extends Link("sqrt") { override def link(mu: Double): Double = math.sqrt(mu) @@ -732,7 +734,7 @@ class GeneralizedLinearRegressionModel private[ml] ( if ($(predictionCol).nonEmpty) { output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) } - if ($(linkPredictionCol).nonEmpty) { + if (hasLinkPredictionCol) { output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)))) } output.toDF() @@ -860,7 +862,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") val predictionCol: String = { - if (origModel.isDefined(origModel.predictionCol) && origModel.getPredictionCol != "") { + if (origModel.isDefined(origModel.predictionCol) && origModel.getPredictionCol.nonEmpty) { origModel.getPredictionCol } else { "prediction_" + java.util.UUID.randomUUID.toString diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index ba0f59e89bb2..d16e8e3f6b25 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -69,8 +69,8 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures setDefault(isotonic -> true, featureIndex -> 0) /** Checks whether the input has weight column. */ - protected[ml] def hasWeightCol: Boolean = { - isDefined(weightCol) && $(weightCol) != "" + private[regression] def hasWeightCol: Boolean = { + isDefined(weightCol) && $(weightCol).nonEmpty } /** 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 a702f02c9143..ff1038cbf185 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 @@ -161,9 +161,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String override protected def train(dataset: Dataset[_]): LinearRegressionModel = { // Extract the number of features before deciding optimization solver. - val numFeatures = dataset.select(col($(featuresCol))).limit(1).rdd.map { - case Row(features: Vector) => features.size - }.first() + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) if (($(solver) == "auto" && $(elasticNetParam) == 0.0 && @@ -242,7 +240,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val coefficients = Vectors.sparse(numFeatures, Seq()) val intercept = yMean - val model = new LinearRegressionModel(uid, coefficients, intercept) + val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) // Handle possible missing or invalid prediction columns val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() @@ -254,7 +252,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String model, Array(0D), Array(0D)) - return copyValues(model.setSummary(trainingSummary)) + return model.setSummary(trainingSummary) } else { require($(regParam) == 0.0, "The standard deviation of the label is zero. " + "Model cannot be regularized.") diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index 332d331a4773..265f2f45c45f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -610,20 +610,31 @@ class LinearRegressionSuite val model1 = new LinearRegression() .setFitIntercept(fitIntercept) .setWeightCol("weight") + .setPredictionCol("myPrediction") .setSolver(solver) .fit(datasetWithWeightConstantLabel) val actual1 = Vectors.dense(model1.intercept, model1.coefficients(0), model1.coefficients(1)) assert(actual1 ~== expected(idx) absTol 1e-4) + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightConstantLabel.schema.fieldNames.toSet + model1.getPredictionCol) + .subsetOf(model1.summary.predictions.schema.fieldNames.toSet)) + val model2 = new LinearRegression() .setFitIntercept(fitIntercept) .setWeightCol("weight") + .setPredictionCol("myPrediction") .setSolver(solver) .fit(datasetWithWeightZeroLabel) val actual2 = Vectors.dense(model2.intercept, model2.coefficients(0), model2.coefficients(1)) assert(actual2 ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1e-4) + + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightZeroLabel.schema.fieldNames.toSet + model2.getPredictionCol) + .subsetOf(model2.summary.predictions.schema.fieldNames.toSet)) + idx += 1 } } @@ -672,7 +683,7 @@ class LinearRegressionSuite test("linear regression model training summary") { Seq("auto", "l-bfgs", "normal").foreach { solver => - val trainer = new LinearRegression().setSolver(solver) + val trainer = new LinearRegression().setSolver(solver).setPredictionCol("myPrediction") val model = trainer.fit(datasetWithDenseFeature) val trainerNoPredictionCol = trainer.setPredictionCol("") val modelNoPredictionCol = trainerNoPredictionCol.fit(datasetWithDenseFeature) @@ -682,7 +693,7 @@ class LinearRegressionSuite assert(modelNoPredictionCol.hasSummary) // Schema should be a superset of the input dataset - assert((datasetWithDenseFeature.schema.fieldNames.toSet + "prediction").subsetOf( + assert((datasetWithDenseFeature.schema.fieldNames.toSet + model.getPredictionCol).subsetOf( model.summary.predictions.schema.fieldNames.toSet)) // Validate that we re-insert a prediction column for evaluation val modelNoPredictionColFieldNames From 4d13348f861fd391c64433a1691c1b7f33a36db1 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 20 May 2016 14:38:25 +0800 Subject: [PATCH 311/313] [SPARK-15367][SQL] Add refreshTable back #### What changes were proposed in this pull request? `refreshTable` was a method in `HiveContext`. It was deleted accidentally while we were migrating the APIs. This PR is to add it back to `HiveContext`. In addition, in `SparkSession`, we put it under the catalog namespace (`SparkSession.catalog.refreshTable`). #### How was this patch tested? Changed the existing test cases to use the function `refreshTable`. Also added a test case for refreshTable in `hivecontext-compatibility` Author: gatorsmile Closes #13156 from gatorsmile/refreshTable. (cherry picked from commit 39fd469078271aa12f3163606000e06e382d35dc) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalog/Catalog.scala | 13 +++++++++ .../spark/sql/execution/datasources/ddl.scala | 21 +++------------ .../spark/sql/internal/CatalogImpl.scala | 27 +++++++++++++++++++ .../spark/sql/internal/SessionState.scala | 4 --- .../sql/hive/MetastoreDataSourcesSuite.scala | 4 +-- .../spark/sql/hive/MultiDatabaseSuite.scala | 4 +-- .../apache/spark/sql/hive/HiveContext.scala | 12 +++++++++ 7 files changed, 59 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 49c07427617f..a99bc3bff6ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -211,4 +211,17 @@ abstract class Catalog { */ def clearCache(): Unit + /** + * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, + * Spark SQL or the external data source library it uses might cache certain metadata about a + * table, such as the location of blocks. When those change outside of Spark SQL, users should + * call this function to invalidate the cache. + * + * If this table is cached as an InMemoryRelation, drop the original cached version and make the + * new version cached lazily. + * + * @since 2.0.0 + */ + def refreshTable(tableName: String): Unit + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 68238dbb46e9..78b1db16826e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -126,24 +126,9 @@ case class RefreshTable(tableIdent: TableIdentifier) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - // Refresh the given table's metadata first. - sparkSession.sessionState.catalog.refreshTable(tableIdent) - - // If this table is cached as a InMemoryColumnarRelation, drop the original - // cached version and make the new version cached lazily. - val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent) - // Use lookupCachedData directly since RefreshTable also takes databaseName. - val isCached = sparkSession.cacheManager.lookupCachedData(logicalPlan).nonEmpty - if (isCached) { - // Create a data frame to represent the table. - // TODO: Use uncacheTable once it supports database name. - val df = Dataset.ofRows(sparkSession, logicalPlan) - // Uncache the logicalPlan. - sparkSession.cacheManager.tryUncacheQuery(df, blocking = true) - // Cache it again. - sparkSession.cacheManager.cacheQuery(df, Some(tableIdent.table)) - } - + // Refresh the given table's metadata. If this table is cached as an InMemoryRelation, + // drop the original cached version and make the new version cached lazily. + sparkSession.catalog.refreshTable(tableIdent.quotedString) Seq.empty[Row] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 473e827f4db1..1371abe189f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -345,6 +345,33 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { sparkSession.cacheManager.lookupCachedData(qName).nonEmpty } + /** + * Refresh the cache entry for a table, if any. For Hive metastore table, the metadata + * is refreshed. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def refreshTable(tableName: String): Unit = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + sessionCatalog.refreshTable(tableIdent) + + // If this table is cached as a InMemoryRelation, drop the original + // cached version and make the new version cached lazily. + val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent) + // Use lookupCachedData directly since RefreshTable also takes databaseName. + val isCached = sparkSession.cacheManager.lookupCachedData(logicalPlan).nonEmpty + if (isCached) { + // Create a data frame to represent the table. + // TODO: Use uncacheTable once it supports database name. + val df = Dataset.ofRows(sparkSession, logicalPlan) + // Uncache the logicalPlan. + sparkSession.cacheManager.tryUncacheQuery(df, blocking = true) + // Cache it again. + sparkSession.cacheManager.cacheQuery(df, Some(tableIdent.table)) + } + } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index f0b8a83dee8c..8f7c6f5d0ca4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -162,10 +162,6 @@ private[sql] class SessionState(sparkSession: SparkSession) { def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan) - def refreshTable(tableName: String): Unit = { - catalog.refreshTable(sqlParser.parseTableIdentifier(tableName)) - } - def invalidateTable(tableName: String): Unit = { catalog.invalidateTable(sqlParser.parseTableIdentifier(tableName)) } 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 00adb9a44b14..686c63065dfc 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 @@ -622,7 +622,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("arrayInParquet") - sessionState.refreshTable("arrayInParquet") + sparkSession.catalog.refreshTable("arrayInParquet") checkAnswer( sql("SELECT a FROM arrayInParquet"), @@ -681,7 +681,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("mapInParquet") - sessionState.refreshTable("mapInParquet") + sparkSession.catalog.refreshTable("mapInParquet") checkAnswer( sql("SELECT a FROM mapInParquet"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 622b043581c5..5b706b043241 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -217,7 +217,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql("ALTER TABLE t ADD PARTITION (p=2)") - hiveContext.sessionState.refreshTable("t") + spark.catalog.refreshTable("t") checkAnswer( spark.table("t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) @@ -249,7 +249,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql(s"ALTER TABLE $db.t ADD PARTITION (p=2)") - hiveContext.sessionState.refreshTable(s"$db.t") + spark.catalog.refreshTable(s"$db.t") checkAnswer( spark.table(s"$db.t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) diff --git a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 75166f6beaa8..415d4c0049d4 100644 --- a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -58,4 +58,16 @@ class HiveContext private[hive]( sparkSession.sharedState.asInstanceOf[HiveSharedState] } + /** + * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, + * Spark SQL or the external data source library it uses might cache certain metadata about a + * table, such as the location of blocks. When those change outside of Spark SQL, users should + * call this function to invalidate the cache. + * + * @since 1.3.0 + */ + def refreshTable(tableName: String): Unit = { + sparkSession.catalog.refreshTable(tableName) + } + } From 4e25d6e8ce9ce88a58fc0ea0e00cc7b68370a62d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 23:43:01 -0700 Subject: [PATCH 312/313] [SPARK-15421][SQL] Validate DDL property values ## What changes were proposed in this pull request? When we parse DDLs involving table or database properties, we need to validate the values. E.g. if we alter a database's property without providing a value: ``` ALTER DATABASE my_db SET DBPROPERTIES('some_key') ``` Then we'll ignore it with Hive, but override the property with the in-memory catalog. Inconsistencies like these arise because we don't validate the property values. In such cases, we should throw exceptions instead. ## How was this patch tested? `DDLCommandSuite` Author: Andrew Or Closes #13205 from andrewor14/ddl-prop-values. (cherry picked from commit 257375019266ab9e3c320e33026318cc31f58ada) Signed-off-by: Andrew Or --- .../spark/sql/execution/SparkSqlParser.scala | 45 +++++++++++++++---- .../execution/command/DDLCommandSuite.scala | 41 +++++++++++++++++ 2 files changed, 77 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8af6d07719c9..ee12bfa7251f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -293,7 +293,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (external) { throw operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } - val options = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) + val options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val provider = ctx.tableProvider.qualifiedName.getText val partitionColumnNames = Option(ctx.partitionColumnNames) @@ -371,6 +371,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { /** * Convert a table property list into a key-value map. + * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. */ override def visitTablePropertyList( ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { @@ -381,6 +382,32 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { }.toMap } + /** + * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified. + */ + private def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v == null }.keys + if (badKeys.nonEmpty) { + throw operationNotAllowed( + s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props + } + + /** + * Parse a list of keys from a [[TablePropertyListContext]], assuming no values are specified. + */ + private def visitPropertyKeys(ctx: TablePropertyListContext): Seq[String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v != null }.keys + if (badKeys.nonEmpty) { + throw operationNotAllowed( + s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props.keys.toSeq + } + /** * A table property key can either be String or a collection of dot separated elements. This * function extracts the property key based on whether its a string literal or a table property @@ -409,7 +436,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.EXISTS != null, Option(ctx.locationSpec).map(visitLocationSpec), Option(ctx.comment).map(string), - Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty)) + Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } /** @@ -424,7 +451,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterDatabaseProperties( ctx.identifier.getText, - visitTablePropertyList(ctx.tablePropertyList)) + visitPropertyKeyValues(ctx.tablePropertyList)) } /** @@ -540,7 +567,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterTableSetProperties( visitTableIdentifier(ctx.tableIdentifier), - visitTablePropertyList(ctx.tablePropertyList), + visitPropertyKeyValues(ctx.tablePropertyList), ctx.VIEW != null) } @@ -557,7 +584,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterTableUnsetProperties( visitTableIdentifier(ctx.tableIdentifier), - visitTablePropertyList(ctx.tablePropertyList).keys.toSeq, + visitPropertyKeys(ctx.tablePropertyList), ctx.EXISTS != null, ctx.VIEW != null) } @@ -575,7 +602,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { AlterTableSerDeProperties( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.STRING).map(string), - Option(ctx.tablePropertyList).map(visitTablePropertyList), + Option(ctx.tablePropertyList).map(visitPropertyKeyValues), // TODO a partition spec is allowed to have optional values. This is currently violated. Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } @@ -783,7 +810,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val comment = Option(ctx.STRING).map(string) val partitionCols = Option(ctx.partitionColumns).toSeq.flatMap(visitCatalogColumns) val cols = Option(ctx.columns).toSeq.flatMap(visitCatalogColumns) - val properties = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) + val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val selectQuery = Option(ctx.query).map(plan) // Note: Hive requires partition columns to be distinct from the schema, so we need @@ -944,7 +971,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { import ctx._ EmptyStorageFormat.copy( serde = Option(string(name)), - serdeProperties = Option(tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty)) + serdeProperties = Option(tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } /** @@ -1001,7 +1028,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { comment = Option(ctx.STRING).map(string), schema, ctx.query, - Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty), + Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty), ctx.EXISTS != null, ctx.REPLACE != null, ctx.TEMPORARY != null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 897170ea57cf..0925a51310e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -57,6 +57,12 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed, expected) } + test("create database - property values must be set") { + assertUnsupported( + sql = "CREATE DATABASE my_db WITH DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("drop database") { val sql1 = "DROP DATABASE IF EXISTS database_name RESTRICT" val sql2 = "DROP DATABASE IF EXISTS database_name CASCADE" @@ -121,6 +127,12 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed2, expected2) } + test("alter database - property values must be set") { + assertUnsupported( + sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("describe database") { // DESCRIBE DATABASE [EXTENDED] db_name; val sql1 = "DESCRIBE DATABASE EXTENDED db_name" @@ -228,6 +240,16 @@ class DDLCommandSuite extends PlanTest { } } + test("create table - property values must be set") { + assertUnsupported( + sql = "CREATE TABLE my_tab TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + assertUnsupported( + sql = "CREATE TABLE my_tab ROW FORMAT SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("create table - location implies external") { val query = "CREATE TABLE my_tab LOCATION '/something/anything'" parser.parsePlan(query) match { @@ -349,6 +371,18 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed3_view, expected3_view) } + test("alter table - property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("alter table unset properties - property values must NOT be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_with_value")) + } + test("alter table: SerDe properties") { val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" val sql2 = @@ -404,6 +438,13 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed5, expected5) } + test("alter table - SerDe property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + // ALTER TABLE table_name ADD [IF NOT EXISTS] PARTITION partition_spec // [LOCATION 'location1'] partition_spec [LOCATION 'location2'] ...; test("alter table: add partition") { From 53c09f065fac9cabe479cd1f205810230eda110d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 23:44:10 -0700 Subject: [PATCH 313/313] [SPARK-15417][SQL][PYTHON] PySpark shell always uses in-memory catalog ## What changes were proposed in this pull request? There is no way to use the Hive catalog in `pyspark-shell`. This is because we used to create a `SparkContext` before calling `SparkSession.enableHiveSupport().getOrCreate()`, which just gets the existing `SparkContext` instead of creating a new one. As a result, `spark.sql.catalogImplementation` was never propagated. ## How was this patch tested? Manual. Author: Andrew Or Closes #13203 from andrewor14/fix-pyspark-shell. (cherry picked from commit c32b1b162e7e5ecc5c823f79ba9f23cbd1407dbf) Signed-off-by: Andrew Or --- python/pyspark/shell.py | 8 +++++--- python/pyspark/sql/session.py | 6 ++++++ 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index adaa3b5a7993..ef46d3065e5e 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,12 +35,11 @@ if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) -sc = SparkContext() -atexit.register(lambda: sc.stop()) +SparkContext._ensure_initialized() try: # Try to access HiveConf, it will raise exception if Hive is not added - sc._jvm.org.apache.hadoop.hive.conf.HiveConf() + SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf() spark = SparkSession.builder\ .enableHiveSupport()\ .getOrCreate() @@ -49,6 +48,9 @@ except TypeError: spark = SparkSession(sc) +sc = spark.sparkContext +atexit.register(lambda: sc.stop()) + # for compatibility sqlContext = spark._wrapped sqlCtx = sqlContext diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 0e04b88265fa..241947537fa2 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -202,6 +202,12 @@ def newSession(self): """ return self.__class__(self._sc, self._jsparkSession.newSession()) + @property + @since(2.0) + def sparkContext(self): + """Returns the underlying :class:`SparkContext`.""" + return self._sc + @property @since(2.0) def conf(self):