From 33814f887aea339c99e14ce7f14ca6fcc6875015 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 18:46:57 +0800 Subject: [PATCH 001/143] [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. --- .../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 5cee2b9af68c2..644e5d65d612c 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 b516297115f04..8d332df029166 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 2f9047b5eb969e0198b8a73e392642ca852ba786 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 18 May 2016 11:48:46 +0100 Subject: [PATCH 002/143] [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. --- .../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 8cb0a295b0773..58664e77d24a5 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 1bcd85e1d533f..acbcb0c4b7b78 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 8d4174124b5c4..e79b1f31643d0 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 60f13d27d0a6e..38728f2693c6c 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 9e6bc7193c13b..141249a427a4c 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 ff022b2dc45ee..a634502e2e4f0 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 3b9feae4a31c9..b02b714168cb0 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 d2f81df1ba67f5d3fc39719d9a67c52d7903e59e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 May 2016 11:51:50 +0100 Subject: [PATCH 003/143] [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. --- .../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 af7532e0c03ec..be9f03d4baaaf 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 c4a45fd855252f0f7130cdcc08486591fadab7df Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 18 May 2016 18:58:24 +0800 Subject: [PATCH 004/143] [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. --- .../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 57e8db7e88fb5..d46c4e7b2b50b 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 420b700695fe8bcdda406c34ad48230b9dfc07f1 Mon Sep 17 00:00:00 2001 From: DLucky Date: Wed, 18 May 2016 12:05:21 +0100 Subject: [PATCH 005/143] [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. --- .../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 adf20dc4b8b16..53587670a5db0 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 c1fd9cacba8e602cc55a893f0beb05ea48c3a1f6 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Wed, 18 May 2016 12:10:32 +0100 Subject: [PATCH 006/143] [SPARK-15263][CORE] Make shuffle service dir cleanup faster by using `rm -rf` ## What changes were proposed in this pull request? Jira: https://issues.apache.org/jira/browse/SPARK-15263 The current logic for directory cleanup is slow because it does directory listing, recurses over child directories, checks for symbolic links, deletes leaf files and finally deletes the dirs when they are empty. There is back-and-forth switching from kernel space to user space while doing this. Since most of the deployment backends would be Unix systems, we could essentially just do `rm -rf` so that entire deletion logic runs in kernel space. The current Java based impl in Spark seems to be similar to what standard libraries like guava and commons IO do (eg. http://svn.apache.org/viewvc/commons/proper/io/trunk/src/main/java/org/apache/commons/io/FileUtils.java?view=markup#l1540). However, guava removed this method in favour of shelling out to an operating system command (like in this PR). See the `Deprecated` note in older javadocs for guava for details : http://google.github.io/guava/releases/10.0.1/api/docs/com/google/common/io/Files.html#deleteRecursively(java.io.File) Ideally, Java should be providing such APIs so that users won't have to do such things to get platform specific code. Also, its not just about speed, but also handling race conditions while doing at FS deletions is tricky. I could find this bug for Java in similar context : http://bugs.java.com/bugdatabase/view_bug.do?bug_id=7148952 ## How was this patch tested? I am relying on existing test cases to test the method. If there are suggestions about testing it, welcome to hear about it. ## Performance gains *Input setup* : Created a nested directory structure of depth 3 and each entry having 50 sub-dirs. The input being cleaned up had total ~125k dirs. Ran both approaches (in isolation) for 6 times to get average numbers: Native Java cleanup | `rm -rf` as a separate process ------------ | ------------- 10.04 sec | 4.11 sec This change made deletion 2.4 times faster for the given test input. Author: Tejas Patil Closes #13042 from tejasapatil/delete_recursive. --- common/network-common/pom.xml | 4 ++ .../apache/spark/network/util/JavaUtils.java | 49 ++++++++++++++++++- .../shuffle/TestShuffleDataContext.java | 24 ++++----- 3 files changed, 61 insertions(+), 16 deletions(-) diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5444ae6d70a92..12d89273d738d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -41,6 +41,10 @@ io.netty netty-all + + org.apache.commons + commons-lang3 + diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java index fbed2f053dc6c..eb534eed24278 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -29,6 +29,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.SystemUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -79,14 +80,32 @@ public static String bytesToString(ByteBuffer b) { return Unpooled.wrappedBuffer(b).toString(StandardCharsets.UTF_8); } - /* + /** * Delete a file or directory and its contents recursively. * Don't follow directories if they are symlinks. - * Throws an exception if deletion is unsuccessful. + * + * @param file Input file / dir to be deleted + * @throws IOException if deletion is unsuccessful */ public static void deleteRecursively(File file) throws IOException { if (file == null) { return; } + // On Unix systems, use operating system command to run faster + // If that does not work out, fallback to the Java IO way + if (SystemUtils.IS_OS_UNIX) { + try { + deleteRecursivelyUsingUnixNative(file); + return; + } catch (IOException e) { + logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " + + "Falling back to Java IO way", file.getAbsolutePath(), e); + } + } + + deleteRecursivelyUsingJavaIO(file); + } + + private static void deleteRecursivelyUsingJavaIO(File file) throws IOException { if (file.isDirectory() && !isSymlink(file)) { IOException savedIOException = null; for (File child : listFilesSafely(file)) { @@ -109,6 +128,32 @@ public static void deleteRecursively(File file) throws IOException { } } + private static void deleteRecursivelyUsingUnixNative(File file) throws IOException { + ProcessBuilder builder = new ProcessBuilder("rm", "-rf", file.getAbsolutePath()); + Process process = null; + int exitCode = -1; + + try { + // In order to avoid deadlocks, consume the stdout (and stderr) of the process + builder.redirectErrorStream(true); + builder.redirectOutput(new File("/dev/null")); + + process = builder.start(); + + exitCode = process.waitFor(); + } catch (Exception e) { + throw new IOException("Failed to delete: " + file.getAbsolutePath(), e); + } finally { + if (process != null && process.isAlive()) { + process.destroy(); + } + } + + if (exitCode != 0 || file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + private static File[] listFilesSafely(File file) throws IOException { if (file.exists()) { File[] files = file.listFiles(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 62a1fb42b023d..81e01949e50fa 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -27,12 +27,17 @@ import com.google.common.io.Files; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.util.JavaUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Manages some sort-shuffle data, including the creation * and cleanup of directories that can be read by the {@link ExternalShuffleBlockResolver}. */ public class TestShuffleDataContext { + private static final Logger logger = LoggerFactory.getLogger(TestShuffleDataContext.class); + public final String[] localDirs; public final int subDirsPerLocalDir; @@ -53,7 +58,11 @@ public void create() { public void cleanup() { for (String localDir : localDirs) { - deleteRecursively(new File(localDir)); + try { + JavaUtils.deleteRecursively(new File(localDir)); + } catch (IOException e) { + logger.warn("Unable to cleanup localDir = " + localDir, e); + } } } @@ -92,17 +101,4 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr public ExecutorShuffleInfo createExecutorInfo(String shuffleManager) { return new ExecutorShuffleInfo(localDirs, subDirsPerLocalDir, shuffleManager); } - - private static void deleteRecursively(File f) { - assert f != null; - if (f.isDirectory()) { - File[] children = f.listFiles(); - if (children != null) { - for (File child : children) { - deleteRecursively(child); - } - } - } - f.delete(); - } } From 8fb1d1c7f3ed1b62625052a532b7388ebec71bbf Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 09:44:21 -0700 Subject: [PATCH 007/143] [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. --- .../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 840f13b39464c..38a21a896e1fe 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 a05a79c88df76..a4a571f15a8c0 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 2be5a16b2d1e4..014aef86b5cc6 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 6807710f9fef1..6c00608302c4e 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 7dc0508784981..e14a23f4a6a83 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 b695aecc13ea1..9a017f29f7d21 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 bdcbd22fd814b..8183f825592c0 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 127789b632b44..ad755529dec64 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 e6e16fff80401..db91329c94cb6 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 fe5abc5c23049..354efe18dbde7 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 278a827644db7..694352ee2af44 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 4a2f65a0ed2b2..383c5b3b0884a 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 3def8b0b1850e..333c23bdaf6d6 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 2c4928ab907ad..38bf7e5e5aec3 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 45f7297048382..4e99a0965780b 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 315ef6a8796f0..cb41457b6653f 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 3d1e67f903ab3512fcad82b94b1825578f8117c9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 18 May 2016 11:18:33 -0700 Subject: [PATCH 008/143] [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. --- 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 e86f44281d492..1790432edd5dc 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 30ab130f29481..7d8d0230b4ab3 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 e8b79afa024123f9d4ceaf0a1043a7e37d913a8d Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 18 May 2016 21:13:12 +0200 Subject: [PATCH 009/143] [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. --- .../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 509c944fed74c..f257382d2205c 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 bbfc415cbb9b7..59b5edc4013e8 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 6aae625fc83f2..80b976914cbdf 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 d7cb65846574b..86c00d91652d1 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 4768d037b77bcec1432a8fcc075a359619a2241d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 18 May 2016 14:22:21 -0700 Subject: [PATCH 010/143] [SPARK-15386][CORE] Master doesn't compile against Java 1.7 / Process.isAlive ## What changes were proposed in this pull request? Remove call to Process.isAlive -- Java 8 only. Introduced in https://github.com/apache/spark/pull/13042 / SPARK-15263 ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #13174 from srowen/SPARK-15386. --- .../src/main/java/org/apache/spark/network/util/JavaUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java index eb534eed24278..f3eaf22c0166e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -144,7 +144,7 @@ private static void deleteRecursivelyUsingUnixNative(File file) throws IOExcepti } catch (Exception e) { throw new IOException("Failed to delete: " + file.getAbsolutePath(), e); } finally { - if (process != null && process.isAlive()) { + if (process != null) { process.destroy(); } } From cc6a47dd81fb2fcf2d2ec158c1537bcc27ed1674 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 May 2016 23:19:55 +0100 Subject: [PATCH 011/143] [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. --- .../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 f1beca29c2aa5..9ab5684d901f0 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 a390c40d67574..40d182cfde231 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 2b3b1d60463f7..92b8ed75d85fc 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 22ee13b6e78c1..99f2bd8bc1f22 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 5d1928ac6b2ca..6ddabfd8ef089 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 fc29b896dae08b957ed15fa681b46162600a4050 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 15:45:59 -0700 Subject: [PATCH 012/143] [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. --- .../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 7933d12e284f1..a7f461381b505 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 f573abf859610..df029e44c54e5 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 84b23453ddb0a97e3d81306de0a5dcb64f88bdd0 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 16:02:52 -0700 Subject: [PATCH 013/143] Revert "[SPARK-15392][SQL] fix default value of size estimation of logical plan" This reverts commit fc29b896dae08b957ed15fa681b46162600a4050. --- .../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 a7f461381b505..7933d12e284f1 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,7 +605,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 df029e44c54e5..f573abf859610 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 32be51fba45f5e07a2a3520293c12dc7765a364d Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Wed, 18 May 2016 16:15:09 -0700 Subject: [PATCH 014/143] [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. --- .../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 011aff4ff6c28..e33fd831ab471 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 f22c0241d930e..f091615a9a714 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 0000000000000..e2719428bb28e --- /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 0000000000000..b8c03daa8c199 --- /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 f61fce5d4102d..b5e51e963f1b6 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 ebfe3a1f2c77e6869c3c36ba67afb7fabe6a94f5 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 May 2016 18:06:38 -0700 Subject: [PATCH 015/143] [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. --- .../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 9166faa54de56..28e4966f918a0 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 cb9a62dfd4e81..c0fa220d34bb6 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 a5f39aaa2314b..71b39c54fa0c2 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 99f156a935b50..a38f1ec09156d 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 7df6e06805a54..fc38369f38c33 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 da575c74f825f..629243bd1a97b 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 ffb606f2c66de..486a440b6f9a8 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} @@ -108,6 +109,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 } } @@ -118,7 +121,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 b02b714168cb0..1935e41185ac0 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 45a9c9dc47815..51538eca644f6 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 9c2a376e413b0701097b0784bd725e4ca87cd837 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 19 May 2016 10:05:53 +0800 Subject: [PATCH 016/143] [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. --- .../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 2409b5d203f40..282f26ce998fe 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 7933d12e284f1..518430f16d712 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 @@ -752,7 +752,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 7ead97bbf6937..81bc973be74a3 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 4987f39ac7a694e1c8b8b82246eb4fbd863201c4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 18 May 2016 19:16:28 -0700 Subject: [PATCH 017/143] [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. --- 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 3824e0a99557c..6b7a341bee889 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 8e6bce90010e3..855c9d666f0bc 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 e33fd831ab471..57a2091fe8c77 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 b1bc5ebdd52ed12aea3fdc7b8f2fa2d00ea09c6b Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 19 May 2016 04:48:36 +0200 Subject: [PATCH 018/143] [DOC][MINOR] ml.feature Scala and Python API sync ## What changes were proposed in this pull request? I reviewed Scala and Python APIs for ml.feature and corrected discrepancies. ## How was this patch tested? Built docs locally, ran style checks Author: Bryan Cutler Closes #13159 from BryanCutler/ml.feature-api-sync. --- .../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 f85f4c65af7d8..08beda6d7515d 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 141d3b924b4fe..dbbaa5aa46f49 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 c0feaa01fc861..2916b6d9df3be 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 2bc9d225ac2f6..d814528ec48d1 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 983b6a5301ae1..497f2ad68ee51 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 5c9117a3ed373461529f9f9306668ed4149c63fb Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 18 May 2016 20:15:00 -0700 Subject: [PATCH 019/143] [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. --- .../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 7d7b4c82fa392..89d2fb9b47971 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 661c21049b62ebfaf788dcbc31d62a09e206265b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 May 2016 21:43:07 -0700 Subject: [PATCH 020/143] [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. --- .../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 84339f439a666..98ce5dd2efd91 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 faf359f54838e..5cfb6d5363a3b 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 3ff991392dfb4..5fced940b38d1 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 1935e41185ac0..52e706285c7ab 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 e2ec32dab8530aa21ec95a27d60b1c22f3d1a18c Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 19 May 2016 08:52:41 +0200 Subject: [PATCH 021/143] [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. --- .../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 ff1eb07dc6058..ca80d0d8bba57 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 2d6d115d54d02..c57e59d01b547 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 3355c8ffa2952..29f1f509608a7 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 5907ebfc11aac8029cfc6d5f8e91cd5f53af54fe Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 May 2016 15:57:44 +0800 Subject: [PATCH 022/143] [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. --- .../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 fc38369f38c33..5e17f8920901a 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 be9f03d4baaaf..6825b65e2b282 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 c94dcb33546f8..4c26c184b7b5b 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 8c92ad82ac5be..7402918c1bbba 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 0000000000000..355b3fc4aa637 --- /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 010dea5b30380..743a27aa7a21d 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 3facca5152e685d9c7da96bff5102169740a4a06 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 10:44:26 +0100 Subject: [PATCH 023/143] [CORE][MINOR] Remove redundant set master in OutputCommitCoordinatorIntegrationSuite ## What changes were proposed in this pull request? Remove redundant set master in OutputCommitCoordinatorIntegrationSuite, as we are already setting it in SparkContext below on line 43. ## How was this patch tested? existing tests Author: Sandeep Singh Closes #13168 from techaddict/minor-1. --- .../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 601f1c378c41f..32cdf16dd3318 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 1052d3644d7eb0e784eb883293ce63a352a3b123 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 10:25:33 -0700 Subject: [PATCH 024/143] [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. --- .../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 41c0aec0ecf96..986f7e0fb0a5e 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 8ecf7f77b2be0a178a8d94d60477876d4ab7517a Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 10:27:17 -0700 Subject: [PATCH 025/143] [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. --- .../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 683ae4aaf407e..c4e882240ffd2 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 85f325f0765a4..e809dd4092afa 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 31f63ac25da43746fdef2a9477f6a79ac046112f Mon Sep 17 00:00:00 2001 From: Pravin Gadakh Date: Thu, 19 May 2016 10:59:07 -0700 Subject: [PATCH 026/143] [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. --- .../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 8204b5af02cff..a47526d36f1a1 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 c0d112d2c53d8..59f9c2adba5fc 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 9308bf119204015c8733fab0c2aef70ff2e41d74 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 11:45:18 -0700 Subject: [PATCH 027/143] [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. --- .../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 b6ecd3cb065ae..d3081ba7accd2 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 cb41457b6653f..cd6b97a855412 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 b7b08dc4b126f..a5b56541c90f7 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 ef7a5e0bcaee45b907a10b73f11c838ef6e23614 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 19 May 2016 11:46:11 -0700 Subject: [PATCH 028/143] [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 a0264ce1acf35..a68ef33d3999c 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 36c93228b94af..8c8768f50bfde 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 21da55cbc375c..489a1c8c3facd 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 54b30d3898746..cf9286e6b97a6 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 ad182086cc3bd7951aaf82693d9bcb56815b43e4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 11:47:17 -0700 Subject: [PATCH 029/143] [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. --- .../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 ca53534b61c4a..20ffe1342e509 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 9ee83b76e71dc..1b325801e27fc 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 faafd1e9dbde1f9c2ad218b90c27c3f25f98e1a4 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 19 May 2016 11:51:59 -0700 Subject: [PATCH 030/143] [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 1313a011c69c2..2c269478ee7ef 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 f5065abf49dea0eac04b0ec219f2d832a0f6730a Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 19 May 2016 11:54:50 -0700 Subject: [PATCH 031/143] [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. --- .../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 009fbaa006574..ba61940b3d5a4 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 e71cd96bf733f0440f818c6efc7a04b68d7cbe45 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 19 May 2016 20:59:19 +0200 Subject: [PATCH 032/143] [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. --- 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 cfcbbfc98e821..25640b1f85250 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 6ac1c3a040f88fae15c46acd73e7e3691f7d3619 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 19 May 2016 12:02:41 -0700 Subject: [PATCH 033/143] [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. --- .../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 3fdd411ac4cc9..4a073d11893c4 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 1fc02d1d4b469..a3472745371f1 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 af2850d4f568c..0f0c1b0702fb1 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 3b8068d3bc478..dedc8f55f01ba 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 4e3cb7a5d965fd490390398ecfe35f1fc05e8511 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 12:05:17 -0700 Subject: [PATCH 034/143] [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. --- .../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 eddc36edc9611..7d63a8f734f0e 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 945830c8bf242..842f42b4c98db 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 6ddabfd8ef089..d986a55959b82 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 b454ef1b204b2..d76a0e657c288 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 0cf9df084fdbe..13cb6a28c3465 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 d223af1496a4b..f684e16c25f7c 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 ce7d51d1c371b..6f7c9f282a586 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 d6de15494fefa..e63c7c581e9b2 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 510a2ee3bfafb..03b532664a0c8 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 08f596f130aa2..7a89b484eb85b 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 9eff42ab2d559..1c467137baa86 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 5ccecc078aa757d3f1f6632aa6df5659490f602f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 12:12:42 -0700 Subject: [PATCH 035/143] [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. --- python/pyspark/sql/dataframe.py | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 9 +++-- .../org/apache/spark/sql/JoinSuite.scala | 2 +- .../apache/spark/sql/StatisticsSuite.scala | 34 +++++++++++++++++++ 4 files changed, 40 insertions(+), 7 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 a68ef33d3999c..4fa799ac55bdf 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 518430f16d712..5d1868980163d 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,14 +599,13 @@ 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) 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/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index a6b83b3d07504..a5d8cb19eadc9 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 0000000000000..9523f6f9f5bbf --- /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 305263954a077d4e56dd1a6b2f493949a9973f55 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 12:32:26 -0700 Subject: [PATCH 036/143] 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 6f7c9f282a586..1fa9b28edf4be 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 5255e55c843c7b67fcb2abb4284b8b1a09bd6672 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 19 May 2016 13:10:51 -0700 Subject: [PATCH 037/143] [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. --- .../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 c62a1eab2016b..383d6d96e8d7c 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 6c6e9fb7c6b3d..c806d6bdf6b63 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 59e6c5560d13def686091391aabe024ecb43174b Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 13:26:41 -0700 Subject: [PATCH 038/143] [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. --- .../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 138e059f94938..afb1080b9b7d5 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 63ca812609b7e..d81b337607556 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 986f7e0fb0a5e..0ab370e3b476e 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 5a83b2870005d..ec60991af64ff 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 ef43a5fe51614eecce2d144cc13b33004a47533a Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 17:24:42 -0700 Subject: [PATCH 039/143] [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. --- .../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 5c9a112ca69ca..ee1956c2d4135 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 1f1cfa0cb28b4..7ebcd297bd549 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 dcf407de676ce99462a62f17cd98c9df56f057df Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 19 May 2016 17:42:59 -0700 Subject: [PATCH 040/143] [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. --- ...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 226d59d0eae88..ef9255794b6ab 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 0000000000000..f11a3fb969db6 --- /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 cb53b2b1aac1d..ff53505549330 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 f8107c7846c9fcabbe2579867574305c7f2028e7 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 17:54:18 -0700 Subject: [PATCH 041/143] [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. --- .../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 ac2cd8726b22e..0ab4459bdb9df 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 @@ -614,7 +614,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 d81b337607556..88b6b27e62d97 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 0ab370e3b476e..790ef1fe8dc94 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 4aa7c2cc0b9b2..e8474d035ee62 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 3e9a3f9db5f41..a702f02c9143c 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 664367781786df7ec52e39950dccd5a09681602c Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 17:56:21 -0700 Subject: [PATCH 042/143] [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. --- .../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 8408516751102..390e9b6444c74 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 fc9099b7ec172..16029dc34863a 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 16ba71aba4e68bbb892d4ceb38d6d1d135d63fd3 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 18:31:05 -0700 Subject: [PATCH 043/143] [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. --- .../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 ce45168a13198..ccad9b3fd52fd 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 01cf649c4f96f64fb4bd09e0e1811cabcc5ead2e Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 20:38:44 -0700 Subject: [PATCH 044/143] [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. --- .../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 79b99095815a1..526bed93fbd24 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 0000000000000..43779878890db --- /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 a81a36d1b1c43..9b209006bc369 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 c76a1947c64f7..5aba4e8f7de07 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 4648926c3421f..74bb46bd217a9 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 b8da04c26a49d..004102103d52c 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 48edbc838c818..6d0604d8f9a5a 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 787909821be9b..c2a9e7b58b470 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 58bc5a448aed3..6194167bda354 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 1ed20b1bfa17d..dd98513f37ecf 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 9d07170fa1d5d..1be6f96f4c942 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 a96b43de15779..87639380bdcf4 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 9d8c09b30c019..b7956b6fd3e9a 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 3c37441a77beb..57696d0150a8b 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 b3e213a49700e..6f877b566875c 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 a4bce2283b860..ac479c08418ce 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 a28f73f10a0b9..df5d34fbe94e6 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 8415fdb84f213..dbc0b1db5c002 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 2b156f3bca5b1..6480b57e1f796 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 52c0bde8f3677..c1928a26b609e 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 0bac2839e179d..27550a3d5c373 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 fedaa77176586..583652badb8fb 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 a8dd44608db32..ca8fae3a48b9d 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 a565c77af4ab1..3dc2e1f896143 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 bef7eb0f995fb..d0a849fd11c7e 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 a5b5dd4088ff8..1077e103a3b89 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 4ea3f2255efa7..1da85ed9dab4e 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 3b5edf1e15aef..7fd9b1feb7f83 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 126aa6298f3de..6cdcdda1a6480 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 d601e7c540e19..4ba13e2e06c8d 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 022dcf94bd006..fa39f4560c8aa 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 b874ccd48b3b1..692d5ad591e84 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 7151e27cde839..da623d1d15702 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 2f10d14da5e4a..c04e2e69541ba 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 5e212e2fc5b3b..6ded42e928250 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 2a090c054fe2b..0f54e684e447d 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 62c6d9b7e390a..8c6bced52dd74 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 7f29b050479fc..3d62b273d2210 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 20edd08a2172d..bf76719937772 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 4e5b87f588e3d..270e636f82117 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 f16585aff4f5d..08d6713ab2bc3 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 d1d618f7de2d8..d41fc0e4dca96 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 6a096d6386550..e9d7e4fdbe8ce 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 de50fb8c4fdb2..05128ea343420 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 64885cc8425d3..3e3abddbee638 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 fdc19a5b3dc47..3451e0773759b 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 f235251e61d42..46e9dd8b59828 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 bf7f1fc71b08e..75b0ec6480196 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 92fc57871cdbf..f427846b9ad10 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 817b962c75007..f67f555e418a7 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 b449108a9b83e..6d114024c31be 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 aa784054d551e..363ab42546d11 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 8b05675d65a1d..dbd4cbfd2b746 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 098bac3bedfff..1458cc72bc17f 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 35087a5e461df..a46b1321b3ca2 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 b2efb2e72e374..cb00977412345 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 ea0ccd7448986..ab554475d59a1 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 373417d3ba7c0..1abaa39eadc22 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 5b464a4722d92..1dcbbcaa0223c 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 66ec2494938e2f9f4403285713d3cba7a1495cf1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 19 May 2016 20:40:17 -0700 Subject: [PATCH 045/143] Closes #11915 Closes #8648 Closes #13089 From d5c47f8ff8c09ff017e896835db044661ee60909 Mon Sep 17 00:00:00 2001 From: Sumedh Mungee Date: Fri, 20 May 2016 12:30:04 +0800 Subject: [PATCH 046/143] [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. --- .../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 177b1390b2f49..227e835e7ed51 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 17591d90e6873f30a042112f56a1686726ccbd60 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Fri, 20 May 2016 12:41:14 +0800 Subject: [PATCH 047/143] [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. --- .../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 9bfc381639140..9cc7b2ac79205 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 690758205efff..1fe143494abad 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 c0fa220d34bb6..58df651da2942 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 2f7422b7420d0..b907f62802175 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 9c1319c1c5e6f..6b7e3714e0b04 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 227e835e7ed51..d4387890b403b 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 324ebbae38767..35a9f44feca64 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 491bdb3ef9db9..c9bd05d0e4e36 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 f2ee0ed4b7ecb2855cc4928a9613a07d45446f4e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 21:53:26 -0700 Subject: [PATCH 048/143] [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. --- .../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 da623d1d15702..7bda219243bf5 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 e8e60c64121b5..486733a390a0c 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 257a239c8d7b0..0e04b88265fa1 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 02dd6547a4adc..78a167eef2e4e 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 a3e2b49556e4c..14d12d30bc0b3 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 629243bd1a97b..f697769bdcdb5 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 85af4faf4d090..d8911f88b0004 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 cb3c46a98bfb4..34187b9a1ae7f 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 b94b84d77a502..045ccc7bd6eae 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 ccad9b3fd52fd..2e17b763a5370 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 8d332df029166..88125a2b4da78 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 bcf70fdc4a497..233b7891d664c 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 df6304d85fe74..7d09bdcebdc3d 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 65bc043076759..0b490fe71c526 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 5d1868980163d..35d67ca2d8c5d 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 eaf993aaed4d2..9f6137d6e3c7c 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 65fe271b69172..b447006761f45 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 0b5a92c256e57..0000000000000 --- 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 38d7b6e25b829..c9594a7e9ab28 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 743a27aa7a21d..460e34a5ff308 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 b489b74fec074..cd6b2647e0be6 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 0000000000000..ec6a2b3575869 --- /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 9523f6f9f5bbf..4de3cf605caa1 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 70a00a43f7db4..2f45db3925a00 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 2a5295d0d2231..8243470b19334 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 d7eae21f9f556..9fe0e9646e31e 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 b5fc51603e160..1753b84ba6af6 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 4fa1754253aff..bd197be655d58 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 1c467137baa86..2374ffaaa5036 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 81bc973be74a3..0296229100a24 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 612cfc7ec7bd6..a34f70ed65b5c 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 51d04f2f4efc6..f969660ddd322 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 cd0256db43aad..9cdf7dea7663e 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 34b8726a922fb..cddf4a1884fa8 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 ff53505549330..e6c0ce95e7b57 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 421f6bca7f865..0cfe260e52152 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 51538eca644f6..853dd0ff3f601 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 620bfa995aa20..79c37faa4e9ba 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 8de223f444f70..638911599aad3 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 d2cb62c617d44..7c74a0308d483 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 6c9ce208dbd6a..622b043581c5d 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 81f3ea8a6e801..8a31a49d97f08 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 aba60da33fe3e..bb351e20c5e99 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 3ba34d435c1f61435c2dddc28650cd111e7c1f33 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 22:14:10 -0700 Subject: [PATCH 049/143] [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. --- .../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 f603cbfb0cc21..7101ca5a17de9 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 0000000000000..bc6852ca7e1fd --- /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 e384c7fbb94cef3c18e8fa8d06159b76b88b5167 Mon Sep 17 00:00:00 2001 From: Oleg Danilov Date: Thu, 19 May 2016 22:23:28 -0700 Subject: [PATCH 050/143] [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. --- .../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 0f0c1b0702fb1..71d5c9960a70c 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 d5e1c5acde95158db38448526c8afad4a6d21dc2 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 19 May 2016 22:55:44 -0700 Subject: [PATCH 051/143] [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. --- .../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 6825b65e2b282..a6fb34cbfb110 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 1fae64e3bc6b1..289c16aef47aa 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 52e706285c7ab..0ffbd6db12be6 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 09a00510c4759ff87abb0b2fdf1630ddf36ca12c Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 19 May 2016 23:03:59 -0700 Subject: [PATCH 052/143] [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. --- .../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 3045f3af360bc..8af6d07719c9f 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 a3472745371f1..d13492e55070b 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 8225bd69c1c7c..df62ba08b8018 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 4c7a6b385c79f4de07a89495afce4f8e73b06086 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Thu, 19 May 2016 23:21:17 -0700 Subject: [PATCH 053/143] [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. --- .../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 8ed6367787203..c69027babba8d 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 47a2940da97caa55bbb8bb8ec1d51c9f6d5041c6 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Thu, 19 May 2016 23:26:11 -0700 Subject: [PATCH 054/143] [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. --- .../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 31a79ddd3fff1..f64155ce3c0cf 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 205ca02962bee..f07020b50397b 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 0ea7cfb7025a0..3426e491dc74a 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 b318b7d87bfdc..7d33be7e81d73 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 fa1010195551a..97aefac025e55 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 bec89f7c3dff0..3d02ce05619ad 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 f8961847f3df2..fca585c2a362b 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 0baf6db607ad9..13ccc2ae7c3d8 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 4263680c6fde3..b06c629802006 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 7463b868ff19b..c514eb0fa51ae 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 d9f94a42b1a0b..676164806e196 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 acd8656b65a69..718f84f6450e0 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 5e203505f1a092e5849ebd01d9ff9e4fc6cdc34a Mon Sep 17 00:00:00 2001 From: sethah Date: Thu, 19 May 2016 23:29:37 -0700 Subject: [PATCH 055/143] [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. --- 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 f6a6937e29f04..f1a21f436f133 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 33e4b7b0d2cc8..8656eb4001f4b 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 4514a358e12f2..bd3d527d9a0e3 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 c44ace91f23f6..3db24a3840599 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 cc353df1ecd00..dae86d84804d0 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 c94b34ebbf4c6ce353c899c571beb34e8db98917 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 23:35:20 -0700 Subject: [PATCH 056/143] [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. --- .../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 cc16c2f038be8..e63eb71080604 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 e8474d035ee62..adbdd345e92ec 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 ba0f59e89bb24..d16e8e3f6b259 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 a702f02c9143c..ff1038cbf185f 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 332d331a47737..265f2f45c45fe 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 39fd469078271aa12f3163606000e06e382d35dc Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 20 May 2016 14:38:25 +0800 Subject: [PATCH 057/143] [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. --- .../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 49c07427617f3..a99bc3bff6eea 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 68238dbb46e9f..78b1db16826e2 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 473e827f4db18..1371abe189f84 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 f0b8a83dee8ca..8f7c6f5d0ca44 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 00adb9a44b14e..686c63065dfcc 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 622b043581c5d..5b706b0432418 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 75166f6beaa87..415d4c0049d40 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 257375019266ab9e3c320e33026318cc31f58ada Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 23:43:01 -0700 Subject: [PATCH 058/143] [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. --- .../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 8af6d07719c9f..ee12bfa7251fd 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 897170ea57cf1..0925a51310e69 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 c32b1b162e7e5ecc5c823f79ba9f23cbd1407dbf Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 23:44:10 -0700 Subject: [PATCH 059/143] [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. --- 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 adaa3b5a79938..ef46d3065e5e8 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 0e04b88265fa1..241947537fa29 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): From 4e739331187f2acdd84a5e65857edb62e58a0f8f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 20 May 2016 13:40:13 +0200 Subject: [PATCH 060/143] [SPARK-15444][PYSPARK][ML][HOTFIX] Default value mismatch of param linkPredictionCol for GeneralizedLinearRegression ## What changes were proposed in this pull request? Default value mismatch of param linkPredictionCol for GeneralizedLinearRegression between PySpark and Scala. That is because default value conflict between #13106 and #13129. This causes ml.tests failed. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13220 from viirya/hotfix-regresstion. --- python/pyspark/ml/regression.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 25640b1f85250..e21dd83923adb 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1303,17 +1303,16 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha @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", linkPredictionCol=""): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None): """ __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", linkPredictionCol="") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None) """ 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", - linkPredictionCol="") + self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -1321,11 +1320,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", linkPredictionCol=""): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None): """ 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", linkPredictionCol="") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None) Sets params for generalized linear regression. """ kwargs = self.setParams._input_kwargs From a3ceb875c64421ced8e52db6d8e51aec9b758e3e Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Fri, 20 May 2016 08:17:19 -0500 Subject: [PATCH 061/143] [SPARK-15203][DEPLOY] The spark daemon shell script error, daemon process start successfully but script output fail message ## What changes were proposed in this pull request? fix the bug: The spark daemon shell script error, daemon process start successfully but script output fail message ## How was this patch tested? existing test. Author: WeichenXu Closes #13172 from WeichenXu123/fix-spark-15203. --- sbin/spark-daemon.sh | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 6ab57df409529..59823571124f1 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -162,6 +162,16 @@ run_command() { esac echo "$newpid" > "$pid" + + #Poll for up to 5 seconds for the java process to start + for i in {1..10} + do + if [[ $(ps -p "$newpid" -o comm=) =~ "java" ]]; then + break + fi + sleep 0.5 + done + sleep 2 # Check if the process has died; in that case we'll tail the log so the user can see if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then From 9a9c6f5c22248c5a891e9d3b788ff12b6b4718b2 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 20 May 2016 09:30:20 -0700 Subject: [PATCH 062/143] [SPARK-15222][SPARKR][ML] SparkR ML examples update in 2.0 ## What changes were proposed in this pull request? Update example code in examples/src/main/r/ml.R to reflect the new algorithms. * spark.glm and glm * spark.survreg * spark.naiveBayes * spark.kmeans ## How was this patch tested? Offline test. Author: Yanbo Liang Closes #13000 from yanboliang/spark-15222. --- examples/src/main/r/ml.R | 129 +++++++++++++++++++++++++++++++++------ 1 file changed, 112 insertions(+), 17 deletions(-) diff --git a/examples/src/main/r/ml.R b/examples/src/main/r/ml.R index a0c903939cbbb..fd35936635334 100644 --- a/examples/src/main/r/ml.R +++ b/examples/src/main/r/ml.R @@ -16,7 +16,7 @@ # # To run this example use -# ./bin/sparkR examples/src/main/r/ml.R +# ./bin/spark-submit examples/src/main/r/ml.R # Load SparkR library into your R session library(SparkR) @@ -25,30 +25,125 @@ library(SparkR) sc <- sparkR.init(appName="SparkR-ML-example") sqlContext <- sparkRSQL.init(sc) -# Train GLM of family 'gaussian' -training1 <- suppressWarnings(createDataFrame(sqlContext, iris)) -test1 <- training1 -model1 <- glm(Sepal_Length ~ Sepal_Width + Species, training1, family = "gaussian") +############################ spark.glm and glm ############################################## + +irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +# Fit a generalized linear model of family "gaussian" with spark.glm +gaussianDF <- irisDF +gaussianTestDF <- irisDF +gaussianGLM <- spark.glm(gaussianDF, Sepal_Length ~ Sepal_Width + Species, family = "gaussian") # Model summary -summary(model1) +summary(gaussianGLM) # Prediction -predictions1 <- predict(model1, test1) -head(select(predictions1, "Sepal_Length", "prediction")) +gaussianPredictions <- predict(gaussianGLM, gaussianTestDF) +showDF(gaussianPredictions) + +# Fit a generalized linear model with glm (R-compliant) +gaussianGLM2 <- glm(Sepal_Length ~ Sepal_Width + Species, gaussianDF, family = "gaussian") +summary(gaussianGLM2) + +# Fit a generalized linear model of family "binomial" with spark.glm +binomialDF <- filter(irisDF, irisDF$Species != "setosa") +binomialTestDF <- binomialDF +binomialGLM <- spark.glm(binomialDF, Species ~ Sepal_Length + Sepal_Width, family = "binomial") + +# Model summary +summary(binomialGLM) + +# Prediction +binomialPredictions <- predict(binomialGLM, binomialTestDF) +showDF(binomialPredictions) + +############################ spark.survreg ############################################## + +# Use the ovarian dataset available in R survival package +library(survival) -# Train GLM of family 'binomial' -training2 <- filter(training1, training1$Species != "setosa") -test2 <- training2 -model2 <- glm(Species ~ Sepal_Length + Sepal_Width, data = training2, family = "binomial") +# Fit an accelerated failure time (AFT) survival regression model with spark.survreg +ovarianDF <- suppressWarnings(createDataFrame(sqlContext, ovarian)) +aftDF <- ovarianDF +aftTestDF <- ovarianDF +aftModel <- spark.survreg(aftDF, Surv(futime, fustat) ~ ecog_ps + rx) # Model summary -summary(model2) +summary(aftModel) + +# Prediction +aftPredictions <- predict(aftModel, aftTestDF) +showDF(aftPredictions) + +############################ spark.naiveBayes ############################################## + +# Fit a Bernoulli naive Bayes model with spark.naiveBayes +titanic <- as.data.frame(Titanic) +titanicDF <- suppressWarnings(createDataFrame(sqlContext, titanic[titanic$Freq > 0, -5])) +nbDF <- titanicDF +nbTestDF <- titanicDF +nbModel <- spark.naiveBayes(nbDF, Survived ~ Class + Sex + Age) + +# Model summary +summary(nbModel) + +# Prediction +nbPredictions <- predict(nbModel, nbTestDF) +showDF(nbPredictions) + +############################ spark.kmeans ############################################## + +# Fit a k-means model with spark.kmeans +irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +kmeansDF <- irisDF +kmeansTestDF <- irisDF +kmeansModel <- spark.kmeans(kmeansDF, ~ Sepal_Length + Sepal_Width + Petal_Length + Petal_Width, + k = 3) + +# Model summary +summary(kmeansModel) + +# Get fitted result from the k-means model +showDF(fitted(kmeansModel)) + +# Prediction +kmeansPredictions <- predict(kmeansModel, kmeansTestDF) +showDF(kmeansPredictions) + +############################ model read/write ############################################## + +irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +# Fit a generalized linear model of family "gaussian" with spark.glm +gaussianDF <- irisDF +gaussianTestDF <- irisDF +gaussianGLM <- spark.glm(gaussianDF, Sepal_Length ~ Sepal_Width + Species, family = "gaussian") + +# Save and then load a fitted MLlib model +modelPath <- tempfile(pattern = "ml", fileext = ".tmp") +write.ml(gaussianGLM, modelPath) +gaussianGLM2 <- read.ml(modelPath) + +# Check model summary +summary(gaussianGLM2) + +# Check model prediction +gaussianPredictions <- predict(gaussianGLM2, gaussianTestDF) +showDF(gaussianPredictions) + +unlink(modelPath) + +############################ fit models with spark.lapply ##################################### + +# Perform distributed training of multiple models with spark.lapply +families <- c("gaussian", "poisson") +train <- function(family) { + model <- glm(Sepal.Length ~ Sepal.Width + Species, iris, family = family) + summary(model) +} +model.summaries <- spark.lapply(sc, families, train) + +# Print the summary of each model +print(model.summaries) -# Prediction (Currently the output of prediction for binomial GLM is the indexed label, -# we need to transform back to the original string label later) -predictions2 <- predict(model2, test2) -head(select(predictions2, "Species", "prediction")) # Stop the SparkContext now sparkR.stop() From d2e1aa97ef5bf7cfffc777a178f44ab8fa775266 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 20 May 2016 09:34:55 -0700 Subject: [PATCH 063/143] [SPARK-15308][SQL] RowEncoder should preserve nested column name. ## What changes were proposed in this pull request? The following code generates wrong schema: ``` val schema = new StructType().add( "struct", new StructType() .add("i", IntegerType, nullable = false) .add( "s", new StructType().add("int", IntegerType, nullable = false), nullable = false), nullable = false) val ds = sqlContext.range(10).map(l => Row(l, Row(l)))(RowEncoder(schema)) ds.printSchema() ``` This should print as follows: ``` root |-- struct: struct (nullable = false) | |-- i: integer (nullable = false) | |-- s: struct (nullable = false) | | |-- int: integer (nullable = false) ``` but the result is: ``` root |-- struct: struct (nullable = false) | |-- col1: integer (nullable = false) | |-- col2: struct (nullable = false) | | |-- col1: integer (nullable = false) ``` This PR fixes `RowEncoder` to preserve nested column name. ## How was this patch tested? Existing tests and I added a test to check if `RowEncoder` preserves nested column name. Author: Takuya UESHIN Closes #13090 from ueshin/issues/SPARK-15308. --- .../sql/catalyst/encoders/RowEncoder.scala | 22 ++++++++++--------- .../catalyst/encoders/RowEncoderSuite.scala | 22 +++++++++++++++++++ 2 files changed, 34 insertions(+), 10 deletions(-) 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 71b39c54fa0c2..2f8ba33f3520f 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 @@ -62,7 +62,7 @@ object RowEncoder { new ExpressionEncoder[Row]( schema, flat = false, - serializer.asInstanceOf[CreateStruct].children, + serializer.asInstanceOf[CreateNamedStruct].flatten, deserializer, ClassTag(cls)) } @@ -148,28 +148,30 @@ object RowEncoder { dataType = t) case StructType(fields) => - val convertedFields = fields.zipWithIndex.map { case (f, i) => + val nonNullOutput = CreateNamedStruct(fields.zipWithIndex.flatMap { case (field, index) => val fieldValue = serializerFor( - GetExternalRowField(inputObject, i, f.name, externalDataTypeForInput(f.dataType)), - f.dataType + GetExternalRowField( + inputObject, index, field.name, externalDataTypeForInput(field.dataType)), + field.dataType ) - if (f.nullable) { + val convertedField = if (field.nullable) { If( - Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil), - Literal.create(null, f.dataType), + Invoke(inputObject, "isNullAt", BooleanType, Literal(index) :: Nil), + Literal.create(null, field.dataType), fieldValue ) } else { fieldValue } - } + Literal(field.name) :: convertedField :: Nil + }) if (inputObject.nullable) { If(IsNull(inputObject), Literal.create(null, inputType), - CreateStruct(convertedFields)) + nonNullOutput) } else { - CreateStruct(convertedFields) + nonNullOutput } } 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 7bb006c173b07..39fcc7225b37e 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,28 @@ class RowEncoderSuite extends SparkFunSuite { assert(encoder.serializer.head.nullable == false) } + test("RowEncoder should preserve nested column name") { + val schema = new StructType().add( + "struct", + new StructType() + .add("i", IntegerType, nullable = false) + .add( + "s", + new StructType().add("int", IntegerType, nullable = false), + nullable = false), + nullable = false) + val encoder = RowEncoder(schema) + assert(encoder.serializer.length == 1) + assert(encoder.serializer.head.dataType == + new StructType() + .add("i", IntegerType, nullable = false) + .add( + "s", + new StructType().add("int", IntegerType, nullable = false), + nullable = false)) + 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)) From e8adc552df80af413e1d31b020489612d13a8770 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 May 2016 09:36:14 -0700 Subject: [PATCH 064/143] [SPARK-15435][SQL] Append Command to all commands ## What changes were proposed in this pull request? We started this convention to append Command suffix to all SQL commands. However, not all commands follow that convention. This patch adds Command suffix to all RunnableCommands. ## How was this patch tested? Updated test cases to reflect the renames. Author: Reynold Xin Closes #13215 from rxin/SPARK-15435. --- .../spark/sql/execution/SparkSqlParser.scala | 91 ++++++++++--------- .../spark/sql/execution/SparkStrategies.scala | 6 +- ...eTable.scala => AnalyzeTableCommand.scala} | 6 +- .../command/createDataSourceTables.scala | 3 +- .../spark/sql/execution/command/ddl.scala | 26 +++--- .../sql/execution/command/functions.scala | 12 +-- .../sql/execution/command/resources.scala | 4 +- .../spark/sql/execution/command/tables.scala | 37 ++++---- .../execution/datasources/DataSource.scala | 2 +- .../datasources/DataSourceStrategy.scala | 4 +- ...cala => InsertIntoDataSourceCommand.scala} | 2 +- ...> InsertIntoHadoopFsRelationCommand.scala} | 12 +-- .../spark/sql/execution/datasources/ddl.scala | 2 +- .../spark/sql/internal/SessionState.scala | 4 +- .../execution/command/DDLCommandSuite.scala | 90 +++++++++--------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- ...scala => CreateTableAsSelectCommand.scala} | 2 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 12 +-- .../spark/sql/hive/StatisticsSuite.scala | 16 ++-- .../apache/spark/sql/hive/parquetSuites.scala | 10 +- 20 files changed, 173 insertions(+), 170 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/command/{AnalyzeTable.scala => AnalyzeTableCommand.scala} (95%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{InsertIntoDataSource.scala => InsertIntoDataSourceCommand.scala} (97%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{InsertIntoHadoopFsRelation.scala => InsertIntoHadoopFsRelationCommand.scala} (93%) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/{CreateTableAsSelect.scala => CreateTableAsSelectCommand.scala} (99%) 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 ee12bfa7251fd..2966eefd07c77 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 @@ -76,8 +76,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AnalyzeTable]] command. This currently only implements the NOSCAN option (other - * options are passed on to Hive) e.g.: + * Create an [[AnalyzeTableCommand]] command. This currently only implements the NOSCAN + * option (other options are passed on to Hive) e.g.: * {{{ * ANALYZE TABLE table COMPUTE STATISTICS NOSCAN; * }}} @@ -86,11 +86,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (ctx.partitionSpec == null && ctx.identifier != null && ctx.identifier.getText.toLowerCase == "noscan") { - AnalyzeTable(visitTableIdentifier(ctx.tableIdentifier).toString) + AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString) } else { // Always just run the no scan analyze. We should fix this and implement full analyze // command in the future. - AnalyzeTable(visitTableIdentifier(ctx.tableIdentifier).toString) + AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString) } } @@ -332,7 +332,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[LoadData]] command. + * Create a [[LoadDataCommand]] command. * * For example: * {{{ @@ -341,7 +341,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { - LoadData( + LoadDataCommand( table = visitTableIdentifier(ctx.tableIdentifier), path = string(ctx.path), isLocal = ctx.LOCAL != null, @@ -351,7 +351,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[TruncateTable]] command. + * Create a [[TruncateTableCommand]] command. * * For example: * {{{ @@ -363,7 +363,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (ctx.identifierList != null) { throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) } - TruncateTable( + TruncateTableCommand( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) ) @@ -422,7 +422,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[CreateDatabase]] command. + * Create a [[CreateDatabaseCommand]] command. * * For example: * {{{ @@ -431,7 +431,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitCreateDatabase(ctx: CreateDatabaseContext): LogicalPlan = withOrigin(ctx) { - CreateDatabase( + CreateDatabaseCommand( ctx.identifier.getText, ctx.EXISTS != null, Option(ctx.locationSpec).map(visitLocationSpec), @@ -440,7 +440,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AlterDatabaseProperties]] command. + * Create an [[AlterDatabasePropertiesCommand]] command. * * For example: * {{{ @@ -449,13 +449,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitSetDatabaseProperties( ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterDatabaseProperties( + AlterDatabasePropertiesCommand( ctx.identifier.getText, visitPropertyKeyValues(ctx.tablePropertyList)) } /** - * Create a [[DropDatabase]] command. + * Create a [[DropDatabaseCommand]] command. * * For example: * {{{ @@ -463,11 +463,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitDropDatabase(ctx: DropDatabaseContext): LogicalPlan = withOrigin(ctx) { - DropDatabase(ctx.identifier.getText, ctx.EXISTS != null, ctx.CASCADE != null) + DropDatabaseCommand(ctx.identifier.getText, ctx.EXISTS != null, ctx.CASCADE != null) } /** - * Create a [[DescribeDatabase]] command. + * Create a [[DescribeDatabaseCommand]] command. * * For example: * {{{ @@ -475,11 +475,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitDescribeDatabase(ctx: DescribeDatabaseContext): LogicalPlan = withOrigin(ctx) { - DescribeDatabase(ctx.identifier.getText, ctx.EXTENDED != null) + DescribeDatabaseCommand(ctx.identifier.getText, ctx.EXTENDED != null) } /** - * Create a [[CreateFunction]] command. + * Create a [[CreateFunctionCommand]] command. * * For example: * {{{ @@ -500,7 +500,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Extract database, name & alias. val functionIdentifier = visitFunctionName(ctx.qualifiedName) - CreateFunction( + CreateFunctionCommand( functionIdentifier.database, functionIdentifier.funcName, string(ctx.className), @@ -509,7 +509,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[DropFunction]] command. + * Create a [[DropFunctionCommand]] command. * * For example: * {{{ @@ -518,7 +518,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitDropFunction(ctx: DropFunctionContext): LogicalPlan = withOrigin(ctx) { val functionIdentifier = visitFunctionName(ctx.qualifiedName) - DropFunction( + DropFunctionCommand( functionIdentifier.database, functionIdentifier.funcName, ctx.EXISTS != null, @@ -526,20 +526,20 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[DropTable]] command. + * Create a [[DropTableCommand]] command. */ override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { if (ctx.PURGE != null) { throw operationNotAllowed("DROP TABLE ... PURGE", ctx) } - DropTable( + DropTableCommand( visitTableIdentifier(ctx.tableIdentifier), ctx.EXISTS != null, ctx.VIEW != null) } /** - * Create a [[AlterTableRename]] command. + * Create a [[AlterTableRenameCommand]] command. * * For example: * {{{ @@ -548,14 +548,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { - AlterTableRename( + AlterTableRenameCommand( visitTableIdentifier(ctx.from), visitTableIdentifier(ctx.to), ctx.VIEW != null) } /** - * Create an [[AlterTableSetProperties]] command. + * Create an [[AlterTableSetPropertiesCommand]] command. * * For example: * {{{ @@ -565,14 +565,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitSetTableProperties( ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetProperties( + AlterTableSetPropertiesCommand( visitTableIdentifier(ctx.tableIdentifier), visitPropertyKeyValues(ctx.tablePropertyList), ctx.VIEW != null) } /** - * Create an [[AlterTableUnsetProperties]] command. + * Create an [[AlterTableUnsetPropertiesCommand]] command. * * For example: * {{{ @@ -582,7 +582,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitUnsetTableProperties( ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterTableUnsetProperties( + AlterTableUnsetPropertiesCommand( visitTableIdentifier(ctx.tableIdentifier), visitPropertyKeys(ctx.tablePropertyList), ctx.EXISTS != null, @@ -590,7 +590,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AlterTableSerDeProperties]] command. + * Create an [[AlterTableSerDePropertiesCommand]] command. * * For example: * {{{ @@ -599,7 +599,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitSetTableSerDe(ctx: SetTableSerDeContext): LogicalPlan = withOrigin(ctx) { - AlterTableSerDeProperties( + AlterTableSerDePropertiesCommand( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.STRING).map(string), Option(ctx.tablePropertyList).map(visitPropertyKeyValues), @@ -608,7 +608,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AlterTableAddPartition]] command. + * Create an [[AlterTableAddPartitionCommand]] command. * * For example: * {{{ @@ -636,14 +636,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Alter View: the location clauses are not allowed. ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec(_) -> None) } - AlterTableAddPartition( + AlterTableAddPartitionCommand( visitTableIdentifier(ctx.tableIdentifier), specsAndLocs, ctx.EXISTS != null) } /** - * Create an [[AlterTableRenamePartition]] command + * Create an [[AlterTableRenamePartitionCommand]] command * * For example: * {{{ @@ -652,14 +652,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitRenameTablePartition( ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenamePartition( + AlterTableRenamePartitionCommand( visitTableIdentifier(ctx.tableIdentifier), visitNonOptionalPartitionSpec(ctx.from), visitNonOptionalPartitionSpec(ctx.to)) } /** - * Create an [[AlterTableDropPartition]] command + * Create an [[AlterTableDropPartitionCommand]] command * * For example: * {{{ @@ -678,14 +678,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (ctx.PURGE != null) { throw operationNotAllowed("ALTER TABLE ... DROP PARTITION ... PURGE", ctx) } - AlterTableDropPartition( + AlterTableDropPartitionCommand( visitTableIdentifier(ctx.tableIdentifier), ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec), ctx.EXISTS != null) } /** - * Create an [[AlterTableSetLocation]] command + * Create an [[AlterTableSetLocationCommand]] command * * For example: * {{{ @@ -693,7 +693,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetLocation( + AlterTableSetLocationCommand( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), visitLocationSpec(ctx.locationSpec)) @@ -759,18 +759,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AddJar]] or [[AddFile]] command depending on the requested resource. + * Create an [[AddJarCommand]] or [[AddFileCommand]] command depending on the requested resource. */ override def visitAddResource(ctx: AddResourceContext): LogicalPlan = withOrigin(ctx) { ctx.identifier.getText.toLowerCase match { - case "file" => AddFile(remainder(ctx.identifier).trim) - case "jar" => AddJar(remainder(ctx.identifier).trim) + case "file" => AddFileCommand(remainder(ctx.identifier).trim) + case "jar" => AddJarCommand(remainder(ctx.identifier).trim) case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) } } /** - * Create a table, returning either a [[CreateTable]] or a [[CreateTableAsSelectLogicalPlan]]. + * Create a table, returning either a [[CreateTableCommand]] or a + * [[CreateTableAsSelectLogicalPlan]]. * * This is not used to create datasource tables, which is handled through * "CREATE TABLE ... USING ...". @@ -868,12 +869,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { selectQuery match { case Some(q) => CreateTableAsSelectLogicalPlan(tableDesc, q, ifNotExists) - case None => CreateTable(tableDesc, ifNotExists) + case None => CreateTableCommand(tableDesc, ifNotExists) } } /** - * Create a [[CreateTableLike]] command. + * Create a [[CreateTableLikeCommand]] command. * * For example: * {{{ @@ -884,7 +885,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitCreateTableLike(ctx: CreateTableLikeContext): LogicalPlan = withOrigin(ctx) { val targetTable = visitTableIdentifier(ctx.target) val sourceTable = visitTableIdentifier(ctx.source) - CreateTableLike(targetTable, sourceTable, ctx.EXISTS != null) + CreateTableLikeCommand(targetTable, sourceTable, ctx.EXISTS != null) } /** 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 5cfb6d5363a3b..3343039ae1cf3 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 @@ -398,7 +398,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { sys.error("Cannot create temporary partitioned table.") case c: CreateTableUsingAsSelect if c.temporary => - val cmd = CreateTempTableUsingAsSelect( + val cmd = CreateTempTableUsingAsSelectCommand( c.tableIdent, c.provider, Array.empty[String], c.mode, c.options, c.child) ExecutedCommandExec(cmd) :: Nil @@ -415,10 +415,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ExecutedCommandExec(cmd) :: Nil case logical.ShowFunctions(db, pattern) => - ExecutedCommandExec(ShowFunctions(db, pattern)) :: Nil + ExecutedCommandExec(ShowFunctionsCommand(db, pattern)) :: Nil case logical.DescribeFunction(function, extended) => - ExecutedCommandExec(DescribeFunction(function, extended)) :: Nil + ExecutedCommandExec(DescribeFunctionCommand(function, extended)) :: Nil case _ => Nil } 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/AnalyzeTableCommand.scala similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index de2db44b0e1b2..a469d4da8613b 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/AnalyzeTableCommand.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} * Right now, it only supports Hive tables and it only updates the size of a Hive table * in the Hive metastore. */ -case class AnalyzeTable(tableName: String) extends RunnableCommand { +case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val sessionState = sparkSession.sessionState @@ -95,7 +95,7 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { sessionState.catalog.alterTable( catalogTable.copy( properties = relation.catalogTable.properties + - (AnalyzeTable.TOTAL_SIZE_FIELD -> newTotalSize.toString))) + (AnalyzeTableCommand.TOTAL_SIZE_FIELD -> newTotalSize.toString))) } case otherRelation => @@ -106,6 +106,6 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { } } -object AnalyzeTable { +object AnalyzeTableCommand { val TOTAL_SIZE_FIELD = "totalSize" } 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 70e5108d938b9..6ca66a22df12d 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 @@ -36,7 +36,7 @@ import org.apache.spark.sql.types._ /** * A command used to create a data source table. * - * Note: This is different from [[CreateTable]]. Please check the syntax for difference. + * Note: This is different from [[CreateTableCommand]]. Please check the syntax for difference. * This is not intended for temporary tables. * * The syntax of using this command in SQL is: @@ -253,6 +253,7 @@ case class CreateDataSourceTableAsSelectCommand( } } + object CreateDataSourceTableUtils extends Logging { /** * Checks if the given name conforms the Hive standard ("[a-zA-z_0-9]+"), 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 49d7fe956f812..dd3f17d525abd 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 @@ -45,7 +45,7 @@ import org.apache.spark.sql.types._ * [WITH DBPROPERTIES (property_name=property_value, ...)]; * }}} */ -case class CreateDatabase( +case class CreateDatabaseCommand( databaseName: String, ifNotExists: Boolean, path: Option[String], @@ -85,7 +85,7 @@ case class CreateDatabase( * DROP DATABASE [IF EXISTS] database_name [RESTRICT|CASCADE]; * }}} */ -case class DropDatabase( +case class DropDatabaseCommand( databaseName: String, ifExists: Boolean, cascade: Boolean) @@ -108,7 +108,7 @@ case class DropDatabase( * ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...) * }}} */ -case class AlterDatabaseProperties( +case class AlterDatabasePropertiesCommand( databaseName: String, props: Map[String, String]) extends RunnableCommand { @@ -134,7 +134,7 @@ case class AlterDatabaseProperties( * DESCRIBE DATABASE [EXTENDED] db_name * }}} */ -case class DescribeDatabase( +case class DescribeDatabaseCommand( databaseName: String, extended: Boolean) extends RunnableCommand { @@ -175,7 +175,7 @@ case class DescribeDatabase( * DROP VIEW [IF EXISTS] [db_name.]view_name; * }}} */ -case class DropTable( +case class DropTableCommand( tableName: TableIdentifier, ifExists: Boolean, isView: Boolean) extends RunnableCommand { @@ -220,7 +220,7 @@ case class DropTable( * ALTER VIEW view1 SET TBLPROPERTIES ('key1' = 'val1', 'key2' = 'val2', ...); * }}} */ -case class AlterTableSetProperties( +case class AlterTableSetPropertiesCommand( tableName: TableIdentifier, properties: Map[String, String], isView: Boolean) @@ -251,7 +251,7 @@ case class AlterTableSetProperties( * ALTER VIEW view1 UNSET TBLPROPERTIES [IF EXISTS] ('key1', 'key2', ...); * }}} */ -case class AlterTableUnsetProperties( +case class AlterTableUnsetPropertiesCommand( tableName: TableIdentifier, propKeys: Seq[String], ifExists: Boolean, @@ -291,7 +291,7 @@ case class AlterTableUnsetProperties( * ALTER TABLE table [PARTITION spec] SET SERDEPROPERTIES serde_properties; * }}} */ -case class AlterTableSerDeProperties( +case class AlterTableSerDePropertiesCommand( tableName: TableIdentifier, serdeClassName: Option[String], serdeProperties: Option[Map[String, String]], @@ -330,7 +330,7 @@ case class AlterTableSerDeProperties( * ALTER TABLE table ADD [IF NOT EXISTS] PARTITION spec [LOCATION 'loc1'] * }}} */ -case class AlterTableAddPartition( +case class AlterTableAddPartitionCommand( tableName: TableIdentifier, partitionSpecsAndLocs: Seq[(TablePartitionSpec, Option[String])], ifNotExists: Boolean) @@ -361,7 +361,7 @@ case class AlterTableAddPartition( * ALTER TABLE table PARTITION spec1 RENAME TO PARTITION spec2; * }}} */ -case class AlterTableRenamePartition( +case class AlterTableRenamePartitionCommand( tableName: TableIdentifier, oldPartition: TablePartitionSpec, newPartition: TablePartitionSpec) @@ -389,7 +389,7 @@ case class AlterTableRenamePartition( * ALTER TABLE table DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE]; * }}} */ -case class AlterTableDropPartition( +case class AlterTableDropPartitionCommand( tableName: TableIdentifier, specs: Seq[TablePartitionSpec], ifExists: Boolean) @@ -420,7 +420,7 @@ case class AlterTableDropPartition( * ALTER TABLE table_name [PARTITION partition_spec] SET LOCATION "loc"; * }}} */ -case class AlterTableSetLocation( +case class AlterTableSetLocationCommand( tableName: TableIdentifier, partitionSpec: Option[TablePartitionSpec], location: String) @@ -459,7 +459,7 @@ case class AlterTableSetLocation( } -private[sql] object DDLUtils { +object DDLUtils { def isDatasourceTable(props: Map[String, String]): Boolean = { props.contains("spark.sql.sources.provider") 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 1ea9bc5299968..d2d8e3ddeae26 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 @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} * AS className [USING JAR\FILE 'uri' [, JAR|FILE 'uri']] * }}} */ -case class CreateFunction( +case class CreateFunctionCommand( databaseName: Option[String], functionName: String, className: String, @@ -81,7 +81,7 @@ case class CreateFunction( * DESCRIBE FUNCTION [EXTENDED] upper; * }}} */ -case class DescribeFunction( +case class DescribeFunctionCommand( functionName: FunctionIdentifier, isExtended: Boolean) extends RunnableCommand { @@ -142,7 +142,7 @@ case class DescribeFunction( * ifExists: returns an error if the function doesn't exist, unless this is true. * isTemp: indicates if it is a temporary function. */ -case class DropFunction( +case class DropFunctionCommand( databaseName: Option[String], functionName: String, ifExists: Boolean, @@ -180,10 +180,10 @@ case class DropFunction( * For the pattern, '*' matches any sequence of characters (including no characters) and * '|' is for alternation. * For example, "show functions like 'yea*|windo*'" will return "window" and "year". - * - * TODO currently we are simply ignore the db */ -case class ShowFunctions(db: Option[String], pattern: Option[String]) extends RunnableCommand { +case class ShowFunctionsCommand(db: Option[String], pattern: Option[String]) + extends RunnableCommand { + override val output: Seq[Attribute] = { val schema = StructType(StructField("function", StringType, nullable = false) :: Nil) schema.toAttributes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala index 29bcb30592516..162d493c1f8ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.types.{IntegerType, StructField, StructType} /** * Adds a jar to the current session so it can be used (for UDFs or serdes). */ -case class AddJar(path: String) extends RunnableCommand { +case class AddJarCommand(path: String) extends RunnableCommand { override val output: Seq[Attribute] = { val schema = StructType( StructField("result", IntegerType, nullable = false) :: Nil) @@ -40,7 +40,7 @@ case class AddJar(path: String) extends RunnableCommand { /** * Adds a file to the current session so it can be used. */ -case class AddFile(path: String) extends RunnableCommand { +case class AddFileCommand(path: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sparkContext.addFile(path) Seq.empty[Row] 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 d13492e55070b..13e63a1befb20 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 @@ -39,9 +39,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils case class CreateTableAsSelectLogicalPlan( - tableDesc: CatalogTable, - child: LogicalPlan, - allowExisting: Boolean) extends UnaryNode with Command { + tableDesc: CatalogTable, + child: LogicalPlan, + allowExisting: Boolean) extends UnaryNode with Command { override def output: Seq[Attribute] = Seq.empty[Attribute] @@ -63,7 +63,7 @@ case class CreateTableAsSelectLogicalPlan( * LIKE [other_db_name.]existing_table_name * }}} */ -case class CreateTableLike( +case class CreateTableLikeCommand( targetTable: TableIdentifier, sourceTable: TableIdentifier, ifNotExists: Boolean) extends RunnableCommand { @@ -115,7 +115,7 @@ case class CreateTableLike( * [AS select_statement]; * }}} */ -case class CreateTable(table: CatalogTable, ifNotExists: Boolean) extends RunnableCommand { +case class CreateTableCommand(table: CatalogTable, ifNotExists: Boolean) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sessionState.catalog.createTable(table, ifNotExists) @@ -134,7 +134,7 @@ case class CreateTable(table: CatalogTable, ifNotExists: Boolean) extends Runnab * ALTER VIEW view1 RENAME TO view2; * }}} */ -case class AlterTableRename( +case class AlterTableRenameCommand( oldName: TableIdentifier, newName: TableIdentifier, isView: Boolean) @@ -159,7 +159,7 @@ case class AlterTableRename( * [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ -case class LoadData( +case class LoadDataCommand( table: TableIdentifier, path: String, isLocal: Boolean, @@ -281,7 +281,7 @@ case class LoadData( * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ -case class TruncateTable( +case class TruncateTableCommand( tableName: TableIdentifier, partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand { @@ -444,16 +444,17 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF append(buffer, "Sort Columns:", sortColumns.mkString("[", ", ", "]"), "") } - DDLUtils.getBucketSpecFromTableProperties(metadata).map { bucketSpec => - appendBucketInfo( - bucketSpec.numBuckets, - bucketSpec.bucketColumnNames, - bucketSpec.sortColumnNames) - }.getOrElse { - appendBucketInfo( - metadata.numBuckets, - metadata.bucketColumnNames, - metadata.sortColumnNames) + DDLUtils.getBucketSpecFromTableProperties(metadata) match { + case Some(bucketSpec) => + appendBucketInfo( + bucketSpec.numBuckets, + bucketSpec.bucketColumnNames, + bucketSpec.sortColumnNames) + case None => + appendBucketInfo( + metadata.numBuckets, + metadata.bucketColumnNames, + metadata.sortColumnNames) } } 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 2e17b763a5370..e5dd4d81d6779 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 @@ -433,7 +433,7 @@ case class DataSource( // ordering of data.logicalPlan (partition columns are all moved after data column). This // will be adjusted within InsertIntoHadoopFsRelation. val plan = - InsertIntoHadoopFsRelation( + InsertIntoHadoopFsRelationCommand( outputPath, partitionColumns.map(UnresolvedAttribute.quoted), bucketSpec, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 0494fafb0e424..a3d87cd38bb8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -66,7 +66,7 @@ private[sql] object DataSourceAnalysis extends Rule[LogicalPlan] { "Cannot overwrite a path that is also being read from.") } - InsertIntoHadoopFsRelation( + InsertIntoHadoopFsRelationCommand( outputPath, t.partitionSchema.fields.map(_.name).map(UnresolvedAttribute(_)), t.bucketSpec, @@ -153,7 +153,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), part, query, overwrite, false) if part.isEmpty => - ExecutedCommandExec(InsertIntoDataSource(l, query, overwrite)) :: Nil + ExecutedCommandExec(InsertIntoDataSourceCommand(l, query, overwrite)) :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index 7b15e496414e3..c3e07f7d00557 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.sources.InsertableRelation /** * Inserts the results of `query` in to a relation that extends [[InsertableRelation]]. */ -private[sql] case class InsertIntoDataSource( +private[sql] case class InsertIntoDataSourceCommand( logicalRelation: LogicalRelation, query: LogicalPlan, overwrite: Boolean) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala similarity index 93% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 4921e4ca6bb7d..1426dcf4697ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -35,11 +35,11 @@ import org.apache.spark.sql.internal.SQLConf /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. - * Writing to dynamic partitions is also supported. Each [[InsertIntoHadoopFsRelation]] issues a - * single write job, and owns a UUID that identifies this job. Each concrete implementation of - * [[HadoopFsRelation]] should use this UUID together with task id to generate unique file path for - * each task output file. This UUID is passed to executor side via a property named - * `spark.sql.sources.writeJobUUID`. + * Writing to dynamic partitions is also supported. Each [[InsertIntoHadoopFsRelationCommand]] + * issues a single write job, and owns a UUID that identifies this job. Each concrete + * implementation of [[HadoopFsRelation]] should use this UUID together with task id to generate + * unique file path for each task output file. This UUID is passed to executor side via a + * property named `spark.sql.sources.writeJobUUID`. * * Different writer containers, [[DefaultWriterContainer]] and [[DynamicPartitionWriterContainer]] * are used to write to normal tables and tables with dynamic partitions. @@ -55,7 +55,7 @@ import org.apache.spark.sql.internal.SQLConf * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is * thrown during job commitment, also aborts the job. */ -private[sql] case class InsertIntoHadoopFsRelation( +private[sql] case class InsertIntoHadoopFsRelationCommand( outputPath: Path, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], 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 78b1db16826e2..edbccde214c61 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 @@ -91,7 +91,7 @@ case class CreateTempTableUsing( } } -case class CreateTempTableUsingAsSelect( +case class CreateTempTableUsingAsSelectCommand( tableIdent: TableIdentifier, provider: String, partitionColumns: Array[String], 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 8f7c6f5d0ca44..939b9195cae99 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.command.AnalyzeTable +import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, FindDataSourceTable, PreInsertCastAndRename, ResolveDataSource} import org.apache.spark.sql.util.ExecutionListenerManager @@ -189,6 +189,6 @@ private[sql] class SessionState(sparkSession: SparkSession) { * in the external catalog. */ def analyze(tableName: String): Unit = { - AnalyzeTable(tableName).run(sparkSession) + AnalyzeTableCommand(tableName).run(sparkSession) } } 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 0925a51310e69..708b878c843a7 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 @@ -48,7 +48,7 @@ class DDLCommandSuite extends PlanTest { |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') """.stripMargin val parsed = parser.parsePlan(sql) - val expected = CreateDatabase( + val expected = CreateDatabaseCommand( "database_name", ifNotExists = true, Some("/home/user/db"), @@ -82,19 +82,19 @@ class DDLCommandSuite extends PlanTest { val parsed6 = parser.parsePlan(sql6) val parsed7 = parser.parsePlan(sql7) - val expected1 = DropDatabase( + val expected1 = DropDatabaseCommand( "database_name", ifExists = true, cascade = false) - val expected2 = DropDatabase( + val expected2 = DropDatabaseCommand( "database_name", ifExists = true, cascade = true) - val expected3 = DropDatabase( + val expected3 = DropDatabaseCommand( "database_name", ifExists = false, cascade = false) - val expected4 = DropDatabase( + val expected4 = DropDatabaseCommand( "database_name", ifExists = false, cascade = true) @@ -116,10 +116,10 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterDatabaseProperties( + val expected1 = AlterDatabasePropertiesCommand( "database_name", Map("a" -> "a", "b" -> "b", "c" -> "c")) - val expected2 = AlterDatabaseProperties( + val expected2 = AlterDatabasePropertiesCommand( "database_name", Map("a" -> "a")) @@ -141,10 +141,10 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = DescribeDatabase( + val expected1 = DescribeDatabaseCommand( "db_name", extended = true) - val expected2 = DescribeDatabase( + val expected2 = DescribeDatabaseCommand( "db_name", extended = false) @@ -167,7 +167,7 @@ class DDLCommandSuite extends PlanTest { """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = CreateFunction( + val expected1 = CreateFunctionCommand( None, "helloworld", "com.matthewrathbone.example.SimpleUDFExample", @@ -175,7 +175,7 @@ class DDLCommandSuite extends PlanTest { FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"), FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")), isTemp = true) - val expected2 = CreateFunction( + val expected2 = CreateFunctionCommand( Some("hello"), "world", "com.matthewrathbone.example.SimpleUDFExample", @@ -198,22 +198,22 @@ class DDLCommandSuite extends PlanTest { val parsed3 = parser.parsePlan(sql3) val parsed4 = parser.parsePlan(sql4) - val expected1 = DropFunction( + val expected1 = DropFunctionCommand( None, "helloworld", ifExists = false, isTemp = true) - val expected2 = DropFunction( + val expected2 = DropFunctionCommand( None, "helloworld", ifExists = true, isTemp = true) - val expected3 = DropFunction( + val expected3 = DropFunctionCommand( Some("hello"), "world", ifExists = false, isTemp = false) - val expected4 = DropFunction( + val expected4 = DropFunctionCommand( Some("hello"), "world", ifExists = true, @@ -231,11 +231,11 @@ class DDLCommandSuite extends PlanTest { containsThesePhrases = Seq("create external table", "location")) val query = "CREATE EXTERNAL TABLE my_tab LOCATION '/something/anything'" parser.parsePlan(query) match { - case ct: CreateTable => + case ct: CreateTableCommand => assert(ct.table.tableType == CatalogTableType.EXTERNAL) assert(ct.table.storage.locationUri == Some("/something/anything")) case other => - fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -253,11 +253,11 @@ class DDLCommandSuite extends PlanTest { test("create table - location implies external") { val query = "CREATE TABLE my_tab LOCATION '/something/anything'" parser.parsePlan(query) match { - case ct: CreateTable => + case ct: CreateTableCommand => assert(ct.table.tableType == CatalogTableType.EXTERNAL) assert(ct.table.storage.locationUri == Some("/something/anything")) case other => - fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -282,7 +282,7 @@ class DDLCommandSuite extends PlanTest { 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," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -308,7 +308,7 @@ class DDLCommandSuite extends PlanTest { assert(ct.partitionColumns.isEmpty) comparePlans(ct.copy(partitionColumns = null), expected) case other => - fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -320,11 +320,11 @@ class DDLCommandSuite extends PlanTest { val sql_view = sql_table.replace("TABLE", "VIEW") val parsed_table = parser.parsePlan(sql_table) val parsed_view = parser.parsePlan(sql_view) - val expected_table = AlterTableRename( + val expected_table = AlterTableRenameCommand( TableIdentifier("table_name", None), TableIdentifier("new_table_name", None), isView = false) - val expected_view = AlterTableRename( + val expected_view = AlterTableRenameCommand( TableIdentifier("table_name", None), TableIdentifier("new_table_name", None), isView = true) @@ -353,11 +353,11 @@ class DDLCommandSuite extends PlanTest { val parsed3_view = parser.parsePlan(sql3_view) val tableIdent = TableIdentifier("table_name", None) - val expected1_table = AlterTableSetProperties( + val expected1_table = AlterTableSetPropertiesCommand( tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false) - val expected2_table = AlterTableUnsetProperties( + val expected2_table = AlterTableUnsetPropertiesCommand( tableIdent, Seq("comment", "test"), ifExists = false, isView = false) - val expected3_table = AlterTableUnsetProperties( + val expected3_table = AlterTableUnsetPropertiesCommand( tableIdent, Seq("comment", "test"), ifExists = true, isView = false) val expected1_view = expected1_table.copy(isView = true) val expected2_view = expected2_table.copy(isView = true) @@ -412,21 +412,21 @@ class DDLCommandSuite extends PlanTest { val parsed4 = parser.parsePlan(sql4) val parsed5 = parser.parsePlan(sql5) val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableSerDeProperties( + val expected1 = AlterTableSerDePropertiesCommand( tableIdent, Some("org.apache.class"), None, None) - val expected2 = AlterTableSerDeProperties( + val expected2 = AlterTableSerDePropertiesCommand( tableIdent, Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) - val expected3 = AlterTableSerDeProperties( + val expected3 = AlterTableSerDePropertiesCommand( tableIdent, None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) - val expected4 = AlterTableSerDeProperties( + val expected4 = AlterTableSerDePropertiesCommand( tableIdent, Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), Some(Map("test" -> null, "dt" -> "2008-08-08", "country" -> "us"))) - val expected5 = AlterTableSerDeProperties( + val expected5 = AlterTableSerDePropertiesCommand( tableIdent, None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), @@ -459,13 +459,13 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableAddPartition( + val expected1 = AlterTableAddPartitionCommand( TableIdentifier("table_name", None), Seq( (Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")), (Map("dt" -> "2009-09-09", "country" -> "uk"), None)), ifNotExists = true) - val expected2 = AlterTableAddPartition( + val expected2 = AlterTableAddPartitionCommand( TableIdentifier("table_name", None), Seq((Map("dt" -> "2008-08-08"), Some("loc"))), ifNotExists = false) @@ -490,7 +490,7 @@ class DDLCommandSuite extends PlanTest { |RENAME TO PARTITION (dt='2008-09-09', country='uk') """.stripMargin val parsed = parser.parsePlan(sql) - val expected = AlterTableRenamePartition( + val expected = AlterTableRenamePartitionCommand( TableIdentifier("table_name", None), Map("dt" -> "2008-08-08", "country" -> "us"), Map("dt" -> "2008-09-09", "country" -> "uk")) @@ -529,7 +529,7 @@ class DDLCommandSuite extends PlanTest { assertUnsupported(sql2_view) val tableIdent = TableIdentifier("table_name", None) - val expected1_table = AlterTableDropPartition( + val expected1_table = AlterTableDropPartitionCommand( tableIdent, Seq( Map("dt" -> "2008-08-08", "country" -> "us"), @@ -565,11 +565,11 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableSetLocation( + val expected1 = AlterTableSetLocationCommand( tableIdent, None, "new location") - val expected2 = AlterTableSetLocation( + val expected2 = AlterTableSetLocationCommand( tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")), "new location") @@ -676,13 +676,13 @@ class DDLCommandSuite extends PlanTest { assertUnsupported(s"DROP TABLE IF EXISTS $tableName2 PURGE") val expected1 = - DropTable(TableIdentifier("tab", Option("db")), ifExists = false, isView = false) + DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = false, isView = false) val expected2 = - DropTable(TableIdentifier("tab", Option("db")), ifExists = true, isView = false) + DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = true, isView = false) val expected3 = - DropTable(TableIdentifier("tab", None), ifExists = false, isView = false) + DropTableCommand(TableIdentifier("tab", None), ifExists = false, isView = false) val expected4 = - DropTable(TableIdentifier("tab", None), ifExists = true, isView = false) + DropTableCommand(TableIdentifier("tab", None), ifExists = true, isView = false) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -700,13 +700,13 @@ class DDLCommandSuite extends PlanTest { val parsed4 = parser.parsePlan(s"DROP VIEW IF EXISTS $viewName2") val expected1 = - DropTable(TableIdentifier("view", Option("db")), ifExists = false, isView = true) + DropTableCommand(TableIdentifier("view", Option("db")), ifExists = false, isView = true) val expected2 = - DropTable(TableIdentifier("view", Option("db")), ifExists = true, isView = true) + DropTableCommand(TableIdentifier("view", Option("db")), ifExists = true, isView = true) val expected3 = - DropTable(TableIdentifier("view", None), ifExists = false, isView = true) + DropTableCommand(TableIdentifier("view", None), ifExists = false, isView = true) val expected4 = - DropTable(TableIdentifier("view", None), ifExists = true, isView = true) + DropTableCommand(TableIdentifier("view", None), ifExists = true, isView = true) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) 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 4c528fbbbeef7..86ab152402625 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 @@ -485,7 +485,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) - execution.CreateTableAsSelect( + execution.CreateTableAsSelectCommand( desc.copy(identifier = TableIdentifier(tblName, Some(dbName))), child, allowExisting) 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/CreateTableAsSelectCommand.scala similarity index 99% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala index 9dfbafae872f6..3fc900961e645 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/CreateTableAsSelectCommand.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.hive.MetastoreRelation * raise exception */ private[hive] -case class CreateTableAsSelect( +case class CreateTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, allowExisting: Boolean) 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 2d8b1f325ac32..30ad392969b4e 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Generate, ScriptTransformation} -import org.apache.spark.sql.execution.command.{CreateTable, CreateTableAsSelectLogicalPlan, CreateTableLike, CreateViewCommand, LoadData} +import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.hive.test.TestHive class HiveDDLCommandSuite extends PlanTest { @@ -37,7 +37,7 @@ class HiveDDLCommandSuite extends PlanTest { private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { parser.parsePlan(sql).collect { - case CreateTable(desc, allowExisting) => (desc, allowExisting) + case CreateTableCommand(desc, allowExisting) => (desc, allowExisting) case CreateTableAsSelectLogicalPlan(desc, _, allowExisting) => (desc, allowExisting) case CreateViewCommand(desc, _, allowExisting, _, _, _) => (desc, allowExisting) }.head @@ -555,7 +555,7 @@ class HiveDDLCommandSuite extends PlanTest { test("create table like") { val v1 = "CREATE TABLE table1 LIKE table2" val (target, source, exists) = parser.parsePlan(v1).collect { - case CreateTableLike(t, s, allowExisting) => (t, s, allowExisting) + case CreateTableLikeCommand(t, s, allowExisting) => (t, s, allowExisting) }.head assert(exists == false) assert(target.database.isEmpty) @@ -565,7 +565,7 @@ class HiveDDLCommandSuite extends PlanTest { val v2 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2" val (target2, source2, exists2) = parser.parsePlan(v2).collect { - case CreateTableLike(t, s, allowExisting) => (t, s, allowExisting) + case CreateTableLikeCommand(t, s, allowExisting) => (t, s, allowExisting) }.head assert(exists2) assert(target2.database.isEmpty) @@ -577,7 +577,7 @@ class HiveDDLCommandSuite extends PlanTest { 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) + case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) }.head assert(table.database.isEmpty) assert(table.table == "table1") @@ -588,7 +588,7 @@ class HiveDDLCommandSuite extends PlanTest { val v2 = "LOAD DATA LOCAL INPATH 'path' OVERWRITE INTO TABLE table1 PARTITION(c='1', d='2')" val (table2, path2, isLocal2, isOverwrite2, partition2) = parser.parsePlan(v2).collect { - case LoadData(t, path, l, o, partition) => (t, path, l, o, partition) + case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) }.head assert(table2.database.isEmpty) assert(table2.table == "table1") 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 7011cd81221c5..1a7b6c0112279 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 @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.command.AnalyzeTable +import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -33,7 +33,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { val parsed = hiveContext.parseSql(analyzeCommand) val operators = parsed.collect { - case a: AnalyzeTable => a + case a: AnalyzeTableCommand => a case o => o } @@ -49,23 +49,23 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { assertAnalyzeCommand( "ANALYZE TABLE Table1 COMPUTE STATISTICS", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS noscan", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 COMPUTE STATISTICS nOscAn", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) } ignore("analyze MetastoreRelations") { 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 7fe158c218461..3e5140fe578de 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 @@ -23,7 +23,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.command.ExecutedCommandExec -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand, LogicalRelation} import org.apache.spark.sql.hive.execution.HiveTableScanExec import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -308,10 +308,10 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt") df.queryExecution.sparkPlan match { - case ExecutedCommandExec(_: InsertIntoHadoopFsRelation) => // OK + case ExecutedCommandExec(_: InsertIntoHadoopFsRelationCommand) => // OK case o => fail("test_insert_parquet should be converted to a " + s"${classOf[HadoopFsRelation ].getCanonicalName} and " + - s"${classOf[InsertIntoDataSource].getCanonicalName} is expected as the SparkPlan. " + + s"${classOf[InsertIntoDataSourceCommand].getCanonicalName} should have been SparkPlan. " + s"However, found a ${o.toString} ") } @@ -338,10 +338,10 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array") df.queryExecution.sparkPlan match { - case ExecutedCommandExec(_: InsertIntoHadoopFsRelation) => // OK + case ExecutedCommandExec(_: InsertIntoHadoopFsRelationCommand) => // OK case o => fail("test_insert_parquet should be converted to a " + s"${classOf[HadoopFsRelation ].getCanonicalName} and " + - s"${classOf[InsertIntoDataSource].getCanonicalName} is expected as the SparkPlan." + + s"${classOf[InsertIntoDataSourceCommand].getCanonicalName} should have been SparkPlan." + s"However, found a ${o.toString} ") } From 2cbe96e64d5f84474b2eb59bed9ce3ab543d8aff Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 20 May 2016 09:38:34 -0700 Subject: [PATCH 065/143] [SPARK-15400][SQL] CreateNamedStruct and CreateNamedStructUnsafe should preserve metadata of value expressions if it is NamedExpression. ## What changes were proposed in this pull request? `CreateNamedStruct` and `CreateNamedStructUnsafe` should preserve metadata of value expressions if it is `NamedExpression` like `CreateStruct` or `CreateStructUnsafe` are doing. ## How was this patch tested? Existing tests. Author: Takuya UESHIN Closes #13193 from ueshin/issues/SPARK-15400. --- .../expressions/complexTypeCreator.scala | 17 ++++++++++++----- .../expressions/ComplexTypeSuite.scala | 18 ++++++++++++++++++ 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index d986d9dca6ed4..f60d278c54873 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -252,9 +252,13 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { private lazy val names = nameExprs.map(_.eval(EmptyRow)) override lazy val dataType: StructType = { - val fields = names.zip(valExprs).map { case (name, valExpr) => - StructField(name.asInstanceOf[UTF8String].toString, - valExpr.dataType, valExpr.nullable, Metadata.empty) + val fields = names.zip(valExprs).map { + case (name, valExpr: NamedExpression) => + StructField(name.asInstanceOf[UTF8String].toString, + valExpr.dataType, valExpr.nullable, valExpr.metadata) + case (name, valExpr) => + StructField(name.asInstanceOf[UTF8String].toString, + valExpr.dataType, valExpr.nullable, Metadata.empty) } StructType(fields) } @@ -365,8 +369,11 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression private lazy val names = nameExprs.map(_.eval(EmptyRow).toString) override lazy val dataType: StructType = { - val fields = names.zip(valExprs).map { case (name, valExpr) => - StructField(name, valExpr.dataType, valExpr.nullable, Metadata.empty) + val fields = names.zip(valExprs).map { + case (name, valExpr: NamedExpression) => + StructField(name, valExpr.dataType, valExpr.nullable, valExpr.metadata) + case (name, valExpr) => + StructField(name, valExpr.dataType, valExpr.nullable, Metadata.empty) } StructType(fields) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 7c009a7360b6f..ec7be4d4b849d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -228,4 +228,22 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { checkErrorMessage(structType, IntegerType, "Field name should be String Literal") checkErrorMessage(otherType, StringType, "Can't extract value from") } + + test("ensure to preserve metadata") { + val metadata = new MetadataBuilder() + .putString("key", "value") + .build() + + def checkMetadata(expr: Expression): Unit = { + assert(expr.dataType.asInstanceOf[StructType]("a").metadata === metadata) + assert(expr.dataType.asInstanceOf[StructType]("b").metadata === Metadata.empty) + } + + val a = AttributeReference("a", IntegerType, metadata = metadata)() + val b = AttributeReference("b", IntegerType)() + checkMetadata(CreateStruct(Seq(a, b))) + checkMetadata(CreateNamedStruct(Seq("a", a, "b", b))) + checkMetadata(CreateStructUnsafe(Seq(a, b))) + checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) + } } From fe2fcb48039ac897242e2cfaed31703fa6116db7 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 20 May 2016 10:27:41 -0700 Subject: [PATCH 066/143] [SPARK-15360][SPARK-SUBMIT] Should print spark-submit usage when no arguments is specified (Please fill in changes proposed in this fix) In 2.0, ./bin/spark-submit doesn't print out usage, but it raises an exception. In this PR, an exception handling is added in the Main.java when the exception is thrown. In the handling code, if there is no additional argument, it prints out usage. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually tested. ./bin/spark-submit Usage: spark-submit [options] [app arguments] Usage: spark-submit --kill [submission ID] --master [spark://...] Usage: spark-submit --status [submission ID] --master [spark://...] Usage: spark-submit run-example [options] example-class [example args] Options: --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or on one of the worker machines inside the cluster ("cluster") (Default: client). --class CLASS_NAME Your application's main class (for Java / Scala apps). --name NAME A name of your application. --jars JARS Comma-separated list of local jars to include on the driver and executor classpaths. --packages Comma-separated list of maven coordinates of jars to include on the driver and executor classpaths. Will search the local maven repo, then maven central and any additional remote repositories given by --repositories. The format for the coordinates should be groupId:artifactId:version. Author: wm624@hotmail.com Closes #13163 from wangmiao1981/submit. --- .../launcher/SparkSubmitCommandBuilder.java | 47 +++++++++++-------- .../SparkSubmitCommandBuilderSuite.java | 13 +++++ 2 files changed, 41 insertions(+), 19 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 76897c4f75199..824500d980eee 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -107,28 +107,37 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { SparkSubmitCommandBuilder(List args) { this.allowsMixedArguments = false; - + this.sparkArgs = new ArrayList<>(); boolean isExample = false; List submitArgs = args; - if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { - this.allowsMixedArguments = true; - appResource = PYSPARK_SHELL; - submitArgs = args.subList(1, args.size()); - } else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) { - this.allowsMixedArguments = true; - appResource = SPARKR_SHELL; - submitArgs = args.subList(1, args.size()); - } else if (args.size() > 0 && args.get(0).equals(RUN_EXAMPLE)) { - isExample = true; - submitArgs = args.subList(1, args.size()); - } - this.sparkArgs = new ArrayList<>(); - this.isExample = isExample; + if (args.size() > 0) { + switch (args.get(0)) { + case PYSPARK_SHELL: + this.allowsMixedArguments = true; + appResource = PYSPARK_SHELL; + submitArgs = args.subList(1, args.size()); + break; + + case SPARKR_SHELL: + this.allowsMixedArguments = true; + appResource = SPARKR_SHELL; + submitArgs = args.subList(1, args.size()); + break; + + case RUN_EXAMPLE: + isExample = true; + submitArgs = args.subList(1, args.size()); + } - OptionParser parser = new OptionParser(); - parser.parse(submitArgs); - this.printInfo = parser.infoRequested; + this.isExample = isExample; + OptionParser parser = new OptionParser(); + parser.parse(submitArgs); + this.printInfo = parser.infoRequested; + } else { + this.isExample = isExample; + this.printInfo = true; + } } @Override @@ -147,7 +156,7 @@ List buildSparkSubmitArgs() { List args = new ArrayList<>(); SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); - if (!allowsMixedArguments) { + if (!allowsMixedArguments && !printInfo) { checkArgument(appResource != null, "Missing application resource."); } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index c16f46a360277..ffe11bc51b722 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -58,6 +58,19 @@ public void testClusterCmdBuilder() throws Exception { testCmdBuilder(false, false); } + @Test + public void testCliHelpAndNoArg() throws Exception { + List helpArgs = Arrays.asList(parser.HELP); + Map env = new HashMap<>(); + List cmd = buildCommand(helpArgs, env); + assertTrue("--help should be contained in the final cmd.", cmd.contains(parser.HELP)); + + List sparkEmptyArgs = Collections.emptyList(); + cmd = buildCommand(sparkEmptyArgs, env); + assertTrue("org.apache.spark.deploy.SparkSubmit should be contained in the final cmd of empty input.", + cmd.contains("org.apache.spark.deploy.SparkSubmit")); + } + @Test public void testCliParser() throws Exception { List sparkSubmitArgs = Arrays.asList( From 5a25cd4ff3eccbbb4653e1fb43dd9026bb2df331 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 20 May 2016 10:44:26 -0700 Subject: [PATCH 067/143] [HOTFIX] disable stress test --- .../apache/spark/sql/execution/joins/HashedRelationSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 a5b56541c90f7..40864c80ebc81 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 @@ -213,7 +213,8 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { longRelation.close() } - test("build HashedRelation with more than 100 millions rows") { + // This test require 4G heap to run, should run it manually + ignore("build HashedRelation with more than 100 millions rows") { val unsafeProj = UnsafeProjection.create( Seq(BoundReference(0, IntegerType, false), BoundReference(1, StringType, true))) From 22947cd0213856442025baf653be588c6c707e36 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 20 May 2016 10:56:35 -0700 Subject: [PATCH 068/143] [SPARK-15165] [SPARK-15205] [SQL] Introduce place holder for comments in generated code ## What changes were proposed in this pull request? This PR introduce place holder for comment in generated code and the purpose is same for #12939 but much safer. Generated code to be compiled doesn't include actual comments but includes place holder instead. Place holders in generated code will be replaced with actual comments only at the time of logging. Also, this PR can resolve SPARK-15205. ## How was this patch tested? Existing tests. Author: Kousuke Saruta Closes #12979 from sarutak/SPARK-15205. --- .../sql/catalyst/expressions/Expression.scala | 6 +- .../expressions/codegen/CodeFormatter.scala | 12 +++- .../expressions/codegen/CodeGenerator.scala | 59 +++++++++++++++++-- .../expressions/codegen/CodegenFallback.scala | 6 +- .../codegen/GenerateMutableProjection.scala | 3 +- .../codegen/GenerateOrdering.scala | 3 +- .../codegen/GeneratePredicate.scala | 3 +- .../codegen/GenerateSafeProjection.scala | 3 +- .../codegen/GenerateUnsafeProjection.scala | 3 +- .../codegen/GenerateUnsafeRowJoiner.scala | 4 +- .../spark/sql/catalyst/util/package.scala | 21 ------- .../codegen/CodeFormatterSuite.scala | 5 +- .../spark/sql/execution/ExistingRDD.scala | 3 +- .../sql/execution/WholeStageCodegenExec.scala | 16 +++-- .../columnar/GenerateColumnAccessor.scala | 5 +- 15 files changed, 95 insertions(+), 57 deletions(-) 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 fab163476f5d1..b4fe151f277a2 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 @@ -21,7 +21,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.types._ //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -97,15 +96,14 @@ abstract class Expression extends TreeNode[Expression] { ctx.subExprEliminationExprs.get(this).map { subExprState => // This expression is repeated which means that the code to evaluate it has already been added // as a function before. In that case, we just re-use it. - val code = s"/* ${toCommentSafeString(this.toString)} */" - ExprCode(code, subExprState.isNull, subExprState.value) + ExprCode(ctx.registerComment(this.toString), subExprState.isNull, subExprState.value) }.getOrElse { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") val ve = doGenCode(ctx, ExprCode("", isNull, value)) if (ve.code.nonEmpty) { // Add `this` in the comment. - ve.copy(s"/* ${toCommentSafeString(this.toString)} */\n" + ve.code.trim) + ve.copy(code = s"${ctx.registerComment(this.toString)}\n" + ve.code.trim) } else { ve } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index ab4831f7abdd0..c7410925da6c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.codegen +import org.apache.commons.lang3.StringUtils + /** * An utility class that indents a block of code based on the curly braces and parentheses. * This is used to prettify generated code when in debug mode (or exceptions). @@ -24,7 +26,15 @@ package org.apache.spark.sql.catalyst.expressions.codegen * Written by Matei Zaharia. */ object CodeFormatter { - def format(code: String): String = new CodeFormatter().addLines(code).result() + def format(code: CodeAndComment): String = { + new CodeFormatter().addLines( + StringUtils.replaceEach( + code.body, + code.comment.keys.toArray, + code.comment.values.toArray) + ).result + } + def stripExtraNewLines(input: String): String = { val code = new StringBuilder var lastLine: String = "dummy" 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 67f671926561a..8b74d606dbb26 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 @@ -199,6 +199,11 @@ class CodegenContext { */ var freshNamePrefix = "" + /** + * The map from a place holder to a corresponding comment + */ + private val placeHolderToComments = new mutable.HashMap[String, String] + /** * Returns a term name that is unique within this instance of a `CodegenContext`. */ @@ -706,6 +711,35 @@ class CodegenContext { if (doSubexpressionElimination) subexpressionElimination(expressions) expressions.map(e => e.genCode(this)) } + + /** + * get a map of the pair of a place holder and a corresponding comment + */ + def getPlaceHolderToComments(): collection.Map[String, String] = placeHolderToComments + + /** + * Register a multi-line comment and return the corresponding place holder + */ + private def registerMultilineComment(text: String): String = { + val placeHolder = s"/*${freshName("c")}*/" + val comment = text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") + placeHolderToComments += (placeHolder -> comment) + placeHolder + } + + /** + * Register a comment and return the corresponding place holder + */ + def registerComment(text: String): String = { + if (text.contains("\n") || text.contains("\r")) { + registerMultilineComment(text) + } else { + val placeHolder = s"/*${freshName("c")}*/" + val safeComment = s"// $text" + placeHolderToComments += (placeHolder -> safeComment) + placeHolder + } + } } /** @@ -716,6 +750,19 @@ abstract class GeneratedClass { def generate(references: Array[Any]): Any } +/** + * A wrapper for the source code to be compiled by [[CodeGenerator]]. + */ +class CodeAndComment(val body: String, val comment: collection.Map[String, String]) + extends Serializable { + override def equals(that: Any): Boolean = that match { + case t: CodeAndComment if t.body == body => true + case _ => false + } + + override def hashCode(): Int = body.hashCode +} + /** * A base class for generators of byte code to perform expression evaluation. Includes a set of * helpers for referring to Catalyst types and building trees that perform evaluation of individual @@ -760,14 +807,14 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. */ - def compile(code: String): GeneratedClass = { + def compile(code: CodeAndComment): GeneratedClass = { cache.get(code) } /** * Compile the Java source code into a Java class, using Janino. */ - private[this] def doCompile(code: String): GeneratedClass = { + private[this] def doCompile(code: CodeAndComment): GeneratedClass = { val evaluator = new ClassBodyEvaluator() evaluator.setParentClassLoader(Utils.getContextOrSparkClassLoader) // Cannot be under package codegen, or fail with java.lang.InstantiationException @@ -788,7 +835,7 @@ object CodeGenerator extends Logging { )) evaluator.setExtendedClass(classOf[GeneratedClass]) - def formatted = CodeFormatter.format(code) + lazy val formatted = CodeFormatter.format(code) logDebug({ // Only add extra debugging info to byte code when we are going to print the source code. @@ -797,7 +844,7 @@ object CodeGenerator extends Logging { }) try { - evaluator.cook("generated.java", code) + evaluator.cook("generated.java", code.body) } catch { case e: Exception => val msg = s"failed to compile: $e\n$formatted" @@ -819,8 +866,8 @@ object CodeGenerator extends Logging { private val cache = CacheBuilder.newBuilder() .maximumSize(100) .build( - new CacheLoader[String, GeneratedClass]() { - override def load(code: String): GeneratedClass = { + new CacheLoader[CodeAndComment, GeneratedClass]() { + override def load(code: CodeAndComment): GeneratedClass = { val startTime = System.nanoTime() val result = doCompile(code) val endTime = System.nanoTime() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala index 2bd77c65c31cf..6a5a3e7933eea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression, Nondeterministic} -import org.apache.spark.sql.catalyst.util.toCommentSafeString /** * A trait that can be used to provide a fallback mode for expression code generation. @@ -36,9 +35,10 @@ trait CodegenFallback extends Expression { val idx = ctx.references.length ctx.references += this val objectTerm = ctx.freshName("obj") + val placeHolder = ctx.registerComment(this.toString) if (nullable) { ev.copy(code = s""" - /* expression: ${toCommentSafeString(this.toString)} */ + $placeHolder Object $objectTerm = ((Expression) references[$idx]).eval($input); boolean ${ev.isNull} = $objectTerm == null; ${ctx.javaType(this.dataType)} ${ev.value} = ${ctx.defaultValue(this.dataType)}; @@ -47,7 +47,7 @@ trait CodegenFallback extends Expression { }""") } else { ev.copy(code = s""" - /* expression: ${toCommentSafeString(this.toString)} */ + $placeHolder Object $objectTerm = ((Expression) references[$idx]).eval($input); ${ctx.javaType(this.dataType)} ${ev.value} = (${ctx.boxedType(this.dataType)}) $objectTerm; """, isNull = "false") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index f143b40443836..1305289e783c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -94,7 +94,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP val allProjections = ctx.splitExpressions(ctx.INPUT_ROW, projectionCodes) val allUpdates = ctx.splitExpressions(ctx.INPUT_ROW, updates) - val code = s""" + val codeBody = s""" public java.lang.Object generate(Object[] references) { return new SpecificMutableProjection(references); } @@ -133,6 +133,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP } """ + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) 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 dc4825cdd8765..1c53d62a5e98a 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 @@ -113,7 +113,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR protected def create(ordering: Seq[SortOrder]): BaseOrdering = { val ctx = newCodeGenContext() val comparisons = genComparisons(ctx, ordering) - val code = s""" + val codeBody = s""" public SpecificOrdering generate(Object[] references) { return new SpecificOrdering(references); } @@ -136,6 +136,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR } }""" + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index dd8e2a289a661..ef44e6b46b798 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -40,7 +40,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool protected def create(predicate: Expression): ((InternalRow) => Boolean) = { val ctx = newCodeGenContext() val eval = predicate.genCode(ctx) - val code = s""" + val codeBody = s""" public SpecificPredicate generate(Object[] references) { return new SpecificPredicate(references); } @@ -61,6 +61,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool } }""" + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") val p = CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index ee1a363145dad..b0b121255357a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -155,7 +155,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] """ } val allExpressions = ctx.splitExpressions(ctx.INPUT_ROW, expressionCodes) - val code = s""" + val codeBody = s""" public java.lang.Object generate(Object[] references) { return new SpecificSafeProjection(references); } @@ -181,6 +181,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] } """ + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 6aa9cbf08bdb9..102f276e9b3ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -362,7 +362,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val ctx = newCodeGenContext() val eval = createCode(ctx, expressions, subexpressionEliminationEnabled) - val code = s""" + val codeBody = s""" public java.lang.Object generate(Object[] references) { return new SpecificUnsafeProjection(references); } @@ -390,6 +390,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } """ + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index b1ffbaa3e94ec..4dc1678ff67bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -157,7 +157,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U }.mkString("\n") // ------------------------ Finally, put everything together --------------------------- // - val code = s""" + val codeBody = s""" |public java.lang.Object generate(Object[] references) { | return new SpecificUnsafeRowJoiner(); |} @@ -193,7 +193,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | } |} """.stripMargin - + val code = new CodeAndComment(codeBody, Map.empty) logDebug(s"SpecificUnsafeRowJoiner($schema1, $schema2):\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) 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 f1d6cab9a5a1c..4005087dad05a 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 @@ -155,27 +155,6 @@ package object util { 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. - */ - def toCommentSafeString(str: String): String = { - val len = math.min(str.length, 128) - val suffix = if (str.length > len) "..." else "" - - // 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 implicit class debugLogging(a: Any) { def debugLogging() { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala index f57b82bb96399..6022f2dbbe948 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala @@ -25,11 +25,12 @@ class CodeFormatterSuite extends SparkFunSuite { def testCase(name: String)(input: String)(expected: String): Unit = { test(name) { - if (CodeFormatter.format(input).trim !== expected.trim) { + val sourceCode = new CodeAndComment(input, Map.empty) + if (CodeFormatter.format(sourceCode).trim !== expected.trim) { fail( s""" |== FAIL: Formatted code doesn't match === - |${sideBySide(CodeFormatter.format(input).trim, expected.trim).mkString("\n")} + |${sideBySide(CodeFormatter.format(sourceCode).trim, expected.trim).mkString("\n")} """.stripMargin) } } 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 d8911f88b0004..ec23a9c41a029 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 @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} -import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -253,7 +252,7 @@ private[sql] case class BatchedDataSourceScanExec( val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } val valueVar = ctx.freshName("value") val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = s"/* ${toCommentSafeString(str)} */\n" + (if (nullable) { + val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { s""" boolean ${isNullVar} = ${columnVar}.isNullAt($ordinal); $javaType ${valueVar} = ${isNullVar} ? ${ctx.defaultValue(dataType)} : ($value); 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 d6f7b6ed35dbd..37fdc362b5e4e 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 @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -79,7 +78,7 @@ trait CodegenSupport extends SparkPlan { this.parent = parent ctx.freshNamePrefix = variablePrefix s""" - |/*** PRODUCE: ${toCommentSafeString(this.simpleString)} */ + |${ctx.registerComment(s"PRODUCE: ${this.simpleString}")} |${doProduce(ctx)} """.stripMargin } @@ -147,8 +146,7 @@ trait CodegenSupport extends SparkPlan { ctx.freshNamePrefix = parent.variablePrefix val evaluated = evaluateRequiredVariables(output, inputVars, parent.usedInputs) s""" - | - |/*** CONSUME: ${toCommentSafeString(parent.simpleString)} */ + |${ctx.registerComment(s"CONSUME: ${parent.simpleString}")} |$evaluated |${parent.doConsume(ctx, inputVars, rowVar)} """.stripMargin @@ -299,7 +297,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co * * @return the tuple of the codegen context and the actual generated source. */ - def doCodeGen(): (CodegenContext, String) = { + def doCodeGen(): (CodegenContext, CodeAndComment) = { val ctx = new CodegenContext val code = child.asInstanceOf[CodegenSupport].produce(ctx, this) val source = s""" @@ -307,9 +305,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co return new GeneratedIterator(references); } - /** Codegened pipeline for: - * ${toCommentSafeString(child.treeString.trim)} - */ + ${ctx.registerComment(s"""Codegend pipeline for\n${child.treeString.trim}""")} final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { private Object[] references; @@ -333,7 +329,9 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co """.trim // try to compile, helpful for debug - val cleanedSource = CodeFormatter.stripExtraNewLines(source) + val cleanedSource = + new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments()) + logDebug(s"\n${CodeFormatter.format(cleanedSource)}") CodeGenerator.compile(cleanedSource) (ctx, cleanedSource) 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 bd5cb800dde3c..e0b48119f61dc 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator, UnsafeRowWriter} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, UnsafeRowWriter} import org.apache.spark.sql.types._ /** @@ -150,7 +150,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera (0 to groupedAccessorsLength - 1).map { i => s"extractors$i();" }.mkString("\n")) } - val code = s""" + val codeBody = s""" import java.nio.ByteBuffer; import java.nio.ByteOrder; import scala.collection.Iterator; @@ -224,6 +224,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera } }""" + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] From dfa61f7b136ae060bbe04e3c0da1148da41018c7 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 20 May 2016 12:38:46 -0700 Subject: [PATCH 069/143] [SPARK-15190][SQL] Support using SQLUserDefinedType for case classes ## What changes were proposed in this pull request? Right now inferring the schema for case classes happens before searching the SQLUserDefinedType annotation, so the SQLUserDefinedType annotation for case classes doesn't work. This PR simply changes the inferring order to resolve it. I also reenabled the java.math.BigDecimal test and added two tests for `List`. ## How was this patch tested? `encodeDecodeTest(UDTCaseClass(new java.net.URI("http://spark.apache.org/")), "udt with case class")` Author: Shixiong Zhu Closes #12965 from zsxwing/SPARK-15190. --- .../spark/sql/catalyst/ScalaReflection.scala | 70 +++++++++---------- .../encoders/ExpressionEncoderSuite.scala | 28 +++++++- 2 files changed, 62 insertions(+), 36 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 58df651da2942..36989a20cb5d5 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 @@ -348,6 +348,23 @@ object ScalaReflection extends ScalaReflection { "toScalaMap", keyData :: valueData :: Nil) + case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + val obj = NewInstance( + udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), + Nil, + dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) + Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) + + case t if UDTRegistration.exists(getClassNameFromType(t)) => + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() + .asInstanceOf[UserDefinedType[_]] + val obj = NewInstance( + udt.getClass, + Nil, + dataType = ObjectType(udt.getClass)) + Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) + case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) @@ -388,23 +405,6 @@ object ScalaReflection extends ScalaReflection { } else { newInstance } - - case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() - val obj = NewInstance( - udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), - Nil, - dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) - - case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] - val obj = NewInstance( - udt.getClass, - Nil, - dataType = ObjectType(udt.getClass)) - Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) } } @@ -522,17 +522,6 @@ object ScalaReflection extends ScalaReflection { 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) => - val fieldValue = Invoke(inputObject, fieldName, dataTypeFor(fieldType)) - val clsName = getClassNameFromType(fieldType) - val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath - expressions.Literal(fieldName) :: serializerFor(fieldValue, fieldType, newPath) :: Nil - }) - val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) - expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) - case t if t <:< localTypeOf[Array[_]] => val TypeRef(_, _, Seq(elementType)) = t toCatalystArray(inputObject, elementType) @@ -645,6 +634,17 @@ object ScalaReflection extends ScalaReflection { dataType = ObjectType(udt.getClass)) Invoke(obj, "serialize", udt.sqlType, inputObject :: Nil) + case t if definedByConstructorParams(t) => + val params = getConstructorParameters(t) + val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => + val fieldValue = Invoke(inputObject, fieldName, dataTypeFor(fieldType)) + val clsName = getClassNameFromType(fieldType) + val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath + expressions.Literal(fieldName) :: serializerFor(fieldValue, fieldType, newPath) :: Nil + }) + val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) + expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) + case other => throw new UnsupportedOperationException( s"No Encoder found for $tpe\n" + walkedTypePath.mkString("\n")) @@ -743,13 +743,6 @@ object ScalaReflection extends ScalaReflection { val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if definedByConstructorParams(t) => - val params = getConstructorParameters(t) - Schema(StructType( - params.map { case (fieldName, fieldType) => - val Schema(dataType, nullable) = schemaFor(fieldType) - StructField(fieldName, dataType, nullable) - }), nullable = true) case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) @@ -775,6 +768,13 @@ object ScalaReflection extends ScalaReflection { case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) + case t if definedByConstructorParams(t) => + val params = getConstructorParameters(t) + Schema(StructType( + params.map { case (fieldName, fieldType) => + val Schema(dataType, nullable) = schemaFor(fieldType) + StructField(fieldName, dataType, nullable) + }), nullable = true) case other => throw new UnsupportedOperationException(s"Schema for type $other is not supported") } 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 d4387890b403b..3d97113b52e39 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 @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.ArrayData -import org.apache.spark.sql.types.{ArrayType, Decimal, ObjectType, StructType} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String case class RepeatedStruct(s: Seq[PrimitiveData]) @@ -86,6 +87,25 @@ class JavaSerializable(val value: Int) extends Serializable { } } +/** For testing UDT for a case class */ +@SQLUserDefinedType(udt = classOf[UDTForCaseClass]) +case class UDTCaseClass(uri: java.net.URI) + +class UDTForCaseClass extends UserDefinedType[UDTCaseClass] { + + override def sqlType: DataType = StringType + + override def serialize(obj: UDTCaseClass): UTF8String = { + UTF8String.fromString(obj.uri.toString) + } + + override def userClass: Class[UDTCaseClass] = classOf[UDTCaseClass] + + override def deserialize(datum: Any): UDTCaseClass = datum match { + case uri: UTF8String => UDTCaseClass(new java.net.URI(uri.toString)) + } +} + class ExpressionEncoderSuite extends PlanTest with AnalysisTest { OuterScopes.addOuterScope(this) @@ -147,6 +167,12 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { encodeDecodeTest(Tuple1[Seq[Int]](null), "null seq in tuple") encodeDecodeTest(Tuple1[Map[String, String]](null), "null map in tuple") + encodeDecodeTest(List(1, 2), "list of int") + encodeDecodeTest(List("a", null), "list with String and null") + + encodeDecodeTest( + UDTCaseClass(new java.net.URI("http://spark.apache.org/")), "udt with case class") + // Kryo encoders encodeDecodeTest("hello", "kryo string")(encoderFor(Encoders.kryo[String])) encodeDecodeTest(new KryoSerializable(15), "kryo object")( From 2ba3ff044900d16d5f6331523526f785864c1e62 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 May 2016 13:00:29 -0700 Subject: [PATCH 070/143] =?UTF-8?q?[SPARK-10216][SQL]=20Revert=20"[]=20Avo?= =?UTF-8?q?id=20creating=20empty=20files=20during=20overwrit=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 8d05a7a from #12855, which seems to have caused regressions when working with empty DataFrames. Author: Michael Armbrust Closes #13181 from marmbrus/revert12855. --- .../datasources/WriterContainer.scala | 221 +++++++++--------- .../spark/sql/hive/hiveWriterContainers.scala | 24 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 41 +--- .../sql/sources/HadoopFsRelationTest.scala | 22 +- 4 files changed, 126 insertions(+), 182 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 7e12bbb2128bf..3b064a5bc489f 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,50 +239,48 @@ private[sql] class DefaultWriterContainer( extends BaseWriterContainer(relation, job, isAppend) { def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - if (iterator.hasNext) { - executorSideSetup(taskContext) - val configuration = taskAttemptContext.getConfiguration - configuration.set("spark.sql.sources.output.path", outputPath) - var writer = newOutputWriter(getWorkPath) - writer.initConverter(dataSchema) + 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 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) + } - 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 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() + def abortTask(): Unit = { + try { + if (writer != null) { + writer.close() } + } finally { + super.abortTask() } } } @@ -365,87 +363,84 @@ private[sql] class DynamicPartitionWriterContainer( } def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - 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) - }) + 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.") - // 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) - }) - } + 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) + }) + } - val sortedIterator = sorter.sortedIterator() + val sortedIterator = sorter.sortedIterator() - // 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) + // 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 } - currentWriter.writeInternal(sortedIterator.getValue) - } - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } + currentKey = nextKey.copy() + logDebug(s"Writing partition: $currentKey") - commitTask() - }(catchBlock = { - if (currentWriter != null) { - currentWriter.close() + currentWriter = newOutputWriter(currentKey, getPartitionString) } - abortTask() - }) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } + 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) } } } 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 706fdbc2604fe..794fe264ead5d 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,21 +178,19 @@ private[hive] class SparkHiveWriterContainer( // this function is executed on executor side def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { - 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)) - } + val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() + executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - close() + 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)) } + + 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 b25684562075c..4a55bcc3b19d0 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._ +import org.apache.spark.sql.{QueryTest, _} 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,35 +216,6 @@ 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 a3183f2977223..62998572eaf4b 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, LocalityTestFileSystem} +import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -879,26 +879,6 @@ 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 0e70fd61b4bc92bd744fc44dd3cbe91443207c72 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 20 May 2016 13:21:53 -0700 Subject: [PATCH 071/143] [SPARK-15438][SQL] improve explain of whole stage codegen ## What changes were proposed in this pull request? Currently, the explain of a query with whole-stage codegen looks like this ``` >>> df = sqlCtx.range(1000);df2 = sqlCtx.range(1000);df.join(pyspark.sql.functions.broadcast(df2), 'id').explain() == Physical Plan == WholeStageCodegen : +- Project [id#1L] : +- BroadcastHashJoin [id#1L], [id#4L], Inner, BuildRight, None : :- Range 0, 1, 4, 1000, [id#1L] : +- INPUT +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint])) +- WholeStageCodegen : +- Range 0, 1, 4, 1000, [id#4L] ``` The problem is that the plan looks much different than logical plan, make us hard to understand the plan (especially when the logical plan is not showed together). This PR will change it to: ``` >>> df = sqlCtx.range(1000);df2 = sqlCtx.range(1000);df.join(pyspark.sql.functions.broadcast(df2), 'id').explain() == Physical Plan == *Project [id#0L] +- *BroadcastHashJoin [id#0L], [id#3L], Inner, BuildRight, None :- *Range 0, 1, 4, 1000, [id#0L] +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])) +- *Range 0, 1, 4, 1000, [id#3L] ``` The `*`before the plan means that it's part of whole-stage codegen, it's easy to understand. ## How was this patch tested? Manually ran some queries and check the explain. Author: Davies Liu Closes #13204 from davies/explain_codegen. --- .../spark/sql/catalyst/trees/TreeNode.scala | 57 +++---------------- .../sql/execution/WholeStageCodegenExec.scala | 29 +++++----- .../sql/execution/exchange/Exchange.scala | 3 - 3 files changed, 22 insertions(+), 67 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 5eb8fdf048e28..e8e2a7bbabcd4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -467,50 +467,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } /** - * All the nodes that will be used to generate tree string. - * - * For example: - * - * WholeStageCodegen - * +-- SortMergeJoin - * |-- InputAdapter - * | +-- Sort - * +-- InputAdapter - * +-- Sort - * - * the treeChildren of WholeStageCodegen will be Seq(Sort, Sort), it will generate a tree string - * like this: - * - * WholeStageCodegen - * : +- SortMergeJoin - * : :- INPUT - * : :- INPUT - * :- Sort - * :- Sort - */ - protected def treeChildren: Seq[BaseType] = children - - /** - * All the nodes that are parts of this node. - * - * For example: - * - * WholeStageCodegen - * +- SortMergeJoin - * |-- InputAdapter - * | +-- Sort - * +-- InputAdapter - * +-- Sort - * - * the innerChildren of WholeStageCodegen will be Seq(SortMergeJoin), it will generate a tree - * string like this: - * - * WholeStageCodegen - * : +- SortMergeJoin - * : :- INPUT - * : :- INPUT - * :- Sort - * :- Sort + * All the nodes that are parts of this node, this is used by subquries. */ protected def innerChildren: Seq[BaseType] = Nil @@ -522,7 +479,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { * `lastChildren` for the root node should be empty. */ def generateTreeString( - depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder): StringBuilder = { + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + prefix: String = ""): StringBuilder = { if (depth > 0) { lastChildren.init.foreach { isLast => val prefixFragment = if (isLast) " " else ": " @@ -533,6 +493,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { builder.append(branch) } + builder.append(prefix) builder.append(simpleString) builder.append("\n") @@ -542,9 +503,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { innerChildren.last.generateTreeString(depth + 2, lastChildren :+ false :+ true, builder) } - if (treeChildren.nonEmpty) { - treeChildren.init.foreach(_.generateTreeString(depth + 1, lastChildren :+ false, builder)) - treeChildren.last.generateTreeString(depth + 1, lastChildren :+ true, builder) + if (children.nonEmpty) { + children.init.foreach(_.generateTreeString(depth + 1, lastChildren :+ false, builder, prefix)) + children.last.generateTreeString(depth + 1, lastChildren :+ true, builder, prefix) } builder 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 37fdc362b5e4e..2a1ce735b74ea 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 @@ -245,9 +245,13 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp """.stripMargin } - override def simpleString: String = "INPUT" - - override def treeChildren: Seq[SparkPlan] = Nil + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + prefix: String = ""): StringBuilder = { + child.generateTreeString(depth, lastChildren, builder, "") + } } object WholeStageCodegenExec { @@ -398,20 +402,13 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co """.stripMargin.trim } - override def innerChildren: Seq[SparkPlan] = { - child :: Nil - } - - private def collectInputs(plan: SparkPlan): Seq[SparkPlan] = plan match { - case InputAdapter(c) => c :: Nil - case other => other.children.flatMap(collectInputs) + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + prefix: String = ""): StringBuilder = { + child.generateTreeString(depth, lastChildren, builder, "*") } - - override def treeChildren: Seq[SparkPlan] = { - collectInputs(child) - } - - override def simpleString: String = "WholeStageCodegen" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala index 9da9df617405d..9a9597d3733e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala @@ -60,9 +60,6 @@ case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchan override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { child.executeBroadcast() } - - // Do not repeat the same tree in explain. - override def treeChildren: Seq[SparkPlan] = Nil } /** From dcac8e6f49918a809fb3f2b8bf666582c479a6eb Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 May 2016 14:49:54 -0700 Subject: [PATCH 072/143] [SPARK-15454][SQL] Filter out files starting with _ ## What changes were proposed in this pull request? Many other systems (e.g. Impala) uses _xxx as staging, and Spark should not be reading those files. ## How was this patch tested? Added a unit test case. Author: Reynold Xin Closes #13227 from rxin/SPARK-15454. --- .../execution/datasources/fileSourceInterfaces.scala | 10 +++++----- .../execution/datasources/HadoopFsRelationSuite.scala | 11 +++++++++++ 2 files changed, 16 insertions(+), 5 deletions(-) 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 88125a2b4da78..e0e569bca48b4 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 @@ -341,11 +341,11 @@ private[sql] object HadoopFsRelation extends Logging { /** Checks if we should filter out this path name. */ def shouldFilterOut(pathName: String): Boolean = { - // TODO: We should try to filter out all files/dirs starting with "." or "_". - // The only reason that we are not doing it now is that Parquet needs to find those - // metadata files from leaf files returned by this methods. We should refactor - // this logic to not mix metadata files with data files. - pathName == "_SUCCESS" || pathName == "_temporary" || pathName.startsWith(".") + // We filter everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + (pathName.startsWith("_") || pathName.startsWith(".")) && + !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala index 89d57653adcbd..3c68dc8bb98d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala @@ -39,4 +39,15 @@ class HadoopFsRelationSuite extends QueryTest with SharedSQLContext { assert(df.queryExecution.logical.statistics.sizeInBytes === BigInt(totalSize)) } } + + test("file filtering") { + assert(!HadoopFsRelation.shouldFilterOut("abcd")) + assert(HadoopFsRelation.shouldFilterOut(".ab")) + assert(HadoopFsRelation.shouldFilterOut("_cd")) + + assert(!HadoopFsRelation.shouldFilterOut("_metadata")) + assert(!HadoopFsRelation.shouldFilterOut("_common_metadata")) + assert(HadoopFsRelation.shouldFilterOut("_ab_metadata")) + assert(HadoopFsRelation.shouldFilterOut("_cd_common_metadata")) + } } From a78d6ce376edf2a8836e01f47b9dff5371058d4c Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 20 May 2016 15:19:28 -0700 Subject: [PATCH 073/143] [SPARK-15078] [SQL] Add all TPCDS 1.4 benchmark queries for SparkSQL ## What changes were proposed in this pull request? Now that SparkSQL supports all TPC-DS queries, this patch adds all 99 benchmark queries inside SparkSQL. ## How was this patch tested? Benchmark only Author: Sameer Agarwal Closes #13188 from sameeragarwal/tpcds-all. --- dev/.rat-excludes | 1 + sql/core/src/test/resources/tpcds/q1.sql | 19 + sql/core/src/test/resources/tpcds/q10.sql | 57 + sql/core/src/test/resources/tpcds/q11.sql | 68 + sql/core/src/test/resources/tpcds/q12.sql | 22 + sql/core/src/test/resources/tpcds/q13.sql | 49 + sql/core/src/test/resources/tpcds/q14a.sql | 120 ++ sql/core/src/test/resources/tpcds/q14b.sql | 95 ++ sql/core/src/test/resources/tpcds/q15.sql | 15 + sql/core/src/test/resources/tpcds/q16.sql | 23 + sql/core/src/test/resources/tpcds/q17.sql | 33 + sql/core/src/test/resources/tpcds/q18.sql | 28 + sql/core/src/test/resources/tpcds/q19.sql | 19 + sql/core/src/test/resources/tpcds/q2.sql | 81 ++ sql/core/src/test/resources/tpcds/q20.sql | 18 + sql/core/src/test/resources/tpcds/q21.sql | 25 + sql/core/src/test/resources/tpcds/q22.sql | 14 + sql/core/src/test/resources/tpcds/q23a.sql | 53 + sql/core/src/test/resources/tpcds/q23b.sql | 68 + sql/core/src/test/resources/tpcds/q24a.sql | 34 + sql/core/src/test/resources/tpcds/q24b.sql | 34 + sql/core/src/test/resources/tpcds/q25.sql | 33 + sql/core/src/test/resources/tpcds/q26.sql | 19 + sql/core/src/test/resources/tpcds/q27.sql | 21 + sql/core/src/test/resources/tpcds/q28.sql | 56 + sql/core/src/test/resources/tpcds/q29.sql | 32 + sql/core/src/test/resources/tpcds/q3.sql | 13 + sql/core/src/test/resources/tpcds/q30.sql | 35 + sql/core/src/test/resources/tpcds/q31.sql | 60 + sql/core/src/test/resources/tpcds/q32.sql | 15 + sql/core/src/test/resources/tpcds/q33.sql | 65 + sql/core/src/test/resources/tpcds/q34.sql | 32 + sql/core/src/test/resources/tpcds/q35.sql | 46 + sql/core/src/test/resources/tpcds/q36.sql | 26 + sql/core/src/test/resources/tpcds/q37.sql | 15 + sql/core/src/test/resources/tpcds/q38.sql | 30 + sql/core/src/test/resources/tpcds/q39a.sql | 47 + sql/core/src/test/resources/tpcds/q39b.sql | 48 + sql/core/src/test/resources/tpcds/q4.sql | 120 ++ sql/core/src/test/resources/tpcds/q40.sql | 25 + sql/core/src/test/resources/tpcds/q41.sql | 49 + sql/core/src/test/resources/tpcds/q42.sql | 18 + sql/core/src/test/resources/tpcds/q43.sql | 33 + sql/core/src/test/resources/tpcds/q44.sql | 46 + sql/core/src/test/resources/tpcds/q45.sql | 21 + sql/core/src/test/resources/tpcds/q46.sql | 32 + sql/core/src/test/resources/tpcds/q47.sql | 63 + sql/core/src/test/resources/tpcds/q48.sql | 63 + sql/core/src/test/resources/tpcds/q49.sql | 126 ++ sql/core/src/test/resources/tpcds/q5.sql | 131 ++ sql/core/src/test/resources/tpcds/q50.sql | 47 + sql/core/src/test/resources/tpcds/q51.sql | 55 + sql/core/src/test/resources/tpcds/q52.sql | 14 + sql/core/src/test/resources/tpcds/q53.sql | 30 + sql/core/src/test/resources/tpcds/q54.sql | 61 + sql/core/src/test/resources/tpcds/q55.sql | 13 + sql/core/src/test/resources/tpcds/q56.sql | 65 + sql/core/src/test/resources/tpcds/q57.sql | 56 + sql/core/src/test/resources/tpcds/q58.sql | 59 + sql/core/src/test/resources/tpcds/q59.sql | 75 + sql/core/src/test/resources/tpcds/q6.sql | 21 + sql/core/src/test/resources/tpcds/q60.sql | 62 + sql/core/src/test/resources/tpcds/q61.sql | 33 + sql/core/src/test/resources/tpcds/q62.sql | 35 + sql/core/src/test/resources/tpcds/q63.sql | 31 + sql/core/src/test/resources/tpcds/q64.sql | 92 ++ sql/core/src/test/resources/tpcds/q65.sql | 33 + sql/core/src/test/resources/tpcds/q66.sql | 240 ++++ sql/core/src/test/resources/tpcds/q67.sql | 38 + sql/core/src/test/resources/tpcds/q68.sql | 34 + sql/core/src/test/resources/tpcds/q69.sql | 38 + sql/core/src/test/resources/tpcds/q7.sql | 19 + sql/core/src/test/resources/tpcds/q70.sql | 38 + sql/core/src/test/resources/tpcds/q71.sql | 44 + sql/core/src/test/resources/tpcds/q72.sql | 33 + sql/core/src/test/resources/tpcds/q73.sql | 30 + sql/core/src/test/resources/tpcds/q74.sql | 58 + sql/core/src/test/resources/tpcds/q75.sql | 76 + sql/core/src/test/resources/tpcds/q76.sql | 47 + sql/core/src/test/resources/tpcds/q77.sql | 100 ++ sql/core/src/test/resources/tpcds/q78.sql | 64 + sql/core/src/test/resources/tpcds/q79.sql | 27 + sql/core/src/test/resources/tpcds/q8.sql | 87 ++ sql/core/src/test/resources/tpcds/q80.sql | 94 ++ sql/core/src/test/resources/tpcds/q81.sql | 38 + sql/core/src/test/resources/tpcds/q82.sql | 15 + sql/core/src/test/resources/tpcds/q83.sql | 56 + sql/core/src/test/resources/tpcds/q84.sql | 19 + sql/core/src/test/resources/tpcds/q85.sql | 82 ++ sql/core/src/test/resources/tpcds/q86.sql | 24 + sql/core/src/test/resources/tpcds/q87.sql | 28 + sql/core/src/test/resources/tpcds/q88.sql | 122 ++ sql/core/src/test/resources/tpcds/q89.sql | 30 + sql/core/src/test/resources/tpcds/q9.sql | 48 + sql/core/src/test/resources/tpcds/q90.sql | 19 + sql/core/src/test/resources/tpcds/q91.sql | 23 + sql/core/src/test/resources/tpcds/q92.sql | 16 + sql/core/src/test/resources/tpcds/q93.sql | 19 + sql/core/src/test/resources/tpcds/q94.sql | 23 + sql/core/src/test/resources/tpcds/q95.sql | 29 + sql/core/src/test/resources/tpcds/q96.sql | 11 + sql/core/src/test/resources/tpcds/q97.sql | 30 + sql/core/src/test/resources/tpcds/q98.sql | 21 + sql/core/src/test/resources/tpcds/q99.sql | 34 + .../benchmark/TPCDSQueryBenchmark.scala | 126 ++ .../datasources/parquet/TPCDSBenchmark.scala | 1226 ----------------- 106 files changed, 4858 insertions(+), 1226 deletions(-) create mode 100755 sql/core/src/test/resources/tpcds/q1.sql create mode 100755 sql/core/src/test/resources/tpcds/q10.sql create mode 100755 sql/core/src/test/resources/tpcds/q11.sql create mode 100755 sql/core/src/test/resources/tpcds/q12.sql create mode 100755 sql/core/src/test/resources/tpcds/q13.sql create mode 100755 sql/core/src/test/resources/tpcds/q14a.sql create mode 100755 sql/core/src/test/resources/tpcds/q14b.sql create mode 100755 sql/core/src/test/resources/tpcds/q15.sql create mode 100755 sql/core/src/test/resources/tpcds/q16.sql create mode 100755 sql/core/src/test/resources/tpcds/q17.sql create mode 100755 sql/core/src/test/resources/tpcds/q18.sql create mode 100755 sql/core/src/test/resources/tpcds/q19.sql create mode 100755 sql/core/src/test/resources/tpcds/q2.sql create mode 100755 sql/core/src/test/resources/tpcds/q20.sql create mode 100755 sql/core/src/test/resources/tpcds/q21.sql create mode 100755 sql/core/src/test/resources/tpcds/q22.sql create mode 100755 sql/core/src/test/resources/tpcds/q23a.sql create mode 100755 sql/core/src/test/resources/tpcds/q23b.sql create mode 100755 sql/core/src/test/resources/tpcds/q24a.sql create mode 100755 sql/core/src/test/resources/tpcds/q24b.sql create mode 100755 sql/core/src/test/resources/tpcds/q25.sql create mode 100755 sql/core/src/test/resources/tpcds/q26.sql create mode 100755 sql/core/src/test/resources/tpcds/q27.sql create mode 100755 sql/core/src/test/resources/tpcds/q28.sql create mode 100755 sql/core/src/test/resources/tpcds/q29.sql create mode 100755 sql/core/src/test/resources/tpcds/q3.sql create mode 100755 sql/core/src/test/resources/tpcds/q30.sql create mode 100755 sql/core/src/test/resources/tpcds/q31.sql create mode 100755 sql/core/src/test/resources/tpcds/q32.sql create mode 100755 sql/core/src/test/resources/tpcds/q33.sql create mode 100755 sql/core/src/test/resources/tpcds/q34.sql create mode 100755 sql/core/src/test/resources/tpcds/q35.sql create mode 100755 sql/core/src/test/resources/tpcds/q36.sql create mode 100755 sql/core/src/test/resources/tpcds/q37.sql create mode 100755 sql/core/src/test/resources/tpcds/q38.sql create mode 100755 sql/core/src/test/resources/tpcds/q39a.sql create mode 100755 sql/core/src/test/resources/tpcds/q39b.sql create mode 100755 sql/core/src/test/resources/tpcds/q4.sql create mode 100755 sql/core/src/test/resources/tpcds/q40.sql create mode 100755 sql/core/src/test/resources/tpcds/q41.sql create mode 100755 sql/core/src/test/resources/tpcds/q42.sql create mode 100755 sql/core/src/test/resources/tpcds/q43.sql create mode 100755 sql/core/src/test/resources/tpcds/q44.sql create mode 100755 sql/core/src/test/resources/tpcds/q45.sql create mode 100755 sql/core/src/test/resources/tpcds/q46.sql create mode 100755 sql/core/src/test/resources/tpcds/q47.sql create mode 100755 sql/core/src/test/resources/tpcds/q48.sql create mode 100755 sql/core/src/test/resources/tpcds/q49.sql create mode 100755 sql/core/src/test/resources/tpcds/q5.sql create mode 100755 sql/core/src/test/resources/tpcds/q50.sql create mode 100755 sql/core/src/test/resources/tpcds/q51.sql create mode 100755 sql/core/src/test/resources/tpcds/q52.sql create mode 100755 sql/core/src/test/resources/tpcds/q53.sql create mode 100755 sql/core/src/test/resources/tpcds/q54.sql create mode 100755 sql/core/src/test/resources/tpcds/q55.sql create mode 100755 sql/core/src/test/resources/tpcds/q56.sql create mode 100755 sql/core/src/test/resources/tpcds/q57.sql create mode 100755 sql/core/src/test/resources/tpcds/q58.sql create mode 100755 sql/core/src/test/resources/tpcds/q59.sql create mode 100755 sql/core/src/test/resources/tpcds/q6.sql create mode 100755 sql/core/src/test/resources/tpcds/q60.sql create mode 100755 sql/core/src/test/resources/tpcds/q61.sql create mode 100755 sql/core/src/test/resources/tpcds/q62.sql create mode 100755 sql/core/src/test/resources/tpcds/q63.sql create mode 100755 sql/core/src/test/resources/tpcds/q64.sql create mode 100755 sql/core/src/test/resources/tpcds/q65.sql create mode 100755 sql/core/src/test/resources/tpcds/q66.sql create mode 100755 sql/core/src/test/resources/tpcds/q67.sql create mode 100755 sql/core/src/test/resources/tpcds/q68.sql create mode 100755 sql/core/src/test/resources/tpcds/q69.sql create mode 100755 sql/core/src/test/resources/tpcds/q7.sql create mode 100755 sql/core/src/test/resources/tpcds/q70.sql create mode 100755 sql/core/src/test/resources/tpcds/q71.sql create mode 100755 sql/core/src/test/resources/tpcds/q72.sql create mode 100755 sql/core/src/test/resources/tpcds/q73.sql create mode 100755 sql/core/src/test/resources/tpcds/q74.sql create mode 100755 sql/core/src/test/resources/tpcds/q75.sql create mode 100755 sql/core/src/test/resources/tpcds/q76.sql create mode 100755 sql/core/src/test/resources/tpcds/q77.sql create mode 100755 sql/core/src/test/resources/tpcds/q78.sql create mode 100755 sql/core/src/test/resources/tpcds/q79.sql create mode 100755 sql/core/src/test/resources/tpcds/q8.sql create mode 100755 sql/core/src/test/resources/tpcds/q80.sql create mode 100755 sql/core/src/test/resources/tpcds/q81.sql create mode 100755 sql/core/src/test/resources/tpcds/q82.sql create mode 100755 sql/core/src/test/resources/tpcds/q83.sql create mode 100755 sql/core/src/test/resources/tpcds/q84.sql create mode 100755 sql/core/src/test/resources/tpcds/q85.sql create mode 100755 sql/core/src/test/resources/tpcds/q86.sql create mode 100755 sql/core/src/test/resources/tpcds/q87.sql create mode 100755 sql/core/src/test/resources/tpcds/q88.sql create mode 100755 sql/core/src/test/resources/tpcds/q89.sql create mode 100755 sql/core/src/test/resources/tpcds/q9.sql create mode 100755 sql/core/src/test/resources/tpcds/q90.sql create mode 100755 sql/core/src/test/resources/tpcds/q91.sql create mode 100755 sql/core/src/test/resources/tpcds/q92.sql create mode 100755 sql/core/src/test/resources/tpcds/q93.sql create mode 100755 sql/core/src/test/resources/tpcds/q94.sql create mode 100755 sql/core/src/test/resources/tpcds/q95.sql create mode 100755 sql/core/src/test/resources/tpcds/q96.sql create mode 100755 sql/core/src/test/resources/tpcds/q97.sql create mode 100755 sql/core/src/test/resources/tpcds/q98.sql create mode 100755 sql/core/src/test/resources/tpcds/q99.sql create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 67a241806d7cb..fb582dec56d51 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -98,3 +98,4 @@ spark-deps-.* .*csv .*tsv org.apache.spark.scheduler.ExternalClusterManager +.*\.sql diff --git a/sql/core/src/test/resources/tpcds/q1.sql b/sql/core/src/test/resources/tpcds/q1.sql new file mode 100755 index 0000000000000..4d20faad8ef58 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q1.sql @@ -0,0 +1,19 @@ +WITH customer_total_return AS +( SELECT + sr_customer_sk AS ctr_customer_sk, + sr_store_sk AS ctr_store_sk, + sum(sr_return_amt) AS ctr_total_return + FROM store_returns, date_dim + WHERE sr_returned_date_sk = d_date_sk AND d_year = 2000 + GROUP BY sr_customer_sk, sr_store_sk) +SELECT c_customer_id +FROM customer_total_return ctr1, store, customer +WHERE ctr1.ctr_total_return > + (SELECT avg(ctr_total_return) * 1.2 + FROM customer_total_return ctr2 + WHERE ctr1.ctr_store_sk = ctr2.ctr_store_sk) + AND s_store_sk = ctr1.ctr_store_sk + AND s_state = 'TN' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q10.sql b/sql/core/src/test/resources/tpcds/q10.sql new file mode 100755 index 0000000000000..5500e1aea1552 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q10.sql @@ -0,0 +1,57 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_county IN ('Rush County', 'Toole County', 'Jefferson County', + 'Dona Ana County', 'La Porte County') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3)) +GROUP BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +ORDER BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q11.sql b/sql/core/src/test/resources/tpcds/q11.sql new file mode 100755 index 0000000000000..3618fb14fa39c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q11.sql @@ -0,0 +1,68 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ss_ext_list_price - ss_ext_discount_amt) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id + , c_first_name + , c_last_name + , d_year + , c_preferred_cust_flag + , c_birth_country + , c_login + , c_email_address + , d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ws_ext_list_price - ws_ext_discount_amt) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + GROUP BY + c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, + c_login, c_email_address, d_year) +SELECT t_s_secyear.customer_preferred_cust_flag +FROM year_total t_s_firstyear + , year_total t_s_secyear + , year_total t_w_firstyear + , year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY t_s_secyear.customer_preferred_cust_flag +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q12.sql b/sql/core/src/test/resources/tpcds/q12.sql new file mode 100755 index 0000000000000..0382737f5aa2c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q12.sql @@ -0,0 +1,22 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) AS itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + web_sales, item, date_dim +WHERE + ws_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q13.sql b/sql/core/src/test/resources/tpcds/q13.sql new file mode 100755 index 0000000000000..32dc9e26097ba --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q13.sql @@ -0,0 +1,49 @@ +SELECT + avg(ss_quantity), + avg(ss_ext_sales_price), + avg(ss_ext_wholesale_cost), + sum(ss_ext_wholesale_cost) +FROM store_sales + , store + , customer_demographics + , household_demographics + , customer_address + , date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND ((ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'M' + AND cd_education_status = 'Advanced Degree' + AND ss_sales_price BETWEEN 100.00 AND 150.00 + AND hd_dep_count = 3 +) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'S' + AND cd_education_status = 'College' + AND ss_sales_price BETWEEN 50.00 AND 100.00 + AND hd_dep_count = 1 + ) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'W' + AND cd_education_status = '2 yr Degree' + AND ss_sales_price BETWEEN 150.00 AND 200.00 + AND hd_dep_count = 1 + )) + AND ((ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('TX', 'OH', 'TX') + AND ss_net_profit BETWEEN 100 AND 200 +) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('OR', 'NM', 'KY') + AND ss_net_profit BETWEEN 150 AND 300 + ) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('VA', 'TX', 'MS') + AND ss_net_profit BETWEEN 50 AND 250 + )) diff --git a/sql/core/src/test/resources/tpcds/q14a.sql b/sql/core/src/test/resources/tpcds/q14a.sql new file mode 100755 index 0000000000000..954ddd41be0e6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q14a.sql @@ -0,0 +1,120 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM ( + SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 2001 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT + channel, + i_brand_id, + i_class_id, + i_category_id, + sum(sales), + sum(number_sales) +FROM ( + SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales) + UNION ALL + SELECT + 'catalog' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(cs_quantity * cs_list_price) sales, + count(*) number_sales + FROM catalog_sales, item, date_dim + WHERE cs_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(cs_quantity * cs_list_price) > (SELECT average_sales FROM avg_sales) + UNION ALL + SELECT + 'web' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ws_quantity * ws_list_price) sales, + count(*) number_sales + FROM web_sales, item, date_dim + WHERE ws_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ws_quantity * ws_list_price) > (SELECT average_sales + FROM avg_sales) + ) y +GROUP BY ROLLUP (channel, i_brand_id, i_class_id, i_category_id) +ORDER BY channel, i_brand_id, i_class_id, i_category_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q14b.sql b/sql/core/src/test/resources/tpcds/q14b.sql new file mode 100755 index 0000000000000..929a8484bf9b4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q14b.sql @@ -0,0 +1,95 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM (SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT * +FROM + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 + 1 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) this_year, + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) last_year +WHERE this_year.i_brand_id = last_year.i_brand_id + AND this_year.i_class_id = last_year.i_class_id + AND this_year.i_category_id = last_year.i_category_id +ORDER BY this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q15.sql b/sql/core/src/test/resources/tpcds/q15.sql new file mode 100755 index 0000000000000..b8182e23b0195 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q15.sql @@ -0,0 +1,15 @@ +SELECT + ca_zip, + sum(cs_sales_price) +FROM catalog_sales, customer, customer_address, date_dim +WHERE cs_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND (substr(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', + '85392', '85460', '80348', '81792') + OR ca_state IN ('CA', 'WA', 'GA') + OR cs_sales_price > 500) + AND cs_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip +ORDER BY ca_zip +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q16.sql b/sql/core/src/test/resources/tpcds/q16.sql new file mode 100755 index 0000000000000..732ad0d848071 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q16.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT cs_order_number) AS `order count `, + sum(cs_ext_ship_cost) AS `total shipping cost `, + sum(cs_net_profit) AS `total net profit ` +FROM + catalog_sales cs1, date_dim, customer_address, call_center +WHERE + d_date BETWEEN '2002-02-01' AND (CAST('2002-02-01' AS DATE) + INTERVAL 60 days) + AND cs1.cs_ship_date_sk = d_date_sk + AND cs1.cs_ship_addr_sk = ca_address_sk + AND ca_state = 'GA' + AND cs1.cs_call_center_sk = cc_call_center_sk + AND cc_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + AND EXISTS(SELECT * + FROM catalog_sales cs2 + WHERE cs1.cs_order_number = cs2.cs_order_number + AND cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM catalog_returns cr1 + WHERE cs1.cs_order_number = cr1.cr_order_number) +ORDER BY count(DISTINCT cs_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q17.sql b/sql/core/src/test/resources/tpcds/q17.sql new file mode 100755 index 0000000000000..4d647f7956004 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q17.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_state, + count(ss_quantity) AS store_sales_quantitycount, + avg(ss_quantity) AS store_sales_quantityave, + stddev_samp(ss_quantity) AS store_sales_quantitystdev, + stddev_samp(ss_quantity) / avg(ss_quantity) AS store_sales_quantitycov, + count(sr_return_quantity) as_store_returns_quantitycount, + avg(sr_return_quantity) as_store_returns_quantityave, + stddev_samp(sr_return_quantity) as_store_returns_quantitystdev, + stddev_samp(sr_return_quantity) / avg(sr_return_quantity) AS store_returns_quantitycov, + count(cs_quantity) AS catalog_sales_quantitycount, + avg(cs_quantity) AS catalog_sales_quantityave, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitystdev, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitycov +FROM store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, store, item +WHERE d1.d_quarter_name = '2001Q1' + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') +GROUP BY i_item_id, i_item_desc, s_state +ORDER BY i_item_id, i_item_desc, s_state +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q18.sql b/sql/core/src/test/resources/tpcds/q18.sql new file mode 100755 index 0000000000000..4055c80fdef51 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q18.sql @@ -0,0 +1,28 @@ +SELECT + i_item_id, + ca_country, + ca_state, + ca_county, + avg(cast(cs_quantity AS DECIMAL(12, 2))) agg1, + avg(cast(cs_list_price AS DECIMAL(12, 2))) agg2, + avg(cast(cs_coupon_amt AS DECIMAL(12, 2))) agg3, + avg(cast(cs_sales_price AS DECIMAL(12, 2))) agg4, + avg(cast(cs_net_profit AS DECIMAL(12, 2))) agg5, + avg(cast(c_birth_year AS DECIMAL(12, 2))) agg6, + avg(cast(cd1.cd_dep_count AS DECIMAL(12, 2))) agg7 +FROM catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd1.cd_demo_sk AND + cs_bill_customer_sk = c_customer_sk AND + cd1.cd_gender = 'F' AND + cd1.cd_education_status = 'Unknown' AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_addr_sk = ca_address_sk AND + c_birth_month IN (1, 6, 8, 9, 12, 2) AND + d_year = 1998 AND + ca_state IN ('MS', 'IN', 'ND', 'OK', 'NM', 'VA', 'MS') +GROUP BY ROLLUP (i_item_id, ca_country, ca_state, ca_county) +ORDER BY ca_country, ca_state, ca_county, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q19.sql b/sql/core/src/test/resources/tpcds/q19.sql new file mode 100755 index 0000000000000..e38ab7f2683f4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q19.sql @@ -0,0 +1,19 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + i_manufact_id, + i_manufact, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item, customer, customer_address, store +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 8 + AND d_moy = 11 + AND d_year = 1998 + AND ss_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5) + AND ss_store_sk = s_store_sk +GROUP BY i_brand, i_brand_id, i_manufact_id, i_manufact +ORDER BY ext_price DESC, brand, brand_id, i_manufact_id, i_manufact +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q2.sql b/sql/core/src/test/resources/tpcds/q2.sql new file mode 100755 index 0000000000000..52c0e90c46740 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q2.sql @@ -0,0 +1,81 @@ +WITH wscs AS +( SELECT + sold_date_sk, + sales_price + FROM (SELECT + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + FROM web_sales) x + UNION ALL + (SELECT + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + FROM catalog_sales)), + wswscs AS + ( SELECT + d_week_seq, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN sales_price + ELSE NULL END) + sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN sales_price + ELSE NULL END) + mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN sales_price + ELSE NULL END) + tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN sales_price + ELSE NULL END) + wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN sales_price + ELSE NULL END) + thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN sales_price + ELSE NULL END) + fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN sales_price + ELSE NULL END) + sat_sales + FROM wscs, date_dim + WHERE d_date_sk = sold_date_sk + GROUP BY d_week_seq) +SELECT + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +FROM + (SELECT + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001) y, + (SELECT + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1) z +WHERE d_week_seq1 = d_week_seq2 - 53 +ORDER BY d_week_seq1 diff --git a/sql/core/src/test/resources/tpcds/q20.sql b/sql/core/src/test/resources/tpcds/q20.sql new file mode 100755 index 0000000000000..7ac6c7a75d8ea --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q20.sql @@ -0,0 +1,18 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(cs_ext_sales_price) AS itemrevenue, + sum(cs_ext_sales_price) * 100 / sum(sum(cs_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM catalog_sales, item, date_dim +WHERE cs_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) +AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q21.sql b/sql/core/src/test/resources/tpcds/q21.sql new file mode 100755 index 0000000000000..550881143f809 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q21.sql @@ -0,0 +1,25 @@ +SELECT * +FROM ( + SELECT + w_warehouse_name, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_after + FROM inventory, warehouse, item, date_dim + WHERE i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = inv_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) + GROUP BY w_warehouse_name, i_item_id) x +WHERE (CASE WHEN inv_before > 0 + THEN inv_after / inv_before + ELSE NULL + END) BETWEEN 2.0 / 3.0 AND 3.0 / 2.0 +ORDER BY w_warehouse_name, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q22.sql b/sql/core/src/test/resources/tpcds/q22.sql new file mode 100755 index 0000000000000..add3b41f7c76c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q22.sql @@ -0,0 +1,14 @@ +SELECT + i_product_name, + i_brand, + i_class, + i_category, + avg(inv_quantity_on_hand) qoh +FROM inventory, date_dim, item, warehouse +WHERE inv_date_sk = d_date_sk + AND inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 +GROUP BY ROLLUP (i_product_name, i_brand, i_class, i_category) +ORDER BY qoh, i_product_name, i_brand, i_class, i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q23a.sql b/sql/core/src/test/resources/tpcds/q23a.sql new file mode 100755 index 0000000000000..37791f643375c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q23a.sql @@ -0,0 +1,53 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales, customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT sum(sales) +FROM ((SELECT cs_quantity * cs_list_price sales +FROM catalog_sales, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer)) + UNION ALL + (SELECT ws_quantity * ws_list_price sales + FROM web_sales, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer))) y +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q23b.sql b/sql/core/src/test/resources/tpcds/q23b.sql new file mode 100755 index 0000000000000..01150197af2ba --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q23b.sql @@ -0,0 +1,68 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales + , customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT + c_last_name, + c_first_name, + sales +FROM ((SELECT + c_last_name, + c_first_name, + sum(cs_quantity * cs_list_price) sales +FROM catalog_sales, customer, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer) + AND cs_bill_customer_sk = c_customer_sk +GROUP BY c_last_name, c_first_name) + UNION ALL + (SELECT + c_last_name, + c_first_name, + sum(ws_quantity * ws_list_price) sales + FROM web_sales, customer, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer) + AND ws_bill_customer_sk = c_customer_sk + GROUP BY c_last_name, c_first_name)) y +ORDER BY c_last_name, c_first_name, sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q24a.sql b/sql/core/src/test/resources/tpcds/q24a.sql new file mode 100755 index 0000000000000..bcc189486634d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q24a.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, + i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'pale' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/sql/core/src/test/resources/tpcds/q24b.sql b/sql/core/src/test/resources/tpcds/q24b.sql new file mode 100755 index 0000000000000..830eb670bcdd2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q24b.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, + i_color, i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'chiffon' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/sql/core/src/test/resources/tpcds/q25.sql b/sql/core/src/test/resources/tpcds/q25.sql new file mode 100755 index 0000000000000..a4d78a3c56adc --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q25.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_net_profit) AS store_sales_profit, + sum(sr_net_loss) AS store_returns_loss, + sum(cs_net_profit) AS catalog_sales_profit +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, + store, item +WHERE + d1.d_moy = 4 + AND d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 4 AND 10 + AND d2.d_year = 2001 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_moy BETWEEN 4 AND 10 + AND d3.d_year = 2001 +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 \ No newline at end of file diff --git a/sql/core/src/test/resources/tpcds/q26.sql b/sql/core/src/test/resources/tpcds/q26.sql new file mode 100755 index 0000000000000..6d395a1d791dd --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q26.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 +FROM catalog_sales, customer_demographics, date_dim, item, promotion +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd_demo_sk AND + cs_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q27.sql b/sql/core/src/test/resources/tpcds/q27.sql new file mode 100755 index 0000000000000..b0e2fd95fd159 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q27.sql @@ -0,0 +1,21 @@ +SELECT + i_item_id, + s_state, + grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, store, item +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_store_sk = s_store_sk AND + ss_cdemo_sk = cd_demo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + d_year = 2002 AND + s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_item_id, s_state) +ORDER BY i_item_id, s_state +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q28.sql b/sql/core/src/test/resources/tpcds/q28.sql new file mode 100755 index 0000000000000..f34c2bb0e34e1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q28.sql @@ -0,0 +1,56 @@ +SELECT * +FROM (SELECT + avg(ss_list_price) B1_LP, + count(ss_list_price) B1_CNT, + count(DISTINCT ss_list_price) B1_CNTD +FROM store_sales +WHERE ss_quantity BETWEEN 0 AND 5 + AND (ss_list_price BETWEEN 8 AND 8 + 10 + OR ss_coupon_amt BETWEEN 459 AND 459 + 1000 + OR ss_wholesale_cost BETWEEN 57 AND 57 + 20)) B1, + (SELECT + avg(ss_list_price) B2_LP, + count(ss_list_price) B2_CNT, + count(DISTINCT ss_list_price) B2_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 6 AND 10 + AND (ss_list_price BETWEEN 90 AND 90 + 10 + OR ss_coupon_amt BETWEEN 2323 AND 2323 + 1000 + OR ss_wholesale_cost BETWEEN 31 AND 31 + 20)) B2, + (SELECT + avg(ss_list_price) B3_LP, + count(ss_list_price) B3_CNT, + count(DISTINCT ss_list_price) B3_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 11 AND 15 + AND (ss_list_price BETWEEN 142 AND 142 + 10 + OR ss_coupon_amt BETWEEN 12214 AND 12214 + 1000 + OR ss_wholesale_cost BETWEEN 79 AND 79 + 20)) B3, + (SELECT + avg(ss_list_price) B4_LP, + count(ss_list_price) B4_CNT, + count(DISTINCT ss_list_price) B4_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 16 AND 20 + AND (ss_list_price BETWEEN 135 AND 135 + 10 + OR ss_coupon_amt BETWEEN 6071 AND 6071 + 1000 + OR ss_wholesale_cost BETWEEN 38 AND 38 + 20)) B4, + (SELECT + avg(ss_list_price) B5_LP, + count(ss_list_price) B5_CNT, + count(DISTINCT ss_list_price) B5_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 25 + AND (ss_list_price BETWEEN 122 AND 122 + 10 + OR ss_coupon_amt BETWEEN 836 AND 836 + 1000 + OR ss_wholesale_cost BETWEEN 17 AND 17 + 20)) B5, + (SELECT + avg(ss_list_price) B6_LP, + count(ss_list_price) B6_CNT, + count(DISTINCT ss_list_price) B6_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 26 AND 30 + AND (ss_list_price BETWEEN 154 AND 154 + 10 + OR ss_coupon_amt BETWEEN 7326 AND 7326 + 1000 + OR ss_wholesale_cost BETWEEN 7 AND 7 + 20)) B6 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q29.sql b/sql/core/src/test/resources/tpcds/q29.sql new file mode 100755 index 0000000000000..3f1fd553f6da8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q29.sql @@ -0,0 +1,32 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_quantity) AS store_sales_quantity, + sum(sr_return_quantity) AS store_returns_quantity, + sum(cs_quantity) AS catalog_sales_quantity +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, + date_dim d3, store, item +WHERE + d1.d_moy = 9 + AND d1.d_year = 1999 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 9 AND 9 + 3 + AND d2.d_year = 1999 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_year IN (1999, 1999 + 1, 1999 + 2) +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q3.sql b/sql/core/src/test/resources/tpcds/q3.sql new file mode 100755 index 0000000000000..181509df9deb7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q3.sql @@ -0,0 +1,13 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + SUM(ss_ext_sales_price) sum_agg +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manufact_id = 128 + AND dt.d_moy = 11 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, sum_agg DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q30.sql b/sql/core/src/test/resources/tpcds/q30.sql new file mode 100755 index 0000000000000..986bef566d2c8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q30.sql @@ -0,0 +1,35 @@ +WITH customer_total_return AS +(SELECT + wr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(wr_return_amt) AS ctr_total_return + FROM web_returns, date_dim, customer_address + WHERE wr_returned_date_sk = d_date_sk + AND d_year = 2002 + AND wr_returning_addr_sk = ca_address_sk + GROUP BY wr_returning_customer_sk, ca_state) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_day, + c_birth_month, + c_birth_year, + c_birth_country, + c_login, + c_email_address, + c_last_review_date, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag + , c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address + , c_last_review_date, ctr_total_return +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q31.sql b/sql/core/src/test/resources/tpcds/q31.sql new file mode 100755 index 0000000000000..3e543d5436407 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q31.sql @@ -0,0 +1,60 @@ +WITH ss AS +(SELECT + ca_county, + d_qoy, + d_year, + sum(ss_ext_sales_price) AS store_sales + FROM store_sales, date_dim, customer_address + WHERE ss_sold_date_sk = d_date_sk + AND ss_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year), + ws AS + (SELECT + ca_county, + d_qoy, + d_year, + sum(ws_ext_sales_price) AS web_sales + FROM web_sales, date_dim, customer_address + WHERE ws_sold_date_sk = d_date_sk + AND ws_bill_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year) +SELECT + ss1.ca_county, + ss1.d_year, + ws2.web_sales / ws1.web_sales web_q1_q2_increase, + ss2.store_sales / ss1.store_sales store_q1_q2_increase, + ws3.web_sales / ws2.web_sales web_q2_q3_increase, + ss3.store_sales / ss2.store_sales store_q2_q3_increase +FROM + ss ss1, ss ss2, ss ss3, ws ws1, ws ws2, ws ws3 +WHERE + ss1.d_qoy = 1 + AND ss1.d_year = 2000 + AND ss1.ca_county = ss2.ca_county + AND ss2.d_qoy = 2 + AND ss2.d_year = 2000 + AND ss2.ca_county = ss3.ca_county + AND ss3.d_qoy = 3 + AND ss3.d_year = 2000 + AND ss1.ca_county = ws1.ca_county + AND ws1.d_qoy = 1 + AND ws1.d_year = 2000 + AND ws1.ca_county = ws2.ca_county + AND ws2.d_qoy = 2 + AND ws2.d_year = 2000 + AND ws1.ca_county = ws3.ca_county + AND ws3.d_qoy = 3 + AND ws3.d_year = 2000 + AND CASE WHEN ws1.web_sales > 0 + THEN ws2.web_sales / ws1.web_sales + ELSE NULL END + > CASE WHEN ss1.store_sales > 0 + THEN ss2.store_sales / ss1.store_sales + ELSE NULL END + AND CASE WHEN ws2.web_sales > 0 + THEN ws3.web_sales / ws2.web_sales + ELSE NULL END + > CASE WHEN ss2.store_sales > 0 + THEN ss3.store_sales / ss2.store_sales + ELSE NULL END +ORDER BY ss1.ca_county diff --git a/sql/core/src/test/resources/tpcds/q32.sql b/sql/core/src/test/resources/tpcds/q32.sql new file mode 100755 index 0000000000000..1a907961e74bb --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q32.sql @@ -0,0 +1,15 @@ +SELECT 1 AS `excess discount amount ` +FROM + catalog_sales, item, date_dim +WHERE + i_manufact_id = 977 + AND i_item_sk = cs_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk + AND cs_ext_discount_amt > ( + SELECT 1.3 * avg(cs_ext_discount_amt) + FROM catalog_sales, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27]' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q33.sql b/sql/core/src/test/resources/tpcds/q33.sql new file mode 100755 index 0000000000000..d24856aa5c1eb --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q33.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_manufact_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), cs AS +(SELECT + i_manufact_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN ( + SELECT i_manufact_id + FROM item + WHERE + i_category IN ('Electronics')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), + ws AS ( + SELECT + i_manufact_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id) +SELECT + i_manufact_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_manufact_id +ORDER BY total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q34.sql b/sql/core/src/test/resources/tpcds/q34.sql new file mode 100755 index 0000000000000..33396bf16e574 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q34.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (date_dim.d_dom BETWEEN 1 AND 3 OR date_dim.d_dom BETWEEN 25 AND 28) + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND (CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL + END) > 1.2 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', + 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + GROUP BY ss_ticket_number, ss_customer_sk) dn, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 15 AND 20 +ORDER BY c_last_name, c_first_name, c_salutation, c_preferred_cust_flag DESC diff --git a/sql/core/src/test/resources/tpcds/q35.sql b/sql/core/src/test/resources/tpcds/q35.sql new file mode 100755 index 0000000000000..cfe4342d8be86 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q35.sql @@ -0,0 +1,46 @@ +SELECT + ca_state, + cd_gender, + cd_marital_status, + count(*) cnt1, + min(cd_dep_count), + max(cd_dep_count), + avg(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + min(cd_dep_employed_count), + max(cd_dep_employed_count), + avg(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + min(cd_dep_college_count), + max(cd_dep_college_count), + avg(cd_dep_college_count) +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4)) +GROUP BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +ORDER BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q36.sql b/sql/core/src/test/resources/tpcds/q36.sql new file mode 100755 index 0000000000000..a8f93df76a34b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q36.sql @@ -0,0 +1,26 @@ +SELECT + sum(ss_net_profit) / sum(ss_ext_sales_price) AS gross_margin, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ss_net_profit) / sum(ss_ext_sales_price) ASC) AS rank_within_parent +FROM + store_sales, date_dim d1, item, store +WHERE + d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN i_category END + , rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q37.sql b/sql/core/src/test/resources/tpcds/q37.sql new file mode 100755 index 0000000000000..11b3821fa48b8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q37.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, catalog_sales +WHERE i_current_price BETWEEN 68 AND 68 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-02-01' AS DATE) AND (cast('2000-02-01' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (677, 940, 694, 808) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND cs_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q38.sql b/sql/core/src/test/resources/tpcds/q38.sql new file mode 100755 index 0000000000000..1c8d53ee2bbfc --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q38.sql @@ -0,0 +1,30 @@ +SELECT count(*) +FROM ( + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM store_sales, date_dim, customer + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + ) hot_cust +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q39a.sql b/sql/core/src/test/resources/tpcds/q39a.sql new file mode 100755 index 0000000000000..9fc4c1701cf21 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q39a.sql @@ -0,0 +1,47 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/sql/core/src/test/resources/tpcds/q39b.sql b/sql/core/src/test/resources/tpcds/q39b.sql new file mode 100755 index 0000000000000..6f8493029fab4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q39b.sql @@ -0,0 +1,48 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 + AND inv1.cov > 1.5 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/sql/core/src/test/resources/tpcds/q4.sql b/sql/core/src/test/resources/tpcds/q4.sql new file mode 100755 index 0000000000000..b9f27fbc9a4a6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q4.sql @@ -0,0 +1,120 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + + ss_ext_sales_price) / 2) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + + cs_ext_sales_price) / 2)) year_total, + 'c' sale_type + FROM customer, catalog_sales, date_dim + WHERE c_customer_sk = cs_bill_customer_sk AND cs_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / + 2)) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk AND ws_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +FROM year_total t_s_firstyear, year_total t_s_secyear, year_total t_c_firstyear, + year_total t_c_secyear, year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_c_secyear.customer_id + AND t_s_firstyear.customer_id = t_c_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_c_firstyear.sale_type = 'c' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_c_secyear.sale_type = 'c' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_c_firstyear.dyear = 2001 + AND t_c_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_c_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END +ORDER BY + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q40.sql b/sql/core/src/test/resources/tpcds/q40.sql new file mode 100755 index 0000000000000..66d8b73ac1c15 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q40.sql @@ -0,0 +1,25 @@ +SELECT + w_state, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_after +FROM + catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + , warehouse, item, date_dim +WHERE + i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = cs_item_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) +GROUP BY w_state, i_item_id +ORDER BY w_state, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q41.sql b/sql/core/src/test/resources/tpcds/q41.sql new file mode 100755 index 0000000000000..25e317e0e201a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q41.sql @@ -0,0 +1,49 @@ +SELECT DISTINCT (i_product_name) +FROM item i1 +WHERE i_manufact_id BETWEEN 738 AND 738 + 40 + AND (SELECT count(*) AS item_cnt +FROM item +WHERE (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'powder' OR i_color = 'khaki') AND + (i_units = 'Ounce' OR i_units = 'Oz') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'brown' OR i_color = 'honeydew') AND + (i_units = 'Bunch' OR i_units = 'Ton') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'floral' OR i_color = 'deep') AND + (i_units = 'N/A' OR i_units = 'Dozen') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'light' OR i_color = 'cornflower') AND + (i_units = 'Box' OR i_units = 'Pound') AND + (i_size = 'medium' OR i_size = 'extra large') + ))) OR + (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'midnight' OR i_color = 'snow') AND + (i_units = 'Pallet' OR i_units = 'Gross') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'cyan' OR i_color = 'papaya') AND + (i_units = 'Cup' OR i_units = 'Dram') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'orange' OR i_color = 'frosted') AND + (i_units = 'Each' OR i_units = 'Tbl') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'forest' OR i_color = 'ghost') AND + (i_units = 'Lb' OR i_units = 'Bundle') AND + (i_size = 'medium' OR i_size = 'extra large') + )))) > 0 +ORDER BY i_product_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q42.sql b/sql/core/src/test/resources/tpcds/q42.sql new file mode 100755 index 0000000000000..4d2e71760d870 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q42.sql @@ -0,0 +1,18 @@ +SELECT + dt.d_year, + item.i_category_id, + item.i_category, + sum(ss_ext_sales_price) +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year + , item.i_category_id + , item.i_category +ORDER BY sum(ss_ext_sales_price) DESC, dt.d_year + , item.i_category_id + , item.i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q43.sql b/sql/core/src/test/resources/tpcds/q43.sql new file mode 100755 index 0000000000000..45411772c1b54 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q43.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + s_store_id, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales +FROM date_dim, store_sales, store +WHERE d_date_sk = ss_sold_date_sk AND + s_store_sk = ss_store_sk AND + s_gmt_offset = -5 AND + d_year = 2000 +GROUP BY s_store_name, s_store_id +ORDER BY s_store_name, s_store_id, sun_sales, mon_sales, tue_sales, wed_sales, + thu_sales, fri_sales, sat_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q44.sql b/sql/core/src/test/resources/tpcds/q44.sql new file mode 100755 index 0000000000000..379e604788625 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q44.sql @@ -0,0 +1,46 @@ +SELECT + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +FROM (SELECT * +FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col ASC) rnk +FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +FROM store_sales ss1 +WHERE ss_store_sk = 4 +GROUP BY ss_item_sk +HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col +FROM store_sales +WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL +GROUP BY ss_store_sk)) V1) V11 +WHERE rnk < 11) asceding, + (SELECT * + FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col DESC) rnk + FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + FROM store_sales ss1 + WHERE ss_store_sk = 4 + GROUP BY ss_item_sk + HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col + FROM store_sales + WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL + GROUP BY ss_store_sk)) V2) V21 + WHERE rnk < 11) descending, + item i1, item i2 +WHERE asceding.rnk = descending.rnk + AND i1.i_item_sk = asceding.item_sk + AND i2.i_item_sk = descending.item_sk +ORDER BY asceding.rnk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q45.sql b/sql/core/src/test/resources/tpcds/q45.sql new file mode 100755 index 0000000000000..907438f196c4c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q45.sql @@ -0,0 +1,21 @@ +SELECT + ca_zip, + ca_city, + sum(ws_sales_price) +FROM web_sales, customer, customer_address, date_dim, item +WHERE ws_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND ws_item_sk = i_item_sk + AND (substr(ca_zip, 1, 5) IN + ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') + OR + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) +) + AND ws_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip, ca_city +ORDER BY ca_zip, ca_city +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q46.sql b/sql/core/src/test/resources/tpcds/q46.sql new file mode 100755 index 0000000000000..0911677dff206 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q46.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics, customer_address + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_dow IN (6, 0) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Fairview', 'Midway', 'Fairview', 'Fairview', 'Fairview') + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q47.sql b/sql/core/src/test/resources/tpcds/q47.sql new file mode 100755 index 0000000000000..cfc37a4cece66 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q47.sql @@ -0,0 +1,63 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + s_store_name, + s_company_name, + d_year, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name + ORDER BY d_year, d_moy) rn + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.s_store_name, + v1.s_company_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.s_store_name = v1_lag.s_store_name AND + v1.s_store_name = v1_lead.s_store_name AND + v1.s_company_name = v1_lag.s_company_name AND + v1.s_company_name = v1_lead.s_company_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q48.sql b/sql/core/src/test/resources/tpcds/q48.sql new file mode 100755 index 0000000000000..fdb9f38e294f7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q48.sql @@ -0,0 +1,63 @@ +SELECT sum(ss_quantity) +FROM store_sales, store, customer_demographics, customer_address, date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND + ( + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'M' + AND + cd_education_status = '4 yr Degree' + AND + ss_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'D' + AND + cd_education_status = '2 yr Degree' + AND + ss_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'S' + AND + cd_education_status = 'College' + AND + ss_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('CO', 'OH', 'TX') + AND ss_net_profit BETWEEN 0 AND 2000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('OR', 'MN', 'KY') + AND ss_net_profit BETWEEN 150 AND 3000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('VA', 'CA', 'MS') + AND ss_net_profit BETWEEN 50 AND 25000 + ) + ) diff --git a/sql/core/src/test/resources/tpcds/q49.sql b/sql/core/src/test/resources/tpcds/q49.sql new file mode 100755 index 0000000000000..9568d8b92d10a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q49.sql @@ -0,0 +1,126 @@ +SELECT + 'web' AS channel, + web.item, + web.return_ratio, + web.return_rank, + web.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + ws.ws_item_sk AS item, + (cast(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + web_sales ws LEFT OUTER JOIN web_returns wr + ON (ws.ws_order_number = wr.wr_order_number AND + ws.ws_item_sk = wr.wr_item_sk) + , date_dim + WHERE + wr.wr_return_amt > 10000 + AND ws.ws_net_profit > 1 + AND ws.ws_net_paid > 0 + AND ws.ws_quantity > 0 + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY ws.ws_item_sk + ) in_web + ) web +WHERE (web.return_rank <= 10 OR web.currency_rank <= 10) +UNION +SELECT + 'catalog' AS channel, + catalog.item, + catalog.return_ratio, + catalog.return_rank, + catalog.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + cs.cs_item_sk AS item, + (cast(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + catalog_sales cs LEFT OUTER JOIN catalog_returns cr + ON (cs.cs_order_number = cr.cr_order_number AND + cs.cs_item_sk = cr.cr_item_sk) + , date_dim + WHERE + cr.cr_return_amount > 10000 + AND cs.cs_net_profit > 1 + AND cs.cs_net_paid > 0 + AND cs.cs_quantity > 0 + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY cs.cs_item_sk + ) in_cat + ) catalog +WHERE (catalog.return_rank <= 10 OR catalog.currency_rank <= 10) +UNION +SELECT + 'store' AS channel, + store.item, + store.return_ratio, + store.return_rank, + store.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + sts.ss_item_sk AS item, + (cast(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + store_sales sts LEFT OUTER JOIN store_returns sr + ON (sts.ss_ticket_number = sr.sr_ticket_number AND sts.ss_item_sk = sr.sr_item_sk) + , date_dim + WHERE + sr.sr_return_amt > 10000 + AND sts.ss_net_profit > 1 + AND sts.ss_net_paid > 0 + AND sts.ss_quantity > 0 + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY sts.ss_item_sk + ) in_store + ) store +WHERE (store.return_rank <= 10 OR store.currency_rank <= 10) +ORDER BY 1, 4, 5 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q5.sql b/sql/core/src/test/resources/tpcds/q5.sql new file mode 100755 index 0000000000000..b87cf3a44827b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q5.sql @@ -0,0 +1,131 @@ +WITH ssr AS +( SELECT + s_store_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ss_store_sk AS store_sk, + ss_sold_date_sk AS date_sk, + ss_ext_sales_price AS sales_price, + ss_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM store_sales + UNION ALL + SELECT + sr_store_sk AS store_sk, + sr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + sr_return_amt AS return_amt, + sr_net_loss AS net_loss + FROM store_returns) + salesreturns, date_dim, store + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND store_sk = s_store_sk + GROUP BY s_store_id), + csr AS + ( SELECT + cp_catalog_page_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + cs_catalog_page_sk AS page_sk, + cs_sold_date_sk AS date_sk, + cs_ext_sales_price AS sales_price, + cs_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM catalog_sales + UNION ALL + SELECT + cr_catalog_page_sk AS page_sk, + cr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + cr_return_amount AS return_amt, + cr_net_loss AS net_loss + FROM catalog_returns + ) salesreturns, date_dim, catalog_page + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND page_sk = cp_catalog_page_sk + GROUP BY cp_catalog_page_id) + , + wsr AS + ( SELECT + web_site_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ws_web_site_sk AS wsr_web_site_sk, + ws_sold_date_sk AS date_sk, + ws_ext_sales_price AS sales_price, + ws_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM web_sales + UNION ALL + SELECT + ws_web_site_sk AS wsr_web_site_sk, + wr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + wr_return_amt AS return_amt, + wr_net_loss AS net_loss + FROM web_returns + LEFT OUTER JOIN web_sales ON + (wr_item_sk = ws_item_sk + AND wr_order_number = ws_order_number) + ) salesreturns, date_dim, web_site + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND wsr_web_site_sk = web_site_sk + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + concat('store', s_store_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', cp_catalog_page_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM wsr + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q50.sql b/sql/core/src/test/resources/tpcds/q50.sql new file mode 100755 index 0000000000000..f1d4b15449edd --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q50.sql @@ -0,0 +1,47 @@ +SELECT + s_store_name, + s_company_id, + s_street_number, + s_street_name, + s_street_type, + s_suite_number, + s_city, + s_county, + s_state, + s_zip, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND + (sr_returned_date_sk - ss_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND + (sr_returned_date_sk - ss_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 90) AND + (sr_returned_date_sk - ss_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + store_sales, store_returns, store, date_dim d1, date_dim d2 +WHERE + d2.d_year = 2001 + AND d2.d_moy = 8 + AND ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND sr_returned_date_sk = d2.d_date_sk + AND ss_customer_sk = sr_customer_sk + AND ss_store_sk = s_store_sk +GROUP BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +ORDER BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q51.sql b/sql/core/src/test/resources/tpcds/q51.sql new file mode 100755 index 0000000000000..62b003eb67b9b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q51.sql @@ -0,0 +1,55 @@ +WITH web_v1 AS ( + SELECT + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + OVER (PARTITION BY ws_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_item_sk IS NOT NULL + GROUP BY ws_item_sk, d_date), + store_v1 AS ( + SELECT + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + OVER (PARTITION BY ss_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ss_item_sk IS NOT NULL + GROUP BY ss_item_sk, d_date) +SELECT * +FROM (SELECT + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) web_cumulative, + max(store_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) store_cumulative +FROM (SELECT + CASE WHEN web.item_sk IS NOT NULL + THEN web.item_sk + ELSE store.item_sk END item_sk, + CASE WHEN web.d_date IS NOT NULL + THEN web.d_date + ELSE store.d_date END d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +FROM web_v1 web FULL OUTER JOIN store_v1 store ON (web.item_sk = store.item_sk + AND web.d_date = store.d_date) + ) x) y +WHERE web_cumulative > store_cumulative +ORDER BY item_sk, d_date +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q52.sql b/sql/core/src/test/resources/tpcds/q52.sql new file mode 100755 index 0000000000000..467d1ae050455 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q52.sql @@ -0,0 +1,14 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, ext_price DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q53.sql b/sql/core/src/test/resources/tpcds/q53.sql new file mode 100755 index 0000000000000..b42c68dcf871b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q53.sql @@ -0,0 +1,30 @@ +SELECT * +FROM + (SELECT + i_manufact_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manufact_id) avg_quarterly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, + 1200 + 7, 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) AND + ((i_category IN ('Books', 'Children', 'Electronics') AND + i_class IN ('personal', 'portable', 'reference', 'self-help') AND + i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR + (i_category IN ('Women', 'Music', 'Men') AND + i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND + i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) + GROUP BY i_manufact_id, d_qoy) tmp1 +WHERE CASE WHEN avg_quarterly_sales > 0 + THEN abs(sum_sales - avg_quarterly_sales) / avg_quarterly_sales + ELSE NULL END > 0.1 +ORDER BY avg_quarterly_sales, + sum_sales, + i_manufact_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q54.sql b/sql/core/src/test/resources/tpcds/q54.sql new file mode 100755 index 0000000000000..897237fb6e10b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q54.sql @@ -0,0 +1,61 @@ +WITH my_customers AS ( + SELECT DISTINCT + c_customer_sk, + c_current_addr_sk + FROM + (SELECT + cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales + UNION ALL + SELECT + ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + FROM web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + WHERE sold_date_sk = d_date_sk + AND item_sk = i_item_sk + AND i_category = 'Women' + AND i_class = 'maternity' + AND c_customer_sk = cs_or_ws_sales.customer_sk + AND d_moy = 12 + AND d_year = 1998 +) + , my_revenue AS ( + SELECT + c_customer_sk, + sum(ss_ext_sales_price) AS revenue + FROM my_customers, + store_sales, + customer_address, + store, + date_dim + WHERE c_current_addr_sk = ca_address_sk + AND ca_county = s_county + AND ca_state = s_state + AND ss_sold_date_sk = d_date_sk + AND c_customer_sk = ss_customer_sk + AND d_month_seq BETWEEN (SELECT DISTINCT d_month_seq + 1 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + AND (SELECT DISTINCT d_month_seq + 3 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + GROUP BY c_customer_sk +) + , segments AS +(SELECT cast((revenue / 50) AS INT) AS segment + FROM my_revenue) +SELECT + segment, + count(*) AS num_customers, + segment * 50 AS segment_base +FROM segments +GROUP BY segment +ORDER BY segment, num_customers +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q55.sql b/sql/core/src/test/resources/tpcds/q55.sql new file mode 100755 index 0000000000000..bc5d888c9ac58 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q55.sql @@ -0,0 +1,13 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 28 + AND d_moy = 11 + AND d_year = 1999 +GROUP BY i_brand, i_brand_id +ORDER BY ext_price DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q56.sql b/sql/core/src/test/resources/tpcds/q56.sql new file mode 100755 index 0000000000000..2fa1738dcfee6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q56.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM + catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q57.sql b/sql/core/src/test/resources/tpcds/q57.sql new file mode 100755 index 0000000000000..cf70d4b905b55 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q57.sql @@ -0,0 +1,56 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + cc_name, + d_year, + d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) + OVER + (PARTITION BY i_category, i_brand, cc_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, cc_name + ORDER BY d_year, d_moy) rn + FROM item, catalog_sales, date_dim, call_center + WHERE cs_item_sk = i_item_sk AND + cs_sold_date_sk = d_date_sk AND + cc_call_center_sk = cs_call_center_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + cc_name, d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.cc_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.cc_name = v1_lag.cc_name AND + v1.cc_name = v1_lead.cc_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q58.sql b/sql/core/src/test/resources/tpcds/q58.sql new file mode 100755 index 0000000000000..5f63f33dc927c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q58.sql @@ -0,0 +1,59 @@ +WITH ss_items AS +(SELECT + i_item_id item_id, + sum(ss_ext_sales_price) ss_item_rev + FROM store_sales, item, date_dim + WHERE ss_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ss_sold_date_sk = d_date_sk + GROUP BY i_item_id), + cs_items AS + (SELECT + i_item_id item_id, + sum(cs_ext_sales_price) cs_item_rev + FROM catalog_sales, item, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND cs_sold_date_sk = d_date_sk + GROUP BY i_item_id), + ws_items AS + (SELECT + i_item_id item_id, + sum(ws_ext_sales_price) ws_item_rev + FROM web_sales, item, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ws_sold_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + ss_items.item_id, + ss_item_rev, + ss_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ss_dev, + cs_item_rev, + cs_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 cs_dev, + ws_item_rev, + ws_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ws_dev, + (ss_item_rev + cs_item_rev + ws_item_rev) / 3 average +FROM ss_items, cs_items, ws_items +WHERE ss_items.item_id = cs_items.item_id + AND ss_items.item_id = ws_items.item_id + AND ss_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev + AND ss_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND cs_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND cs_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND ws_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND ws_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev +ORDER BY item_id, ss_item_rev +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q59.sql b/sql/core/src/test/resources/tpcds/q59.sql new file mode 100755 index 0000000000000..3cef2027680b0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q59.sql @@ -0,0 +1,75 @@ +WITH wss AS +(SELECT + d_week_seq, + ss_store_sk, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + GROUP BY d_week_seq, ss_store_sk +) +SELECT + s_store_name1, + s_store_id1, + d_week_seq1, + sun_sales1 / sun_sales2, + mon_sales1 / mon_sales2, + tue_sales1 / tue_sales2, + wed_sales1 / wed_sales2, + thu_sales1 / thu_sales2, + fri_sales1 / fri_sales2, + sat_sales1 / sat_sales2 +FROM + (SELECT + s_store_name s_store_name1, + wss.d_week_seq d_week_seq1, + s_store_id s_store_id1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 AND 1212 + 11) y, + (SELECT + s_store_name s_store_name2, + wss.d_week_seq d_week_seq2, + s_store_id s_store_id2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 + 12 AND 1212 + 23) x +WHERE s_store_id1 = s_store_id2 + AND d_week_seq1 = d_week_seq2 - 52 +ORDER BY s_store_name1, s_store_id1, d_week_seq1 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q6.sql b/sql/core/src/test/resources/tpcds/q6.sql new file mode 100755 index 0000000000000..f0f5cf05aebda --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q6.sql @@ -0,0 +1,21 @@ +SELECT + a.ca_state state, + count(*) cnt +FROM + customer_address a, customer c, store_sales s, date_dim d, item i +WHERE a.ca_address_sk = c.c_current_addr_sk + AND c.c_customer_sk = s.ss_customer_sk + AND s.ss_sold_date_sk = d.d_date_sk + AND s.ss_item_sk = i.i_item_sk + AND d.d_month_seq = + (SELECT DISTINCT (d_month_seq) + FROM date_dim + WHERE d_year = 2000 AND d_moy = 1) + AND i.i_current_price > 1.2 * + (SELECT avg(j.i_current_price) + FROM item j + WHERE j.i_category = i.i_category) +GROUP BY a.ca_state +HAVING count(*) >= 10 +ORDER BY cnt +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q60.sql b/sql/core/src/test/resources/tpcds/q60.sql new file mode 100755 index 0000000000000..41b963f44ba13 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q60.sql @@ -0,0 +1,62 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY i_item_id, total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q61.sql b/sql/core/src/test/resources/tpcds/q61.sql new file mode 100755 index 0000000000000..b0a872b4b80e1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q61.sql @@ -0,0 +1,33 @@ +SELECT + promotions, + total, + cast(promotions AS DECIMAL(15, 4)) / cast(total AS DECIMAL(15, 4)) * 100 +FROM + (SELECT sum(ss_ext_sales_price) promotions + FROM store_sales, store, promotion, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_promo_sk = p_promo_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) promotional_sales, + (SELECT sum(ss_ext_sales_price) total + FROM store_sales, store, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) all_sales +ORDER BY promotions, total +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q62.sql b/sql/core/src/test/resources/tpcds/q62.sql new file mode 100755 index 0000000000000..8a414f154bdc8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q62.sql @@ -0,0 +1,35 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + web_name, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND + (ws_ship_date_sk - ws_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND + (ws_ship_date_sk - ws_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND + (ws_ship_date_sk - ws_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + web_sales, warehouse, ship_mode, web_site, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_ship_date_sk = d_date_sk + AND ws_warehouse_sk = w_warehouse_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND ws_web_site_sk = web_site_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +ORDER BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q63.sql b/sql/core/src/test/resources/tpcds/q63.sql new file mode 100755 index 0000000000000..ef6867e0a9451 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q63.sql @@ -0,0 +1,31 @@ +SELECT * +FROM (SELECT + i_manager_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manager_id) avg_monthly_sales +FROM item + , store_sales + , date_dim + , store +WHERE ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, 1200 + 7, + 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) + AND ((i_category IN ('Books', 'Children', 'Electronics') + AND i_class IN ('personal', 'portable', 'refernece', 'self-help') + AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR (i_category IN ('Women', 'Music', 'Men') + AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') + AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) +GROUP BY i_manager_id, d_moy) tmp1 +WHERE CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY i_manager_id + , avg_monthly_sales + , sum_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q64.sql b/sql/core/src/test/resources/tpcds/q64.sql new file mode 100755 index 0000000000000..8ec1d31b61afe --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q64.sql @@ -0,0 +1,92 @@ +WITH cs_ui AS +(SELECT + cs_item_sk, + sum(cs_ext_list_price) AS sale, + sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit) AS refund + FROM catalog_sales + , catalog_returns + WHERE cs_item_sk = cr_item_sk + AND cs_order_number = cr_order_number + GROUP BY cs_item_sk + HAVING sum(cs_ext_list_price) > 2 * sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit)), + cross_sales AS + (SELECT + i_product_name product_name, + i_item_sk item_sk, + s_store_name store_name, + s_zip store_zip, + ad1.ca_street_number b_street_number, + ad1.ca_street_name b_streen_name, + ad1.ca_city b_city, + ad1.ca_zip b_zip, + ad2.ca_street_number c_street_number, + ad2.ca_street_name c_street_name, + ad2.ca_city c_city, + ad2.ca_zip c_zip, + d1.d_year AS syear, + d2.d_year AS fsyear, + d3.d_year s2year, + count(*) cnt, + sum(ss_wholesale_cost) s1, + sum(ss_list_price) s2, + sum(ss_coupon_amt) s3 + FROM store_sales, store_returns, cs_ui, date_dim d1, date_dim d2, date_dim d3, + store, customer, customer_demographics cd1, customer_demographics cd2, + promotion, household_demographics hd1, household_demographics hd2, + customer_address ad1, customer_address ad2, income_band ib1, income_band ib2, item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk = cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk AND + ss_item_sk = i_item_sk AND + ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number AND + ss_item_sk = cs_ui.cs_item_sk AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk AND + c_first_sales_date_sk = d2.d_date_sk AND + c_first_shipto_date_sk = d3.d_date_sk AND + ss_promo_sk = p_promo_sk AND + hd1.hd_income_band_sk = ib1.ib_income_band_sk AND + hd2.hd_income_band_sk = ib2.ib_income_band_sk AND + cd1.cd_marital_status <> cd2.cd_marital_status AND + i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium') AND + i_current_price BETWEEN 64 AND 64 + 10 AND + i_current_price BETWEEN 64 + 1 AND 64 + 15 + GROUP BY i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, + ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, + ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year + ) +SELECT + cs1.product_name, + cs1.store_name, + cs1.store_zip, + cs1.b_street_number, + cs1.b_streen_name, + cs1.b_city, + cs1.b_zip, + cs1.c_street_number, + cs1.c_street_name, + cs1.c_city, + cs1.c_zip, + cs1.syear, + cs1.cnt, + cs1.s1, + cs1.s2, + cs1.s3, + cs2.s1, + cs2.s2, + cs2.s3, + cs2.syear, + cs2.cnt +FROM cross_sales cs1, cross_sales cs2 +WHERE cs1.item_sk = cs2.item_sk AND + cs1.syear = 1999 AND + cs2.syear = 1999 + 1 AND + cs2.cnt <= cs1.cnt AND + cs1.store_name = cs2.store_name AND + cs1.store_zip = cs2.store_zip +ORDER BY cs1.product_name, cs1.store_name, cs2.cnt diff --git a/sql/core/src/test/resources/tpcds/q65.sql b/sql/core/src/test/resources/tpcds/q65.sql new file mode 100755 index 0000000000000..aad04be1bcdf0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q65.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand +FROM store, item, + (SELECT + ss_store_sk, + avg(revenue) AS ave + FROM + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sa + GROUP BY ss_store_sk) sb, + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sc +WHERE sb.ss_store_sk = sc.ss_store_sk AND + sc.revenue <= 0.1 * sb.ave AND + s_store_sk = sc.ss_store_sk AND + i_item_sk = sc.ss_item_sk +ORDER BY s_store_name, i_item_desc +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q66.sql b/sql/core/src/test/resources/tpcds/q66.sql new file mode 100755 index 0000000000000..f826b4164372a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q66.sql @@ -0,0 +1,240 @@ +SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + ship_carriers, + year, + sum(jan_sales) AS jan_sales, + sum(feb_sales) AS feb_sales, + sum(mar_sales) AS mar_sales, + sum(apr_sales) AS apr_sales, + sum(may_sales) AS may_sales, + sum(jun_sales) AS jun_sales, + sum(jul_sales) AS jul_sales, + sum(aug_sales) AS aug_sales, + sum(sep_sales) AS sep_sales, + sum(oct_sales) AS oct_sales, + sum(nov_sales) AS nov_sales, + sum(dec_sales) AS dec_sales, + sum(jan_sales / w_warehouse_sq_ft) AS jan_sales_per_sq_foot, + sum(feb_sales / w_warehouse_sq_ft) AS feb_sales_per_sq_foot, + sum(mar_sales / w_warehouse_sq_ft) AS mar_sales_per_sq_foot, + sum(apr_sales / w_warehouse_sq_ft) AS apr_sales_per_sq_foot, + sum(may_sales / w_warehouse_sq_ft) AS may_sales_per_sq_foot, + sum(jun_sales / w_warehouse_sq_ft) AS jun_sales_per_sq_foot, + sum(jul_sales / w_warehouse_sq_ft) AS jul_sales_per_sq_foot, + sum(aug_sales / w_warehouse_sq_ft) AS aug_sales_per_sq_foot, + sum(sep_sales / w_warehouse_sq_ft) AS sep_sales_per_sq_foot, + sum(oct_sales / w_warehouse_sq_ft) AS oct_sales_per_sq_foot, + sum(nov_sales / w_warehouse_sq_ft) AS nov_sales_per_sq_foot, + sum(dec_sales / w_warehouse_sq_ft) AS dec_sales_per_sq_foot, + sum(jan_net) AS jan_net, + sum(feb_net) AS feb_net, + sum(mar_net) AS mar_net, + sum(apr_net) AS apr_net, + sum(may_net) AS may_net, + sum(jun_net) AS jun_net, + sum(jul_net) AS jul_net, + sum(aug_net) AS aug_net, + sum(sep_net) AS sep_net, + sum(oct_net) AS oct_net, + sum(nov_net) AS nov_net, + sum(dec_net) AS dec_net +FROM ( + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS dec_net + FROM + web_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + ws_warehouse_sk = w_warehouse_sk + AND ws_sold_date_sk = d_date_sk + AND ws_sold_time_sk = t_time_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year) + UNION ALL + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS dec_net + FROM + catalog_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND cs_sold_time_sk = t_time_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year + ) + ) x +GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, + ship_carriers, year +ORDER BY w_warehouse_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q67.sql b/sql/core/src/test/resources/tpcds/q67.sql new file mode 100755 index 0000000000000..f66e2252bdbd4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q67.sql @@ -0,0 +1,38 @@ +SELECT * +FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sumsales, + rank() + OVER (PARTITION BY i_category + ORDER BY sumsales DESC) rk + FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sum(coalesce(ss_sales_price * ss_quantity, 0)) sumsales + FROM store_sales, date_dim, store, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ROLLUP (i_category, i_class, i_brand, i_product_name, d_year, d_qoy, + d_moy, s_store_id)) dw1) dw2 +WHERE rk <= 100 +ORDER BY + i_category, i_class, i_brand, i_product_name, d_year, + d_qoy, d_moy, s_store_id, sumsales, rk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q68.sql b/sql/core/src/test/resources/tpcds/q68.sql new file mode 100755 index 0000000000000..adb8a7189dad7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q68.sql @@ -0,0 +1,34 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + extended_price, + extended_tax, + list_price +FROM (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_ext_sales_price) extended_price, + sum(ss_ext_list_price) list_price, + sum(ss_ext_tax) extended_tax +FROM store_sales, date_dim, store, household_demographics, customer_address +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Midway', 'Fairview') +GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, + customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, ss_ticket_number +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q69.sql b/sql/core/src/test/resources/tpcds/q69.sql new file mode 100755 index 0000000000000..1f0ee64f565a3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q69.sql @@ -0,0 +1,38 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_state IN ('KY', 'GA', 'NM') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + (NOT exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + NOT exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2)) +GROUP BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +ORDER BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q7.sql b/sql/core/src/test/resources/tpcds/q7.sql new file mode 100755 index 0000000000000..6630a00548403 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q7.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, item, promotion +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_cdemo_sk = cd_demo_sk AND + ss_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q70.sql b/sql/core/src/test/resources/tpcds/q70.sql new file mode 100755 index 0000000000000..625011b212fe0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q70.sql @@ -0,0 +1,38 @@ +SELECT + sum(ss_net_profit) AS total_sum, + s_state, + s_county, + grouping(s_state) + grouping(s_county) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(s_state) + grouping(s_county), + CASE WHEN grouping(s_county) = 0 + THEN s_state END + ORDER BY sum(ss_net_profit) DESC) AS rank_within_parent +FROM + store_sales, date_dim d1, store +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + AND s_state IN + (SELECT s_state + FROM + (SELECT + s_state AS s_state, + rank() + OVER (PARTITION BY s_state + ORDER BY sum(ss_net_profit) DESC) AS ranking + FROM store_sales, store, date_dim + WHERE d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + GROUP BY s_state) tmp1 + WHERE ranking <= 5) +GROUP BY ROLLUP (s_state, s_county) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN s_state END + , rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q71.sql b/sql/core/src/test/resources/tpcds/q71.sql new file mode 100755 index 0000000000000..8d724b9244e11 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q71.sql @@ -0,0 +1,44 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + t_hour, + t_minute, + sum(ext_price) ext_price +FROM item, + (SELECT + ws_ext_sales_price AS ext_price, + ws_sold_date_sk AS sold_date_sk, + ws_item_sk AS sold_item_sk, + ws_sold_time_sk AS time_sk + FROM web_sales, date_dim + WHERE d_date_sk = ws_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + cs_ext_sales_price AS ext_price, + cs_sold_date_sk AS sold_date_sk, + cs_item_sk AS sold_item_sk, + cs_sold_time_sk AS time_sk + FROM catalog_sales, date_dim + WHERE d_date_sk = cs_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + ss_ext_sales_price AS ext_price, + ss_sold_date_sk AS sold_date_sk, + ss_item_sk AS sold_item_sk, + ss_sold_time_sk AS time_sk + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + ) AS tmp, time_dim +WHERE + sold_item_sk = i_item_sk + AND i_manager_id = 1 + AND time_sk = t_time_sk + AND (t_meal_time = 'breakfast' OR t_meal_time = 'dinner') +GROUP BY i_brand, i_brand_id, t_hour, t_minute +ORDER BY ext_price DESC, brand_id diff --git a/sql/core/src/test/resources/tpcds/q72.sql b/sql/core/src/test/resources/tpcds/q72.sql new file mode 100755 index 0000000000000..99b3eee54aa1a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q72.sql @@ -0,0 +1,33 @@ +SELECT + i_item_desc, + w_warehouse_name, + d1.d_week_seq, + count(CASE WHEN p_promo_sk IS NULL + THEN 1 + ELSE 0 END) no_promo, + count(CASE WHEN p_promo_sk IS NOT NULL + THEN 1 + ELSE 0 END) promo, + count(*) total_cnt +FROM catalog_sales + JOIN inventory ON (cs_item_sk = inv_item_sk) + JOIN warehouse ON (w_warehouse_sk = inv_warehouse_sk) + JOIN item ON (i_item_sk = cs_item_sk) + JOIN customer_demographics ON (cs_bill_cdemo_sk = cd_demo_sk) + JOIN household_demographics ON (cs_bill_hdemo_sk = hd_demo_sk) + JOIN date_dim d1 ON (cs_sold_date_sk = d1.d_date_sk) + JOIN date_dim d2 ON (inv_date_sk = d2.d_date_sk) + JOIN date_dim d3 ON (cs_ship_date_sk = d3.d_date_sk) + LEFT OUTER JOIN promotion ON (cs_promo_sk = p_promo_sk) + LEFT OUTER JOIN catalog_returns ON (cr_item_sk = cs_item_sk AND cr_order_number = cs_order_number) +WHERE d1.d_week_seq = d2.d_week_seq + AND inv_quantity_on_hand < cs_quantity + AND d3.d_date > (cast(d1.d_date AS DATE) + interval 5 days) + AND hd_buy_potential = '>10000' + AND d1.d_year = 1999 + AND hd_buy_potential = '>10000' + AND cd_marital_status = 'D' + AND d1.d_year = 1999 +GROUP BY i_item_desc, w_warehouse_name, d1.d_week_seq +ORDER BY total_cnt DESC, i_item_desc, w_warehouse_name, d_week_seq +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q73.sql b/sql/core/src/test/resources/tpcds/q73.sql new file mode 100755 index 0000000000000..881be2e9024d7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q73.sql @@ -0,0 +1,30 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN + household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL END > 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN ('Williamson County', 'Franklin Parish', 'Bronx County', 'Orange County') + GROUP BY ss_ticket_number, ss_customer_sk) dj, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 1 AND 5 +ORDER BY cnt DESC diff --git a/sql/core/src/test/resources/tpcds/q74.sql b/sql/core/src/test/resources/tpcds/q74.sql new file mode 100755 index 0000000000000..154b26d6802a3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q74.sql @@ -0,0 +1,58 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ss_net_paid) year_total, + 's' sale_type + FROM + customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ws_net_paid) year_total, + 'w' sale_type + FROM + customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name +FROM + year_total t_s_firstyear, year_total t_s_secyear, + year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.year = 2001 + AND t_s_secyear.year = 2001 + 1 + AND t_w_firstyear.year = 2001 + AND t_w_secyear.year = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY 1, 1, 1 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q75.sql b/sql/core/src/test/resources/tpcds/q75.sql new file mode 100755 index 0000000000000..2a143232b5196 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q75.sql @@ -0,0 +1,76 @@ +WITH all_sales AS ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + SUM(sales_cnt) AS sales_cnt, + SUM(sales_amt) AS sales_amt + FROM ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + cs_quantity - COALESCE(cr_return_quantity, 0) AS sales_cnt, + cs_ext_sales_price - COALESCE(cr_return_amount, 0.0) AS sales_amt + FROM catalog_sales + JOIN item ON i_item_sk = cs_item_sk + JOIN date_dim ON d_date_sk = cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ss_quantity - COALESCE(sr_return_quantity, 0) AS sales_cnt, + ss_ext_sales_price - COALESCE(sr_return_amt, 0.0) AS sales_amt + FROM store_sales + JOIN item ON i_item_sk = ss_item_sk + JOIN date_dim ON d_date_sk = ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ws_quantity - COALESCE(wr_return_quantity, 0) AS sales_cnt, + ws_ext_sales_price - COALESCE(wr_return_amt, 0.0) AS sales_amt + FROM web_sales + JOIN item ON i_item_sk = ws_item_sk + JOIN date_dim ON d_date_sk = ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number = wr_order_number + AND ws_item_sk = wr_item_sk) + WHERE i_category = 'Books') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) +SELECT + prev_yr.d_year AS prev_year, + curr_yr.d_year AS year, + curr_yr.i_brand_id, + curr_yr.i_class_id, + curr_yr.i_category_id, + curr_yr.i_manufact_id, + prev_yr.sales_cnt AS prev_yr_cnt, + curr_yr.sales_cnt AS curr_yr_cnt, + curr_yr.sales_cnt - prev_yr.sales_cnt AS sales_cnt_diff, + curr_yr.sales_amt - prev_yr.sales_amt AS sales_amt_diff +FROM all_sales curr_yr, all_sales prev_yr +WHERE curr_yr.i_brand_id = prev_yr.i_brand_id + AND curr_yr.i_class_id = prev_yr.i_class_id + AND curr_yr.i_category_id = prev_yr.i_category_id + AND curr_yr.i_manufact_id = prev_yr.i_manufact_id + AND curr_yr.d_year = 2002 + AND prev_yr.d_year = 2002 - 1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17, 2)) / CAST(prev_yr.sales_cnt AS DECIMAL(17, 2)) < 0.9 +ORDER BY sales_cnt_diff +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q76.sql b/sql/core/src/test/resources/tpcds/q76.sql new file mode 100755 index 0000000000000..815fa922be19d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q76.sql @@ -0,0 +1,47 @@ +SELECT + channel, + col_name, + d_year, + d_qoy, + i_category, + COUNT(*) sales_cnt, + SUM(ext_sales_price) sales_amt +FROM ( + SELECT + 'store' AS channel, + ss_store_sk col_name, + d_year, + d_qoy, + i_category, + ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_store_sk IS NULL + AND ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + UNION ALL + SELECT + 'web' AS channel, + ws_ship_customer_sk col_name, + d_year, + d_qoy, + i_category, + ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_ship_customer_sk IS NULL + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk = i_item_sk + UNION ALL + SELECT + 'catalog' AS channel, + cs_ship_addr_sk col_name, + d_year, + d_qoy, + i_category, + cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_ship_addr_sk IS NULL + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk = i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q77.sql b/sql/core/src/test/resources/tpcds/q77.sql new file mode 100755 index 0000000000000..7830f96e76515 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q77.sql @@ -0,0 +1,100 @@ +WITH ss AS +(SELECT + s_store_sk, + sum(ss_ext_sales_price) AS sales, + sum(ss_net_profit) AS profit + FROM store_sales, date_dim, store + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + GROUP BY s_store_sk), + sr AS + (SELECT + s_store_sk, + sum(sr_return_amt) AS returns, + sum(sr_net_loss) AS profit_loss + FROM store_returns, date_dim, store + WHERE sr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND sr_store_sk = s_store_sk + GROUP BY s_store_sk), + cs AS + (SELECT + cs_call_center_sk, + sum(cs_ext_sales_price) AS sales, + sum(cs_net_profit) AS profit + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + GROUP BY cs_call_center_sk), + cr AS + (SELECT + sum(cr_return_amount) AS returns, + sum(cr_net_loss) AS profit_loss + FROM catalog_returns, date_dim + WHERE cr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03]' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days)), + ws AS + (SELECT + wp_web_page_sk, + sum(ws_ext_sales_price) AS sales, + sum(ws_net_profit) AS profit + FROM web_sales, date_dim, web_page + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ws_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk), + wr AS + (SELECT + wp_web_page_sk, + sum(wr_return_amt) AS returns, + sum(wr_net_loss) AS profit_loss + FROM web_returns, date_dim, web_page + WHERE wr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND wr_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + ss.s_store_sk AS id, + sales, + coalesce(returns, 0) AS returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ss + LEFT JOIN sr + ON ss.s_store_sk = sr.s_store_sk + UNION ALL + SELECT + 'catalog channel' AS channel, + cs_call_center_sk AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM cs, cr + UNION ALL + SELECT + 'web channel' AS channel, + ws.wp_web_page_sk AS id, + sales, + coalesce(returns, 0) returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ws + LEFT JOIN wr + ON ws.wp_web_page_sk = wr.wp_web_page_sk + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q78.sql b/sql/core/src/test/resources/tpcds/q78.sql new file mode 100755 index 0000000000000..07b0940e26882 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q78.sql @@ -0,0 +1,64 @@ +WITH ws AS +(SELECT + d_year AS ws_sold_year, + ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + FROM web_sales + LEFT JOIN web_returns ON wr_order_number = ws_order_number AND ws_item_sk = wr_item_sk + JOIN date_dim ON ws_sold_date_sk = d_date_sk + WHERE wr_order_number IS NULL + GROUP BY d_year, ws_item_sk, ws_bill_customer_sk +), + cs AS + (SELECT + d_year AS cs_sold_year, + cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + FROM catalog_sales + LEFT JOIN catalog_returns ON cr_order_number = cs_order_number AND cs_item_sk = cr_item_sk + JOIN date_dim ON cs_sold_date_sk = d_date_sk + WHERE cr_order_number IS NULL + GROUP BY d_year, cs_item_sk, cs_bill_customer_sk + ), + ss AS + (SELECT + d_year AS ss_sold_year, + ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + FROM store_sales + LEFT JOIN store_returns ON sr_ticket_number = ss_ticket_number AND ss_item_sk = sr_item_sk + JOIN date_dim ON ss_sold_date_sk = d_date_sk + WHERE sr_ticket_number IS NULL + GROUP BY d_year, ss_item_sk, ss_customer_sk + ) +SELECT + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) ratio, + ss_qty store_qty, + ss_wc store_wholesale_cost, + ss_sp store_sales_price, + coalesce(ws_qty, 0) + coalesce(cs_qty, 0) other_chan_qty, + coalesce(ws_wc, 0) + coalesce(cs_wc, 0) other_chan_wholesale_cost, + coalesce(ws_sp, 0) + coalesce(cs_sp, 0) other_chan_sales_price +FROM ss + LEFT JOIN ws + ON (ws_sold_year = ss_sold_year AND ws_item_sk = ss_item_sk AND ws_customer_sk = ss_customer_sk) + LEFT JOIN cs + ON (cs_sold_year = ss_sold_year AND cs_item_sk = ss_item_sk AND cs_customer_sk = ss_customer_sk) +WHERE coalesce(ws_qty, 0) > 0 AND coalesce(cs_qty, 0) > 0 AND ss_sold_year = 2000 +ORDER BY + ratio, + ss_qty DESC, ss_wc DESC, ss_sp DESC, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q79.sql b/sql/core/src/test/resources/tpcds/q79.sql new file mode 100755 index 0000000000000..08f86dc2032aa --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q79.sql @@ -0,0 +1,27 @@ +SELECT + c_last_name, + c_first_name, + substr(s_city, 1, 30), + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + store.s_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (household_demographics.hd_dep_count = 6 OR + household_demographics.hd_vehicle_count > 2) + AND date_dim.d_dow = 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_number_employees BETWEEN 200 AND 295 + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city) ms, customer +WHERE ss_customer_sk = c_customer_sk +ORDER BY c_last_name, c_first_name, substr(s_city, 1, 30), profit +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q8.sql b/sql/core/src/test/resources/tpcds/q8.sql new file mode 100755 index 0000000000000..497725111f4f3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q8.sql @@ -0,0 +1,87 @@ +SELECT + s_store_name, + sum(ss_net_profit) +FROM store_sales, date_dim, store, + (SELECT ca_zip + FROM ( + (SELECT substr(ca_zip, 1, 5) ca_zip + FROM customer_address + WHERE substr(ca_zip, 1, 5) IN ( + '24128','76232','65084','87816','83926','77556','20548', + '26231','43848','15126','91137','61265','98294','25782', + '17920','18426','98235','40081','84093','28577','55565', + '17183','54601','67897','22752','86284','18376','38607', + '45200','21756','29741','96765','23932','89360','29839', + '25989','28898','91068','72550','10390','18845','47770', + '82636','41367','76638','86198','81312','37126','39192', + '88424','72175','81426','53672','10445','42666','66864', + '66708','41248','48583','82276','18842','78890','49448', + '14089','38122','34425','79077','19849','43285','39861', + '66162','77610','13695','99543','83444','83041','12305', + '57665','68341','25003','57834','62878','49130','81096', + '18840','27700','23470','50412','21195','16021','76107', + '71954','68309','18119','98359','64544','10336','86379', + '27068','39736','98569','28915','24206','56529','57647', + '54917','42961','91110','63981','14922','36420','23006', + '67467','32754','30903','20260','31671','51798','72325', + '85816','68621','13955','36446','41766','68806','16725', + '15146','22744','35850','88086','51649','18270','52867', + '39972','96976','63792','11376','94898','13595','10516', + '90225','58943','39371','94945','28587','96576','57855', + '28488','26105','83933','25858','34322','44438','73171', + '30122','34102','22685','71256','78451','54364','13354', + '45375','40558','56458','28286','45266','47305','69399', + '83921','26233','11101','15371','69913','35942','15882', + '25631','24610','44165','99076','33786','70738','26653', + '14328','72305','62496','22152','10144','64147','48425', + '14663','21076','18799','30450','63089','81019','68893', + '24996','51200','51211','45692','92712','70466','79994', + '22437','25280','38935','71791','73134','56571','14060', + '19505','72425','56575','74351','68786','51650','20004', + '18383','76614','11634','18906','15765','41368','73241', + '76698','78567','97189','28545','76231','75691','22246', + '51061','90578','56691','68014','51103','94167','57047', + '14867','73520','15734','63435','25733','35474','24676', + '94627','53535','17879','15559','53268','59166','11928', + '59402','33282','45721','43933','68101','33515','36634', + '71286','19736','58058','55253','67473','41918','19515', + '36495','19430','22351','77191','91393','49156','50298', + '87501','18652','53179','18767','63193','23968','65164', + '68880','21286','72823','58470','67301','13394','31016', + '70372','67030','40604','24317','45748','39127','26065', + '77721','31029','31880','60576','24671','45549','13376', + '50016','33123','19769','22927','97789','46081','72151', + '15723','46136','51949','68100','96888','64528','14171', + '79777','28709','11489','25103','32213','78668','22245', + '15798','27156','37930','62971','21337','51622','67853', + '10567','38415','15455','58263','42029','60279','37125', + '56240','88190','50308','26859','64457','89091','82136', + '62377','36233','63837','58078','17043','30010','60099', + '28810','98025','29178','87343','73273','30469','64034', + '39516','86057','21309','90257','67875','40162','11356', + '73650','61810','72013','30431','22461','19512','13375', + '55307','30625','83849','68908','26689','96451','38193', + '46820','88885','84935','69035','83144','47537','56616', + '94983','48033','69952','25486','61547','27385','61860', + '58048','56910','16807','17871','35258','31387','35458', + '35576')) + INTERSECT + (SELECT ca_zip + FROM + (SELECT + substr(ca_zip, 1, 5) ca_zip, + count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk AND + c_preferred_cust_flag = 'Y' + GROUP BY ca_zip + HAVING count(*) > 10) A1) + ) A2 + ) V1 +WHERE ss_store_sk = s_store_sk + AND ss_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 1998 + AND (substr(s_zip, 1, 2) = substr(V1.ca_zip, 1, 2)) +GROUP BY s_store_name +ORDER BY s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q80.sql b/sql/core/src/test/resources/tpcds/q80.sql new file mode 100755 index 0000000000000..433db87d2a858 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q80.sql @@ -0,0 +1,94 @@ +WITH ssr AS +(SELECT + s_store_id AS store_id, + sum(ss_ext_sales_price) AS sales, + sum(coalesce(sr_return_amt, 0)) AS returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) AS profit + FROM store_sales + LEFT OUTER JOIN store_returns ON + (ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number) + , + date_dim, store, item, promotion + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + AND ss_item_sk = i_item_sk + AND i_current_price > 50 + AND ss_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY s_store_id), + csr AS + (SELECT + cp_catalog_page_id AS catalog_page_id, + sum(cs_ext_sales_price) AS sales, + sum(coalesce(cr_return_amount, 0)) AS returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) AS profit + FROM catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_item_sk = cr_item_sk AND + cs_order_number = cr_order_number) + , + date_dim, catalog_page, item, promotion + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND cs_catalog_page_sk = cp_catalog_page_sk + AND cs_item_sk = i_item_sk + AND i_current_price > 50 + AND cs_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY cp_catalog_page_id), + wsr AS + (SELECT + web_site_id, + sum(ws_ext_sales_price) AS sales, + sum(coalesce(wr_return_amt, 0)) AS returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) AS profit + FROM web_sales + LEFT OUTER JOIN web_returns ON + (ws_item_sk = wr_item_sk AND ws_order_number = wr_order_number) + , + date_dim, web_site, item, promotion + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ws_web_site_sk = web_site_sk + AND ws_item_sk = i_item_sk + AND i_current_price > 50 + AND ws_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM (SELECT + 'store channel' AS channel, + concat('store', store_id) AS id, + sales, + returns, + profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', catalog_page_id) AS id, + sales, + returns, + profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + profit + FROM wsr) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q81.sql b/sql/core/src/test/resources/tpcds/q81.sql new file mode 100755 index 0000000000000..18f0ffa7e8f4c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q81.sql @@ -0,0 +1,38 @@ +WITH customer_total_return AS +(SELECT + cr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(cr_return_amt_inc_tax) AS ctr_total_return + FROM catalog_returns, date_dim, customer_address + WHERE cr_returned_date_sk = d_date_sk + AND d_year = 2000 + AND cr_returning_addr_sk = ca_address_sk + GROUP BY cr_returning_customer_sk, ca_state ) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + ca_street_number, + ca_street_name, + ca_street_type, + ca_suite_number, + ca_city, + ca_county, + ca_state, + ca_zip, + ca_country, + ca_gmt_offset, + ca_location_type, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name + , ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset + , ca_location_type, ctr_total_return +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q82.sql b/sql/core/src/test/resources/tpcds/q82.sql new file mode 100755 index 0000000000000..20942cfeb0787 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q82.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, store_sales +WHERE i_current_price BETWEEN 62 AND 62 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-05-25' AS DATE) AND (cast('2000-05-25' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (129, 270, 821, 423) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND ss_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q83.sql b/sql/core/src/test/resources/tpcds/q83.sql new file mode 100755 index 0000000000000..53c10c7ded6c1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q83.sql @@ -0,0 +1,56 @@ +WITH sr_items AS +(SELECT + i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + FROM store_returns, item, date_dim + WHERE sr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND sr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + cr_items AS + (SELECT + i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + FROM catalog_returns, item, date_dim + WHERE cr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND cr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + wr_items AS + (SELECT + i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + FROM web_returns, item, date_dim + WHERE wr_item_sk = i_item_sk AND d_date IN + (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND wr_returned_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + sr_items.item_id, + sr_item_qty, + sr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 sr_dev, + cr_item_qty, + cr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 cr_dev, + wr_item_qty, + wr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 wr_dev, + (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 average +FROM sr_items, cr_items, wr_items +WHERE sr_items.item_id = cr_items.item_id + AND sr_items.item_id = wr_items.item_id +ORDER BY sr_items.item_id, sr_item_qty +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q84.sql b/sql/core/src/test/resources/tpcds/q84.sql new file mode 100755 index 0000000000000..a1076b57ced5c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q84.sql @@ -0,0 +1,19 @@ +SELECT + c_customer_id AS customer_id, + concat(c_last_name, ', ', c_first_name) AS customername +FROM customer + , customer_address + , customer_demographics + , household_demographics + , income_band + , store_returns +WHERE ca_city = 'Edgewood' + AND c_current_addr_sk = ca_address_sk + AND ib_lower_bound >= 38128 + AND ib_upper_bound <= 38128 + 50000 + AND ib_income_band_sk = hd_income_band_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND sr_cdemo_sk = cd_demo_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q85.sql b/sql/core/src/test/resources/tpcds/q85.sql new file mode 100755 index 0000000000000..cf718b0f8adec --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q85.sql @@ -0,0 +1,82 @@ +SELECT + substr(r_reason_desc, 1, 20), + avg(ws_quantity), + avg(wr_refunded_cash), + avg(wr_fee) +FROM web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason +WHERE ws_web_page_sk = wp_web_page_sk + AND ws_item_sk = wr_item_sk + AND ws_order_number = wr_order_number + AND ws_sold_date_sk = d_date_sk AND d_year = 2000 + AND cd1.cd_demo_sk = wr_refunded_cdemo_sk + AND cd2.cd_demo_sk = wr_returning_cdemo_sk + AND ca_address_sk = wr_refunded_addr_sk + AND r_reason_sk = wr_reason_sk + AND + ( + ( + cd1.cd_marital_status = 'M' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'Advanced Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd1.cd_marital_status = 'S' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'College' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd1.cd_marital_status = 'W' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = '2 yr Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ca_country = 'United States' + AND + ca_state IN ('IN', 'OH', 'NJ') + AND ws_net_profit BETWEEN 100 AND 200 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('WI', 'CT', 'KY') + AND ws_net_profit BETWEEN 150 AND 300 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('LA', 'IA', 'AR') + AND ws_net_profit BETWEEN 50 AND 250 + ) + ) +GROUP BY r_reason_desc +ORDER BY substr(r_reason_desc, 1, 20) + , avg(ws_quantity) + , avg(wr_refunded_cash) + , avg(wr_fee) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q86.sql b/sql/core/src/test/resources/tpcds/q86.sql new file mode 100755 index 0000000000000..789a4abf7b5f7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q86.sql @@ -0,0 +1,24 @@ +SELECT + sum(ws_net_paid) AS total_sum, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ws_net_paid) DESC) AS rank_within_parent +FROM + web_sales, date_dim d1, item +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ws_sold_date_sk + AND i_item_sk = ws_item_sk +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC, + CASE WHEN lochierarchy = 0 + THEN i_category END, + rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q87.sql b/sql/core/src/test/resources/tpcds/q87.sql new file mode 100755 index 0000000000000..4aaa9f39dce9e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q87.sql @@ -0,0 +1,28 @@ +SELECT count(*) +FROM ((SELECT DISTINCT + c_last_name, + c_first_name, + d_date +FROM store_sales, date_dim, customer +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + ) cool_cust diff --git a/sql/core/src/test/resources/tpcds/q88.sql b/sql/core/src/test/resources/tpcds/q88.sql new file mode 100755 index 0000000000000..25bcd90f41ab6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q88.sql @@ -0,0 +1,122 @@ +SELECT * +FROM + (SELECT count(*) h8_30_to_9 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 8 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s1, + (SELECT count(*) h9_to_9_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s2, + (SELECT count(*) h9_30_to_10 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s3, + (SELECT count(*) h10_to_10_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s4, + (SELECT count(*) h10_30_to_11 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s5, + (SELECT count(*) h11_to_11_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s6, + (SELECT count(*) h11_30_to_12 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s7, + (SELECT count(*) h12_to_12_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 12 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s8 diff --git a/sql/core/src/test/resources/tpcds/q89.sql b/sql/core/src/test/resources/tpcds/q89.sql new file mode 100755 index 0000000000000..75408cb0323f8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q89.sql @@ -0,0 +1,30 @@ +SELECT * +FROM ( + SELECT + i_category, + i_class, + i_brand, + s_store_name, + s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_year IN (1999) AND + ((i_category IN ('Books', 'Electronics', 'Sports') AND + i_class IN ('computers', 'stereo', 'football')) + OR (i_category IN ('Men', 'Jewelry', 'Women') AND + i_class IN ('shirts', 'birdal', 'dresses'))) + GROUP BY i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +WHERE CASE WHEN (avg_monthly_sales <> 0) + THEN (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q9.sql b/sql/core/src/test/resources/tpcds/q9.sql new file mode 100755 index 0000000000000..de3db9d988f1e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q9.sql @@ -0,0 +1,48 @@ +SELECT + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) > 62316685 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) END bucket1, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) > 19045798 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) END bucket2, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) > 365541424 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) END bucket3, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) > 216357808 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) END bucket4, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) > 184483884 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) END bucket5 +FROM reason +WHERE r_reason_sk = 1 diff --git a/sql/core/src/test/resources/tpcds/q90.sql b/sql/core/src/test/resources/tpcds/q90.sql new file mode 100755 index 0000000000000..85e35bf8bf8ec --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q90.sql @@ -0,0 +1,19 @@ +SELECT cast(amc AS DECIMAL(15, 4)) / cast(pmc AS DECIMAL(15, 4)) am_pm_ratio +FROM (SELECT count(*) amc +FROM web_sales, household_demographics, time_dim, web_page +WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 8 AND 8 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) at, + (SELECT count(*) pmc + FROM web_sales, household_demographics, time_dim, web_page + WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 19 AND 19 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) pt +ORDER BY am_pm_ratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q91.sql b/sql/core/src/test/resources/tpcds/q91.sql new file mode 100755 index 0000000000000..9ca7ce00ac775 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q91.sql @@ -0,0 +1,23 @@ +SELECT + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +FROM + call_center, catalog_returns, date_dim, customer, customer_address, + customer_demographics, household_demographics +WHERE + cr_call_center_sk = cc_call_center_sk + AND cr_returned_date_sk = d_date_sk + AND cr_returning_customer_sk = c_customer_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND ca_address_sk = c_current_addr_sk + AND d_year = 1998 + AND d_moy = 11 + AND ((cd_marital_status = 'M' AND cd_education_status = 'Unknown') + OR (cd_marital_status = 'W' AND cd_education_status = 'Advanced Degree')) + AND hd_buy_potential LIKE 'Unknown%' + AND ca_gmt_offset = -7 +GROUP BY cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status +ORDER BY sum(cr_net_loss) DESC diff --git a/sql/core/src/test/resources/tpcds/q92.sql b/sql/core/src/test/resources/tpcds/q92.sql new file mode 100755 index 0000000000000..99129c3bd9e5b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q92.sql @@ -0,0 +1,16 @@ +SELECT sum(ws_ext_discount_amt) AS `Excess Discount Amount ` +FROM web_sales, item, date_dim +WHERE i_manufact_id = 350 + AND i_item_sk = ws_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + AND ws_ext_discount_amt > + ( + SELECT 1.3 * avg(ws_ext_discount_amt) + FROM web_sales, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + ) +ORDER BY sum(ws_ext_discount_amt) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q93.sql b/sql/core/src/test/resources/tpcds/q93.sql new file mode 100755 index 0000000000000..222dc31c1f561 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q93.sql @@ -0,0 +1,19 @@ +SELECT + ss_customer_sk, + sum(act_sales) sumsales +FROM (SELECT + ss_item_sk, + ss_ticket_number, + ss_customer_sk, + CASE WHEN sr_return_quantity IS NOT NULL + THEN (ss_quantity - sr_return_quantity) * ss_sales_price + ELSE (ss_quantity * ss_sales_price) END act_sales +FROM store_sales + LEFT OUTER JOIN store_returns + ON (sr_item_sk = ss_item_sk AND sr_ticket_number = ss_ticket_number) + , + reason +WHERE sr_reason_sk = r_reason_sk AND r_reason_desc = 'reason 28') t +GROUP BY ss_customer_sk +ORDER BY sumsales, ss_customer_sk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q94.sql b/sql/core/src/test/resources/tpcds/q94.sql new file mode 100755 index 0000000000000..d6de3d75b82d2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q94.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 days) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND EXISTS(SELECT * + FROM web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM web_returns wr1 + WHERE ws1.ws_order_number = wr1.wr_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q95.sql b/sql/core/src/test/resources/tpcds/q95.sql new file mode 100755 index 0000000000000..df71f00bd6c0b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q95.sql @@ -0,0 +1,29 @@ +WITH ws_wh AS +(SELECT + ws1.ws_order_number, + ws1.ws_warehouse_sk wh1, + ws2.ws_warehouse_sk wh2 + FROM web_sales ws1, web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 DAY) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND ws1.ws_order_number IN (SELECT ws_order_number + FROM ws_wh) + AND ws1.ws_order_number IN (SELECT wr_order_number + FROM web_returns, ws_wh + WHERE wr_order_number = ws_wh.ws_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q96.sql b/sql/core/src/test/resources/tpcds/q96.sql new file mode 100755 index 0000000000000..7ab17e7bc4597 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q96.sql @@ -0,0 +1,11 @@ +SELECT count(*) +FROM store_sales, household_demographics, time_dim, store +WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 20 + AND time_dim.t_minute >= 30 + AND household_demographics.hd_dep_count = 7 + AND store.s_store_name = 'ese' +ORDER BY count(*) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q97.sql b/sql/core/src/test/resources/tpcds/q97.sql new file mode 100755 index 0000000000000..e7e0b1a05259d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q97.sql @@ -0,0 +1,30 @@ +WITH ssci AS ( + SELECT + ss_customer_sk customer_sk, + ss_item_sk item_sk + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ss_customer_sk, ss_item_sk), + csci AS ( + SELECT + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY cs_bill_customer_sk, cs_item_sk) +SELECT + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL + THEN 1 + ELSE 0 END) store_only, + sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) catalog_only, + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) store_and_catalog +FROM ssci + FULL OUTER JOIN csci ON (ssci.customer_sk = csci.customer_sk + AND ssci.item_sk = csci.item_sk) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q98.sql b/sql/core/src/test/resources/tpcds/q98.sql new file mode 100755 index 0000000000000..bb10d4bf8da23 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q98.sql @@ -0,0 +1,21 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ss_ext_sales_price) AS itemrevenue, + sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + store_sales, item, date_dim +WHERE + ss_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio diff --git a/sql/core/src/test/resources/tpcds/q99.sql b/sql/core/src/test/resources/tpcds/q99.sql new file mode 100755 index 0000000000000..f1a3d4d2b7fe9 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q99.sql @@ -0,0 +1,34 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + cc_name, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND + (cs_ship_date_sk - cs_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND + (cs_ship_date_sk - cs_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND + (cs_ship_date_sk - cs_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + catalog_sales, warehouse, ship_mode, call_center, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND cs_ship_date_sk = d_date_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND cs_call_center_sk = cc_call_center_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, cc_name +ORDER BY substr(w_warehouse_name, 1, 20), sm_type, cc_name +LIMIT 100 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala new file mode 100644 index 0000000000000..398d8d98be9b3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -0,0 +1,126 @@ +/* + * 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.io.File + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure TPCDS query performance. + * To run this: + * spark-submit --class --jars + */ +object TPCDSQueryBenchmark { + val conf = + new SparkConf() + .setMaster("local[1]") + .setAppName("test-sql-context") + .set("spark.sql.parquet.compression.codec", "snappy") + .set("spark.sql.shuffle.partitions", "4") + .set("spark.driver.memory", "3g") + .set("spark.executor.memory", "3g") + .set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) + + val spark = SparkSession.builder.config(conf).getOrCreate() + + val tables = Seq("catalog_page", "catalog_returns", "customer", "customer_address", + "customer_demographics", "date_dim", "household_demographics", "inventory", "item", + "promotion", "store", "store_returns", "catalog_sales", "web_sales", "store_sales", + "web_returns", "web_site", "reason", "call_center", "warehouse", "ship_mode", "income_band", + "time_dim", "web_page") + + def setupTables(dataLocation: String): Map[String, Long] = { + tables.map { tableName => + spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) + tableName -> spark.table(tableName).count() + }.toMap + } + + def tpcdsAll(dataLocation: String, queries: Seq[String]): Unit = { + require(dataLocation.nonEmpty, + "please modify the value of dataLocation to point to your local TPCDS data") + val tableSizes = setupTables(dataLocation) + spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + queries.foreach { name => + val queryString = fileToString(new File(Thread.currentThread().getContextClassLoader + .getResource(s"tpcds/$name.sql").getFile)) + + // This is an indirect hack to estimate the size of each query's input by traversing the + // logical plan and adding up the sizes of all tables that appear in the plan. Note that this + // currently doesn't take WITH subqueries into account which might lead to fairly inaccurate + // per-row processing time for those cases. + val queryRelations = scala.collection.mutable.HashSet[String]() + spark.sql(queryString).queryExecution.logical.map { + case ur @ UnresolvedRelation(t: TableIdentifier, _) => + queryRelations.add(t.table) + case lp: LogicalPlan => + lp.expressions.foreach { _ foreach { + case subquery: SubqueryExpression => + subquery.plan.foreach { + case ur @ UnresolvedRelation(t: TableIdentifier, _) => + queryRelations.add(t.table) + case _ => + } + case _ => + } + } + case _ => + } + val numRows = queryRelations.map(tableSizes.getOrElse(_, 0L)).sum + val benchmark = new Benchmark(s"TPCDS Snappy", numRows, 5) + benchmark.addCase(name) { i => + spark.sql(queryString).collect() + } + benchmark.run() + } + } + + def main(args: Array[String]): Unit = { + + // List of all TPC-DS queries + val tpcdsQueries = Seq( + "q1", "q2", "q3", "q4", "q5", "q6", "q7", "q8", "q9", "q10", "q11", + "q12", "q13", "q14a", "q14b", "q15", "q16", "q17", "q18", "q19", "q20", + "q21", "q22", "q23a", "q23b", "q24a", "q24b", "q25", "q26", "q27", "q28", "q29", "q30", + "q31", "q32", "q33", "q34", "q35", "q36", "q37", "q38", "q39a", "q39b", "q40", + "q41", "q42", "q43", "q44", "q45", "q46", "q47", "q48", "q49", "q50", + "q51", "q52", "q53", "q54", "q55", "q56", "q57", "q58", "q59", "q60", + "q61", "q62", "q63", "q64", "q65", "q66", "q67", "q68", "q69", "q70", + "q71", "q72", "q73", "q74", "q75", "q76", "q77", "q78", "q79", "q80", + "q81", "q82", "q83", "q84", "q85", "q86", "q87", "q88", "q89", "q90", + "q91", "q92", "q93", "q94", "q95", "q96", "q97", "q98", "q99") + + // In order to run this benchmark, please follow the instructions at + // https://github.com/databricks/spark-sql-perf/blob/master/README.md to generate the TPCDS data + // locally (preferably with a scale factor of 5 for benchmarking). Thereafter, the value of + // dataLocation below needs to be set to the location where the generated data is stored. + val dataLocation = "" + + tpcdsAll(dataLocation, queries = tpcdsQueries) + } +} 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 deleted file mode 100644 index 228ae6f840ccf..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala +++ /dev/null @@ -1,1226 +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.datasources.parquet - -import org.apache.spark.SparkConf -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.util.Benchmark - -/** - * Benchmark to measure TPCDS query performance. - * To run this: - * spark-submit --class --jars - */ -object TPCDSBenchmark { - val conf = new SparkConf() - conf.set("spark.sql.parquet.compression.codec", "snappy") - conf.set("spark.sql.shuffle.partitions", "4") - conf.set("spark.driver.memory", "3g") - conf.set("spark.executor.memory", "3g") - conf.set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) - conf.setMaster("local[1]") - conf.setAppName("test-sql-context") - - val spark = SparkSession.builder.config(conf).getOrCreate() - - // These queries a subset of the TPCDS benchmark queries and are taken from - // https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/ - // sql/perf/tpcds/ImpalaKitQueries.scala - val tpcds = Seq( - ("q19", """ - |select - | i_brand_id, - | i_brand, - | i_manufact_id, - | i_manufact, - | sum(ss_ext_sales_price) ext_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join customer on (store_sales.ss_customer_sk = customer.c_customer_sk) - | join customer_address on - | (customer.c_current_addr_sk = customer_address.ca_address_sk) - |where - | ss_sold_date_sk between 2451484 and 2451513 - | and d_moy = 11 - | and d_year = 1999 - | and i_manager_id = 7 - | and substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5) - |group by - | i_brand, - | i_brand_id, - | i_manufact_id, - | i_manufact - |order by - | ext_price desc, - | i_brand, - | i_brand_id, - | i_manufact_id, - | i_manufact - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q19 1710 / 1858 8.7 114.5 1.0X - */ - - ("q27", """ - |select - | i_item_id, - | s_state, - | avg(ss_quantity) agg1, - | avg(ss_list_price) agg2, - | avg(ss_coupon_amt) agg3, - | avg(ss_sales_price) agg4 - |from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join customer_demographics on - | (store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | ss_sold_date_sk between 2450815 and 2451179 -- partition key filter - | and d_year = 1998 - | and cd_gender = 'F' - | and cd_marital_status = 'W' - | and cd_education_status = 'Primary' - | and s_state in ('WI', 'CA', 'TX', 'FL', 'WA', 'TN') - |group by - | i_item_id, - | s_state - |order by - | i_item_id, - | s_state - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q27 2016 / 2180 8.2 122.6 1.0X - */ - - ("q3", """ - |select - | dt.d_year, - | item.i_brand_id brand_id, - | item.i_brand brand, - | sum(ss_ext_sales_price) sum_agg - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim dt on (dt.d_date_sk = store_sales.ss_sold_date_sk) - |where - | item.i_manufact_id = 436 - | and dt.d_moy = 12 - | and (ss_sold_date_sk between 2451149 and 2451179 - | or ss_sold_date_sk between 2451514 and 2451544 - | or ss_sold_date_sk between 2451880 and 2451910 - | or ss_sold_date_sk between 2452245 and 2452275 - | or ss_sold_date_sk between 2452610 and 2452640) - |group by - | d_year, - | item.i_brand, - | item.i_brand_id - |order by - | d_year, - | sum_agg desc, - | brand_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q3 1073 / 1140 13.5 73.9 1.0X - */ - - ("q34", """ - |select - | c_last_name, - | c_first_name, - | c_salutation, - | c_preferred_cust_flag, - | ss_ticket_number, - | cnt - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | count(*) cnt - | from - | store_sales - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | date_dim.d_year in (1998, 1998 + 1, 1998 + 2) - | and (date_dim.d_dom between 1 and 3 - | or date_dim.d_dom between 25 and 28) - | and (household_demographics.hd_buy_potential = '>10000' - | or household_demographics.hd_buy_potential = 'unknown') - | and household_demographics.hd_vehicle_count > 0 - | and (case when household_demographics.hd_vehicle_count > 0 then - | household_demographics.hd_dep_count / household_demographics.hd_vehicle_count - | else null end) > 1.2 - | and ss_sold_date_sk between 2450816 and 2451910 -- partition key filter - | group by - | ss_ticket_number, - | ss_customer_sk - | ) dn - |join customer on (dn.ss_customer_sk = customer.c_customer_sk) - |where - | cnt between 15 and 20 - |order by - | c_last_name, - | c_first_name, - | c_salutation, - | c_preferred_cust_flag desc, - | ss_ticket_number, - | cnt - |limit 1000 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q34 1482 / 1734 10.0 100.4 1.0X - */ - - ("q42", """ - |select - | d_year, - | i_category_id, - | i_category, - | sum(ss_ext_sales_price) as total_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim dt on (dt.d_date_sk = store_sales.ss_sold_date_sk) - |where - | item.i_manager_id = 1 - | and dt.d_moy = 12 - | and dt.d_year = 1998 - | and ss_sold_date_sk between 2451149 and 2451179 -- partition key filter - |group by - | d_year, - | i_category_id, - | i_category - |order by - | total_price desc, - | d_year, - | i_category_id, - | i_category - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q42 1125 / 1357 12.9 77.4 1.0X - */ - - ("q43", """ - |select - | s_store_name, - | s_store_id, - | sum(case when (d_day_name = 'Sunday') then ss_sales_price else null end) sun_sales, - | sum(case when (d_day_name = 'Monday') then ss_sales_price else null end) mon_sales, - | sum(case when (d_day_name = 'Tuesday') then - | ss_sales_price else null end) tue_sales, - | sum(case when (d_day_name = 'Wednesday') then - | ss_sales_price else null end) wed_sales, - | sum(case when (d_day_name = 'Thursday') then - | ss_sales_price else null end) thu_sales, - | sum(case when (d_day_name = 'Friday') then ss_sales_price else null end) fri_sales, - | sum(case when (d_day_name = 'Saturday') then - | ss_sales_price else null end) sat_sales - |from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | s_gmt_offset = -5 - | and d_year = 1998 - | and ss_sold_date_sk between 2450816 and 2451179 -- partition key filter - |group by - | s_store_name, - | s_store_id - |order by - | s_store_name, - | s_store_id, - | sun_sales, - | mon_sales, - | tue_sales, - | wed_sales, - | thu_sales, - | fri_sales, - | sat_sales - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q43 1681 / 1985 8.6 116.1 1.0X - */ - - ("q46", """ - |select - | c_last_name, - | c_first_name, - | ca_city, - | bought_city, - | ss_ticket_number, - | amt, - | profit - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | ca_city bought_city, - | sum(ss_coupon_amt) amt, - | sum(ss_net_profit) profit - | from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join customer_address on - | (store_sales.ss_addr_sk = customer_address.ca_address_sk) - | where - | store.s_city in ('Midway', 'Concord', 'Spring Hill', 'Brownsville', 'Greenville') - | and (household_demographics.hd_dep_count = 5 - | or household_demographics.hd_vehicle_count = 3) - | and date_dim.d_dow in (6, 0) - | and date_dim.d_year in (1999, 1999 + 1, 1999 + 2) - | group by - | ss_ticket_number, - | ss_customer_sk, - | ss_addr_sk, - | ca_city - | ) dn - | join customer on (dn.ss_customer_sk = customer.c_customer_sk) - | join customer_address current_addr on - | (customer.c_current_addr_sk = current_addr.ca_address_sk) - |where - | current_addr.ca_city <> bought_city - |order by - | c_last_name, - | c_first_name, - | ca_city, - | bought_city, - | ss_ticket_number - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q46 2948 / 3218 5.1 196.1 1.0X - */ - - ("q52", """ - |select - | d_year, - | i_brand_id, - | i_brand, - | sum(ss_ext_sales_price) ext_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim dt on (store_sales.ss_sold_date_sk = dt.d_date_sk) - |where - | i_manager_id = 1 - | and d_moy = 12 - | and d_year = 1998 - | and ss_sold_date_sk between 2451149 and 2451179 -- partition key filter - |group by - | d_year, - | i_brand, - | i_brand_id - |order by - | d_year, - | ext_price desc, - | i_brand_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q52 1099 / 1228 13.2 75.7 1.0X - */ - - ("q53", """ - |select - | * - |from - | (select - | i_manufact_id, - | sum(ss_sales_price) sum_sales - | from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq in(1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, - | 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11) - | and ( - | (i_category in('Books', 'Children', 'Electronics') - | and i_class in('personal', 'portable', 'reference', 'self-help') - | and i_brand in('scholaramalgamalg #14', 'scholaramalgamalg #7', - | 'exportiunivamalg #9', 'scholaramalgamalg #9') - | ) - | or - | (i_category in('Women', 'Music', 'Men') - | and i_class in('accessories', 'classical', 'fragrances', 'pants') - | and i_brand in('amalgimporto #1', 'edu packscholar #1', - | 'exportiimporto #1', 'importoamalg #1') - | ) - | ) - | group by - | i_manufact_id, - | d_qoy - | ) tmp1 - |order by - | sum_sales, - | i_manufact_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q53 968 / 1020 15.0 66.6 1.0X - */ - - ("q55", """ - |select - | i_brand_id, - | i_brand, - | sum(ss_ext_sales_price) ext_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | i_manager_id = 36 - | and d_moy = 12 - | and d_year = 2001 - | and ss_sold_date_sk between 2452245 and 2452275 -- partition key filter - |group by - | i_brand, - | i_brand_id - |order by - | ext_price desc, - | i_brand_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q55 1002 / 1020 14.5 69.0 1.0X - */ - - ("q59", """ - |select - | s_store_name1, - | s_store_id1, - | d_week_seq1, - | sun_sales1 / sun_sales2, - | mon_sales1 / mon_sales2, - | tue_sales1 / tue_sales2, - | wed_sales1 / wed_sales2, - | thu_sales1 / thu_sales2, - | fri_sales1 / fri_sales2, - | sat_sales1 / sat_sales2 - |from - | (select - | s_store_name s_store_name1, - | wss.d_week_seq d_week_seq1, - | s_store_id s_store_id1, - | sun_sales sun_sales1, - | mon_sales mon_sales1, - | tue_sales tue_sales1, - | wed_sales wed_sales1, - | thu_sales thu_sales1, - | fri_sales fri_sales1, - | sat_sales sat_sales1 - | from - | (select - | d_week_seq, - | ss_store_sk, - | sum(case when(d_day_name = 'Sunday') then - | ss_sales_price else null end) sun_sales, - | sum(case when(d_day_name = 'Monday') then - | ss_sales_price else null end) mon_sales, - | sum(case when(d_day_name = 'Tuesday') then - | ss_sales_price else null end) tue_sales, - | sum(case when(d_day_name = 'Wednesday') then - | ss_sales_price else null end) wed_sales, - | sum(case when(d_day_name = 'Thursday') then - | ss_sales_price else null end) thu_sales, - | sum(case when(d_day_name = 'Friday') then - | ss_sales_price else null end) fri_sales, - | sum(case when(d_day_name = 'Saturday') then - | ss_sales_price else null end) sat_sales - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451088 and 2451452 - | group by - | d_week_seq, - | ss_store_sk - | ) wss - | join store on (wss.ss_store_sk = store.s_store_sk) - | join date_dim d on (wss.d_week_seq = d.d_week_seq) - | where - | d_month_seq between 1185 and 1185 + 11 - | ) y - | join - | (select - | s_store_name s_store_name2, - | wss.d_week_seq d_week_seq2, - | s_store_id s_store_id2, - | sun_sales sun_sales2, - | mon_sales mon_sales2, - | tue_sales tue_sales2, - | wed_sales wed_sales2, - | thu_sales thu_sales2, - | fri_sales fri_sales2, - | sat_sales sat_sales2 - | from - | (select - | d_week_seq, - | ss_store_sk, - | sum(case when(d_day_name = 'Sunday') then - | ss_sales_price else null end) sun_sales, - | sum(case when(d_day_name = 'Monday') then - | ss_sales_price else null end) mon_sales, - | sum(case when(d_day_name = 'Tuesday') then - | ss_sales_price else null end) tue_sales, - | sum(case when(d_day_name = 'Wednesday') then - | ss_sales_price else null end) wed_sales, - | sum(case when(d_day_name = 'Thursday') then - | ss_sales_price else null end) thu_sales, - | sum(case when(d_day_name = 'Friday') then - | ss_sales_price else null end) fri_sales, - | sum(case when(d_day_name = 'Saturday') then - | ss_sales_price else null end) sat_sales - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451088 and 2451452 - | group by - | d_week_seq, - | ss_store_sk - | ) wss - | join store on (wss.ss_store_sk = store.s_store_sk) - | join date_dim d on (wss.d_week_seq = d.d_week_seq) - | where - | d_month_seq between 1185 + 12 and 1185 + 23 - | ) x - | on (y.s_store_id1 = x.s_store_id2) - |where - | d_week_seq1 = d_week_seq2 - 52 - |order by - | s_store_name1, - | s_store_id1, - | d_week_seq1 - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q59 1624 / 1663 17.9 55.8 1.0X - */ - - ("q63", """ - |select - | * - |from - | (select - | i_manager_id, - | sum(ss_sales_price) sum_sales - | from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq in (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, - | 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11) - | and ( - | (i_category in('Books', 'Children', 'Electronics') - | and i_class in('personal', 'portable', 'refernece', 'self-help') - | and i_brand in('scholaramalgamalg #14', 'scholaramalgamalg #7', - | 'exportiunivamalg #9', 'scholaramalgamalg #9') - | ) - | or - | (i_category in('Women', 'Music', 'Men') - | and i_class in('accessories', 'classical', 'fragrances', 'pants') - | and i_brand in('amalgimporto #1', 'edu packscholar #1', - | 'exportiimporto #1', 'importoamalg #1') - | ) - | ) - | group by - | i_manager_id, - | d_moy - | ) tmp1 - |order by - | i_manager_id, - | sum_sales - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q63 979 / 1006 14.8 67.4 1.0X - */ - - ("q65", """ - |select - | s_store_name, - | i_item_desc, - | sc.revenue, - | i_current_price, - | i_wholesale_cost, - | i_brand - |from - | (select - | ss_store_sk, - | ss_item_sk, - | sum(ss_sales_price) as revenue - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq between 1212 and 1212 + 11 - | group by - | ss_store_sk, - | ss_item_sk - | ) sc - | join item on (sc.ss_item_sk = item.i_item_sk) - | join store on (sc.ss_store_sk = store.s_store_sk) - | join - | (select - | ss_store_sk, - | avg(revenue) as ave - | from - | (select - | ss_store_sk, - | ss_item_sk, - | sum(ss_sales_price) as revenue - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq between 1212 and 1212 + 11 - | group by - | ss_store_sk, - | ss_item_sk - | ) sa - | group by - | ss_store_sk - | ) sb on (sc.ss_store_sk = sb.ss_store_sk) -- 676 rows - |where - | sc.revenue <= 0.1 * sb.ave - |order by - | s_store_name, - | i_item_desc - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q65 7770 / 8097 3.7 267.9 1.0X - */ - - ("q68", """ - |select - | c_last_name, - | c_first_name, - | ca_city, - | bought_city, - | ss_ticket_number, - | extended_price, - | extended_tax, - | list_price - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | ca_city bought_city, - | sum(ss_ext_sales_price) extended_price, - | sum(ss_ext_list_price) list_price, - | sum(ss_ext_tax) extended_tax - | from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join customer_address on - | (store_sales.ss_addr_sk = customer_address.ca_address_sk) - | where - | store.s_city in('Midway', 'Fairview') - | --and date_dim.d_dom between 1 and 2 - | --and date_dim.d_year in(1999, 1999 + 1, 1999 + 2) - | -- and ss_date between '1999-01-01' and '2001-12-31' - | -- and dayofmonth(ss_date) in (1,2) - | and (household_demographics.hd_dep_count = 5 - | or household_demographics.hd_vehicle_count = 3) - | and d_date between '1999-01-01' and '1999-03-31' - | and ss_sold_date_sk between 2451180 and 2451269 - | -- partition key filter (3 months) - | group by - | ss_ticket_number, - | ss_customer_sk, - | ss_addr_sk, - | ca_city - | ) dn - | join customer on (dn.ss_customer_sk = customer.c_customer_sk) - | join customer_address current_addr on - | (customer.c_current_addr_sk = current_addr.ca_address_sk) - |where - | current_addr.ca_city <> bought_city - |order by - | c_last_name, - | ss_ticket_number - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q68 3105 / 3405 4.8 206.5 1.0X - */ - - ("q7", """ - |select - | i_item_id, - | avg(ss_quantity) agg1, - | avg(ss_list_price) agg2, - | avg(ss_coupon_amt) agg3, - | avg(ss_sales_price) agg4 - |from - | store_sales - | join customer_demographics on - | (store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join promotion on (store_sales.ss_promo_sk = promotion.p_promo_sk) - | join date_dim on (ss_sold_date_sk = d_date_sk) - |where - | cd_gender = 'F' - | and cd_marital_status = 'W' - | and cd_education_status = 'Primary' - | and (p_channel_email = 'N' - | or p_channel_event = 'N') - | and d_year = 1998 - | and ss_sold_date_sk between 2450815 and 2451179 -- partition key filter - |group by - | i_item_id - |order by - | i_item_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q7 2042 / 2333 8.1 124.2 1.0X - */ - - ("q73", """ - |select - | c_last_name, - | c_first_name, - | c_salutation, - | c_preferred_cust_flag, - | ss_ticket_number, - | cnt - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | count(*) cnt - | from - | store_sales - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | -- join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | store.s_county in - | ('Williamson County','Franklin Parish','Bronx County','Orange County') - | -- and date_dim.d_dom between 1 and 2 - | -- and date_dim.d_year in(1998, 1998 + 1, 1998 + 2) - | -- and ss_date between '1999-01-01' and '2001-12-02' - | -- and dayofmonth(ss_date) in (1,2) - | -- partition key filter - | -- and ss_sold_date_sk in (2450816, 2450846, 2450847, 2450874, 2450875, 2450905, - | -- 2450906, 2450935, 2450936, 2450966, 2450967, - | -- 2450996, 2450997, 2451027, 2451028, 2451058, 2451059, - | -- 2451088, 2451089, 2451119, 2451120, 2451149, - | -- 2451150, 2451180, 2451181, 2451211, 2451212, 2451239, - | -- 2451240, 2451270, 2451271, 2451300, 2451301, - | -- 2451331, 2451332, 2451361, 2451362, 2451392, 2451393, - | -- 2451423, 2451424, 2451453, 2451454, 2451484, - | -- 2451485, 2451514, 2451515, 2451545, 2451546, 2451576, - | -- 2451577, 2451605, 2451606, 2451636, 2451637, - | -- 2451666, 2451667, 2451697, 2451698, 2451727, 2451728, - | -- 2451758, 2451759, 2451789, 2451790, 2451819, - | -- 2451820, 2451850, 2451851, 2451880, 2451881) - | and (household_demographics.hd_buy_potential = '>10000' - | or household_demographics.hd_buy_potential = 'unknown') - | and household_demographics.hd_vehicle_count > 0 - | and case when household_demographics.hd_vehicle_count > 0 then - | household_demographics.hd_dep_count / household_demographics.hd_vehicle_count - | else null end > 1 - | and ss_sold_date_sk between 2451180 and 2451269 - | -- partition key filter (3 months) - | group by - | ss_ticket_number, - | ss_customer_sk - | ) dj - | join customer on (dj.ss_customer_sk = customer.c_customer_sk) - |where - | cnt between 1 and 5 - |order by - | cnt desc - |limit 1000 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q73 1124 / 1221 13.1 76.5 1.0X - */ - - ("q79", """ - |select - | c_last_name, - | c_first_name, - | substr(s_city, 1, 30) as city, - | ss_ticket_number, - | amt, - | profit - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | s_city, - | sum(ss_coupon_amt) amt, - | sum(ss_net_profit) profit - | from - | store_sales - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | where - | store.s_number_employees between 200 and 295 - | and (household_demographics.hd_dep_count = 8 - | or household_demographics.hd_vehicle_count > 0) - | and date_dim.d_dow = 1 - | and date_dim.d_year in (1998, 1998 + 1, 1998 + 2) - | -- and ss_date between '1998-01-01' and '2000-12-25' - | -- 156 days - | and d_date between '1999-01-01' and '1999-03-31' - | and ss_sold_date_sk between 2451180 and 2451269 -- partition key filter - | group by - | ss_ticket_number, - | ss_customer_sk, - | ss_addr_sk, - | s_city - | ) ms - | join customer on (ms.ss_customer_sk = customer.c_customer_sk) - |order by - | c_last_name, - | c_first_name, - | city, - | profit - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q79 2029 / 2488 7.3 137.5 1.0X - */ - - ("q8", - """ - |select s_store_name - | ,sum(ss_net_profit) - | from store_sales - | ,date_dim - | ,store, - | (select distinct a01.ca_zip - | from - | (SELECT substr(ca_zip,1,5) ca_zip - | FROM customer_address - | WHERE substr(ca_zip,1,5) IN ('89436', '30868', '65085', '22977', '83927', '77557', - | '58429', '40697', '80614', '10502', '32779', - | '91137', '61265', '98294', '17921', '18427', '21203', '59362', '87291', '84093', - | '21505', '17184', '10866', '67898', '25797', - | '28055', '18377', '80332', '74535', '21757', '29742', '90885', '29898', '17819', - | '40811', '25990', '47513', '89531', '91068', - | '10391', '18846', '99223', '82637', '41368', '83658', '86199', '81625', '26696', - | '89338', '88425', '32200', '81427', '19053', - | '77471', '36610', '99823', '43276', '41249', '48584', '83550', '82276', '18842', - | '78890', '14090', '38123', '40936', '34425', - | '19850', '43286', '80072', '79188', '54191', '11395', '50497', '84861', '90733', - | '21068', '57666', '37119', '25004', '57835', - | '70067', '62878', '95806', '19303', '18840', '19124', '29785', '16737', '16022', - | '49613', '89977', '68310', '60069', '98360', - | '48649', '39050', '41793', '25002', '27413', '39736', '47208', '16515', '94808', - | '57648', '15009', '80015', '42961', '63982', - | '21744', '71853', '81087', '67468', '34175', '64008', '20261', '11201', '51799', - | '48043', '45645', '61163', '48375', '36447', - | '57042', '21218', '41100', '89951', '22745', '35851', '83326', '61125', '78298', - | '80752', '49858', '52940', '96976', '63792', - | '11376', '53582', '18717', '90226', '50530', '94203', '99447', '27670', '96577', - | '57856', '56372', '16165', '23427', '54561', - | '28806', '44439', '22926', '30123', '61451', '92397', '56979', '92309', '70873', - | '13355', '21801', '46346', '37562', '56458', - | '28286', '47306', '99555', '69399', '26234', '47546', '49661', '88601', '35943', - | '39936', '25632', '24611', '44166', '56648', - | '30379', '59785', '11110', '14329', '93815', '52226', '71381', '13842', '25612', - | '63294', '14664', '21077', '82626', '18799', - | '60915', '81020', '56447', '76619', '11433', '13414', '42548', '92713', '70467', - | '30884', '47484', '16072', '38936', '13036', - | '88376', '45539', '35901', '19506', '65690', '73957', '71850', '49231', '14276', - | '20005', '18384', '76615', '11635', '38177', - | '55607', '41369', '95447', '58581', '58149', '91946', '33790', '76232', '75692', - | '95464', '22246', '51061', '56692', '53121', - | '77209', '15482', '10688', '14868', '45907', '73520', '72666', '25734', '17959', - | '24677', '66446', '94627', '53535', '15560', - | '41967', '69297', '11929', '59403', '33283', '52232', '57350', '43933', '40921', - | '36635', '10827', '71286', '19736', '80619', - | '25251', '95042', '15526', '36496', '55854', '49124', '81980', '35375', '49157', - | '63512', '28944', '14946', '36503', '54010', - | '18767', '23969', '43905', '66979', '33113', '21286', '58471', '59080', '13395', - | '79144', '70373', '67031', '38360', '26705', - | '50906', '52406', '26066', '73146', '15884', '31897', '30045', '61068', '45550', - | '92454', '13376', '14354', '19770', '22928', - | '97790', '50723', '46081', '30202', '14410', '20223', '88500', '67298', '13261', - | '14172', '81410', '93578', '83583', '46047', - | '94167', '82564', '21156', '15799', '86709', '37931', '74703', '83103', '23054', - | '70470', '72008', '35709', '91911', '69998', - | '20961', '70070', '63197', '54853', '88191', '91830', '49521', '19454', '81450', - | '89091', '62378', '31904', '61869', '51744', - | '36580', '85778', '36871', '48121', '28810', '83712', '45486', '67393', '26935', - | '42393', '20132', '55349', '86057', '21309', - | '80218', '10094', '11357', '48819', '39734', '40758', '30432', '21204', '29467', - | '30214', '61024', '55307', '74621', '11622', - | '68908', '33032', '52868', '99194', '99900', '84936', '69036', '99149', '45013', - | '32895', '59004', '32322', '14933', '32936', - | '33562', '72550', '27385', '58049', '58200', '16808', '21360', '32961', '18586', - | '79307', '15492')) a01 - | inner join - | (select ca_zip - | from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt - | FROM customer_address, customer - | WHERE ca_address_sk = c_current_addr_sk and - | c_preferred_cust_flag='Y' - | group by ca_zip - | having count(*) > 10)A1 - | ) b11 - | on (a01.ca_zip = b11.ca_zip )) A2 - | where ss_store_sk = s_store_sk - | and ss_sold_date_sk = d_date_sk - | and ss_sold_date_sk between 2451271 and 2451361 - | and d_qoy = 2 and d_year = 1999 - | and (substr(s_zip,1,2) = substr(a2.ca_zip,1,2)) - | group by s_store_name - | order by s_store_name - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q8 1737 / 2197 8.7 115.6 1.0X - */ - - ("q82", """ - |select - | i_item_id, - | i_item_desc, - | i_current_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join inventory on (item.i_item_sk = inventory.inv_item_sk) - | join date_dim on (inventory.inv_date_sk = date_dim.d_date_sk) - |where - | i_current_price between 30 and 30 + 30 - | and i_manufact_id in (437, 129, 727, 663) - | and inv_quantity_on_hand between 100 and 500 - |group by - | i_item_id, - | i_item_desc, - | i_current_price - |order by - | i_item_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q82 9399 / 10245 6.8 147.2 1.0X - */ - - ("q89", """ - |select - | * - |from - | (select - | i_category, - | i_class, - | i_brand, - | s_store_name, - | s_company_name, - | d_moy, - | sum(ss_sales_price) sum_sales - | from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451545 and 2451910 -- partition key filter - | and d_year in (2000) - | and ((i_category in('Home', 'Books', 'Electronics') - | and i_class in('wallpaper', 'parenting', 'musical')) - | or (i_category in('Shoes', 'Jewelry', 'Men') - | and i_class in('womens', 'birdal', 'pants')) - | ) - | group by - | i_category, - | i_class, - | i_brand, - | s_store_name, - | s_company_name, - | d_moy - | ) tmp1 - |order by - | sum_sales, - | s_store_name - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q89 1122 / 1274 12.9 77.2 1.0X - */ - - ("q98", """ - |select - | i_item_desc, - | i_category, - | i_class, - | i_current_price, - | sum(ss_ext_sales_price) as itemrevenue - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | ss_sold_date_sk between 2451911 and 2451941 - | -- partition key filter (1 calendar month) - | and d_date between '2001-01-01' and '2001-01-31' - | and i_category in('Jewelry', 'Sports', 'Books') - |group by - | i_item_id, - | i_item_desc, - | i_category, - | i_class, - | i_current_price - |order by - | i_category, - | i_class, - | i_item_id, - | i_item_desc - | -- revenueratio - |limit 1000 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q98 1235 / 1542 11.8 85.0 1.0X - */ - - ("ss_max", """ - |select - | count(*) as total, - | max(ss_sold_date_sk) as max_ss_sold_date_sk, - | max(ss_sold_time_sk) as max_ss_sold_time_sk, - | max(ss_item_sk) as max_ss_item_sk, - | max(ss_customer_sk) as max_ss_customer_sk, - | max(ss_cdemo_sk) as max_ss_cdemo_sk, - | max(ss_hdemo_sk) as max_ss_hdemo_sk, - | max(ss_addr_sk) as max_ss_addr_sk, - | max(ss_store_sk) as max_ss_store_sk, - | max(ss_promo_sk) as max_ss_promo_sk - |from store_sales - """.stripMargin) - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - ss_max 2305 / 2731 6.2 160.0 1.0X - */ - - ).toArray - - val tables = Seq("customer", "customer_address", "customer_demographics", "date_dim", - "household_demographics", "inventory", "item", "promotion", "store", "catalog_sales", - "web_sales", "store_sales") - - def setupTables(dataLocation: String): Map[String, Long] = { - tables.map { tableName => - spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) - tableName -> spark.table(tableName).count() - }.toMap - } - - def tpcdsAll(dataLocation: String): Unit = { - require(dataLocation.nonEmpty, - "please modify the value of dataLocation to point to your local TPCDS data") - val tableSizes = setupTables(dataLocation) - spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - tpcds.filter(q => q._1 != "").foreach { - case (name: String, query: String) => - val numRows = spark.sql(query).queryExecution.logical.map { - case ur@UnresolvedRelation(t: TableIdentifier, _) => - tableSizes.getOrElse(t.table, throw new RuntimeException(s"${t.table} not found.")) - case _ => 0L - }.sum - val benchmark = new Benchmark("TPCDS Snappy (scale = 5)", numRows, 5) - benchmark.addCase(name) { i => - spark.sql(query).collect() - } - benchmark.run() - } - } - - def main(args: Array[String]): Unit = { - - // In order to run this benchmark, please follow the instructions at - // https://github.com/databricks/spark-sql-perf/blob/master/README.md to generate the TPCDS data - // locally (preferably with a scale factor of 5 for benchmarking). Thereafter, the value of - // dataLocation below needs to be set to the location where the generated data is stored. - val dataLocation = "" - - tpcdsAll(dataLocation) - } -} From 06c9f520714e07259c6f8ce6f9ea5a230a278cb5 Mon Sep 17 00:00:00 2001 From: tedyu Date: Fri, 20 May 2016 18:13:18 -0500 Subject: [PATCH 074/143] [SPARK-15273] YarnSparkHadoopUtil#getOutOfMemoryErrorArgument should respect OnOutOfMemoryError parameter given by user ## What changes were proposed in this pull request? As Nirav reported in this thread: http://search-hadoop.com/m/q3RTtdF3yNLMd7u YarnSparkHadoopUtil#getOutOfMemoryErrorArgument previously specified 'kill %p' unconditionally. We should respect the parameter given by user. ## How was this patch tested? Existing tests Author: tedyu Closes #13057 from tedyu/master. --- .../spark/deploy/yarn/ExecutorRunnable.scala | 9 ++------ .../deploy/yarn/YarnSparkHadoopUtil.scala | 23 ++++++++++++------- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 3d370e6d71426..fc753b7e750cb 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -211,15 +211,10 @@ private[yarn] class ExecutorRunnable( Seq("--user-class-path", "file:" + absPath) }.toSeq + YarnSparkHadoopUtil.addOutOfMemoryErrorArgument(javaOpts) val commands = prefixEnv ++ Seq( YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", - "-server", - // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in - // an inconsistent state. - // TODO: If the OOM is not recoverable by rescheduling it on different node, then do - // 'something' to fail job ... akin to blacklisting trackers in mapred ? - YarnSparkHadoopUtil.getOutOfMemoryErrorArgument) ++ + "-server") ++ javaOpts ++ Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", "--driver-url", masterAddress.toString, 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 44181610d770d..de6cd946137cb 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 @@ -24,7 +24,8 @@ import java.security.PrivilegedExceptionAction import java.util.regex.Matcher import java.util.regex.Pattern -import scala.collection.mutable.HashMap +import scala.collection.JavaConverters._ +import scala.collection.mutable.{HashMap, ListBuffer} import scala.reflect.runtime._ import scala.util.Try @@ -405,6 +406,12 @@ object YarnSparkHadoopUtil { } /** + * Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. + * Not killing the task leaves various aspects of the executor and (to some extent) the jvm in + * an inconsistent state. + * TODO: If the OOM is not recoverable by rescheduling it on different node, then do + * 'something' to fail job ... akin to blacklisting trackers in mapred ? + * * The handler if an OOM Exception is thrown by the JVM must be configured on Windows * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'. * @@ -415,14 +422,14 @@ object YarnSparkHadoopUtil { * the behavior of '%' in a .cmd file: it gets interpreted as an incomplete environment * variable. Windows .cmd files escape a '%' by '%%'. Thus, the correct way of writing * '%%p' in an escaped way is '%%%%p'. - * - * @return The correct OOM Error handler JVM option, platform dependent. */ - def getOutOfMemoryErrorArgument: String = { - if (Utils.isWindows) { - escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") - } else { - "-XX:OnOutOfMemoryError='kill %p'" + private[yarn] def addOutOfMemoryErrorArgument(javaOpts: ListBuffer[String]): Unit = { + if (!javaOpts.exists(_.contains("-XX:OnOutOfMemoryError"))) { + if (Utils.isWindows) { + javaOpts += escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") + } else { + javaOpts += "-XX:OnOutOfMemoryError='kill %p'" + } } } From 127bf1bb07967e2e4f99ad7abaa7f6fab3b3f407 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 20 May 2016 16:40:33 -0700 Subject: [PATCH 075/143] [SPARK-15031][EXAMPLE] Use SparkSession in examples ## What changes were proposed in this pull request? Use `SparkSession` according to [SPARK-15031](https://issues.apache.org/jira/browse/SPARK-15031) `MLLLIB` is not recommended to use now, so examples in `MLLIB` are ignored in this PR. `StreamingContext` can not be directly obtained from `SparkSession`, so example in `Streaming` are ignored too. cc andrewor14 ## How was this patch tested? manual tests with spark-submit Author: Zheng RuiFeng Closes #13164 from zhengruifeng/use_sparksession_ii. --- .../org/apache/spark/examples/JavaHdfsLR.java | 14 ++++++++------ .../apache/spark/examples/JavaLogQuery.java | 12 ++++++++---- .../apache/spark/examples/JavaPageRank.java | 13 +++++++------ .../apache/spark/examples/JavaSparkPi.java | 12 ++++++++---- .../spark/examples/JavaStatusTrackerDemo.java | 19 ++++++++++++------- .../org/apache/spark/examples/JavaTC.java | 15 ++++++++++----- .../apache/spark/examples/JavaWordCount.java | 15 +++++++++------ examples/src/main/python/als.py | 12 +++++++++--- examples/src/main/python/avro_inputformat.py | 12 +++++++++--- examples/src/main/python/kmeans.py | 12 ++++++++---- .../src/main/python/logistic_regression.py | 13 +++++++++---- examples/src/main/python/pagerank.py | 11 +++++++---- .../src/main/python/parquet_inputformat.py | 12 +++++++++--- examples/src/main/python/pi.py | 12 +++++++++--- examples/src/main/python/sort.py | 13 +++++++++---- .../src/main/python/transitive_closure.py | 12 +++++++++--- examples/src/main/python/wordcount.py | 13 +++++++++---- .../apache/spark/examples/BroadcastTest.scala | 16 ++++++++++++---- .../spark/examples/DFSReadWriteTest.scala | 13 ++++++++----- .../examples/ExceptionHandlingTest.scala | 12 ++++++++---- .../apache/spark/examples/GroupByTest.scala | 12 ++++++++---- .../org/apache/spark/examples/HdfsTest.scala | 12 +++++++----- .../spark/examples/MultiBroadcastTest.scala | 13 +++++++++---- .../examples/SimpleSkewedGroupByTest.scala | 13 ++++++++----- .../spark/examples/SkewedGroupByTest.scala | 13 +++++++++---- .../org/apache/spark/examples/SparkALS.scala | 12 ++++++++---- .../apache/spark/examples/SparkHdfsLR.scala | 16 +++++++++------- .../apache/spark/examples/SparkKMeans.scala | 15 +++++++++------ .../org/apache/spark/examples/SparkLR.scala | 13 +++++++++---- .../apache/spark/examples/SparkPageRank.scala | 13 ++++++++----- .../org/apache/spark/examples/SparkPi.scala | 11 +++++++---- .../org/apache/spark/examples/SparkTC.scala | 11 +++++++---- .../examples/sql/hive/HiveFromSpark.scala | 2 +- .../apache/spark/sql/JavaDataFrameSuite.java | 6 +++--- 34 files changed, 279 insertions(+), 146 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 f64155ce3c0cf..ded442096c257 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -17,11 +17,10 @@ package org.apache.spark.examples; -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.api.java.function.Function2; +import org.apache.spark.sql.SparkSession; import java.io.Serializable; import java.util.Arrays; @@ -122,9 +121,12 @@ public static void main(String[] args) { showWarning(); - SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - JavaRDD lines = sc.textFile(args[0]); + SparkSession spark = SparkSession + .builder() + .appName("JavaHdfsLR") + .getOrCreate(); + + JavaRDD lines = spark.read().text(args[0]).javaRDD(); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[1]); @@ -152,6 +154,6 @@ public static void main(String[] args) { System.out.print("Final w: "); printWeights(w); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index ebb0687b14ae0..7775443861661 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -20,12 +20,13 @@ import com.google.common.collect.Lists; import scala.Tuple2; import scala.Tuple3; -import org.apache.spark.SparkConf; + 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.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; import java.io.Serializable; import java.util.List; @@ -99,9 +100,12 @@ public static Stats extractStats(String line) { } public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaLogQuery") + .getOrCreate(); - SparkConf sparkConf = new SparkConf().setAppName("JavaLogQuery"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); JavaRDD dataSet = (args.length == 1) ? jsc.textFile(args[0]) : jsc.parallelize(exampleApacheLogs); @@ -123,6 +127,6 @@ public Stats call(Stats stats, Stats stats2) { for (Tuple2 t : output) { System.out.println(t._1() + "\t" + t._2()); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 229d1234414e5..128b5ab17c8d4 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -26,14 +26,13 @@ import com.google.common.collect.Iterables; -import org.apache.spark.SparkConf; 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.api.java.function.Function2; import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; /** * Computes the PageRank of URLs from an input file. Input file should @@ -73,15 +72,17 @@ public static void main(String[] args) throws Exception { showWarning(); - SparkConf sparkConf = new SparkConf().setAppName("JavaPageRank"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaPageRank") + .getOrCreate(); // Loads in input file. It should be in format of: // URL neighbor URL // URL neighbor URL // URL neighbor URL // ... - JavaRDD lines = ctx.textFile(args[0], 1); + JavaRDD lines = spark.read().text(args[0]).javaRDD(); // Loads all URLs from input file and initialize their neighbors. JavaPairRDD> links = lines.mapToPair( @@ -132,6 +133,6 @@ public Double call(Double sum) { System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 04a57a6bfb58b..7df145e3117b8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -17,11 +17,11 @@ package org.apache.spark.examples; -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.api.java.function.Function2; +import org.apache.spark.sql.SparkSession; import java.util.ArrayList; import java.util.List; @@ -33,8 +33,12 @@ public final class JavaSparkPi { public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName("JavaSparkPi"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaSparkPi") + .getOrCreate(); + + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); int slices = (args.length == 1) ? Integer.parseInt(args[0]) : 2; int n = 100000 * slices; @@ -61,6 +65,6 @@ public Integer call(Integer integer, Integer integer2) { System.out.println("Pi is roughly " + 4.0 * count / n); - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java index e68ec74c3ed54..6f899c772eb98 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java @@ -17,13 +17,14 @@ package org.apache.spark.examples; -import org.apache.spark.SparkConf; import org.apache.spark.SparkJobInfo; import org.apache.spark.SparkStageInfo; import org.apache.spark.api.java.JavaFutureAction; 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.SparkSession; + import java.util.Arrays; import java.util.List; @@ -44,11 +45,15 @@ public T call(T x) throws Exception { } public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName(APP_NAME); - final JavaSparkContext sc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName(APP_NAME) + .getOrCreate(); + + final JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); // Example of implementing a progress reporter for a simple job. - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( + JavaRDD rdd = jsc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( new IdentityWithDelay()); JavaFutureAction> jobFuture = rdd.collectAsync(); while (!jobFuture.isDone()) { @@ -58,13 +63,13 @@ public static void main(String[] args) throws Exception { continue; } int currentJobId = jobIds.get(jobIds.size() - 1); - SparkJobInfo jobInfo = sc.statusTracker().getJobInfo(currentJobId); - SparkStageInfo stageInfo = sc.statusTracker().getStageInfo(jobInfo.stageIds()[0]); + SparkJobInfo jobInfo = jsc.statusTracker().getJobInfo(currentJobId); + SparkStageInfo stageInfo = jsc.statusTracker().getStageInfo(jobInfo.stageIds()[0]); System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() + " active, " + stageInfo.numCompletedTasks() + " complete"); } System.out.println("Job results are: " + jobFuture.get()); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index ca10384212da2..f12ca77ed1eb0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -25,10 +25,10 @@ import scala.Tuple2; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; /** * Transitive closure on a graph, implemented in Java. @@ -64,10 +64,15 @@ public Tuple2 call(Tuple2> t } public static void main(String[] args) { - SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaTC") + .getOrCreate(); + + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); + Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; - JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); + JavaPairRDD tc = jsc.parallelizePairs(generateGraph(), slices).cache(); // Linear transitive closure: each round grows paths by one edge, // by joining the graph's edges with the already-discovered paths. @@ -94,6 +99,6 @@ public Tuple2 call(Tuple2 e) { } while (nextCount != oldCount); System.out.println("TC has " + tc.count() + " edges."); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 3ff5412b934f0..1caee60e348d2 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -18,13 +18,13 @@ package org.apache.spark.examples; import scala.Tuple2; -import org.apache.spark.SparkConf; + 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.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; import java.util.Arrays; import java.util.Iterator; @@ -41,9 +41,12 @@ public static void main(String[] args) throws Exception { System.exit(1); } - SparkConf sparkConf = new SparkConf().setAppName("JavaWordCount"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); - JavaRDD lines = ctx.textFile(args[0], 1); + SparkSession spark = SparkSession + .builder() + .appName("JavaWordCount") + .getOrCreate(); + + JavaRDD lines = spark.read().text(args[0]).javaRDD(); JavaRDD words = lines.flatMap(new FlatMapFunction() { @Override @@ -72,6 +75,6 @@ public Integer call(Integer i1, Integer i2) { for (Tuple2 tuple : output) { System.out.println(tuple._1() + ": " + tuple._2()); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index f07020b50397b..81562e20a927d 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -28,7 +28,7 @@ import numpy as np from numpy.random import rand from numpy import matrix -from pyspark import SparkContext +from pyspark.sql import SparkSession LAMBDA = 0.01 # regularization np.random.seed(42) @@ -62,7 +62,13 @@ def update(i, vec, mat, ratings): example. Please use pyspark.ml.recommendation.ALS for more conventional use.""", file=sys.stderr) - sc = SparkContext(appName="PythonALS") + spark = SparkSession\ + .builder\ + .appName("PythonALS")\ + .getOrCreate() + + sc = spark._sc + M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 F = int(sys.argv[3]) if len(sys.argv) > 3 else 10 @@ -99,4 +105,4 @@ def update(i, vec, mat, ratings): print("Iteration %d:" % i) print("\nRMSE: %5.4f\n" % error) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index da368ac628a49..3f65e8f79a65c 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -19,8 +19,8 @@ import sys -from pyspark import SparkContext from functools import reduce +from pyspark.sql import SparkSession """ Read data file users.avro in local Spark distro: @@ -64,7 +64,13 @@ exit(-1) path = sys.argv[1] - sc = SparkContext(appName="AvroKeyInputFormat") + + spark = SparkSession\ + .builder\ + .appName("AvroKeyInputFormat")\ + .getOrCreate() + + sc = spark._sc conf = None if len(sys.argv) == 3: @@ -82,4 +88,4 @@ for k in output: print(k) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 3426e491dc74a..92e0a3ae2ee60 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -27,7 +27,7 @@ import sys import numpy as np -from pyspark import SparkContext +from pyspark.sql import SparkSession def parseVector(line): @@ -55,8 +55,12 @@ def closestPoint(p, centers): 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]) + spark = SparkSession\ + .builder\ + .appName("PythonKMeans")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) data = lines.map(parseVector).cache() K = int(sys.argv[2]) convergeDist = float(sys.argv[3]) @@ -79,4 +83,4 @@ def closestPoint(p, centers): print("Final centers: " + str(kPoints)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 7d33be7e81d73..01c938454b108 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -27,7 +27,7 @@ import sys import numpy as np -from pyspark import SparkContext +from pyspark.sql import SparkSession D = 10 # Number of dimensions @@ -55,8 +55,13 @@ def readPointBatch(iterator): 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() + spark = SparkSession\ + .builder\ + .appName("PythonLR")\ + .getOrCreate() + + points = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0])\ + .mapPartitions(readPointBatch).cache() iterations = int(sys.argv[2]) # Initialize w to a random value @@ -80,4 +85,4 @@ def add(x, y): print("Final w: " + str(w)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index 2fdc9773d4eb1..a399a9c37c5d5 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -25,7 +25,7 @@ import sys from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession def computeContribs(urls, rank): @@ -51,14 +51,17 @@ def parseNeighbors(urls): file=sys.stderr) # Initialize the spark context. - sc = SparkContext(appName="PythonPageRank") + spark = SparkSession\ + .builder\ + .appName("PythonPageRank")\ + .getOrCreate() # Loads in input file. It should be in format of: # URL neighbor URL # URL neighbor URL # URL neighbor URL # ... - lines = sc.textFile(sys.argv[1], 1) + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) # Loads all URLs from input file and initialize their neighbors. links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() @@ -79,4 +82,4 @@ def parseNeighbors(urls): for (link, rank) in ranks.collect(): print("%s has rank: %s." % (link, rank)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index e1fd85b082c08..2f09f4d5730e8 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -18,7 +18,7 @@ import sys -from pyspark import SparkContext +from pyspark.sql import SparkSession """ Read data file users.parquet in local Spark distro: @@ -47,7 +47,13 @@ exit(-1) path = sys.argv[1] - sc = SparkContext(appName="ParquetInputFormat") + + spark = SparkSession\ + .builder\ + .appName("ParquetInputFormat")\ + .getOrCreate() + + sc = spark._sc parquet_rdd = sc.newAPIHadoopFile( path, @@ -59,4 +65,4 @@ for k in output: print(k) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 92e5cf45abc8b..5db03e4a21091 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -20,14 +20,20 @@ from random import random from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": """ Usage: pi [partitions] """ - sc = SparkContext(appName="PythonPi") + spark = SparkSession\ + .builder\ + .appName("PythonPi")\ + .getOrCreate() + + sc = spark._sc + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * partitions @@ -39,4 +45,4 @@ def f(_): count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) print("Pi is roughly %f" % (4.0 * count / n)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index b6c2916254056..81898cf6d5ce6 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -19,15 +19,20 @@ import sys -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": if len(sys.argv) != 2: print("Usage: sort ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="PythonSort") - lines = sc.textFile(sys.argv[1], 1) + + spark = SparkSession\ + .builder\ + .appName("PythonSort")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) sortedCount = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (int(x), 1)) \ .sortByKey() @@ -37,4 +42,4 @@ for (num, unitcount) in output: print(num) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 3d61250d8b230..37c41dcd03dc6 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -20,7 +20,7 @@ import sys from random import Random -from pyspark import SparkContext +from pyspark.sql import SparkSession numEdges = 200 numVertices = 100 @@ -41,7 +41,13 @@ def generateGraph(): """ Usage: transitive_closure [partitions] """ - sc = SparkContext(appName="PythonTransitiveClosure") + spark = SparkSession\ + .builder\ + .appName("PythonTransitiveClosure")\ + .getOrCreate() + + sc = spark._sc + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 tc = sc.parallelize(generateGraph(), partitions).cache() @@ -67,4 +73,4 @@ def generateGraph(): print("TC has %i edges" % tc.count()) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index 7c0143607b61d..3d5e44d5b2df1 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -20,15 +20,20 @@ import sys from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": if len(sys.argv) != 2: print("Usage: wordcount ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="PythonWordCount") - lines = sc.textFile(sys.argv[1], 1) + + spark = SparkSession\ + .builder\ + .appName("PythonWordCount")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) counts = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (x, 1)) \ .reduceByKey(add) @@ -36,4 +41,4 @@ for (word, count) in output: print("%s: %i" % (word, count)) - sc.stop() + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index af5a815f6ec76..c50f25d951947 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -18,7 +18,8 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession /** * Usage: BroadcastTest [slices] [numElem] [blockSize] @@ -28,9 +29,16 @@ object BroadcastTest { val blockSize = if (args.length > 2) args(2) else "4096" - val sparkConf = new SparkConf().setAppName("Broadcast Test") + val sparkConf = new SparkConf() .set("spark.broadcast.blockSize", blockSize) - val sc = new SparkContext(sparkConf) + + val spark = SparkSession + .builder + .config(sparkConf) + .appName("Broadcast Test") + .getOrCreate() + + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 @@ -48,7 +56,7 @@ object BroadcastTest { println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6)) } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index 7bf023667dcae..4b5e36c73642d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -22,7 +22,7 @@ import java.io.File import scala.io.Source._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Simple test for reading and writing to a distributed @@ -101,11 +101,14 @@ object DFSReadWriteTest { val fileContents = readFile(localFilePath.toString()) val localWordCount = runLocalWordCount(fileContents) - println("Creating SparkConf") - val conf = new SparkConf().setAppName("DFS Read Write Test") + println("Creating SparkSession") + val spark = SparkSession + .builder + .appName("DFS Read Write Test") + .getOrCreate() println("Creating SparkContext") - val sc = new SparkContext(conf) + val sc = spark.sparkContext println("Writing local file to DFS") val dfsFilename = dfsDirPath + "/dfs_read_write_test" @@ -124,7 +127,7 @@ object DFSReadWriteTest { .values .sum - sc.stop() + spark.stop() if (localWordCount == dfsWordCount) { println(s"Success! Local Word Count ($localWordCount) " + diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index d42f63e87052e..6a1bbed290235 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -17,18 +17,22 @@ package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession object ExceptionHandlingTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("ExceptionHandlingTest") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("ExceptionHandlingTest") + .getOrCreate() + val sc = spark.sparkContext + sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") } } - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 4db229b5dec32..0cb61d7495fa6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -20,20 +20,24 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] */ object GroupByTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("GroupBy Test") + val spark = SparkSession + .builder + .appName("GroupBy Test") + .getOrCreate() + var numMappers = if (args.length > 0) args(0).toInt else 2 var numKVPairs = if (args.length > 1) args(1).toInt else 1000 var valSize = if (args.length > 2) args(2).toInt else 1000 var numReducers = if (args.length > 3) args(3).toInt else numMappers - val sc = new SparkContext(sparkConf) + val sc = spark.sparkContext val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random @@ -50,7 +54,7 @@ object GroupByTest { println(pairs1.groupByKey(numReducers).count()) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index 124dc9af6390f..aa8de69839e28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -18,7 +18,7 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark._ +import org.apache.spark.sql.SparkSession object HdfsTest { @@ -29,9 +29,11 @@ object HdfsTest { System.err.println("Usage: HdfsTest ") System.exit(1) } - val sparkConf = new SparkConf().setAppName("HdfsTest") - val sc = new SparkContext(sparkConf) - val file = sc.textFile(args(0)) + val spark = SparkSession + .builder + .appName("HdfsTest") + .getOrCreate() + val file = spark.read.text(args(0)).rdd val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { val start = System.currentTimeMillis() @@ -39,7 +41,7 @@ object HdfsTest { val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 3eb0c2772337a..961ab99200d29 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -18,8 +18,9 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession + /** * Usage: MultiBroadcastTest [slices] [numElem] @@ -27,8 +28,12 @@ import org.apache.spark.rdd.RDD object MultiBroadcastTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("Multi-Broadcast Test") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("Multi-Broadcast Test") + .getOrCreate() + + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 @@ -51,7 +56,7 @@ object MultiBroadcastTest { // Collect the small RDD so we can print the observed sizes locally. observedSizes.collect().foreach(i => println(i)) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index ec07e6323ee9a..255c2bfcee669 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -20,23 +20,26 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Usage: SimpleSkewedGroupByTest [numMappers] [numKVPairs] [valSize] [numReducers] [ratio] */ object SimpleSkewedGroupByTest { def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("SimpleSkewedGroupByTest") + .getOrCreate() + + val sc = spark.sparkContext - val sparkConf = new SparkConf().setAppName("SimpleSkewedGroupByTest") var numMappers = if (args.length > 0) args(0).toInt else 2 var numKVPairs = if (args.length > 1) args(1).toInt else 1000 var valSize = if (args.length > 2) args(2).toInt else 1000 var numReducers = if (args.length > 3) args(3).toInt else numMappers var ratio = if (args.length > 4) args(4).toInt else 5.0 - val sc = new SparkContext(sparkConf) - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random var result = new Array[(Int, Array[Byte])](numKVPairs) @@ -64,7 +67,7 @@ object SimpleSkewedGroupByTest { // .map{case (k,v) => (k, v.size)} // .collectAsMap) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 8e4c2b6229755..efd40147f7ca7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -20,20 +20,25 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] */ object SkewedGroupByTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("GroupBy Test") + val spark = SparkSession + .builder + .appName("GroupBy Test") + .getOrCreate() + + val sc = spark.sparkContext + var numMappers = if (args.length > 0) args(0).toInt else 2 var numKVPairs = if (args.length > 1) args(1).toInt else 1000 var valSize = if (args.length > 2) args(2).toInt else 1000 var numReducers = if (args.length > 3) args(3).toInt else numMappers - val sc = new SparkContext(sparkConf) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random @@ -54,7 +59,7 @@ object SkewedGroupByTest { println(pairs1.groupByKey(numReducers).count()) - sc.stop() + spark.stop() } } // scalastyle:on println 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 b06c629802006..8a3d08f459783 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples import org.apache.commons.math3.linear._ -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Alternating least squares matrix factorization. @@ -108,8 +108,12 @@ object SparkALS { println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS") - val sparkConf = new SparkConf().setAppName("SparkALS") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkALS") + .getOrCreate() + + val sc = spark.sparkContext val R = generateR() @@ -135,7 +139,7 @@ object SparkALS { println() } - sc.stop() + spark.stop() } private def randomVector(n: Int): RealVector = 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 c514eb0fa51ae..84f133e0116a2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -23,9 +23,8 @@ import java.util.Random import scala.math.exp import breeze.linalg.{DenseVector, Vector} -import org.apache.hadoop.conf.Configuration -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Logistic regression based classification. @@ -67,11 +66,14 @@ object SparkHdfsLR { showWarning() - val sparkConf = new SparkConf().setAppName("SparkHdfsLR") + val spark = SparkSession + .builder + .appName("SparkHdfsLR") + .getOrCreate() + val inputPath = args(0) - val conf = new Configuration() - val sc = new SparkContext(sparkConf) - val lines = sc.textFile(inputPath) + val lines = spark.read.text(inputPath).rdd + val points = lines.map(parsePoint).cache() val ITERATIONS = args(1).toInt @@ -88,7 +90,7 @@ object SparkHdfsLR { } println("Final w: " + w) - sc.stop() + spark.stop() } } // scalastyle:on println 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 676164806e196..aa93c93c441f4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples import breeze.linalg.{squaredDistance, DenseVector, Vector} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * K-means clustering. @@ -66,14 +66,17 @@ object SparkKMeans { showWarning() - val sparkConf = new SparkConf().setAppName("SparkKMeans") - val sc = new SparkContext(sparkConf) - val lines = sc.textFile(args(0)) + val spark = SparkSession + .builder + .appName("SparkKMeans") + .getOrCreate() + + val lines = spark.read.text(args(0)).rdd val data = lines.map(parseVector _).cache() val K = args(1).toInt val convergeDist = args(2).toDouble - val kPoints = data.takeSample(withReplacement = false, K, 42).toArray + val kPoints = data.takeSample(withReplacement = false, K, 42) var tempDist = 1.0 while(tempDist > convergeDist) { @@ -97,7 +100,7 @@ object SparkKMeans { println("Final centers:") kPoints.foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println 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 718f84f6450e0..8ef3aab6570d6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -24,7 +24,7 @@ import scala.math.exp import breeze.linalg.{DenseVector, Vector} -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Logistic regression based classification. @@ -63,8 +63,13 @@ object SparkLR { showWarning() - val sparkConf = new SparkConf().setAppName("SparkLR") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkLR") + .getOrCreate() + + val sc = spark.sparkContext + val numSlices = if (args.length > 0) args(0).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() @@ -82,7 +87,7 @@ object SparkLR { println("Final w: " + w) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 2664ddbb87d23..b7c363c7d4fae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -18,7 +18,7 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Computes the PageRank of URLs from an input file. Input file should @@ -50,10 +50,13 @@ object SparkPageRank { showWarning() - val sparkConf = new SparkConf().setAppName("PageRank") + val spark = SparkSession + .builder + .appName("SparkPageRank") + .getOrCreate() + val iters = if (args.length > 1) args(1).toInt else 10 - val ctx = new SparkContext(sparkConf) - val lines = ctx.textFile(args(0), 1) + val lines = spark.read.text(args(0)).rdd val links = lines.map{ s => val parts = s.split("\\s+") (parts(0), parts(1)) @@ -71,7 +74,7 @@ object SparkPageRank { val output = ranks.collect() output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + ".")) - ctx.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 818d4f2b81f82..5be8f3b073410 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -20,16 +20,19 @@ package org.apache.spark.examples import scala.math.random -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** Computes an approximation to pi */ object SparkPi { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("Spark Pi") - val spark = new SparkContext(conf) + val spark = SparkSession + .builder + .appName("Spark Pi") + .getOrCreate() + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow - val count = spark.parallelize(1 until n, slices).map { i => + val count = sc.parallelize(1 until n, slices).map { i => val x = random * 2 - 1 val y = random * 2 - 1 if (x*x + y*y < 1) 1 else 0 diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index fc7a1f859f602..46aa68b8b8195 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -21,7 +21,7 @@ package org.apache.spark.examples import scala.collection.mutable import scala.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Transitive closure on a graph. @@ -42,10 +42,13 @@ object SparkTC { } def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("SparkTC") - val spark = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkTC") + .getOrCreate() + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 - var tc = spark.parallelize(generateGraph, slices).cache() + var tc = sc.parallelize(generateGraph, slices).cache() // Linear transitive closure: each round grows paths by one edge, // by joining the graph's edges with the already-discovered paths. 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 7293cb51b2613..59bdfa09ad0f8 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 @@ -22,7 +22,7 @@ import java.io.File import com.google.common.io.{ByteStreams, Files} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkConf import org.apache.spark.sql._ object HiveFromSpark { 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 35a9f44feca64..1e8f1062c53ee 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 @@ -168,8 +168,8 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { Assert.assertEquals( new StructField("d", new ArrayType(DataTypes.StringType, true), true, Metadata.empty()), schema.apply("d")); - Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, Metadata.empty()), - schema.apply("e")); + 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, @@ -189,7 +189,7 @@ 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) + // Java.math.BigInteger is equavient to Spark Decimal(38,0) Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); } From 021c19702c720b4466b016498917d47f99000e13 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Fri, 20 May 2016 16:41:57 -0700 Subject: [PATCH 076/143] [SPARK-15456][PYSPARK] Fixed PySpark shell context initialization when HiveConf not present ## What changes were proposed in this pull request? When PySpark shell cannot find HiveConf, it will fallback to create a SparkSession from a SparkContext. This fixes a bug caused by using a variable to SparkContext before it was initialized. ## How was this patch tested? Manually starting PySpark shell and using the SparkContext Author: Bryan Cutler Closes #13237 from BryanCutler/pyspark-shell-session-context-SPARK-15456. --- python/pyspark/shell.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index ef46d3065e5e8..ac5ce87a3f0fd 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -44,9 +44,9 @@ .enableHiveSupport()\ .getOrCreate() except py4j.protocol.Py4JError: - spark = SparkSession(sc) + spark = SparkSession.builder.getOrCreate() except TypeError: - spark = SparkSession(sc) + spark = SparkSession.builder.getOrCreate() sc = spark.sparkContext atexit.register(lambda: sc.stop()) From 45b7557e61d440612d4ce49c31b5ef242fdefa54 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 May 2016 22:01:55 -0700 Subject: [PATCH 077/143] [SPARK-15424][SPARK-15437][SPARK-14807][SQL] Revert Create a hivecontext-compatibility module ## What changes were proposed in this pull request? I initially asked to create a hivecontext-compatibility module to put the HiveContext there. But we are so close to Spark 2.0 release and there is only a single class in it. It seems overkill to have an entire package, which makes it more inconvenient, for a single class. ## How was this patch tested? Tests were moved. Author: Reynold Xin Closes #13207 from rxin/SPARK-15424. --- dev/run-tests.py | 2 +- dev/sparktestsupport/modules.py | 12 ---- pom.xml | 1 - project/SparkBuild.scala | 6 +- .../apache/spark/sql/hive/HiveContext.scala | 0 .../hive/HiveContextCompatibilitySuite.scala | 5 +- sql/hivecontext-compatibility/pom.xml | 57 ------------------- 7 files changed, 6 insertions(+), 77 deletions(-) rename sql/{hivecontext-compatibility => hive}/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala (100%) rename sql/{hivecontext-compatibility => hive}/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala (95%) delete mode 100644 sql/hivecontext-compatibility/pom.xml diff --git a/dev/run-tests.py b/dev/run-tests.py index 7b3269752bac6..2030c4ab2308b 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -110,7 +110,7 @@ def determine_modules_to_test(changed_modules): ['graphx', 'examples'] >>> x = [x.name for x in determine_modules_to_test([modules.sql])] >>> x # doctest: +NORMALIZE_WHITESPACE - ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver', 'hivecontext-compatibility', + ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver', 'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] """ modules_to_test = set() diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 0d6aa7422a534..8e2364d2f71cb 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -158,18 +158,6 @@ def __hash__(self): ) -hivecontext_compatibility = Module( - name="hivecontext-compatibility", - dependencies=[hive], - source_file_regexes=[ - "sql/hivecontext-compatibility/", - ], - sbt_test_goals=[ - "hivecontext-compatibility/test" - ] -) - - sketch = Module( name="sketch", dependencies=[tags], diff --git a/pom.xml b/pom.xml index 9c13af17e4ed5..e778f77fa3151 100644 --- a/pom.xml +++ b/pom.xml @@ -100,7 +100,6 @@ sql/catalyst sql/core sql/hive - sql/hivecontext-compatibility assembly external/flume external/flume-sink diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 3ad9873f43679..f08ca7001f34d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -39,8 +39,8 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, hiveCompatibility) = Seq( - "catalyst", "sql", "hive", "hive-thriftserver", "hivecontext-compatibility" + val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer) = Seq( + "catalyst", "sql", "hive", "hive-thriftserver" ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( @@ -339,7 +339,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( - spark, hive, hiveThriftServer, hiveCompatibility, catalyst, repl, networkCommon, networkShuffle, networkYarn, + spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, unsafe, tags, sketch, mllibLocal ).contains(x) } diff --git a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala similarity index 100% rename from sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala diff --git a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala similarity index 95% rename from sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala index 1c1db72e27ffe..3aa8174702513 100644 --- a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive import org.scalatest.BeforeAndAfterEach -import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEach { @@ -29,7 +29,7 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac override def beforeAll(): Unit = { super.beforeAll() - sc = new SparkContext("local[4]", "test") + sc = SparkContext.getOrCreate(new SparkConf().setMaster("local").setAppName("test")) HiveUtils.newTemporaryConfiguration(useInMemoryDerby = true).foreach { case (k, v) => sc.hadoopConfiguration.set(k, v) } @@ -47,7 +47,6 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac override def afterAll(): Unit = { try { - sc.stop() sc = null hc = null } finally { diff --git a/sql/hivecontext-compatibility/pom.xml b/sql/hivecontext-compatibility/pom.xml deleted file mode 100644 index ed9ef8e27919e..0000000000000 --- a/sql/hivecontext-compatibility/pom.xml +++ /dev/null @@ -1,57 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-hivecontext-compatibility_2.11 - jar - Spark Project HiveContext Compatibility - http://spark.apache.org/ - - hivecontext-compatibility - - - - - org.apache.spark - spark-hive_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - From 666bf2e8357cb9a61eaa2270c414e635d9c331ce Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Sat, 21 May 2016 06:39:47 -0500 Subject: [PATCH 078/143] [SPARK-15445][SQL] Build fails for java 1.7 after adding java.mathBigInteger support ## What changes were proposed in this pull request? Using longValue() and then checking whether the value is in the range for a long manually. ## How was this patch tested? Existing tests Author: Sandeep Singh Closes #13223 from techaddict/SPARK-15445. --- .../org/apache/spark/sql/types/Decimal.scala | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) 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 b907f62802175..31604bad0ff80 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,6 +17,7 @@ package org.apache.spark.sql.types +import java.lang.{Long => JLong} import java.math.{BigInteger, MathContext, RoundingMode} import org.apache.spark.annotation.DeveloperApi @@ -132,17 +133,15 @@ final class Decimal extends Ordered[Decimal] with Serializable { * 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") - } + // TODO: Remove this once we migrate to java8 and use longValueExact() instead. + require( + bigintval.compareTo(LONG_MAX_BIG_INT) <= 0 && bigintval.compareTo(LONG_MIN_BIG_INT) >= 0, + s"BigInteger $bigintval too large for decimal") + this.decimalVal = null + this.longVal = bigintval.longValue() + this._precision = DecimalType.MAX_PRECISION + this._scale = 0 + this } /** @@ -382,6 +381,9 @@ object Decimal { private[sql] val ZERO = Decimal(0) private[sql] val ONE = Decimal(1) + private val LONG_MAX_BIG_INT = BigInteger.valueOf(JLong.MAX_VALUE) + private val LONG_MIN_BIG_INT = BigInteger.valueOf(JLong.MIN_VALUE) + def apply(value: Double): Decimal = new Decimal().set(value) def apply(value: Long): Decimal = new Decimal().set(value) From f39621c998a0fe91a5115f3f843c3ca8dd71c1ab Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 21 May 2016 08:11:14 -0700 Subject: [PATCH 079/143] [SPARK-15462][SQL][TEST] unresolved === false` is enough in testcases. ## What changes were proposed in this pull request? In only `catalyst` module, there exists 8 evaluation test cases on unresolved expressions. But, in real-world situation, those cases doesn't happen since they occurs exceptions before evaluations. ```scala scala> sql("select format_number(null, 3)") res0: org.apache.spark.sql.DataFrame = [format_number(CAST(NULL AS DOUBLE), 3): string] scala> sql("select format_number(cast(null as NULL), 3)") org.apache.spark.sql.catalyst.parser.ParseException: DataType null() is not supported.(line 1, pos 34) ``` This PR makes those testcases more realistic. ```scala - checkEvaluation(FormatNumber(Literal.create(null, NullType), Literal(3)), null) + assert(FormatNumber(Literal.create(null, NullType), Literal(3)).resolved === false) ``` Also, this PR also removes redundant `resolved` checking in `FoldablePropagation` optimizer. ## How was this patch tested? Pass the modified Jenkins tests. Author: Dongjoon Hyun Closes #13241 from dongjoon-hyun/SPARK-15462. --- .../apache/spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- .../apache/spark/sql/catalyst/expressions/CastSuite.scala | 7 ------- .../sql/catalyst/expressions/DateExpressionsSuite.scala | 6 +++--- .../sql/catalyst/expressions/StringExpressionsSuite.scala | 2 +- 4 files changed, 5 insertions(+), 12 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 a6fb34cbfb110..5e998d61880ea 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 @@ -673,7 +673,7 @@ 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 a: Alias if a.child.foldable => (a.toAttribute, a) } case _ => Nil }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 43af3592070fe..dfda7c50f2c05 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -548,7 +548,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = false)) assert(ret.resolved === false) - checkEvaluation(ret, Seq(null, true, false)) } { @@ -607,7 +606,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = false)) assert(ret.resolved === false) - checkEvaluation(ret, Map("a" -> null, "b" -> true, "c" -> false)) } { val ret = cast(map_notNull, MapType(IntegerType, StringType, valueContainsNull = true)) @@ -714,7 +712,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("b", BooleanType, nullable = true), StructField("c", BooleanType, nullable = false)))) assert(ret.resolved === false) - checkEvaluation(ret, InternalRow(null, true, false)) } { @@ -755,10 +752,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("l", LongType, nullable = true))))))) assert(ret.resolved === false) - checkEvaluation(ret, Row( - Seq(123, null, null), - Map("a" -> null, "b" -> true, "c" -> false), - Row(0L))) } test("cast between string and interval") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 53c66d8a754ed..6118a34d29eaa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -143,7 +143,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Seconds") { - checkEvaluation(Second(Literal.create(null, DateType)), null) + assert(Second(Literal.create(null, DateType)).resolved === false) checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) checkEvaluation(Second(Literal(ts)), 15) @@ -176,7 +176,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Hour") { - checkEvaluation(Hour(Literal.create(null, DateType)), null) + assert(Hour(Literal.create(null, DateType)).resolved === false) checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) checkEvaluation(Hour(Literal(ts)), 13) @@ -195,7 +195,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Minute") { - checkEvaluation(Minute(Literal.create(null, DateType)), null) + assert(Minute(Literal.create(null, DateType)).resolved === false) checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) checkEvaluation(Minute(Literal(ts)), 10) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 2cf8ca7000edc..c09c64fd6bf62 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -688,7 +688,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { Literal(Decimal(123123324123L) * Decimal(123123.21234d)), Literal(4)), "15,159,339,180,002,773.2778") checkEvaluation(FormatNumber(Literal.create(null, IntegerType), Literal(3)), null) - checkEvaluation(FormatNumber(Literal.create(null, NullType), Literal(3)), null) + assert(FormatNumber(Literal.create(null, NullType), Literal(3)).resolved === false) } test("find in set") { From 5e1ee28984b169eaab5d2f832921d32cf09de915 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Sat, 21 May 2016 08:36:08 -0700 Subject: [PATCH 080/143] [SPARK-15114][SQL] Column name generated by typed aggregate is super verbose ## What changes were proposed in this pull request? Generate a shorter default alias for `AggregateExpression `, In this PR, aggregate function name along with a index is used for generating the alias name. ```SQL val ds = Seq(1, 3, 2, 5).toDS() ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)).show() ``` Output before change. ```SQL +-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+ |typedsumdouble(unresolveddeserializer(upcast(input[0, int], IntegerType, - root class: "scala.Int"), value#1), upcast(value))|typedaverage(unresolveddeserializer(upcast(input[0, int], IntegerType, - root class: "scala.Int"), value#1), newInstance(class scala.Tuple2))| +-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+ | 11.0| 2.75| +-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+ ``` Output after change: ```SQL +-----------------+---------------+ |typedsumdouble_c1|typedaverage_c2| +-----------------+---------------+ | 11.0| 2.75| +-----------------+---------------+ ``` Note: There is one test in ParquetSuites.scala which shows that that the system picked alias name is not usable and is rejected. [test](https://github.com/apache/spark/blob/master/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala#L672-#L687) ## How was this patch tested? A new test was added in DataSetAggregatorSuite. Author: Dilip Biswal Closes #13045 from dilipbiswal/spark-15114. --- .../spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++-- .../spark/sql/catalyst/analysis/unresolved.scala | 7 +++++-- .../main/scala/org/apache/spark/sql/Column.scala | 16 +++++++++++++++- .../spark/sql/RelationalGroupedDataset.scala | 3 +++ .../spark/sql/DatasetAggregatorSuite.scala | 12 ++++++++++++ 5 files changed, 39 insertions(+), 5 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 2c269478ee7ef..9a92330f75f6f 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 @@ -177,14 +177,16 @@ class Analyzer( private def assignAliases(exprs: Seq[NamedExpression]) = { exprs.zipWithIndex.map { case (expr, i) => - expr.transformUp { case u @ UnresolvedAlias(child, optionalAliasName) => + expr.transformUp { case u @ UnresolvedAlias(child, optGenAliasFunc) => child match { case ne: NamedExpression => ne 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, toPrettySQL(e))() - case e => Alias(e, optionalAliasName.getOrElse(toPrettySQL(e)))() + case e if optGenAliasFunc.isDefined => + Alias(child, optGenAliasFunc.get.apply(e))() + case e => Alias(e, toPrettySQL(e))() } } }.asInstanceOf[Seq[NamedExpression]] 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 1f1897dc36df2..e953eda7843c9 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 @@ -325,10 +325,13 @@ case class UnresolvedExtractValue(child: Expression, extraction: Expression) * Holds the expression that has yet to be aliased. * * @param child The computation that is needs to be resolved during analysis. - * @param aliasName The name if specified to be associated with the result of computing [[child]] + * @param aliasFunc The function if specified to be called to generate an alias to associate + * with the result of computing [[child]] * */ -case class UnresolvedAlias(child: Expression, aliasName: Option[String] = None) +case class UnresolvedAlias( + child: Expression, + aliasFunc: Option[Expression => String] = None) extends UnaryExpression with NamedExpression with Unevaluable { override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 9b8334d334e4d..204af719b2c59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -24,6 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression @@ -37,6 +38,14 @@ private[sql] object Column { def apply(expr: Expression): Column = new Column(expr) def unapply(col: Column): Option[Expression] = Some(col.expr) + + private[sql] def generateAlias(e: Expression): String = { + e match { + case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => + a.aggregateFunction.toString + case expr => usePrettyExpression(expr).sql + } + } } /** @@ -145,7 +154,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { case jt: JsonTuple => MultiAlias(jt, Nil) - case func: UnresolvedFunction => UnresolvedAlias(func, Some(usePrettyExpression(func).sql)) + case func: UnresolvedFunction => UnresolvedAlias(func, Some(Column.generateAlias)) // If we have a top level Cast, there is a chance to give it a better alias, if there is a // NamedExpression under this Cast. @@ -156,9 +165,14 @@ class Column(protected[sql] val expr: Expression) extends Logging { case other => Alias(expr, usePrettyExpression(expr).sql)() } + case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => + UnresolvedAlias(a, Some(Column.generateAlias)) + case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } + + override def toString: String = usePrettyExpression(expr).sql override def equals(that: Any): Boolean = that match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 4f5bf633fab2e..b0e48a6553a45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Pivot} import org.apache.spark.sql.catalyst.util.usePrettyExpression +import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.NumericType @@ -73,6 +74,8 @@ class RelationalGroupedDataset protected[sql]( private[this] def alias(expr: Expression): NamedExpression = expr match { case u: UnresolvedAttribute => UnresolvedAlias(u) case expr: NamedExpression => expr + case a: AggregateExpression if (a.aggregateFunction.isInstanceOf[TypedAggregateExpression]) => + UnresolvedAlias(a, Some(Column.generateAlias)) case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index f1585ca3ff318..ead7bd9642eca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -240,4 +240,16 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { val df2 = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") checkAnswer(df2.agg(RowAgg.toColumn as "b").select("b"), Row(6) :: Nil) } + + test("spark-15114 shorter system generated alias names") { + val ds = Seq(1, 3, 2, 5).toDS() + assert(ds.select(typed.sum((i: Int) => i)).columns.head === "TypedSumDouble(int)") + val ds2 = ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)) + assert(ds2.columns.head === "TypedSumDouble(int)") + assert(ds2.columns.last === "TypedAverage(int)") + val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") + assert(df.groupBy($"j").agg(RowAgg.toColumn).columns.last == + "RowAgg(org.apache.spark.sql.Row)") + assert(df.groupBy($"j").agg(RowAgg.toColumn as "agg1").columns.last == "agg1") + } } From 201a51f36682726d78d9d2fe2c388093bb860ee0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 21 May 2016 12:46:25 -0700 Subject: [PATCH 081/143] [SPARK-15452][SQL] Mark aggregator API as experimental ## What changes were proposed in this pull request? The Aggregator API was introduced in 2.0 for Dataset. All typed Dataset APIs should still be marked as experimental in 2.0. ## How was this patch tested? N/A - annotation only change. Author: Reynold Xin Closes #13226 from rxin/SPARK-15452. --- .../scala/org/apache/spark/sql/expressions/Aggregator.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala index baae9dd2d5e3e..51179a528c503 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.expressions +import org.apache.spark.annotation.Experimental import org.apache.spark.sql.{Dataset, Encoder, TypedColumn} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression /** + * :: Experimental :: * A base class for user-defined aggregations, which can be used in [[Dataset]] operations to take * all of the elements of a group and reduce them to a single value. * @@ -48,6 +50,7 @@ import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression * @tparam OUT The type of the final output result. * @since 1.6.0 */ +@Experimental abstract class Aggregator[-IN, BUF, OUT] extends Serializable { /** From c18fa464f404ed2612f8c4d355cb0544b355975b Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Sat, 21 May 2016 16:08:31 -0700 Subject: [PATCH 082/143] [SPARK-15280] Input/Output] Refactored OrcOutputWriter and moved serialization to a new class. ## What changes were proposed in this pull request? Refactoring: Separated ORC serialization logic from OrcOutputWriter and moved to a new class called OrcSerializer. ## How was this patch tested? Manual tests & existing tests. Author: Ergin Seyfe Closes #13066 from seyfe/orc_serializer. --- .../spark/sql/hive/orc/OrcRelation.scala | 84 ++++++++++--------- 1 file changed, 45 insertions(+), 39 deletions(-) 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 6e55137dd78e1..38f50c112a236 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 @@ -149,39 +149,70 @@ private[sql] class DefaultSource } } -private[orc] class OrcOutputWriter( - path: String, - bucketId: Option[Int], - dataSchema: StructType, - context: TaskAttemptContext) - extends OutputWriter with HiveInspectors { +private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) + extends HiveInspectors { + + def serialize(row: InternalRow): Writable = { + wrapOrcStruct(cachedOrcStruct, structOI, row) + serializer.serialize(cachedOrcStruct, structOI) + } - private val serializer = { + private[this] val serializer = { val table = new Properties() table.setProperty("columns", dataSchema.fieldNames.mkString(",")) table.setProperty("columns.types", dataSchema.map(_.dataType.catalogString).mkString(":")) val serde = new OrcSerde - val configuration = context.getConfiguration - serde.initialize(configuration, table) + serde.initialize(conf, table) serde } - // Object inspector converted from the schema of the relation to be written. - private val structOI = { + // Object inspector converted from the schema of the relation to be serialized. + private[this] val structOI = { val typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(dataSchema.catalogString) OrcStruct.createObjectInspector(typeInfo.asInstanceOf[StructTypeInfo]) .asInstanceOf[SettableStructObjectInspector] } + private[this] val cachedOrcStruct = structOI.create().asInstanceOf[OrcStruct] + + private[this] def wrapOrcStruct( + struct: OrcStruct, + oi: SettableStructObjectInspector, + row: InternalRow): Unit = { + val fieldRefs = oi.getAllStructFieldRefs + var i = 0 + while (i < fieldRefs.size) { + + oi.setStructFieldData( + struct, + fieldRefs.get(i), + wrap( + row.get(i, dataSchema(i).dataType), + fieldRefs.get(i).getFieldObjectInspector, + dataSchema(i).dataType)) + i += 1 + } + } +} + +private[orc] class OrcOutputWriter( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext) + extends OutputWriter { + + private[this] val conf = context.getConfiguration + + private[this] val serializer = new OrcSerializer(dataSchema, conf) + // `OrcRecordWriter.close()` creates an empty file if no rows are written at all. We use this // flag to decide whether `OrcRecordWriter.close()` needs to be called. private var recordWriterInstantiated = false private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - - val conf = context.getConfiguration val uniqueWriteJobId = conf.get("spark.sql.sources.writeJobUUID") val taskAttemptId = context.getTaskAttemptID val partition = taskAttemptId.getTaskID.getId @@ -206,33 +237,8 @@ private[orc] class OrcOutputWriter( override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") - private def wrapOrcStruct( - struct: OrcStruct, - oi: SettableStructObjectInspector, - row: InternalRow): Unit = { - val fieldRefs = oi.getAllStructFieldRefs - var i = 0 - while (i < fieldRefs.size) { - - oi.setStructFieldData( - struct, - fieldRefs.get(i), - wrap( - row.get(i, dataSchema(i).dataType), - fieldRefs.get(i).getFieldObjectInspector, - dataSchema(i).dataType)) - i += 1 - } - } - - val cachedOrcStruct = structOI.create().asInstanceOf[OrcStruct] - override protected[sql] def writeInternal(row: InternalRow): Unit = { - wrapOrcStruct(cachedOrcStruct, structOI, row) - - recordWriter.write( - NullWritable.get(), - serializer.serialize(cachedOrcStruct, structOI)) + recordWriter.write(NullWritable.get(), serializer.serialize(row)) } override def close(): Unit = { From 8f0a3d5bcba313dc3b70d4aa9a8ba2aa2d276062 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 21 May 2016 20:07:34 -0700 Subject: [PATCH 083/143] [SPARK-15330][SQL] Implement Reset Command #### What changes were proposed in this pull request? Like `Set` Command in Hive, `Reset` is also supported by Hive. See the link: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli Below is the related Hive JIRA: https://issues.apache.org/jira/browse/HIVE-3202 This PR is to implement such a command for resetting the SQL-related configuration to the default values. One of the use case shown in HIVE-3202 is listed below: > For the purpose of optimization we set various configs per query. It's worthy but all those configs should be reset every time for next query. #### How was this patch tested? Added a test case. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13121 from gatorsmile/resetCommand. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 12 +++++ .../sql/execution/command/SetCommand.scala | 16 ++++++ .../spark/sql/internal/SQLConfSuite.scala | 49 ++++++++++++++++++- .../hive/thriftserver/SparkSQLCLIDriver.scala | 6 +-- 5 files changed, 82 insertions(+), 5 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 06ac37b7f83ed..848c59e3b8a53 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 @@ -120,6 +120,7 @@ statement | ADD identifier .*? #addResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration + | RESET #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -633,7 +634,7 @@ nonReserved | GROUPING | CUBE | ROLLUP | EXPLAIN | FORMAT | LOGICAL | FORMATTED | CODEGEN | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF - | SET + | SET | RESET | VIEW | REPLACE | IF | NO | DATA @@ -748,6 +749,7 @@ MAP: 'MAP'; STRUCT: 'STRUCT'; COMMENT: 'COMMENT'; SET: 'SET'; +RESET: 'RESET'; DATA: 'DATA'; START: 'START'; TRANSACTION: 'TRANSACTION'; 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 2966eefd07c77..2e3ac9706daf8 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 @@ -75,6 +75,18 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } } + /** + * Create a [[ResetCommand]] logical plan. + * Example SQL : + * {{{ + * RESET; + * }}} + */ + override def visitResetConfiguration( + ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { + ResetCommand + } + /** * Create an [[AnalyzeTableCommand]] command. This currently only implements the NOSCAN * option (other options are passed on to Hive) e.g.: 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 282f26ce998fe..b0e2d03af070d 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 @@ -116,3 +116,19 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm override def run(sparkSession: SparkSession): Seq[Row] = runFunc(sparkSession) } + +/** + * This command is for resetting SQLConf to the default values. Command that runs + * {{{ + * reset; + * }}} + */ +case object ResetCommand extends RunnableCommand with Logging { + + override def run(sparkSession: SparkSession): Seq[Row] = { + sparkSession.sessionState.conf.clear() + Seq.empty[Row] + } + + override val output: Seq[Attribute] = Seq.empty +} 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 0296229100a24..f8227e3bd6ee8 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 @@ -99,7 +99,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { test("deprecated property") { spark.sqlContext.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) - try{ + try { sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") assert(spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) === 10) } finally { @@ -107,6 +107,53 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + test("reset - public conf") { + spark.sqlContext.conf.clear() + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + try { + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) + sql(s"reset") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + } + } + + test("reset - internal conf") { + spark.sqlContext.conf.clear() + val original = spark.conf.get(SQLConf.NATIVE_VIEW) + try { + assert(spark.conf.get(SQLConf.NATIVE_VIEW) === true) + sql(s"set ${SQLConf.NATIVE_VIEW.key}=false") + assert(spark.conf.get(SQLConf.NATIVE_VIEW) === false) + assert(sql(s"set").where(s"key = '${SQLConf.NATIVE_VIEW.key}'").count() == 1) + sql(s"reset") + assert(spark.conf.get(SQLConf.NATIVE_VIEW) === true) + assert(sql(s"set").where(s"key = '${SQLConf.NATIVE_VIEW.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.NATIVE_VIEW}=$original") + } + } + + test("reset - user-defined conf") { + spark.sqlContext.conf.clear() + val userDefinedConf = "x.y.z.reset" + try { + assert(spark.conf.getOption(userDefinedConf).isEmpty) + sql(s"set $userDefinedConf=false") + assert(spark.conf.get(userDefinedConf) === "false") + assert(sql(s"set").where(s"key = '$userDefinedConf'").count() == 1) + sql(s"reset") + assert(spark.conf.getOption(userDefinedConf).isEmpty) + } finally { + spark.conf.unset(userDefinedConf) + } + } + test("invalid conf value") { spark.sqlContext.conf.clear() val e = intercept[IllegalArgumentException] { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 1402e0a687290..33ff8aee79969 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -32,8 +32,8 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, CommandProcessor, - CommandProcessorFactory, SetProcessor} +import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, CommandProcessor} +import org.apache.hadoop.hive.ql.processors.{CommandProcessorFactory, ResetProcessor, SetProcessor} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket @@ -312,7 +312,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (proc != null) { // scalastyle:off println if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || - proc.isInstanceOf[AddResourceProcessor]) { + proc.isInstanceOf[AddResourceProcessor] || proc.isInstanceOf[ResetProcessor]) { val driver = new SparkSQLDriver driver.init() From df9adb5ec994f054b2fa58e492867bbc5a60c234 Mon Sep 17 00:00:00 2001 From: xin Wu Date: Sat, 21 May 2016 21:41:12 -0700 Subject: [PATCH 084/143] [SPARK-15206][SQL] add testcases for distinct aggregate in having clause ## What changes were proposed in this pull request? Add new test cases for including distinct aggregate in having clause in 2.0 branch. This is a followup PR for [#12974](https://github.com/apache/spark/pull/12974), which is for 1.6 branch. Author: xin Wu Closes #12984 from xwu0226/SPARK-15206. --- .../execution/AggregationQuerySuite.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) 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 a2bae2e81f30a..9fc5628b28dbb 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 @@ -958,6 +958,37 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(11) :: Nil) } } + + test("SPARK-15206: single distinct aggregate function in having clause") { + checkAnswer( + sql( + """ + |select key, count(distinct value1) + |from agg2 group by key + |having count(distinct value1) > 0 + """.stripMargin), + Seq( + Row(null, 3), + Row(1, 2), + Row(2, 2) + ) + ) + } + + test("SPARK-15206: multiple distinct aggregate function in having clause") { + checkAnswer( + sql( + """ + |select key, count(distinct value1), count(distinct value2) + |from agg2 group by key + |having count(distinct value1) > 0 and count(distinct value2) = 3 + """.stripMargin), + Seq( + Row(null, 3, 3), + Row(1, 2, 3) + ) + ) + } } From 223f6339088434eb3590c2f42091a38f05f1e5db Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Sat, 21 May 2016 23:01:14 -0700 Subject: [PATCH 085/143] [SPARK-15415][SQL] Fix BroadcastHint when autoBroadcastJoinThreshold is 0 or -1 ## What changes were proposed in this pull request? This PR makes BroadcastHint more deterministic by using a special isBroadcastable property instead of setting the sizeInBytes to 1. See https://issues.apache.org/jira/browse/SPARK-15415 ## How was this patch tested? Added testcases to test if the broadcast hash join is included in the plan when the BroadcastHint is supplied and also tests for propagation of the joins. Author: Jurriaan Pruis Closes #13244 from jurriaan/broadcast-hint. --- .../catalyst/plans/logical/LogicalPlan.scala | 3 +- .../catalyst/plans/logical/Statistics.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 29 +++-- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../execution/joins/BroadcastJoinSuite.scala | 103 +++++++++++++++--- 5 files changed, 114 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 45ac126a72f5c..4984f235b412c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -313,7 +313,8 @@ abstract class UnaryNode extends LogicalPlan { // (product of children). sizeInBytes = 1 } - Statistics(sizeInBytes = sizeInBytes) + + child.statistics.copy(sizeInBytes = sizeInBytes) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index 9ac4c3a2a56c8..63f86ad09412c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -32,4 +32,4 @@ package org.apache.spark.sql.catalyst.plans.logical * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it * defaults to the product of children's `sizeInBytes`. */ -private[sql] case class Statistics(sizeInBytes: BigInt) +private[sql] case class Statistics(sizeInBytes: BigInt, isBroadcastable: Boolean = false) 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 732b0d7919c39..bed48b6f6101b 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 @@ -163,7 +163,9 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation val leftSize = left.statistics.sizeInBytes val rightSize = right.statistics.sizeInBytes val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize - Statistics(sizeInBytes = sizeInBytes) + val isBroadcastable = left.statistics.isBroadcastable || right.statistics.isBroadcastable + + Statistics(sizeInBytes = sizeInBytes, isBroadcastable = isBroadcastable) } } @@ -183,7 +185,7 @@ case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(le duplicateResolved override def statistics: Statistics = { - Statistics(sizeInBytes = left.statistics.sizeInBytes) + left.statistics.copy() } } @@ -330,6 +332,16 @@ case class Join( case UsingJoin(_, _) => false case _ => resolvedExceptNatural } + + override def statistics: Statistics = joinType match { + case LeftAnti | LeftSemi => + // LeftSemi and LeftAnti won't ever be bigger than left + left.statistics.copy() + case _ => + // make sure we don't propagate isBroadcastable in other joins, because + // they could explode the size. + super.statistics.copy(isBroadcastable = false) + } } /** @@ -338,9 +350,8 @@ case class Join( case class BroadcastHint(child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output - // We manually set statistics of BroadcastHint to smallest value to make sure - // the plan wrapped by BroadcastHint will be considered to broadcast later. - override def statistics: Statistics = Statistics(sizeInBytes = 1) + // set isBroadcastable to true so the child will be broadcasted + override def statistics: Statistics = super.statistics.copy(isBroadcastable = true) } case class InsertIntoTable( @@ -465,7 +476,7 @@ case class Aggregate( override def statistics: Statistics = { if (groupingExpressions.isEmpty) { - Statistics(sizeInBytes = 1) + super.statistics.copy(sizeInBytes = 1) } else { super.statistics } @@ -638,7 +649,7 @@ case class GlobalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryN override lazy val statistics: Statistics = { val limit = limitExpr.eval().asInstanceOf[Int] val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum - Statistics(sizeInBytes = sizeInBytes) + child.statistics.copy(sizeInBytes = sizeInBytes) } } @@ -653,7 +664,7 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo override lazy val statistics: Statistics = { val limit = limitExpr.eval().asInstanceOf[Int] val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum - Statistics(sizeInBytes = sizeInBytes) + child.statistics.copy(sizeInBytes = sizeInBytes) } } @@ -690,7 +701,7 @@ case class Sample( if (sizeInBytes == 0) { sizeInBytes = 1 } - Statistics(sizeInBytes = sizeInBytes) + child.statistics.copy(sizeInBytes = sizeInBytes) } override protected def otherCopyArgs: Seq[AnyRef] = isTableSample :: Nil 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 3343039ae1cf3..664e7f566145b 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 @@ -92,7 +92,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * Matches a plan whose output should be small enough to be used in broadcast join. */ private def canBroadcast(plan: LogicalPlan): Boolean = { - plan.statistics.sizeInBytes <= conf.autoBroadcastJoinThreshold + plan.statistics.isBroadcastable || + plan.statistics.sizeInBytes <= conf.autoBroadcastJoinThreshold } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 730ec43556c95..e681b88685b58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -22,9 +22,12 @@ import scala.reflect.ClassTag import org.scalatest.BeforeAndAfterAll import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext} -import org.apache.spark.sql.{QueryTest, SparkSession} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} import org.apache.spark.sql.execution.exchange.EnsureRequirements +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils /** * Test various broadcast join operators. @@ -33,7 +36,9 @@ import org.apache.spark.sql.functions._ * unsafe map in [[org.apache.spark.sql.execution.joins.UnsafeHashedRelation]] is not triggered * without serializing the hashed relation, which does not happen in local mode. */ -class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { +class BroadcastJoinSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + protected var spark: SparkSession = null /** @@ -56,30 +61,100 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { /** * Test whether the specified broadcast join updates the peak execution memory accumulator. */ - private def testBroadcastJoin[T: ClassTag](name: String, joinType: String): Unit = { + private def testBroadcastJoinPeak[T: ClassTag](name: String, joinType: String): Unit = { AccumulatorSuite.verifyPeakExecutionMemorySet(spark.sparkContext, name) { - val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") - val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") - // Comparison at the end is for broadcast left semi join - val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") - val df3 = df1.join(broadcast(df2), joinExpression, joinType) - val plan = - EnsureRequirements(spark.sessionState.conf).apply(df3.queryExecution.sparkPlan) - assert(plan.collect { case p: T => p }.size === 1) + val plan = testBroadcastJoin[T](joinType) plan.executeCollect() } } + private def testBroadcastJoin[T: ClassTag](joinType: String, + forceBroadcast: Boolean = false): SparkPlan = { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + var df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") + + // Comparison at the end is for broadcast left semi join + val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") + val df3 = if (forceBroadcast) { + df1.join(broadcast(df2), joinExpression, joinType) + } else { + df1.join(df2, joinExpression, joinType) + } + val plan = + EnsureRequirements(spark.sessionState.conf).apply(df3.queryExecution.sparkPlan) + assert(plan.collect { case p: T => p }.size === 1) + + return plan + } + test("unsafe broadcast hash join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast hash join", "inner") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast hash join", "inner") } test("unsafe broadcast hash outer join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast hash outer join", "left_outer") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast hash outer join", "left_outer") } test("unsafe broadcast left semi join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast left semi join", "leftsemi") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast left semi join", "leftsemi") + } + + test("broadcast hint isn't bothered by authBroadcastJoinThreshold set to low values") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + testBroadcastJoin[BroadcastHashJoinExec]("inner", true) + } + } + + test("broadcast hint isn't bothered by a disabled authBroadcastJoinThreshold") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + testBroadcastJoin[BroadcastHashJoinExec]("inner", true) + } + } + + test("broadcast hint isn't propagated after a join") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") + val df3 = df1.join(broadcast(df2), Seq("key"), "inner").drop(df2("key")) + + val df4 = spark.createDataFrame(Seq((1, "5"), (2, "5"))).toDF("key", "value") + val df5 = df4.join(df3, Seq("key"), "inner") + + val plan = + EnsureRequirements(spark.sessionState.conf).apply(df5.queryExecution.sparkPlan) + + assert(plan.collect { case p: BroadcastHashJoinExec => p }.size === 1) + assert(plan.collect { case p: SortMergeJoinExec => p }.size === 1) + } } + private def assertBroadcastJoin(df : Dataset[Row]) : Unit = { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + val joined = df1.join(df, Seq("key"), "inner") + + val plan = + EnsureRequirements(spark.sessionState.conf).apply(joined.queryExecution.sparkPlan) + + assert(plan.collect { case p: BroadcastHashJoinExec => p }.size === 1) + } + + test("broadcast hint is propagated correctly") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"), (3, "2"))).toDF("key", "value") + val broadcasted = broadcast(df2) + val df3 = spark.createDataFrame(Seq((2, "2"), (3, "3"))).toDF("key", "value") + + val cases = Seq(broadcasted.limit(2), + broadcasted.filter("value < 10"), + broadcasted.sample(true, 0.5), + broadcasted.distinct(), + broadcasted.groupBy("value").agg(min($"key").as("key")), + // except and intersect are semi/anti-joins which won't return more data then + // their left argument, so the broadcast hint should be propagated here + broadcasted.except(df3), + broadcasted.intersect(df3)) + + cases.foreach(assertBroadcastJoin) + } + } } From 6cb8f836da197eec17d33e4a547340c15e59d091 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 21 May 2016 23:12:27 -0700 Subject: [PATCH 086/143] [SPARK-15396][SQL][DOC] It can't connect hive metastore database #### What changes were proposed in this pull request? The `hive.metastore.warehouse.dir` property in hive-site.xml is deprecated since Spark 2.0.0. Users might not be able to connect to the existing metastore if they do not use the new conf parameter `spark.sql.warehouse.dir`. This PR is to update the document and example for explaining the latest changes in the configuration of default location of database. Below is the screenshot of the latest generated docs: screenshot 2016-05-20 08 38 10 screenshot 2016-05-20 08 53 26 screenshot 2016-05-20 08 53 37 No change is made in the R's example. screenshot 2016-05-20 08 54 38 #### How was this patch tested? N/A Author: gatorsmile Closes #13225 from gatorsmile/document. --- docs/sql-programming-guide.md | 72 +++++++++++-------- .../examples/sql/hive/HiveFromSpark.scala | 11 +-- 2 files changed, 50 insertions(+), 33 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a9e1f9d5ce46f..940c1d77045ad 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1663,43 +1663,50 @@ Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (
-When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can still create a `HiveContext`. When not configured by the -hive-site.xml, the context automatically creates `metastore_db` in the current directory and -creates `warehouse` directory indicated by HiveConf, which defaults to `/user/hive/warehouse`. -Note that you may need to grant write privilege on `/user/hive/warehouse` to the user who starts -the spark application. +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the spark application. {% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc) +// warehouse_location points to the default location for managed databases and tables +val conf = new SparkConf().setAppName("HiveFromSpark").set("spark.sql.warehouse.dir", warehouse_location) +val spark = SparkSession.builder.config(conf).enableHiveSupport().getOrCreate() -sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -sqlContext.sql("FROM src SELECT key, value").collect().foreach(println) +spark.sql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
-When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to -the `sql` method a `HiveContext` also provides an `hql` method, which allows queries to be -expressed in HiveQL. +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the spark application. {% highlight java %} -// sc is an existing JavaSparkContext. -HiveContext sqlContext = new org.apache.spark.sql.hive.HiveContext(sc.sc); +SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate(); -sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); +spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = sqlContext.sql("FROM src SELECT key, value").collect(); +Row[] results = spark.sql("FROM src SELECT key, value").collect(); {% endhighlight %} @@ -1707,18 +1714,25 @@ Row[] results = sqlContext.sql("FROM src SELECT key, value").collect();
-When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in the MetaStore and writing queries using HiveQL. +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the spark application. + {% highlight python %} -# sc is an existing SparkContext. -from pyspark.sql import HiveContext -sqlContext = HiveContext(sc) +from pyspark.sql import SparkSession +spark = SparkSession.builder.enableHiveSupport().getOrCreate() -sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = sqlContext.sql("FROM src SELECT key, value").collect() +results = spark.sql("FROM src SELECT key, value").collect() {% endhighlight %} 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 59bdfa09ad0f8..d3bb7e4398cd3 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 @@ -37,10 +37,13 @@ object HiveFromSpark { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") - // 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. + // When working with Hive, one must instantiate `SparkSession` with Hive support, including + // connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined + // functions. Users who do not have an existing Hive deployment can still enable Hive support. + // When not configured by the hive-site.xml, the context automatically creates `metastore_db` + // in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, + // which defaults to the directory `spark-warehouse` in the current directory that the spark + // application is started. val spark = SparkSession.builder .config(sparkConf) .enableHiveSupport() From 6d0bfb9601b02749d673bbceea5c36e5bc1c3825 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 21 May 2016 23:12:56 -0700 Subject: [PATCH 087/143] Small documentation and style fix. --- .../sql/catalyst/plans/logical/Statistics.scala | 3 ++- .../sql/execution/joins/BroadcastJoinSuite.scala | 15 ++++++--------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index 63f86ad09412c..6e6cc6962c007 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -31,5 +31,6 @@ package org.apache.spark.sql.catalyst.plans.logical * * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it * defaults to the product of children's `sizeInBytes`. + * @param isBroadcastable If true, output is small enough to be used in a broadcast join. */ -private[sql] case class Statistics(sizeInBytes: BigInt, isBroadcastable: Boolean = false) +case class Statistics(sizeInBytes: BigInt, isBroadcastable: Boolean = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index e681b88685b58..be4bf5b447565 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.joins import scala.reflect.ClassTag -import org.scalatest.BeforeAndAfterAll - import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext} import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} import org.apache.spark.sql.execution.exchange.EnsureRequirements @@ -68,10 +66,11 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { } } - private def testBroadcastJoin[T: ClassTag](joinType: String, - forceBroadcast: Boolean = false): SparkPlan = { + private def testBroadcastJoin[T: ClassTag]( + joinType: String, + forceBroadcast: Boolean = false): SparkPlan = { val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") - var df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") // Comparison at the end is for broadcast left semi join val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") @@ -80,11 +79,9 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { } else { df1.join(df2, joinExpression, joinType) } - val plan = - EnsureRequirements(spark.sessionState.conf).apply(df3.queryExecution.sparkPlan) + val plan = EnsureRequirements(spark.sessionState.conf).apply(df3.queryExecution.sparkPlan) assert(plan.collect { case p: T => p }.size === 1) - - return plan + plan } test("unsafe broadcast hash join updates peak execution memory") { From a11175eecacd4a57325dab29fff9ebfad819f22f Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 21 May 2016 23:56:10 -0700 Subject: [PATCH 088/143] [SPARK-15312][SQL] Detect Duplicate Key in Partition Spec and Table Properties #### What changes were proposed in this pull request? When there are duplicate keys in the partition specs or table properties, we always use the last value and ignore all the previous values. This is caused by the function call `toMap`. partition specs or table properties are widely used in multiple DDL statements. This PR is to detect the duplicates and issue an exception if found. #### How was this patch tested? Added test cases in DDLSuite Author: gatorsmile Closes #13095 from gatorsmile/detectDuplicate. --- .../spark/sql/catalyst/parser/AstBuilder.scala | 15 ++++++--------- .../spark/sql/catalyst/parser/ParserUtils.scala | 7 +++++++ .../sql/catalyst/parser/PlanParserSuite.scala | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 7 +++++-- .../sql/execution/command/DDLCommandSuite.scala | 15 +++++++++++++++ 5 files changed, 34 insertions(+), 12 deletions(-) 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 2d7d0f9032956..cace026701384 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 @@ -125,14 +125,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val namedQuery = visitNamedQuery(nCtx) (namedQuery.alias, namedQuery) } - // Check for duplicate names. - ctes.groupBy(_._1).filter(_._2.size > 1).foreach { - case (name, _) => - throw new ParseException( - s"Name '$name' is used for multiple common table expressions", ctx) - } - + checkDuplicateKeys(ctes, ctx) With(query, ctes.toMap) } } @@ -220,11 +214,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { */ override def visitPartitionSpec( ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) { - ctx.partitionVal.asScala.map { pVal => + val parts = ctx.partitionVal.asScala.map { pVal => val name = pVal.identifier.getText.toLowerCase val value = Option(pVal.constant).map(visitStringConstant) name -> value - }.toMap + } + // Check for duplicate partition columns in one spec. + checkDuplicateKeys(parts, ctx) + parts.toMap } /** 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 58e2bdb6e24fb..9619884edeafe 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 @@ -43,6 +43,13 @@ object ParserUtils { new ParseException(s"Operation not allowed: $message", ctx) } + /** Check if duplicate keys exist in a set of key-value pairs. */ + def checkDuplicateKeys[T](keyPairs: Seq[(String, T)], ctx: ParserRuleContext): Unit = { + keyPairs.groupBy(_._1).filter(_._2.size > 1).foreach { case (key, _) => + throw new ParseException(s"Found duplicate keys '$key'.", ctx) + } + } + /** Get the code that creates the given node. */ def source(ctx: ParserRuleContext): String = { val stream = ctx.getStart.getInputStream 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 25d87d93bec42..5811d32cd9e62 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 @@ -108,7 +108,7 @@ class PlanParserSuite extends PlanTest { "cte2" -> table("cte1").select(star()))) intercept( "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1", - "Name 'cte1' is used for multiple common table expressions") + "Found duplicate keys 'cte1'") } test("simple select query") { 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 2e3ac9706daf8..c517b8b55fadb 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 @@ -387,11 +387,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitTablePropertyList( ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { - ctx.tableProperty.asScala.map { property => + val properties = ctx.tableProperty.asScala.map { property => val key = visitTablePropertyKey(property.key) val value = Option(property.value).map(string).orNull key -> value - }.toMap + } + // Check for duplicate property names. + checkDuplicateKeys(properties, ctx) + properties.toMap } /** 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 708b878c843a7..54f98a6232e91 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 @@ -665,6 +665,21 @@ class DDLCommandSuite extends PlanTest { assert(parsed.isInstanceOf[Project]) } + test("duplicate keys in table properties") { + val e = intercept[ParseException] { + parser.parsePlan("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2')") + }.getMessage + assert(e.contains("Found duplicate keys 'key1'")) + } + + test("duplicate columns in partition specs") { + val e = intercept[ParseException] { + parser.parsePlan( + "ALTER TABLE dbx.tab1 PARTITION (a='1', a='2') RENAME TO PARTITION (a='100', a='200')") + }.getMessage + assert(e.contains("Found duplicate keys 'a'")) + } + test("drop table") { val tableName1 = "db.tab" val tableName2 = "tab" From 845e447fa03bf0a53ed79fa7e240af94dc152d2c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 22 May 2016 00:03:37 -0700 Subject: [PATCH 089/143] [SPARK-15459][SQL] Make Range logical and physical explain consistent ## What changes were proposed in this pull request? This patch simplifies the implementation of Range operator and make the explain string consistent between logical plan and physical plan. To do this, I changed RangeExec to embed a Range logical plan in it. Before this patch (note that the logical Range and physical Range actually output different information): ``` == Optimized Logical Plan == Range 0, 100, 2, 2, [id#8L] == Physical Plan == *Range 0, 2, 2, 50, [id#8L] ``` After this patch: If step size is 1: ``` == Optimized Logical Plan == Range(0, 100, splits=2) == Physical Plan == *Range(0, 100, splits=2) ``` If step size is not 1: ``` == Optimized Logical Plan == Range (0, 100, step=2, splits=2) == Physical Plan == *Range (0, 100, step=2, splits=2) ``` ## How was this patch tested? N/A Author: Reynold Xin Closes #13239 from rxin/SPARK-15459. --- .../plans/logical/basicLogicalOperators.scala | 18 ++++++++++--- .../catalog/SessionCatalogSuite.scala | 16 ++++++------ .../spark/sql/execution/SparkStrategies.scala | 4 +-- .../execution/basicPhysicalOperators.scala | 26 +++++++++---------- .../spark/sql/internal/CatalogSuite.scala | 2 +- 5 files changed, 37 insertions(+), 29 deletions(-) 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 bed48b6f6101b..b1b3e00de1456 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 @@ -431,8 +431,11 @@ case class Range( end: Long, step: Long, numSlices: Int, - output: Seq[Attribute]) extends LeafNode with MultiInstanceRelation { - require(step != 0, "step cannot be 0") + output: Seq[Attribute]) + extends LeafNode with MultiInstanceRelation { + + require(step != 0, s"step ($step) cannot be 0") + val numElements: BigInt = { val safeStart = BigInt(start) val safeEnd = BigInt(end) @@ -444,13 +447,20 @@ case class Range( } } - override def newInstance(): Range = - Range(start, end, step, numSlices, output.map(_.newInstance())) + override def newInstance(): Range = copy(output = output.map(_.newInstance())) override def statistics: Statistics = { val sizeInBytes = LongType.defaultSize * numElements Statistics( sizeInBytes = sizeInBytes ) } + + override def simpleString: String = { + if (step == 1) { + s"Range ($start, $end, splits=$numSlices)" + } else { + s"Range ($start, $end, step=$step, splits=$numSlices)" + } + } } case class Aggregate( 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 91e2e077cf3ce..a4dc03cd8b260 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 @@ -197,8 +197,8 @@ class SessionCatalogSuite extends SparkFunSuite { test("create temp table") { val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable1 = Range(1, 10, 1, 10, Seq()) - val tempTable2 = Range(1, 20, 2, 10, Seq()) + val tempTable1 = Range(1, 10, 1, 10) + val tempTable2 = Range(1, 20, 2, 10) catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) catalog.createTempView("tbl2", tempTable2, overrideIfExists = false) assert(catalog.getTempTable("tbl1") == Option(tempTable1)) @@ -243,7 +243,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("drop temp table") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) @@ -304,7 +304,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("rename temp table") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") assert(sessionCatalog.getTempTable("tbl1") == Option(tempTable)) @@ -383,7 +383,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("lookup table relation") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - val tempTable1 = Range(1, 10, 1, 10, Seq()) + val tempTable1 = Range(1, 10, 1, 10) val metastoreTable1 = externalCatalog.getTable("db2", "tbl1") sessionCatalog.createTempView("tbl1", tempTable1, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") @@ -422,7 +422,7 @@ class SessionCatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(TableIdentifier("tbl1", Some("db1")))) assert(!catalog.tableExists(TableIdentifier("tbl2", Some("db1")))) // If database is explicitly specified, do not check temporary tables - val tempTable = Range(1, 10, 1, 10, Seq()) + val tempTable = Range(1, 10, 1, 10) catalog.createTempView("tbl3", tempTable, overrideIfExists = false) assert(!catalog.tableExists(TableIdentifier("tbl3", Some("db2")))) // If database is not explicitly specified, check the current database @@ -434,7 +434,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("list tables without pattern") { val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.createTempView("tbl4", tempTable, overrideIfExists = false) assert(catalog.listTables("db1").toSet == @@ -451,7 +451,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("list tables with pattern") { val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.createTempView("tbl4", tempTable, overrideIfExists = false) assert(catalog.listTables("db1", "*").toSet == catalog.listTables("db1").toSet) 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 664e7f566145b..555a2f4c01d0e 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 @@ -360,8 +360,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { generator, join = join, outer = outer, g.output, planLater(child)) :: Nil case logical.OneRowRelation => execution.RDDScanExec(Nil, singleRowRdd, "OneRowRelation") :: Nil - case r @ logical.Range(start, end, step, numSlices, output) => - execution.RangeExec(start, step, numSlices, r.numElements, output) :: Nil + case r : logical.Range => + execution.RangeExec(r) :: Nil case logical.RepartitionByExpression(expressions, child, nPartitions) => exchange.ShuffleExchange(HashPartitioning( expressions, nPartitions.getOrElse(numPartitions)), planLater(child)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index d492fa7c412df..89bde6ad73f3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, ExpressionCanonicalizer} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.LongType +import org.apache.spark.sql.types.{LongType, StructField, StructType} import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} /** Physical plan for Project. */ @@ -305,22 +305,18 @@ case class SampleExec( /** - * Physical plan for range (generating a range of 64 bit numbers. - * - * @param start first number in the range, inclusive. - * @param step size of the step increment. - * @param numSlices number of partitions. - * @param numElements total number of elements to output. - * @param output output attributes. + * Physical plan for range (generating a range of 64 bit numbers). */ -case class RangeExec( - start: Long, - step: Long, - numSlices: Int, - numElements: BigInt, - output: Seq[Attribute]) +case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) extends LeafExecNode with CodegenSupport { + def start: Long = range.start + def step: Long = range.step + def numSlices: Int = range.numSlices + def numElements: BigInt = range.numElements + + override val output: Seq[Attribute] = range.output + private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -458,6 +454,8 @@ case class RangeExec( } } } + + override def simpleString: String = range.simpleString } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index e4d4cecd5b5dc..cd434f7887db6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -58,7 +58,7 @@ class CatalogSuite } private def createTempTable(name: String): Unit = { - sessionCatalog.createTempView(name, Range(1, 2, 3, 4, Seq()), overrideIfExists = true) + sessionCatalog.createTempView(name, Range(1, 2, 3, 4), overrideIfExists = true) } private def dropTable(name: String, db: Option[String] = None): Unit = { From 1ffa608ba5a849739a56047bda8b157b86b08650 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 22 May 2016 02:08:18 -0700 Subject: [PATCH 090/143] [SPARK-15428][SQL] Disable multiple streaming aggregations ## What changes were proposed in this pull request? Incrementalizing plans of with multiple streaming aggregation is tricky and we dont have the necessary support for "delta" to implement correctly. So disabling the support for multiple streaming aggregations. ## How was this patch tested? Additional unit tests Author: Tathagata Das Closes #13210 from tdas/SPARK-15428. --- .../UnsupportedOperationChecker.scala | 18 +++++++-- .../analysis/UnsupportedOperationsSuite.scala | 39 +++++++++++++------ .../streaming/StreamingAggregationSuite.scala | 19 --------- 3 files changed, 41 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index aadc1d31bd4b2..0e08bf013c8d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -55,10 +55,20 @@ object UnsupportedOperationChecker { case _: InsertIntoTable => throwError("InsertIntoTable is not supported with streaming DataFrames/Datasets") - case Aggregate(_, _, child) if child.isStreaming && outputMode == Append => - throwError( - "Aggregations are not supported on streaming DataFrames/Datasets in " + - "Append output mode. Consider changing output mode to Update.") + case Aggregate(_, _, child) if child.isStreaming => + if (outputMode == Append) { + throwError( + "Aggregations are not supported on streaming DataFrames/Datasets in " + + "Append output mode. Consider changing output mode to Update.") + } + val moreStreamingAggregates = child.find { + case Aggregate(_, _, grandchild) if grandchild.isStreaming => true + case _ => false + } + if (moreStreamingAggregates.nonEmpty) { + throwError("Multiple streaming aggregations are not supported with " + + "streaming DataFrames/Datasets") + } case Join(left, right, joinType, _) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 50baebe8bf4de..674277bdbe15d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.IntegerType @@ -95,6 +96,26 @@ class UnsupportedOperationsSuite extends SparkFunSuite { outputMode = Append, Seq("aggregation", "append output mode")) + // Multiple streaming aggregations not supported + def aggExprs(name: String): Seq[NamedExpression] = Seq(Count("*").as(name)) + + assertSupportedInStreamingPlan( + "aggregate - multiple batch aggregations", + Aggregate(Nil, aggExprs("c"), Aggregate(Nil, aggExprs("d"), batchRelation)), + Update) + + assertSupportedInStreamingPlan( + "aggregate - multiple aggregations but only one streaming aggregation", + Aggregate(Nil, aggExprs("c"), batchRelation).join( + Aggregate(Nil, aggExprs("d"), streamRelation), joinType = Inner), + Update) + + assertNotSupportedInStreamingPlan( + "aggregate - multiple streaming aggregations", + Aggregate(Nil, aggExprs("c"), Aggregate(Nil, aggExprs("d"), streamRelation)), + outputMode = Update, + expectedMsgs = Seq("multiple streaming aggregations")) + // Inner joins: Stream-stream not supported testBinaryOperationInStreamingPlan( "inner join", @@ -354,17 +375,11 @@ class UnsupportedOperationsSuite extends SparkFunSuite { val e = intercept[AnalysisException] { testBody } - - if (!expectedMsgs.map(_.toLowerCase).forall(e.getMessage.toLowerCase.contains)) { - fail( - s"""Exception message should contain the following substrings: - | - | ${expectedMsgs.mkString("\n ")} - | - |Actual exception message: - | - | ${e.getMessage} - """.stripMargin) + expectedMsgs.foreach { m => + if (!e.getMessage.toLowerCase.contains(m.toLowerCase)) { + fail(s"Exception message should contain: '$m', " + + s"actual exception message:\n\t'${e.getMessage}'") + } } } } 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 0f5fc9ca72d98..7104d01c4a2a1 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 @@ -84,25 +84,6 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be ) } - test("multiple aggregations") { - val inputData = MemoryStream[Int] - - val aggregated = - inputData.toDF() - .groupBy($"value") - .agg(count("*") as 'count) - .groupBy($"value" % 2) - .agg(sum($"count")) - .as[(Int, Long)] - - testStream(aggregated)( - AddData(inputData, 1, 2, 3, 4), - CheckLastBatch((0, 2), (1, 2)), - AddData(inputData, 1, 3, 5), - CheckLastBatch((1, 5)) - ) - } - testQuietly("midbatch failure") { val inputData = MemoryStream[Int] FailureSinglton.firstTime = true From 7920296bf8f313e010205937d3ebcbbc7b1a1d9e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 22 May 2016 08:08:46 -0500 Subject: [PATCH 091/143] [SPARK-15430][SQL] Fix potential ConcurrentModificationException for ListAccumulator ## What changes were proposed in this pull request? In `ListAccumulator` we create an unmodifiable view for underlying list. However, it doesn't prevent the underlying to be modified further. So as we access the unmodifiable list, the underlying list can be modified in the same time. It could cause `java.util.ConcurrentModificationException`. We can observe such exception in recent tests. To fix it, we can copy a list of the underlying list and then create the unmodifiable view of this list instead. ## How was this patch tested? The exception might be difficult to test. Existing tests should be passed. Author: Liang-Chi Hsieh Closes #13211 from viirya/fix-concurrentmodify. --- .../main/scala/org/apache/spark/util/AccumulatorV2.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 13cb6a28c3465..21ba46024d3b4 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.{lang => jl} import java.io.ObjectInputStream +import java.util.ArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong @@ -415,7 +416,7 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { - private val _list: java.util.List[T] = new java.util.ArrayList[T] + private val _list: java.util.List[T] = new ArrayList[T] override def isZero: Boolean = _list.isEmpty @@ -437,7 +438,9 @@ class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def value: java.util.List[T] = java.util.Collections.unmodifiableList(_list) + override def value: java.util.List[T] = _list.synchronized { + java.util.Collections.unmodifiableList(new ArrayList[T](_list)) + } private[spark] def setValue(newValue: java.util.List[T]): Unit = { _list.clear() From 72288fd67edc00f56e2e47eab2ef58fe4ff8c177 Mon Sep 17 00:00:00 2001 From: Bo Meng Date: Sun, 22 May 2016 08:10:54 -0500 Subject: [PATCH 092/143] [SPARK-15468][SQL] fix some typos ## What changes were proposed in this pull request? Fix some typos while browsing the codes. ## How was this patch tested? None and obvious. Author: Bo Meng Author: bomeng Closes #13246 from bomeng/typo. --- core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala | 2 +- .../catalyst/expressions/codegen/GenerateSafeProjection.scala | 2 +- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../org/apache/spark/sql/catalyst/planning/patterns.scala | 4 ++-- .../scala/org/apache/spark/sql/types/UDTRegistration.scala | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) 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 21ba46024d3b4..0b9a47c1124b1 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -55,7 +55,7 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { /** * Returns true if this accumulator has been registered. Note that all accumulators must be - * registered before ues, or it will throw exception. + * registered before use, or it will throw exception. */ final def isRegistered: Boolean = metadata != null && AccumulatorContext.get(metadata.id).isDefined diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index b0b121255357a..214dc40641f85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -48,7 +48,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] val tmp = ctx.freshName("tmp") val output = ctx.freshName("safeRow") val values = ctx.freshName("values") - // These expressions could be splitted into multiple functions + // These expressions could be split into multiple functions ctx.addMutableState("Object[]", values, s"this.$values = null;") val rowClass = classOf[GenericInternalRow].getName 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 cace026701384..a13c03a529f37 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 @@ -771,7 +771,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * ******************************************************************************************** */ /** * Create an expression from the given context. This method just passes the context on to the - * vistor and only takes care of typing (We assume that the visitor returns an Expression here). + * visitor and only takes care of typing (We assume that the visitor returns an Expression here). */ protected def expression(ctx: ParserRuleContext): Expression = typedVisit(ctx) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 00656191354f2..f42e67ca6ec20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -227,8 +227,8 @@ object IntegerIndex { * - Unnamed grouping expressions are named so that they can be referred to across phases of * aggregation * - Aggregations that appear multiple times are deduplicated. - * - The compution of the aggregations themselves is separated from the final result. For example, - * the `count` in `count + 1` will be split into an [[AggregateExpression]] and a final + * - The computation of the aggregations themselves is separated from the final result. For + * example, the `count` in `count + 1` will be split into an [[AggregateExpression]] and a final * computation that computes `count.resultAttribute + 1`. */ object PhysicalAggregation { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala index 0f24e51ed2b76..20ec75c70615b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils * This object keeps the mappings between user classes and their User Defined Types (UDTs). * Previously we use the annotation `SQLUserDefinedType` to register UDTs for user classes. * However, by doing this, we add SparkSQL dependency on user classes. This object provides - * alterntive approach to register UDTs for user classes. + * alternative approach to register UDTs for user classes. */ private[spark] object UDTRegistration extends Serializable with Logging { From 3eff65f82d5fc29b21f330966ac763489d217f4d Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Sun, 22 May 2016 09:19:28 -0500 Subject: [PATCH 093/143] [MINOR] More than 100 chars in line in SparkSubmitCommandBuilderSuite ## What changes were proposed in this pull request? More than 100 chars in line. ## How was this patch tested? Author: Sandeep Singh Closes #13249 from techaddict/fix-1. --- .../apache/spark/launcher/SparkSubmitCommandBuilderSuite.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index ffe11bc51b722..d451651555768 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -67,7 +67,8 @@ public void testCliHelpAndNoArg() throws Exception { List sparkEmptyArgs = Collections.emptyList(); cmd = buildCommand(sparkEmptyArgs, env); - assertTrue("org.apache.spark.deploy.SparkSubmit should be contained in the final cmd of empty input.", + assertTrue( + "org.apache.spark.deploy.SparkSubmit should be contained in the final cmd of empty input.", cmd.contains("org.apache.spark.deploy.SparkSubmit")); } From fc44b694bf5162b3a044768da4627b9969909829 Mon Sep 17 00:00:00 2001 From: wangyang Date: Sun, 22 May 2016 19:30:14 -0700 Subject: [PATCH 094/143] [SPARK-15379][SQL] check special invalid date ## What changes were proposed in this pull request? When invalid date string like "2015-02-29 00:00:00" are cast as date or timestamp using spark sql, it used to not return null but another valid date (2015-03-01 in this case). In this pr, invalid date string like "2016-02-29" and "2016-04-31" are returned as null when cast as date or timestamp. ## How was this patch tested? Unit tests are added. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: wangyang Closes #13169 from wangyang1992/invalid_date. --- .../sql/catalyst/util/DateTimeUtils.scala | 27 ++++++++++++++++--- .../catalyst/util/DateTimeUtilsSuite.scala | 19 +++++++++++++ 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index f84c6592c6ae5..e08328a32079e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -58,6 +58,7 @@ object DateTimeUtils { final val YearZero = -17999 final val toYearZero = to2001 + 7304850 final val TimeZoneGMT = TimeZone.getTimeZone("GMT") + final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) @transient lazy val defaultTimeZone = TimeZone.getDefault @@ -333,8 +334,7 @@ object DateTimeUtils { digitsMilli += 1 } - if (!justTime && (segments(0) < 0 || segments(0) > 9999 || segments(1) < 1 || - segments(1) > 12 || segments(2) < 1 || segments(2) > 31)) { + if (!justTime && isInvalidDate(segments(0), segments(1), segments(2))) { return None } @@ -414,10 +414,10 @@ object DateTimeUtils { return None } segments(i) = currentSegmentValue - if (segments(0) < 0 || segments(0) > 9999 || segments(1) < 1 || segments(1) > 12 || - segments(2) < 1 || segments(2) > 31) { + if (isInvalidDate(segments(0), segments(1), segments(2))) { return None } + val c = threadLocalGmtCalendar.get() c.clear() c.set(segments(0), segments(1) - 1, segments(2), 0, 0, 0) @@ -425,6 +425,25 @@ object DateTimeUtils { Some((c.getTimeInMillis / MILLIS_PER_DAY).toInt) } + /** + * Return true if the date is invalid. + */ + private def isInvalidDate(year: Int, month: Int, day: Int): Boolean = { + if (year < 0 || year > 9999 || month < 1 || month > 12 || day < 1 || day > 31) { + return true + } + if (month == 2) { + if (isLeapYear(year) && day > 29) { + return true + } else if (!isLeapYear(year) && day > 28) { + return true + } + } else if (!MonthOf31Days.contains(month) && day > 30) { + return true + } + false + } + /** * Returns the microseconds since year zero (-17999) from microseconds since epoch. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 6745b4b6c3c67..28e30c2219e3f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -353,6 +353,25 @@ class DateTimeUtilsSuite extends SparkFunSuite { c.getTimeInMillis * 1000 + 123456) } + test("SPARK-15379: special invalid date string") { + // Test stringToDate + assert(stringToDate( + UTF8String.fromString("2015-02-29 00:00:00")).isEmpty) + assert(stringToDate( + UTF8String.fromString("2015-04-31 00:00:00")).isEmpty) + assert(stringToDate(UTF8String.fromString("2015-02-29")).isEmpty) + assert(stringToDate(UTF8String.fromString("2015-04-31")).isEmpty) + + + // Test stringToTimestamp + assert(stringToTimestamp( + UTF8String.fromString("2015-02-29 00:00:00")).isEmpty) + assert(stringToTimestamp( + UTF8String.fromString("2015-04-31 00:00:00")).isEmpty) + assert(stringToTimestamp(UTF8String.fromString("2015-02-29")).isEmpty) + assert(stringToTimestamp(UTF8String.fromString("2015-04-31")).isEmpty) + } + test("hours") { val c = Calendar.getInstance() c.set(2015, 2, 18, 13, 2, 11) From dafcb05c2ef8e09f45edfb7eabf58116c23975a0 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Sun, 22 May 2016 23:32:39 -0700 Subject: [PATCH 095/143] [SPARK-15425][SQL] Disallow cross joins by default ## What changes were proposed in this pull request? In order to prevent users from inadvertently writing queries with cartesian joins, this patch introduces a new conf `spark.sql.crossJoin.enabled` (set to `false` by default) that if not set, results in a `SparkException` if the query contains one or more cartesian products. ## How was this patch tested? Added a test to verify the new behavior in `JoinSuite`. Additionally, `SQLQuerySuite` and `SQLMetricsSuite` were modified to explicitly enable cartesian products. Author: Sameer Agarwal Closes #13209 from sameeragarwal/disallow-cartesian. --- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../joins/BroadcastNestedLoopJoinExec.scala | 14 +++++- .../joins/CartesianProductExec.scala | 11 +++++ .../apache/spark/sql/internal/SQLConf.scala | 9 +++- .../org/apache/spark/sql/JoinSuite.scala | 31 +++++++++---- .../org/apache/spark/sql/SQLQuerySuite.scala | 32 +++++++------ .../sql/execution/joins/InnerJoinSuite.scala | 3 +- .../execution/metric/SQLMetricsSuite.scala | 46 ++++++++++--------- .../execution/HiveCompatibilitySuite.scala | 4 ++ .../sql/hive/execution/HiveQuerySuite.scala | 6 +++ 10 files changed, 113 insertions(+), 46 deletions(-) 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 555a2f4c01d0e..c46cecc71f37e 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 @@ -190,7 +190,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } // This join could be very slow or OOM joins.BroadcastNestedLoopJoinExec( - planLater(left), planLater(right), buildSide, joinType, condition) :: Nil + planLater(left), planLater(right), buildSide, joinType, condition, + withinBroadcastThreshold = false) :: Nil // --- Cases where this strategy does not apply --------------------------------------------- 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 2a250ecce6de4..4d43765f8fcd3 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 @@ -19,12 +19,14 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.sql.AnalysisException 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._ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.collection.{BitSet, CompactBuffer} case class BroadcastNestedLoopJoinExec( @@ -32,7 +34,8 @@ case class BroadcastNestedLoopJoinExec( right: SparkPlan, buildSide: BuildSide, joinType: JoinType, - condition: Option[Expression]) extends BinaryExecNode { + condition: Option[Expression], + withinBroadcastThreshold: Boolean = true) extends BinaryExecNode { override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -337,6 +340,15 @@ case class BroadcastNestedLoopJoinExec( ) } + protected override def doPrepare(): Unit = { + if (!withinBroadcastThreshold && !sqlContext.conf.crossJoinEnabled) { + throw new AnalysisException("Both sides of this join are outside the broadcasting " + + "threshold and computing it could be prohibitively expensive. To explicitly enable it, " + + s"please set ${SQLConf.CROSS_JOINS_ENABLED.key} = true") + } + super.doPrepare() + } + protected override def doExecute(): RDD[InternalRow] = { val broadcastedRelation = broadcast.executeBroadcast[Array[InternalRow]]() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 8d7ecc442a9e1..88f78a7a73bce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -19,11 +19,13 @@ package org.apache.spark.sql.execution.joins import org.apache.spark._ import org.apache.spark.rdd.{CartesianPartition, CartesianRDD, RDD} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter @@ -88,6 +90,15 @@ case class CartesianProductExec( override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + protected override def doPrepare(): Unit = { + if (!sqlContext.conf.crossJoinEnabled) { + throw new AnalysisException("Cartesian joins could be prohibitively expensive and are " + + "disabled by default. To explicitly enable them, please set " + + s"${SQLConf.CROSS_JOINS_ENABLED.key} = true") + } + super.doPrepare() + } + protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") 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 35d67ca2d8c5d..f3064eb6ac6d6 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 @@ -338,9 +338,14 @@ object SQLConf { .booleanConf .createWithDefault(true) + val CROSS_JOINS_ENABLED = SQLConfigBuilder("spark.sql.crossJoin.enabled") + .doc("When false, we will throw an error if a query contains a cross join") + .booleanConf + .createWithDefault(false) + val ORDER_BY_ORDINAL = SQLConfigBuilder("spark.sql.orderByOrdinal") .doc("When true, the ordinal numbers are treated as the position in the select list. " + - "When false, the ordinal numbers in order/sort By clause are ignored.") + "When false, the ordinal numbers in order/sort by clause are ignored.") .booleanConf .createWithDefault(true) @@ -622,6 +627,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def bucketingEnabled: Boolean = getConf(SQLConf.BUCKETING_ENABLED) + def crossJoinEnabled: Boolean = getConf(SQLConf.CROSS_JOINS_ENABLED) + // Do not use a value larger than 4000 as the default value of this property. // See the comments of SCHEMA_STRING_LENGTH_THRESHOLD above for more information. def schemaStringLengthThreshold: Int = getConf(SCHEMA_STRING_LENGTH_THRESHOLD) 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 a5d8cb19eadc9..55836737083de 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 @@ -62,7 +62,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("join operator selection") { spark.cacheManager.clearCache() - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0") { + withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[SortMergeJoinExec]), @@ -204,13 +205,27 @@ class JoinSuite extends QueryTest with SharedSQLContext { testData.rdd.flatMap(row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq) } - test("cartisian product join") { - checkAnswer( - testData3.join(testData3), - Row(1, null, 1, null) :: - Row(1, null, 2, 2) :: - Row(2, 2, 1, null) :: - Row(2, 2, 2, 2) :: Nil) + test("cartesian product join") { + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "false") { + val e = intercept[Exception] { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } + assert(e.getMessage.contains("Cartesian joins could be prohibitively expensive and are " + + "disabled by default")) + } } test("left outer join") { 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 460e34a5ff308..b1f848fdc89a3 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 @@ -104,9 +104,11 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ).toDF("a", "b", "c").createOrReplaceTempView("cachedData") spark.catalog.cacheTable("cachedData") - checkAnswer( - sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"), - Row(0) :: Row(81) :: Nil) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"), + Row(0) :: Row(81) :: Nil) + } } test("self join with aliases") { @@ -435,10 +437,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("left semi greater than predicate") { - checkAnswer( - sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"), - Seq(Row(3, 1), Row(3, 2)) - ) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"), + Seq(Row(3, 1), Row(3, 2)) + ) + } } test("left semi greater than predicate and equal operator") { @@ -824,12 +828,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("cartesian product join") { - checkAnswer( - testData3.join(testData3), - Row(1, null, 1, null) :: - Row(1, null, 2, 2) :: - Row(2, 2, 1, null) :: - Row(2, 2, 2, 2) :: Nil) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } } test("left outer join") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 7caeb3be549d9..27f6abcd9509a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -187,7 +187,8 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using CartesianProduct") { - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => CartesianProductExec(left, right, Some(condition())), expectedAnswer.map(Row.fromTuple), 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 7a89b484eb85b..12940c86fe650 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,6 +29,7 @@ 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.{AccumulatorContext, JsonProtocol, Utils} @@ -237,16 +238,18 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("BroadcastNestedLoopJoin metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) testDataForJoin.createOrReplaceTempView("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = spark.sql( - "SELECT * FROM testData2 left JOIN testDataForJoin ON " + - "testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a") - testSparkPlanMetrics(df, 3, Map( - 1L -> ("BroadcastNestedLoopJoin", Map( - "number of output rows" -> 12L))) - ) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + withTempTable("testDataForJoin") { + // Assume the execution plan is + // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = spark.sql( + "SELECT * FROM testData2 left JOIN testDataForJoin ON " + + "testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a") + testSparkPlanMetrics(df, 3, Map( + 1L -> ("BroadcastNestedLoopJoin", Map( + "number of output rows" -> 12L))) + ) + } } } @@ -263,17 +266,18 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } test("CartesianProduct metrics") { - val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.createOrReplaceTempView("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = spark.sql( - "SELECT * FROM testData2 JOIN testDataForJoin") - testSparkPlanMetrics(df, 1, Map( - 0L -> ("CartesianProduct", Map( - "number of output rows" -> 12L))) - ) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) + testDataForJoin.createOrReplaceTempView("testDataForJoin") + withTempTable("testDataForJoin") { + // Assume the execution plan is + // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = spark.sql( + "SELECT * FROM testData2 JOIN testDataForJoin") + testSparkPlanMetrics(df, 1, Map( + 0L -> ("CartesianProduct", Map("number of output rows" -> 12L))) + ) + } } } 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 54fb440b3374b..a8645f7cd31d7 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 @@ -40,6 +40,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalColumnBatchSize = TestHive.conf.columnBatchSize private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalConvertMetastoreOrc = TestHive.sessionState.convertMetastoreOrc + private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -61,6 +62,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Ensures that the plans generation use metastore relation and not OrcRelation // Was done because SqlBuilder does not work with plans having logical relation TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, false) + // Ensures that cross joins are enabled so that we can test them + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) RuleExecutor.resetTime() } @@ -72,6 +75,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, originalConvertMetastoreOrc) + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) TestHive.sessionState.functionRegistry.restore() // For debugging dump some statistics about how much time was spent in various optimizer rules 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 2aaaaadb6afa9..e179021491a69 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 @@ -35,6 +35,7 @@ import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.internal.SQLConf case class TestData(a: Int, b: String) @@ -48,6 +49,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ + private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled + override def beforeAll() { super.beforeAll() TestHive.setCacheTables(true) @@ -55,6 +58,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting Locale.setDefault(Locale.US) + // Ensures that cross joins are enabled so that we can test them + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) } override def afterAll() { @@ -63,6 +68,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) } finally { super.afterAll() } From 80091b8a6840b562cf76341926e5b828d4def7e2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 23 May 2016 10:48:25 -0700 Subject: [PATCH 096/143] [SPARK-14031][SQL] speedup CSV writer ## What changes were proposed in this pull request? Currently, we create an CSVWriter for every row, it's very expensive and memory hungry, took about 15 seconds to write out 1 mm rows (two columns). This PR will write the rows in batch mode, create a CSVWriter for every 1k rows, which could write out 1 mm rows in about 1 seconds (15X faster). ## How was this patch tested? Manually benchmark it. Author: Davies Liu Closes #13229 from davies/csv_writer. --- .../execution/datasources/csv/CSVParser.scala | 19 +++++++++++---- .../datasources/csv/CSVRelation.scala | 23 ++++++++++++------- 2 files changed, 29 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index ae797a1e0749f..111995da9cbad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -76,17 +76,26 @@ private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten writerSettings.setQuoteAllFields(false) writerSettings.setHeaders(headers: _*) - def writeRow(row: Seq[String], includeHeader: Boolean): String = { - val buffer = new ByteArrayOutputStream() - val outputWriter = new OutputStreamWriter(buffer, StandardCharsets.UTF_8) - val writer = new CsvWriter(outputWriter, writerSettings) + private var buffer = new ByteArrayOutputStream() + private var writer = new CsvWriter( + new OutputStreamWriter(buffer, StandardCharsets.UTF_8), + writerSettings) + def writeRow(row: Seq[String], includeHeader: Boolean): Unit = { if (includeHeader) { writer.writeHeaders() } writer.writeRow(row.toArray: _*) + } + + def flush(): String = { writer.close() - buffer.toString.stripLineEnd + val lines = buffer.toString.stripLineEnd + buffer = new ByteArrayOutputStream() + writer = new CsvWriter( + new OutputStreamWriter(buffer, StandardCharsets.UTF_8), + writerSettings) + lines } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 4f2d4387b180f..9849484dcec85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -176,8 +176,8 @@ private[sql] class CsvOutputWriter( }.getRecordWriter(context) } - private var firstRow: Boolean = params.headerFlag - + private val FLUSH_BATCH_SIZE = 1024L + private var records: Long = 0L private val csvWriter = new LineCsvWriter(params, dataSchema.fieldNames.toSeq) private def rowToString(row: Seq[Any]): Seq[String] = row.map { field => @@ -191,16 +191,23 @@ private[sql] class CsvOutputWriter( override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") override protected[sql] def writeInternal(row: InternalRow): Unit = { - // TODO: Instead of converting and writing every row, we should use the univocity buffer - val resultString = csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), firstRow) - if (firstRow) { - firstRow = false + csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), records == 0L && params.headerFlag) + records += 1 + if (records % FLUSH_BATCH_SIZE == 0) { + flush() + } + } + + private def flush(): Unit = { + val lines = csvWriter.flush() + if (lines.nonEmpty) { + text.set(lines) + recordWriter.write(NullWritable.get(), text) } - text.set(resultString) - recordWriter.write(NullWritable.get(), text) } override def close(): Unit = { + flush() recordWriter.close(context) } } From 07c36a2f07fcf5da6fb395f830ebbfc10eb27dcc Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 23 May 2016 11:13:27 -0700 Subject: [PATCH 097/143] [SPARK-15471][SQL] ScalaReflection cleanup ## What changes were proposed in this pull request? 1. simplify the logic of deserializing option type. 2. simplify the logic of serializing array type, and remove silentSchemaFor 3. remove some unnecessary code. ## How was this patch tested? existing tests Author: Wenchen Fan Closes #13250 from cloud-fan/encoder. --- .../spark/sql/catalyst/ScalaReflection.scala | 105 ++++-------------- .../expressions/objects/objects.scala | 4 +- 2 files changed, 21 insertions(+), 88 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 36989a20cb5d5..bdd40f340235b 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 @@ -22,7 +22,7 @@ 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 +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** @@ -72,6 +72,7 @@ object ScalaReflection extends ScalaReflection { case t if t <:< definitions.ByteTpe => ByteType case t if t <:< definitions.BooleanTpe => BooleanType case t if t <:< localTypeOf[Array[Byte]] => BinaryType + case t if t <:< localTypeOf[CalendarInterval] => CalendarIntervalType case t if t <:< localTypeOf[Decimal] => DecimalType.SYSTEM_DEFAULT case _ => val className = getClassNameFromType(tpe) @@ -189,7 +190,6 @@ object ScalaReflection extends ScalaReflection { case _ => UpCast(expr, expected, walkedTypePath) } - val className = getClassNameFromType(tpe) tpe match { case t if !dataTypeFor(t).isInstanceOf[ObjectType] => getPath @@ -239,16 +239,14 @@ object ScalaReflection extends ScalaReflection { DateTimeUtils.getClass, ObjectType(classOf[java.sql.Date]), "toJavaDate", - getPath :: Nil, - propagateNull = true) + getPath :: Nil) case t if t <:< localTypeOf[java.sql.Timestamp] => StaticInvoke( DateTimeUtils.getClass, ObjectType(classOf[java.sql.Timestamp]), "toJavaTimestamp", - getPath :: Nil, - propagateNull = true) + getPath :: Nil) case t if t <:< localTypeOf[java.lang.String] => Invoke(getPath, "toString", ObjectType(classOf[String])) @@ -437,17 +435,17 @@ object ScalaReflection extends ScalaReflection { walkedTypePath: Seq[String]): Expression = ScalaReflectionLock.synchronized { def toCatalystArray(input: Expression, elementType: `Type`): Expression = { - val externalDataType = dataTypeFor(elementType) - val Schema(catalystType, nullable) = silentSchemaFor(elementType) - if (isNativeType(externalDataType)) { - NewInstance( - classOf[GenericArrayData], - input :: Nil, - dataType = ArrayType(catalystType, nullable)) - } else { - val clsName = getClassNameFromType(elementType) - val newPath = s"""- array element class: "$clsName"""" +: walkedTypePath - MapObjects(serializerFor(_, elementType, newPath), input, externalDataType) + dataTypeFor(elementType) match { + case dt: ObjectType => + val clsName = getClassNameFromType(elementType) + val newPath = s"""- array element class: "$clsName"""" +: walkedTypePath + MapObjects(serializerFor(_, elementType, newPath), input, dt) + + case dt => + NewInstance( + classOf[GenericArrayData], + input :: Nil, + dataType = ArrayType(dt, schemaFor(elementType).nullable)) } } @@ -457,63 +455,10 @@ object ScalaReflection extends ScalaReflection { tpe match { case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t - optType match { - // For primitive types we must manually unbox the value of the object. - case t if t <:< definitions.IntTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Integer]), inputObject), - "intValue", - IntegerType) - case t if t <:< definitions.LongTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Long]), inputObject), - "longValue", - LongType) - case t if t <:< definitions.DoubleTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Double]), inputObject), - "doubleValue", - DoubleType) - case t if t <:< definitions.FloatTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Float]), inputObject), - "floatValue", - FloatType) - case t if t <:< definitions.ShortTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Short]), inputObject), - "shortValue", - ShortType) - case t if t <:< definitions.ByteTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Byte]), inputObject), - "byteValue", - ByteType) - case t if t <:< definitions.BooleanTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Boolean]), inputObject), - "booleanValue", - BooleanType) - - // For non-primitives, we can just extract the object from the Option and then recurse. - case other => - val className = getClassNameFromType(optType) - val newPath = s"""- option value class: "$className"""" +: walkedTypePath - - val optionObjectType: DataType = other match { - // Special handling is required for arrays, as getClassFromType() will fail - // since Scala Arrays map to native Java constructs. E.g. "Array[Int]" will map to - // the Java type "[I". - case arr if arr <:< localTypeOf[Array[_]] => arrayClassFor(t) - case cls => ObjectType(getClassFromType(cls)) - } - val unwrapped = UnwrapOption(optionObjectType, inputObject) - - expressions.If( - IsNull(unwrapped), - expressions.Literal.create(null, silentSchemaFor(optType).dataType), - serializerFor(unwrapped, optType, newPath)) - } + val className = getClassNameFromType(optType) + val newPath = s"""- option value class: "$className"""" +: walkedTypePath + val unwrapped = UnwrapOption(dataTypeFor(optType), inputObject) + 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 @@ -704,18 +649,6 @@ object ScalaReflection extends ScalaReflection { /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor[T: TypeTag]: Schema = schemaFor(localTypeOf[T]) - /** - * Returns a catalyst DataType and its nullability for the given Scala Type using reflection. - * - * Unlike `schemaFor`, this method won't throw exception for un-supported type, it will return - * `NullType` silently instead. - */ - def silentSchemaFor(tpe: `Type`): Schema = try { - schemaFor(tpe) - } catch { - case _: UnsupportedOperationException => Schema(NullType, nullable = true) - } - /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized { tpe match { 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 5e17f8920901a..2f2323fa3a25f 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 @@ -289,8 +289,8 @@ case class UnwrapOption( ${inputObject.code} final boolean ${ev.isNull} = ${inputObject.isNull} || ${inputObject.value}.isEmpty(); - $javaType ${ev.value} = - ${ev.isNull} ? ${ctx.defaultValue(javaType)} : ($javaType) ${inputObject.value}.get(); + $javaType ${ev.value} = ${ev.isNull} ? + ${ctx.defaultValue(javaType)} : (${ctx.boxedType(javaType)}) ${inputObject.value}.get(); """ ev.copy(code = code) } From 2585d2b322f3b6b85a0a12ddf7dcde957453000d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 23 May 2016 11:55:03 -0700 Subject: [PATCH 098/143] [SPARK-15279][SQL] Catch conflicting SerDe when creating table ## What changes were proposed in this pull request? The user may do something like: ``` CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS PARQUET CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS ... SERDE 'myserde' CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ORC CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ... SERDE 'myserde' ``` None of these should be allowed because the SerDe's conflict. As of this patch: - `ROW FORMAT DELIMITED` is only compatible with `TEXTFILE` - `ROW FORMAT SERDE` is only compatible with `TEXTFILE`, `RCFILE` and `SEQUENCEFILE` ## How was this patch tested? New tests in `DDLCommandSuite`. Author: Andrew Or Closes #13068 from andrewor14/row-format-conflict. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 60 ++++++++++-- .../execution/command/DDLCommandSuite.scala | 94 +++++++++++++++---- .../spark/sql/hive/HiveDDLCommandSuite.scala | 4 +- 4 files changed, 129 insertions(+), 33 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 848c59e3b8a53..8ea8f766295ec 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -267,8 +267,8 @@ createFileFormat ; fileFormat - : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING (SERDE serdeCls=STRING)? #tableFileFormat - | identifier #genericFileFormat + : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING #tableFileFormat + | identifier #genericFileFormat ; storageHandler 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 c517b8b55fadb..6e4af9500c3c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -796,14 +796,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * Expected format: * {{{ - * CREATE [TEMPORARY] [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name - * [(col1 data_type [COMMENT col_comment], ...)] + * CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name + * [(col1[:] data_type [COMMENT col_comment], ...)] * [COMMENT table_comment] - * [PARTITIONED BY (col3 data_type [COMMENT col_comment], ...)] - * [CLUSTERED BY (col1, ...) [SORTED BY (col1 [ASC|DESC], ...)] INTO num_buckets BUCKETS] - * [SKEWED BY (col1, col2, ...) ON ((col_value, col_value, ...), ...) [STORED AS DIRECTORIES]] + * [PARTITIONED BY (col2[:] data_type [COMMENT col_comment], ...)] * [ROW FORMAT row_format] - * [STORED AS file_format | STORED BY storage_handler_class [WITH SERDEPROPERTIES (...)]] + * [STORED AS file_format] * [LOCATION path] * [TBLPROPERTIES (property_name=property_value, ...)] * [AS select_statement]; @@ -849,6 +847,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { compressed = false, serdeProperties = Map()) } + validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) .getOrElse(EmptyStorageFormat) val rowStorage = Option(ctx.rowFormat).map(visitRowFormat).getOrElse(EmptyStorageFormat) @@ -905,6 +904,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { /** * Create a [[CatalogStorageFormat]] for creating tables. + * + * Format: STORED AS ... */ override def visitCreateFileFormat( ctx: CreateFileFormatContext): CatalogStorageFormat = withOrigin(ctx) { @@ -932,9 +933,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: TableFileFormatContext): CatalogStorageFormat = withOrigin(ctx) { EmptyStorageFormat.copy( inputFormat = Option(string(ctx.inFmt)), - outputFormat = Option(string(ctx.outFmt)), - serde = Option(ctx.serdeCls).map(string) - ) + outputFormat = Option(string(ctx.outFmt))) } /** @@ -1018,6 +1017,49 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { EmptyStorageFormat.copy(serdeProperties = entries.toMap) } + /** + * Throw a [[ParseException]] if the user specified incompatible SerDes through ROW FORMAT + * and STORED AS. + * + * The following are allowed. Anything else is not: + * ROW FORMAT SERDE ... STORED AS [SEQUENCEFILE | RCFILE | TEXTFILE] + * ROW FORMAT DELIMITED ... STORED AS TEXTFILE + * ROW FORMAT ... STORED AS INPUTFORMAT ... OUTPUTFORMAT ... + */ + private def validateRowFormatFileFormat( + rowFormatCtx: RowFormatContext, + createFileFormatCtx: CreateFileFormatContext, + parentCtx: ParserRuleContext): Unit = { + if (rowFormatCtx == null || createFileFormatCtx == null) { + return + } + (rowFormatCtx, createFileFormatCtx.fileFormat) match { + case (_, ffTable: TableFileFormatContext) => // OK + case (rfSerde: RowFormatSerdeContext, ffGeneric: GenericFileFormatContext) => + ffGeneric.identifier.getText.toLowerCase match { + case ("sequencefile" | "textfile" | "rcfile") => // OK + case fmt => + throw operationNotAllowed( + s"ROW FORMAT SERDE is incompatible with format '$fmt', which also specifies a serde", + parentCtx) + } + case (rfDelimited: RowFormatDelimitedContext, ffGeneric: GenericFileFormatContext) => + ffGeneric.identifier.getText.toLowerCase match { + case "textfile" => // OK + case fmt => throw operationNotAllowed( + s"ROW FORMAT DELIMITED is only compatible with 'textfile', not '$fmt'", parentCtx) + } + case _ => + // should never happen + def str(ctx: ParserRuleContext): String = { + (0 until ctx.getChildCount).map { i => ctx.getChild(i).getText }.mkString(" ") + } + throw operationNotAllowed( + s"Unexpected combination of ${str(rowFormatCtx)} and ${str(createFileFormatCtx)}", + parentCtx) + } + } + /** * Create or replace a view. This creates a [[CreateViewCommand]] command. * 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 54f98a6232e91..eab1f55712f6f 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.command +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, FunctionResource} import org.apache.spark.sql.catalyst.catalog.FunctionResourceType @@ -25,9 +27,10 @@ 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.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { private val parser = new SparkSqlParser(new SQLConf) @@ -40,6 +43,15 @@ class DDLCommandSuite extends PlanTest { containsThesePhrases.foreach { p => assert(e.getMessage.toLowerCase.contains(p.toLowerCase)) } } + private def parseAs[T: ClassTag](query: String): T = { + parser.parsePlan(query) match { + case t: T => t + case other => + fail(s"Expected to parse ${classTag[T].runtimeClass} from query," + + s"got ${other.getClass.getName}: $query") + } + } + test("create database") { val sql = """ @@ -225,19 +237,69 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed4, expected4) } + test("create table - row format and table file format") { + val createTableStart = "CREATE TABLE my_tab ROW FORMAT" + val fileFormat = s"STORED AS INPUTFORMAT 'inputfmt' OUTPUTFORMAT 'outputfmt'" + val query1 = s"$createTableStart SERDE 'anything' $fileFormat" + val query2 = s"$createTableStart DELIMITED FIELDS TERMINATED BY ' ' $fileFormat" + + // No conflicting serdes here, OK + val parsed1 = parseAs[CreateTableCommand](query1) + assert(parsed1.table.storage.serde == Some("anything")) + assert(parsed1.table.storage.inputFormat == Some("inputfmt")) + assert(parsed1.table.storage.outputFormat == Some("outputfmt")) + val parsed2 = parseAs[CreateTableCommand](query2) + assert(parsed2.table.storage.serde.isEmpty) + assert(parsed2.table.storage.inputFormat == Some("inputfmt")) + assert(parsed2.table.storage.outputFormat == Some("outputfmt")) + } + + test("create table - row format serde and generic file format") { + val allSources = Seq("parquet", "orc", "avro", "sequencefile", "rcfile", "textfile") + val supportedSources = Set("sequencefile", "rcfile", "textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS $s" + if (supportedSources.contains(s)) { + val ct = parseAs[CreateTableCommand](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s, new SQLConf) + assert(hiveSerde.isDefined) + assert(ct.table.storage.serde == Some("anything")) + assert(ct.table.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.table.storage.outputFormat == hiveSerde.get.outputFormat) + } else { + assertUnsupported(query, Seq("row format serde", "incompatible", s)) + } + } + } + + test("create table - row format delimited and generic file format") { + val allSources = Seq("parquet", "orc", "avro", "sequencefile", "rcfile", "textfile") + val supportedSources = Set("textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS $s" + if (supportedSources.contains(s)) { + val ct = parseAs[CreateTableCommand](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s, new SQLConf) + assert(hiveSerde.isDefined) + assert(ct.table.storage.serde == hiveSerde.get.serde) + assert(ct.table.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.table.storage.outputFormat == hiveSerde.get.outputFormat) + } else { + assertUnsupported(query, Seq("row format delimited", "only compatible with 'textfile'", s)) + } + } + } + test("create external table - location must be specified") { assertUnsupported( sql = "CREATE EXTERNAL TABLE my_tab", containsThesePhrases = Seq("create external table", "location")) val query = "CREATE EXTERNAL TABLE my_tab LOCATION '/something/anything'" - parser.parsePlan(query) match { - case ct: CreateTableCommand => - assert(ct.table.tableType == CatalogTableType.EXTERNAL) - assert(ct.table.storage.locationUri == Some("/something/anything")) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") - } + val ct = parseAs[CreateTableCommand](query) + assert(ct.table.tableType == CatalogTableType.EXTERNAL) + assert(ct.table.storage.locationUri == Some("/something/anything")) } test("create table - property values must be set") { @@ -252,14 +314,9 @@ class DDLCommandSuite extends PlanTest { test("create table - location implies external") { val query = "CREATE TABLE my_tab LOCATION '/something/anything'" - parser.parsePlan(query) match { - case ct: CreateTableCommand => - assert(ct.table.tableType == CatalogTableType.EXTERNAL) - assert(ct.table.storage.locationUri == Some("/something/anything")) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") - } + val ct = parseAs[CreateTableCommand](query) + assert(ct.table.tableType == CatalogTableType.EXTERNAL) + assert(ct.table.storage.locationUri == Some("/something/anything")) } test("create table using - with partitioned by") { @@ -551,8 +608,7 @@ class DDLCommandSuite extends PlanTest { test("alter table: set file format (not allowed)") { assertUnsupported( - "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + - "OUTPUTFORMAT 'test' SERDE 'test'") + "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' OUTPUTFORMAT 'test'") assertUnsupported( "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + "SET FILEFORMAT PARQUET") 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 30ad392969b4e..96c8fa6b70501 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 @@ -61,7 +61,7 @@ class HiveDDLCommandSuite extends PlanTest { |country STRING COMMENT 'country of origination') |COMMENT 'This is the staging page view table' |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') - |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\054' STORED AS RCFILE + |STORED AS RCFILE |LOCATION '/user/external/page_view' |TBLPROPERTIES ('p1'='v1', 'p2'='v2') |AS SELECT * FROM src""".stripMargin @@ -88,8 +88,6 @@ class HiveDDLCommandSuite extends PlanTest { assert(desc.partitionColumns == CatalogColumn("dt", "string", comment = Some("date type")) :: CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) - assert(desc.storage.serdeProperties == - Map((serdeConstants.SERIALIZATION_FORMAT, "\u002C"), (serdeConstants.FIELD_DELIM, "\u002C"))) assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) assert(desc.storage.serde == From 37c617e4f580482b59e1abbe3c0c27c7125cf605 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 23 May 2016 14:19:25 -0700 Subject: [PATCH 099/143] [MINOR][SQL][DOCS] Add notes of the deterministic assumption on UDF functions ## What changes were proposed in this pull request? Spark assumes that UDF functions are deterministic. This PR adds explicit notes about that. ## How was this patch tested? It's only about docs. Author: Dongjoon Hyun Closes #13087 from dongjoon-hyun/SPARK-15282. --- python/pyspark/sql/functions.py | 3 +++ .../org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 1 + sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 3 +++ .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 3 +++ .../src/main/scala/org/apache/spark/sql/UDFRegistration.scala | 1 + .../org/apache/spark/sql/expressions/UserDefinedFunction.scala | 3 +++ .../scala/org/apache/spark/sql/internal/SessionState.scala | 1 + 7 files changed, 15 insertions(+) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index dac842c0ce8c0..716b16fdc9530 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1756,6 +1756,9 @@ def __call__(self, *cols): @since(1.3) def udf(f, returnType=StringType()): """Creates a :class:`Column` expression representing a user defined function (UDF). + Note that the user-defined functions must be deterministic. Due to optimization, + duplicate invocations may be eliminated or the function may even be invoked more times than + it is present in the query. >>> from pyspark.sql.types import IntegerType >>> slen = udf(lambda s: len(s), IntegerType()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 0038cf65e2993..21390644bc0b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.types.DataType /** * User-defined function. + * Note that the user-defined functions must be deterministic. * @param function The user defined scala function to run. * Note that if you use primitive parameters, you are not able to check if it is * null or not, and the UDF will return null for you if the primitive input is 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 14d12d30bc0b3..7013e316ead83 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 @@ -199,6 +199,9 @@ class SQLContext private[sql]( /** * A collection of methods for registering user-defined functions (UDF). + * Note that the user-defined functions must be deterministic. Due to optimization, + * duplicate invocations may be eliminated or the function may even be invoked more times than + * it is present in the query. * * The following example registers a Scala closure as UDF: * {{{ 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 f697769bdcdb5..5c87c844185c5 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 @@ -145,6 +145,9 @@ class SparkSession private( /** * A collection of methods for registering user-defined functions (UDF). + * Note that the user-defined functions must be deterministic. Due to optimization, + * duplicate invocations may be eliminated or the function may even be invoked more times than + * it is present in the query. * * The following example registers a Scala closure as UDF: * {{{ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 3a043dcc6af22..b006236481a29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.types.DataType /** * Functions for registering user-defined functions. Use [[SQLContext.udf]] to access this. + * Note that the user-defined functions must be deterministic. * * @since 1.3.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index bd35d19aa20bb..49fdec57558e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -25,6 +25,9 @@ import org.apache.spark.sql.types.DataType /** * A user-defined function. To create one, use the `udf` functions in [[functions]]. + * Note that the user-defined functions must be deterministic. Due to optimization, + * duplicate invocations may be eliminated or the function may even be invoked more times than + * it is present in the query. * As an example: * {{{ * // Defined a UDF that returns true or false based on some numeric score. 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 939b9195cae99..c9cc2ba04a413 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 @@ -100,6 +100,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { /** * Interface exposed to the user for registering user-defined functions. + * Note that the user-defined functions must be deterministic. */ lazy val udf: UDFRegistration = new UDFRegistration(functionRegistry) From 03c7b7c4b9374f0cb6a29aeaf495bd21c2563de4 Mon Sep 17 00:00:00 2001 From: sureshthalamati Date: Mon, 23 May 2016 17:15:19 -0700 Subject: [PATCH 100/143] [SPARK-15315][SQL] Adding error check to the CSV datasource writer for unsupported complex data types. ## What changes were proposed in this pull request? Adds error handling to the CSV writer for unsupported complex data types. Currently garbage gets written to the output csv files if the data frame schema has complex data types. ## How was this patch tested? Added new unit test case. Author: sureshthalamati Closes #13105 from sureshthalamati/csv_complex_types_SPARK-15315. --- .../datasources/csv/DefaultSource.scala | 14 +++++++++++- .../execution/datasources/csv/CSVSuite.scala | 22 +++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) 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 f47ed76cba765..057bde1a75f07 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 @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration /** @@ -86,6 +86,7 @@ class DefaultSource extends FileFormat with DataSourceRegister { job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { + verifySchema(dataSchema) val conf = job.getConfiguration val csvOptions = new CSVOptions(options) csvOptions.compressionCodec.foreach { codec => @@ -172,4 +173,15 @@ class DefaultSource extends FileFormat with DataSourceRegister { .mapPartitions(_.map(pair => new String(pair._2.getBytes, 0, pair._2.getLength, charset))) } } + + private def verifySchema(schema: StructType): Unit = { + schema.foreach { field => + field.dataType match { + case _: ArrayType | _: MapType | _: StructType => + throw new UnsupportedOperationException( + s"CSV data source does not support ${field.dataType.simpleString} data type.") + case _ => + } + } + } } 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 27d6dc9197d27..bae290776f6e1 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 @@ -31,6 +31,8 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { + import testImplicits._ + private val carsFile = "cars.csv" private val carsMalformedFile = "cars-malformed.csv" private val carsFile8859 = "cars_iso-8859-1.csv" @@ -582,4 +584,24 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(numbers.count() == 8) } + + test("error handling for unsupported data types.") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + var msg = intercept[UnsupportedOperationException] { + Seq((1, "Tesla")).toDF("a", "b").selectExpr("struct(a, b)").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support struct data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, Map("Tesla" -> 3))).toDF("id", "cars").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support map data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, Array("Tesla", "Chevy", "Ford"))).toDF("id", "brands").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support array data type")) + } + } } From a8e97d17b91684e68290d9f18a43622232aa94e7 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 23 May 2016 17:20:29 -0700 Subject: [PATCH 101/143] [MINOR][SPARKR][DOC] Add a description for running unit tests in Windows ## What changes were proposed in this pull request? This PR adds the description for running unit tests in Windows. ## How was this patch tested? On a bare machine (Window 7, 32bits), this was manually built and tested. Author: hyukjinkwon Closes #13217 from HyukjinKwon/minor-r-doc. --- R/README.md | 8 +++++++- R/WINDOWS.md | 20 ++++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/R/README.md b/R/README.md index 810bfc14e977e..044f95312ae8e 100644 --- a/R/README.md +++ b/R/README.md @@ -1,11 +1,13 @@ # R on Spark SparkR is an R package that provides a light-weight frontend to use Spark from R. + ### Installing sparkR Libraries of sparkR need to be created in `$SPARK_HOME/R/lib`. This can be done by running the script `$SPARK_HOME/R/install-dev.sh`. By default the above script uses the system wide installation of R. However, this can be changed to any user installed location of R by setting the environment variable `R_HOME` the full path of the base directory where R is installed, before running install-dev.sh script. Example: + ``` # where /home/username/R is where R is installed and /home/username/R/bin contains the files R and RScript export R_HOME=/home/username/R @@ -17,6 +19,7 @@ export R_HOME=/home/username/R #### Build Spark Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run + ``` build/mvn -DskipTests -Psparkr package ``` @@ -38,6 +41,7 @@ To set other options like driver memory, executor memory etc. you can pass in th #### Using SparkR from RStudio If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example + ``` # Set this to where Spark is installed Sys.setenv(SPARK_HOME="/Users/username/spark") @@ -64,13 +68,15 @@ To run one of them, use `./bin/spark-submit `. For example: ./bin/spark-submit examples/src/main/r/dataframe.R -You can also run the unit-tests for SparkR by running (you need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first): +You can also run the unit tests for SparkR by running. You need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first: R -e 'install.packages("testthat", repos="http://cran.us.r-project.org")' ./R/run-tests.sh ### Running on YARN + The `./bin/spark-submit` can also be used to submit jobs to YARN clusters. You will need to set YARN conf dir before doing so. For example on CDH you can run + ``` export YARN_CONF_DIR=/etc/hadoop/conf ./bin/spark-submit --master yarn examples/src/main/r/dataframe.R diff --git a/R/WINDOWS.md b/R/WINDOWS.md index 3f889c0ca3d1e..f948ed3974794 100644 --- a/R/WINDOWS.md +++ b/R/WINDOWS.md @@ -11,3 +11,23 @@ include Rtools and R in `PATH`. directory in Maven in `PATH`. 4. Set `MAVEN_OPTS` as described in [Building Spark](http://spark.apache.org/docs/latest/building-spark.html). 5. Open a command shell (`cmd`) in the Spark directory and run `mvn -DskipTests -Psparkr package` + +## Unit tests + +To run the SparkR unit tests on Windows, the following steps are required —assuming you are in the Spark root directory and do not have Apache Hadoop installed already: + +1. Create a folder to download Hadoop related files for Windows. For example, `cd ..` and `mkdir hadoop`. + +2. Download the relevant Hadoop bin package from [steveloughran/winutils](https://github.com/steveloughran/winutils). While these are not official ASF artifacts, they are built from the ASF release git hashes by a Hadoop PMC member on a dedicated Windows VM. For further reading, consult [Windows Problems on the Hadoop wiki](https://wiki.apache.org/hadoop/WindowsProblems). + +3. Install the files into `hadoop\bin`; make sure that `winutils.exe` and `hadoop.dll` are present. + +4. Set the environment variable `HADOOP_HOME` to the full path to the newly created `hadoop` directory. + +5. Run unit tests for SparkR by running the command below. You need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first: + + ``` + R -e "install.packages('testthat', repos='http://cran.us.r-project.org')" + .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defualt.name="file:///" R\pkg\tests\run-all.R + ``` + From 01659bc50cd3d53815d205d005c3678e714c08e0 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 23 May 2016 17:32:01 -0700 Subject: [PATCH 102/143] [SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively ## What changes were proposed in this pull request? Currently command `ADD FILE|JAR ` is supported natively in SparkSQL. However, when this command is run, the file/jar is added to the resources that can not be looked up by `LIST FILE(s)|JAR(s)` command because the `LIST` command is passed to Hive command processor in Spark-SQL or simply not supported in Spark-shell. There is no way users can find out what files/jars are added to the spark context. Refer to [Hive commands](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli) This PR is to support following commands: `LIST (FILE[s] [filepath ...] | JAR[s] [jarfile ...])` ### For example: ##### LIST FILE(s) ``` scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt") res1: org.apache.spark.sql.DataFrame = [] scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt") res2: org.apache.spark.sql.DataFrame = [] scala> spark.sql("list file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt").show(false) +----------------------------------------------+ |result | +----------------------------------------------+ |hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt| +----------------------------------------------+ scala> spark.sql("list files").show(false) +----------------------------------------------+ |result | +----------------------------------------------+ |hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt| |hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt | +----------------------------------------------+ ``` ##### LIST JAR(s) ``` scala> spark.sql("add jar /Users/xinwu/spark/core/src/test/resources/TestUDTF.jar") res9: org.apache.spark.sql.DataFrame = [result: int] scala> spark.sql("list jar TestUDTF.jar").show(false) +---------------------------------------------+ |result | +---------------------------------------------+ |spark://192.168.1.234:50131/jars/TestUDTF.jar| +---------------------------------------------+ scala> spark.sql("list jars").show(false) +---------------------------------------------+ |result | +---------------------------------------------+ |spark://192.168.1.234:50131/jars/TestUDTF.jar| +---------------------------------------------+ ``` ## How was this patch tested? New test cases are added for Spark-SQL, Spark-Shell and SparkContext API code path. Author: Xin Wu Author: xin Wu Closes #13212 from xwu0226/list_command. --- .../scala/org/apache/spark/SparkContext.scala | 10 +++ core/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../org/apache/spark/SparkContextSuite.scala | 14 ++++- .../spark/sql/catalyst/parser/SqlBase.g4 | 5 +- .../spark/sql/execution/SparkSqlParser.scala | 39 ++++++++++-- .../sql/execution/command/resources.scala | 57 +++++++++++++++++- .../hive/thriftserver/SparkSQLCLIDriver.scala | 10 ++- .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../sql/hive/thriftserver/CliSuite.scala | 19 ++++++ .../sql/hive/execution/HiveQuerySuite.scala | 12 ++++ 10 files changed, 149 insertions(+), 17 deletions(-) create mode 100644 core/src/test/resources/TestUDTF.jar create mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6cdd0d298f37..351024bea47e6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1386,6 +1386,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli addFile(path, false) } + /** + * Returns a list of file paths that are added to resources. + */ + def listFiles(): Seq[String] = addedFiles.keySet.toSeq + /** * Add a file to be downloaded with this Spark job on every node. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported @@ -1724,6 +1729,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli postEnvironmentUpdate() } + /** + * Returns a list of jar files that are added to resources. + */ + def listJars(): Seq[String] = addedJars.keySet.toSeq + // Shut down the SparkContext. def stop() { if (LiveListenerBus.withinListenerThread.value) { diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d | JAR[S] ) + * LIST (FILE[S] [filepath ...] | JAR[S] [jarpath ...]) + * }}} */ - override def visitAddResource(ctx: AddResourceContext): LogicalPlan = withOrigin(ctx) { - ctx.identifier.getText.toLowerCase match { - case "file" => AddFileCommand(remainder(ctx.identifier).trim) - case "jar" => AddJarCommand(remainder(ctx.identifier).trim) - case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) + override def visitManageResource(ctx: ManageResourceContext): LogicalPlan = withOrigin(ctx) { + val mayebePaths = remainder(ctx.identifier).trim + ctx.op.getType match { + case SqlBaseParser.ADD => + ctx.identifier.getText.toLowerCase match { + case "file" => AddFileCommand(mayebePaths) + case "jar" => AddJarCommand(mayebePaths) + case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) + } + case SqlBaseParser.LIST => + ctx.identifier.getText.toLowerCase match { + case "files" | "file" => + if (mayebePaths.length > 0) { + ListFilesCommand(mayebePaths.split("\\s+")) + } else { + ListFilesCommand() + } + case "jars" | "jar" => + if (mayebePaths.length > 0) { + ListJarsCommand(mayebePaths.split("\\s+")) + } else { + ListJarsCommand() + } + case other => throw operationNotAllowed(s"LIST with resource type '$other'", ctx) + } + case _ => throw operationNotAllowed(s"Other types of operation on resources", ctx) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala index 162d493c1f8ce..20b08946675d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala @@ -17,9 +17,14 @@ package org.apache.spark.sql.execution.command +import java.io.File +import java.net.URI + +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} /** * Adds a jar to the current session so it can be used (for UDFs or serdes). @@ -46,3 +51,51 @@ case class AddFileCommand(path: String) extends RunnableCommand { Seq.empty[Row] } } + +/** + * Returns a list of file paths that are added to resources. + * If file paths are provided, return the ones that are added to resources. + */ +case class ListFilesCommand(files: Seq[String] = Seq.empty[String]) extends RunnableCommand { + override val output: Seq[Attribute] = { + AttributeReference("Results", StringType, nullable = false)() :: Nil + } + override def run(sparkSession: SparkSession): Seq[Row] = { + val fileList = sparkSession.sparkContext.listFiles() + if (files.size > 0) { + files.map { f => + val uri = new URI(f) + val schemeCorrectedPath = uri.getScheme match { + case null | "local" => new File(f).getCanonicalFile.toURI.toString + case _ => f + } + new Path(schemeCorrectedPath).toUri.toString + }.collect { + case f if fileList.contains(f) => f + }.map(Row(_)) + } else { + fileList.map(Row(_)) + } + } +} + +/** + * Returns a list of jar files that are added to resources. + * If jar files are provided, return the ones that are added to resources. + */ +case class ListJarsCommand(jars: Seq[String] = Seq.empty[String]) extends RunnableCommand { + override val output: Seq[Attribute] = { + AttributeReference("Results", StringType, nullable = false)() :: Nil + } + override def run(sparkSession: SparkSession): Seq[Row] = { + val jarList = sparkSession.sparkContext.listJars() + if (jars.nonEmpty) { + for { + jarName <- jars.map(f => new Path(f).getName) + jarPath <- jarList if jarPath.contains(jarName) + } yield Row(jarPath) + } else { + jarList.map(Row(_)) + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 33ff8aee79969..7389e18aefb1b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -32,8 +32,7 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, CommandProcessor} -import org.apache.hadoop.hive.ql.processors.{CommandProcessorFactory, ResetProcessor, SetProcessor} +import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket @@ -295,9 +294,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { System.exit(0) } if (tokens(0).toLowerCase(Locale.ENGLISH).equals("source") || - cmd_trimmed.startsWith("!") || - tokens(0).toLowerCase.equals("list") || - isRemoteMode) { + cmd_trimmed.startsWith("!") || isRemoteMode) { val start = System.currentTimeMillis() super.processCmd(cmd) val end = System.currentTimeMillis() @@ -312,7 +309,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (proc != null) { // scalastyle:off println if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || - proc.isInstanceOf[AddResourceProcessor] || proc.isInstanceOf[ResetProcessor]) { + proc.isInstanceOf[AddResourceProcessor] || proc.isInstanceOf[ListResourceProcessor] || + proc.isInstanceOf[ResetProcessor] ) { val driver = new SparkSQLDriver driver.init() diff --git a/sql/hive-thriftserver/src/test/resources/TestUDTF.jar b/sql/hive-thriftserver/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d "This is a test for Spark-11624") } + + test("list jars") { + val jarFile = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + runCliWithin(2.minute)( + s"ADD JAR $jarFile" -> "", + s"LIST JARS" -> "TestUDTF.jar", + s"List JAR $jarFile" -> "TestUDTF.jar" + ) + } + + test("list files") { + val dataFilePath = Thread.currentThread().getContextClassLoader + .getResource("data/files/small_kv.txt") + runCliWithin(2.minute)( + s"ADD FILE $dataFilePath" -> "", + s"LIST FILES" -> "small_kv.txt", + s"LIST FILE $dataFilePath" -> "small_kv.txt" + ) + } } 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 e179021491a69..e0f6ccf04dd33 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 @@ -876,6 +876,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") sql("select * from src join t1 on src.key = t1.a") sql("DROP TABLE t1") + assert(sql("list jars"). + filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + assert(sql("list jar"). + filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + val testJar2 = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath + sql(s"ADD JAR $testJar2") + assert(sql(s"list jar $testJar").count() == 1) } test("CREATE TEMPORARY FUNCTION") { @@ -899,6 +906,11 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } assert(checkAddFileRDD.first()) + assert(sql("list files"). + filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file"). + filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) } createQueryTest("dynamic_partition", From 5afd927a47aa7ede3039234f2f7262e2247aa2ae Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 23 May 2016 18:03:45 -0700 Subject: [PATCH 103/143] [SPARK-15311][SQL] Disallow DML on Regular Tables when Using In-Memory Catalog #### What changes were proposed in this pull request? So far, when using In-Memory Catalog, we allow DDL operations for the tables. However, the corresponding DML operations are not supported for the tables that are neither temporary nor data source tables. For example, ```SQL CREATE TABLE tabName(i INT, j STRING) SELECT * FROM tabName INSERT OVERWRITE TABLE tabName SELECT 1, 'a' ``` In the above example, before this PR fix, we will get very confusing exception messages for either `SELECT` or `INSERT` ``` org.apache.spark.sql.AnalysisException: unresolved operator 'SimpleCatalogRelation default, CatalogTable(`default`.`tbl`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(i,int,true,None), CatalogColumn(j,string,true,None)),List(),List(),List(),-1,,1463928681802,-1,Map(),None,None,None,List()), None; ``` This PR is to issue appropriate exceptions in this case. The message will be like ``` org.apache.spark.sql.AnalysisException: Please enable Hive support when operating non-temporary tables: `tbl`; ``` #### How was this patch tested? Added a test case in `DDLSuite`. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13093 from gatorsmile/selectAfterCreate. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 15 ++++++ .../plans/logical/basicLogicalOperators.scala | 3 +- .../sql/execution/command/DDLSuite.scala | 48 +++++++++++++++++++ 3 files changed, 65 insertions(+), 1 deletion(-) 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 28aa249888b30..cd242d78a4123 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.catalog.SimpleCatalogRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.UsingJoin @@ -305,6 +306,20 @@ trait CheckAnalysis extends PredicateHelper { |Conflicting attributes: ${conflictingAttributes.mkString(",")} """.stripMargin) + case s: SimpleCatalogRelation => + failAnalysis( + s""" + |Please enable Hive support when selecting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + + case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => + failAnalysis( + s""" + |Please enable Hive support when inserting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + case o if !o.resolved => failAnalysis( s"unresolved operator ${operator.simpleString}") 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 b1b3e00de1456..ca0096eeb2087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -377,7 +377,8 @@ case class InsertIntoTable( } assert(overwrite || !ifNotExists) - override lazy val resolved: Boolean = childrenResolved && expectedColumns.forall { expected => + override lazy val resolved: Boolean = + childrenResolved && table.resolved && expectedColumns.forall { expected => child.output.size == expected.size && child.output.zip(expected).forall { case (childAttr, tableAttr) => DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) 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 d72dc092e277c..64f5a4ac479a7 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 @@ -22,6 +22,7 @@ import java.io.File import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach +import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, NoSuchPartitionException, NoSuchTableException} @@ -1044,6 +1045,53 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { ) } + test("select/insert into the managed table") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + val tabName = "tbl" + withTable(tabName) { + sql(s"CREATE TABLE $tabName (i INT, j STRING)") + val catalogTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + assert(catalogTable.tableType == CatalogTableType.MANAGED) + + var message = intercept[AnalysisException] { + sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") + }.getMessage + assert(message.contains("Please enable Hive support when inserting the regular tables")) + message = intercept[AnalysisException] { + sql(s"SELECT * FROM $tabName") + }.getMessage + assert(message.contains("Please enable Hive support when selecting the regular tables")) + } + } + + test("select/insert into external table") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + withTempDir { tempDir => + val tabName = "tbl" + withTable(tabName) { + sql( + s""" + |CREATE EXTERNAL TABLE $tabName (i INT, j STRING) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |LOCATION '$tempDir' + """.stripMargin) + val catalogTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + assert(catalogTable.tableType == CatalogTableType.EXTERNAL) + + var message = intercept[AnalysisException] { + sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") + }.getMessage + assert(message.contains("Please enable Hive support when inserting the regular tables")) + message = intercept[AnalysisException] { + sql(s"SELECT * FROM $tabName") + }.getMessage + assert(message.contains("Please enable Hive support when selecting the regular tables")) + } + } + } + test("drop default database") { Seq("true", "false").foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { From a15ca5533db91fefaf3248255a59c4d94eeda1a9 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Mon, 23 May 2016 18:14:48 -0700 Subject: [PATCH 104/143] [SPARK-15464][ML][MLLIB][SQL][TESTS] Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code ## What changes were proposed in this pull request? Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code. ## How was this patch tested? Existing test. Author: WeichenXu Closes #13242 from WeichenXu123/python_doctest_update_sparksession. --- python/pyspark/ml/classification.py | 38 ++--- python/pyspark/ml/clustering.py | 22 +-- python/pyspark/ml/evaluation.py | 20 +-- python/pyspark/ml/feature.py | 66 ++++----- python/pyspark/ml/recommendation.py | 18 +-- python/pyspark/ml/regression.py | 46 ++++--- python/pyspark/ml/tuning.py | 18 +-- python/pyspark/mllib/classification.py | 10 +- python/pyspark/mllib/evaluation.py | 10 +- python/pyspark/mllib/feature.py | 10 +- python/pyspark/mllib/fpm.py | 9 +- python/pyspark/mllib/linalg/distributed.py | 12 +- python/pyspark/mllib/random.py | 10 +- python/pyspark/mllib/regression.py | 10 +- python/pyspark/mllib/stat/_statistics.py | 10 +- python/pyspark/mllib/tree.py | 9 +- python/pyspark/mllib/util.py | 10 +- python/pyspark/sql/catalog.py | 14 +- python/pyspark/sql/column.py | 12 +- python/pyspark/sql/conf.py | 11 +- python/pyspark/sql/functions.py | 153 +++++++++++---------- python/pyspark/sql/group.py | 12 +- 22 files changed, 298 insertions(+), 232 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index a1c3f7298409c..ea660d7808fa4 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -498,7 +498,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -512,7 +512,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred 1 >>> model.featureImportances SparseVector(1, {0: 1.0}) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -520,7 +520,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred DenseVector([1.0, 0.0]) >>> result.rawPrediction DenseVector([1.0, 0.0]) - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 @@ -627,7 +627,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -639,7 +639,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0, 1.0]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -647,7 +647,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred 0 >>> numpy.argmax(result.rawPrediction) 0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> rfc_path = temp_path + "/rfc" @@ -754,7 +754,7 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -766,10 +766,10 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> gbtc_path = temp_path + "gbtc" @@ -885,7 +885,7 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H >>> from pyspark.sql import Row >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... Row(label=0.0, features=Vectors.dense([0.0, 0.0])), ... Row(label=0.0, features=Vectors.dense([0.0, 1.0])), ... Row(label=1.0, features=Vectors.dense([1.0, 0.0]))]) @@ -1029,7 +1029,7 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, Number of outputs has to be equal to the total number of labels. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (0.0, Vectors.dense([0.0, 0.0])), ... (1.0, Vectors.dense([0.0, 1.0])), ... (1.0, Vectors.dense([1.0, 0.0])), @@ -1040,7 +1040,7 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, [2, 5, 2] >>> model.weights.size 27 - >>> testDF = sqlContext.createDataFrame([ + >>> testDF = spark.createDataFrame([ ... (Vectors.dense([1.0, 0.0]),), ... (Vectors.dense([0.0, 0.0]),)], ["features"]) >>> model.transform(testDF).show() @@ -1467,21 +1467,23 @@ def _to_java(self): if __name__ == "__main__": import doctest import pyspark.ml.classification - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.classification.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.classification tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.classification tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index ac7183d2efff5..a457904e7880a 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -73,7 +73,7 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte ... (Vectors.dense([0.75, 0.935]),), ... (Vectors.dense([-0.83, -0.68]),), ... (Vectors.dense([-0.91, -0.76]),)] - >>> df = sqlContext.createDataFrame(data, ["features"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> gm = GaussianMixture(k=3, tol=0.0001, ... maxIter=10, seed=10) >>> model = gm.fit(df) @@ -197,7 +197,7 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol >>> 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"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> kmeans = KMeans(k=2, seed=1) >>> model = kmeans.fit(df) >>> centers = model.clusterCenters() @@ -350,7 +350,7 @@ class BisectingKMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte >>> 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"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> bkm = BisectingKMeans(k=2, minDivisibleClusterSize=1.0) >>> model = bkm.fit(df) >>> centers = model.clusterCenters() @@ -627,7 +627,7 @@ class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInter >>> from pyspark.ml.linalg import Vectors, SparseVector >>> from pyspark.ml.clustering import LDA - >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], + >>> df = spark.createDataFrame([[1, Vectors.dense([0.0, 1.0])], ... [2, SparseVector(2, {0: 1.0})],], ["id", "features"]) >>> lda = LDA(k=2, seed=1, optimizer="em") >>> model = lda.fit(df) @@ -933,21 +933,23 @@ def getKeepLastCheckpoint(self): if __name__ == "__main__": import doctest import pyspark.ml.clustering - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.clustering.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.clustering tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.clustering tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 16029dc34863a..b8b2b37af553d 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -114,7 +114,7 @@ class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPrediction >>> 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"]) + >>> dataset = spark.createDataFrame(scoreAndLabels, ["raw", "label"]) ... >>> evaluator = BinaryClassificationEvaluator(rawPredictionCol="raw") >>> evaluator.evaluate(dataset) @@ -181,7 +181,7 @@ class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): >>> scoreAndLabels = [(-28.98343821, -27.0), (20.21491975, 21.5), ... (-25.98418959, -22.0), (30.69731842, 33.0), (74.69283752, 71.0)] - >>> dataset = sqlContext.createDataFrame(scoreAndLabels, ["raw", "label"]) + >>> dataset = spark.createDataFrame(scoreAndLabels, ["raw", "label"]) ... >>> evaluator = RegressionEvaluator(predictionCol="raw") >>> evaluator.evaluate(dataset) @@ -253,7 +253,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio >>> scoreAndLabels = [(0.0, 0.0), (0.0, 1.0), (0.0, 0.0), ... (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)] - >>> dataset = sqlContext.createDataFrame(scoreAndLabels, ["prediction", "label"]) + >>> dataset = spark.createDataFrame(scoreAndLabels, ["prediction", "label"]) ... >>> evaluator = MulticlassClassificationEvaluator(predictionCol="prediction") >>> evaluator.evaluate(dataset) @@ -313,17 +313,19 @@ def setParams(self, predictionCol="prediction", labelCol="label", if __name__ == "__main__": import doctest - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.evaluation tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.evaluation tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 497f2ad68ee51..93745c70c4fe6 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -66,7 +66,7 @@ class Binarizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java Binarize a column of continuous features given a threshold. - >>> df = sqlContext.createDataFrame([(0.5,)], ["values"]) + >>> df = spark.createDataFrame([(0.5,)], ["values"]) >>> binarizer = Binarizer(threshold=1.0, inputCol="values", outputCol="features") >>> binarizer.transform(df).head().features 0.0 @@ -131,7 +131,7 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav Maps a column of continuous features to a column of feature buckets. - >>> df = sqlContext.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) + >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], ... inputCol="values", outputCol="buckets") >>> bucketed = bucketizer.transform(df).collect() @@ -206,7 +206,7 @@ class CountVectorizer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Extracts a vocabulary from document collections and generates a :py:attr:`CountVectorizerModel`. - >>> df = sqlContext.createDataFrame( + >>> df = spark.createDataFrame( ... [(0, ["a", "b", "c"]), (1, ["a", "b", "b", "c", "a"])], ... ["label", "raw"]) >>> cv = CountVectorizer(inputCol="raw", outputCol="vectors") @@ -381,7 +381,7 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit `_. >>> from pyspark.ml.linalg import Vectors - >>> df1 = sqlContext.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) + >>> df1 = spark.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) >>> dct = DCT(inverse=False, inputCol="vec", outputCol="resultVec") >>> df2 = dct.transform(df1) >>> df2.head().resultVec @@ -448,7 +448,7 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada by a scalar multiplier. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([2.0, 1.0, 3.0]),)], ["values"]) + >>> df = spark.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") >>> ep.transform(df).head().eprod @@ -516,7 +516,7 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, Java it is advisable to use a power of two as the numFeatures parameter; otherwise the features will not be mapped evenly to the columns. - >>> df = sqlContext.createDataFrame([(["a", "b", "c"],)], ["words"]) + >>> df = spark.createDataFrame([(["a", "b", "c"],)], ["words"]) >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") >>> hashingTF.transform(df).head().features SparseVector(10, {0: 1.0, 1: 1.0, 2: 1.0}) @@ -583,7 +583,7 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab Compute the Inverse Document Frequency (IDF) given a collection of documents. >>> from pyspark.ml.linalg import DenseVector - >>> df = sqlContext.createDataFrame([(DenseVector([1.0, 2.0]),), + >>> df = spark.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") >>> model = idf.fit(df) @@ -671,7 +671,7 @@ class MaxAbsScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav any sparsity. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> df = spark.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> maScaler = MaxAbsScaler(inputCol="a", outputCol="scaled") >>> model = maScaler.fit(df) >>> model.transform(df).show() @@ -758,7 +758,7 @@ class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav transformer will be DenseVector even for sparse input. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> df = spark.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> mmScaler = MinMaxScaler(inputCol="a", outputCol="scaled") >>> model = mmScaler.fit(df) >>> model.originalMin @@ -889,7 +889,7 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr When the input array length is less than n (number of elements per n-gram), no n-grams are returned. - >>> df = sqlContext.createDataFrame([Row(inputTokens=["a", "b", "c", "d", "e"])]) + >>> df = spark.createDataFrame([Row(inputTokens=["a", "b", "c", "d", "e"])]) >>> ngram = NGram(n=2, inputCol="inputTokens", outputCol="nGrams") >>> ngram.transform(df).head() Row(inputTokens=[u'a', u'b', u'c', u'd', u'e'], nGrams=[u'a b', u'b c', u'c d', u'd e']) @@ -963,7 +963,7 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav >>> 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"]) + >>> df = spark.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) >>> normalizer = Normalizer(p=2.0, inputCol="dense", outputCol="features") >>> normalizer.transform(df).head().features DenseVector([0.6, -0.8]) @@ -1115,7 +1115,7 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) + >>> df = spark.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) >>> px = PolynomialExpansion(degree=2, inputCol="dense", outputCol="expanded") >>> px.transform(df).head().expanded DenseVector([0.5, 0.25, 2.0, 1.0, 4.0]) @@ -1182,7 +1182,7 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav covering all real values. This attempts to find numBuckets partitions based on a sample of data, but it may find fewer depending on the data sample values. - >>> df = sqlContext.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) + >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) >>> qds = QuantileDiscretizer(numBuckets=2, ... inputCol="values", outputCol="buckets", seed=123) >>> qds.getSeed() @@ -1272,7 +1272,7 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, length. It returns an array of strings that can be empty. - >>> df = sqlContext.createDataFrame([("A B c",)], ["text"]) + >>> df = spark.createDataFrame([("A B c",)], ["text"]) >>> reTokenizer = RegexTokenizer(inputCol="text", outputCol="words") >>> reTokenizer.transform(df).head() Row(text=u'A B c', words=[u'a', u'b', u'c']) @@ -1400,7 +1400,7 @@ class SQLTransformer(JavaTransformer, JavaMLReadable, JavaMLWritable): Currently we only support SQL syntax like 'SELECT ... FROM __THIS__' where '__THIS__' represents the underlying table of the input dataset. - >>> df = sqlContext.createDataFrame([(0, 1.0, 3.0), (2, 2.0, 5.0)], ["id", "v1", "v2"]) + >>> df = spark.createDataFrame([(0, 1.0, 3.0), (2, 2.0, 5.0)], ["id", "v1", "v2"]) >>> sqlTrans = SQLTransformer( ... statement="SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__") >>> sqlTrans.transform(df).head() @@ -1461,7 +1461,7 @@ class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, J statistics on the samples in the training set. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> df = spark.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") >>> model = standardScaler.fit(df) >>> model.mean @@ -1718,7 +1718,7 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadabl A feature transformer that filters out stop words from input. Note: null values from input array are preserved unless adding null to stopWords explicitly. - >>> df = sqlContext.createDataFrame([(["a", "b", "c"],)], ["text"]) + >>> df = spark.createDataFrame([(["a", "b", "c"],)], ["text"]) >>> remover = StopWordsRemover(inputCol="text", outputCol="words", stopWords=["b"]) >>> remover.transform(df).head().words == ['a', 'c'] True @@ -1810,7 +1810,7 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java A tokenizer that converts the input string to lowercase and then splits it by white spaces. - >>> df = sqlContext.createDataFrame([("a b c",)], ["text"]) + >>> df = spark.createDataFrame([("a b c",)], ["text"]) >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) @@ -1864,7 +1864,7 @@ class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol, JavaMLReadabl A feature transformer that merges multiple columns into a vector column. - >>> df = sqlContext.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) + >>> df = spark.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) >>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features") >>> vecAssembler.transform(df).head().features DenseVector([1.0, 0.0, 3.0]) @@ -1944,7 +1944,7 @@ class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Ja - Add option for allowing unknown categories. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([-1.0, 0.0]),), + >>> df = spark.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") >>> model = indexer.fit(df) @@ -2074,7 +2074,7 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J followed by the selected names (in the order given). >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.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]),), ... (Vectors.dense([0.6, -1.1, -3.0, 4.5, 3.3]),)], ["features"]) @@ -2161,7 +2161,7 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, Has natural language processing or machine learning process. >>> sent = ("a b " * 100 + "a c " * 10).split(" ") - >>> doc = sqlContext.createDataFrame([(sent,), (sent,)], ["sentence"]) + >>> doc = spark.createDataFrame([(sent,), (sent,)], ["sentence"]) >>> word2Vec = Word2Vec(vectorSize=5, seed=42, inputCol="sentence", outputCol="model") >>> model = word2Vec.fit(doc) >>> model.getVectors().show() @@ -2345,7 +2345,7 @@ class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab >>> 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=2, inputCol="features", outputCol="pca_features") >>> model = pca.fit(df) >>> model.transform(df).collect()[0].pca_features @@ -2447,7 +2447,7 @@ class RFormula(JavaEstimator, HasFeaturesCol, HasLabelCol, JavaMLReadable, JavaM operators, including '~', '.', ':', '+', and '-'. Also see the `R formula docs `_. - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, 1.0, "a"), ... (0.0, 2.0, "b"), ... (0.0, 0.0, "a") @@ -2561,7 +2561,7 @@ class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, Ja categorical label. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame( + >>> df = spark.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), ... (Vectors.dense([1.0, 0.0, 15.0, 0.1]), 0.0)], @@ -2656,8 +2656,7 @@ def selectedFeatures(self): import tempfile import pyspark.ml.feature - from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext + from pyspark.sql import Row, SparkSession globs = globals().copy() features = pyspark.ml.feature.__dict__.copy() @@ -2665,19 +2664,22 @@ def selectedFeatures(self): # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.feature tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.feature tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark testData = sc.parallelize([Row(id=0, label="a"), Row(id=1, label="b"), Row(id=2, label="c"), Row(id=3, label="a"), Row(id=4, label="a"), Row(id=5, label="c")], 2) - globs['stringIndDf'] = sqlContext.createDataFrame(testData) + globs['stringIndDf'] = spark.createDataFrame(testData) temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 86c00d91652d1..bac2a3029fd71 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -65,7 +65,7 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha indicated user preferences rather than explicit ratings given to items. - >>> df = sqlContext.createDataFrame( + >>> df = spark.createDataFrame( ... [(0, 0, 4.0), (0, 1, 2.0), (1, 1, 3.0), (1, 2, 4.0), (2, 1, 1.0), (2, 2, 5.0)], ... ["user", "item", "rating"]) >>> als = ALS(rank=10, maxIter=5) @@ -74,7 +74,7 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha 10 >>> model.userFactors.orderBy("id").collect() [Row(id=0, features=[...]), Row(id=1, ...), Row(id=2, ...)] - >>> test = sqlContext.createDataFrame([(0, 2), (1, 0), (2, 0)], ["user", "item"]) + >>> test = spark.createDataFrame([(0, 2), (1, 0), (2, 0)], ["user", "item"]) >>> predictions = sorted(model.transform(test).collect(), key=lambda r: r[0]) >>> predictions[0] Row(user=0, item=2, prediction=-0.13807615637779236) @@ -370,21 +370,23 @@ def itemFactors(self): if __name__ == "__main__": import doctest import pyspark.ml.recommendation - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.recommendation.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.recommendation tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.recommendation tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index e21dd83923adb..8f58594a66360 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -55,19 +55,19 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction - L2 + L1 (elastic net) >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = 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") >>> model = lr.fit(df) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> abs(model.transform(test0).head().prediction - (-1.0)) < 0.001 True >>> abs(model.coefficients[0] - 1.0) < 0.001 True >>> abs(model.intercept - 0.0) < 0.001 True - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> abs(model.transform(test1).head().prediction - 1.0) < 0.001 True >>> lr.setParams("vector") @@ -413,12 +413,12 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti Only univariate (single feature) algorithm supported. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> ir = IsotonicRegression() >>> model = ir.fit(df) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 >>> model.boundaries @@ -643,7 +643,7 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi It supports both continuous and categorical features. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> dt = DecisionTreeRegressor(maxDepth=2, varianceCol="variance") @@ -654,10 +654,10 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi 3 >>> model.featureImportances SparseVector(1, {0: 1.0}) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> dtr_path = temp_path + "/dtr" @@ -809,7 +809,7 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2, seed=42) @@ -818,10 +818,10 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 0.5 >>> rfr_path = temp_path + "/rfr" @@ -921,7 +921,7 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> gbt = GBTRegressor(maxIter=5, maxDepth=2, seed=42) @@ -932,10 +932,10 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> gbtr_path = temp_path + "gbtr" @@ -1056,7 +1056,7 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi .. seealso:: `AFT Model `_ >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0), 1.0), ... (0.0, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) >>> aftsr = AFTSurvivalRegression() @@ -1257,7 +1257,7 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha .. seealso:: `GLM `_ >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(0.0, 0.0)), ... (1.0, Vectors.dense(1.0, 2.0)), ... (2.0, Vectors.dense(0.0, 0.0)), @@ -1603,21 +1603,23 @@ def pValues(self): if __name__ == "__main__": import doctest import pyspark.ml.regression - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.regression.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.regression tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.regression tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 4f7a6b0f7b73b..fe87b6cdb9369 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -152,7 +152,7 @@ class CrossValidator(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator >>> from pyspark.ml.linalg import Vectors - >>> dataset = sqlContext.createDataFrame( + >>> dataset = spark.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), ... (Vectors.dense([0.5]), 0.0), @@ -311,7 +311,7 @@ class TrainValidationSplit(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator >>> from pyspark.ml.linalg import Vectors - >>> dataset = sqlContext.createDataFrame( + >>> dataset = spark.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), ... (Vectors.dense([0.5]), 0.0), @@ -456,17 +456,19 @@ def copy(self, extra=None): if __name__ == "__main__": import doctest - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.tuning tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.tuning tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index fe5b6844bfceb..f186217031193 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -756,12 +756,16 @@ def update(rdd): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession import pyspark.mllib.classification globs = pyspark.mllib.classification.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.classification tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 22e68ea5b4511..5f32f092c7cbe 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -516,12 +516,16 @@ def accuracy(self): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession import pyspark.mllib.evaluation globs = pyspark.mllib.evaluation.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.evaluation tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 90559f6cfbe43..e31c75c1e8675 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -732,11 +732,15 @@ def transform(self, vector): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.feature tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index f339e50891166..ab4066f7d68ba 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -183,16 +183,21 @@ class FreqSequence(namedtuple("FreqSequence", ["sequence", "freq"])): def _test(): import doctest + from pyspark.sql import SparkSession import pyspark.mllib.fpm globs = pyspark.mllib.fpm.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.fpm tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/mllib/linalg/distributed.py b/python/pyspark/mllib/linalg/distributed.py index af34ce346b0ca..ea4f27cf4ffe9 100644 --- a/python/pyspark/mllib/linalg/distributed.py +++ b/python/pyspark/mllib/linalg/distributed.py @@ -1184,16 +1184,18 @@ def toCoordinateMatrix(self): def _test(): import doctest - from pyspark import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession from pyspark.mllib.linalg import Matrices import pyspark.mllib.linalg.distributed globs = pyspark.mllib.linalg.distributed.__dict__.copy() - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) - globs['sqlContext'] = SQLContext(globs['sc']) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.linalg.distributed tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext globs['Matrices'] = Matrices (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 6a3c643b66417..61213ddf62e8b 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -409,13 +409,17 @@ def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed= def _test(): import doctest - from pyspark.context import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.random tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 639c5eabaa23b..43d9072a24e4d 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -824,12 +824,16 @@ def update(rdd): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession import pyspark.mllib.regression globs = pyspark.mllib.regression.__dict__.copy() - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.regression tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index 36c8f48a4a882..b0a85240b289a 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -306,11 +306,15 @@ def kolmogorovSmirnovTest(data, distName="norm", *params): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.stat.statistics tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index f7ea466b43291..8be76fcefe542 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -657,9 +657,14 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, def _test(): import doctest globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + from pyspark.sql import SparkSession + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.tree tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 39bc6586dd582..a316ee1ad45ff 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -347,13 +347,17 @@ def generateLinearRDD(sc, nexamples, nfeatures, eps, def _test(): import doctest - from pyspark.context import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.util tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 812dbba59e277..3033f147bc969 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -244,21 +244,23 @@ def _reset(self): def _test(): import os import doctest - from pyspark.context import SparkContext - from pyspark.sql.session import SparkSession + from pyspark.sql import SparkSession import pyspark.sql.catalog os.chdir(os.environ["SPARK_HOME"]) globs = pyspark.sql.catalog.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') - globs['sc'] = sc - globs['spark'] = SparkSession(sc) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.catalog tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod( pyspark.sql.catalog, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 5b26e9469801e..4b99f3058b75c 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -434,13 +434,15 @@ def __repr__(self): def _test(): import doctest - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession import pyspark.sql.column globs = pyspark.sql.column.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.column tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) @@ -448,7 +450,7 @@ def _test(): (failure_count, test_count) = doctest.testmod( pyspark.sql.column, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index 609d882a95a32..792c420ca6386 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -71,11 +71,14 @@ def _test(): os.chdir(os.environ["SPARK_HOME"]) globs = pyspark.sql.conf.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') - globs['sc'] = sc - globs['spark'] = SparkSession(sc) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.conf tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod(pyspark.sql.conf, globs=globs) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 716b16fdc9530..1f15eec645382 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -212,7 +212,7 @@ def broadcast(df): def coalesce(*cols): """Returns the first column that is not null. - >>> cDf = sqlContext.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b")) + >>> cDf = spark.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b")) >>> cDf.show() +----+----+ | a| b| @@ -252,7 +252,7 @@ def corr(col1, col2): >>> a = range(20) >>> b = [2 * x for x in range(20)] - >>> df = sqlContext.createDataFrame(zip(a, b), ["a", "b"]) + >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) >>> df.agg(corr("a", "b").alias('c')).collect() [Row(c=1.0)] """ @@ -267,7 +267,7 @@ def covar_pop(col1, col2): >>> a = [1] * 10 >>> b = [1] * 10 - >>> df = sqlContext.createDataFrame(zip(a, b), ["a", "b"]) + >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) >>> df.agg(covar_pop("a", "b").alias('c')).collect() [Row(c=0.0)] """ @@ -282,7 +282,7 @@ def covar_samp(col1, col2): >>> a = [1] * 10 >>> b = [1] * 10 - >>> df = sqlContext.createDataFrame(zip(a, b), ["a", "b"]) + >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) >>> df.agg(covar_samp("a", "b").alias('c')).collect() [Row(c=0.0)] """ @@ -373,7 +373,7 @@ def input_file_name(): def isnan(col): """An expression that returns true iff the column is NaN. - >>> df = sqlContext.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) + >>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) >>> df.select(isnan("a").alias("r1"), isnan(df.a).alias("r2")).collect() [Row(r1=False, r2=False), Row(r1=True, r2=True)] """ @@ -385,7 +385,7 @@ def isnan(col): def isnull(col): """An expression that returns true iff the column is null. - >>> df = sqlContext.createDataFrame([(1, None), (None, 2)], ("a", "b")) + >>> df = spark.createDataFrame([(1, None), (None, 2)], ("a", "b")) >>> df.select(isnull("a").alias("r1"), isnull(df.a).alias("r2")).collect() [Row(r1=False, r2=False), Row(r1=True, r2=True)] """ @@ -432,7 +432,7 @@ def nanvl(col1, col2): Both inputs should be floating point columns (DoubleType or FloatType). - >>> df = sqlContext.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) + >>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) >>> df.select(nanvl("a", "b").alias("r1"), nanvl(df.a, df.b).alias("r2")).collect() [Row(r1=1.0, r2=1.0), Row(r1=2.0, r2=2.0)] """ @@ -470,7 +470,7 @@ def round(col, scale=0): Round the given value to `scale` decimal places using HALF_UP rounding mode if `scale` >= 0 or at integral part when `scale` < 0. - >>> sqlContext.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect() + >>> spark.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect() [Row(r=3.0)] """ sc = SparkContext._active_spark_context @@ -483,7 +483,7 @@ def bround(col, scale=0): Round the given value to `scale` decimal places using HALF_EVEN rounding mode if `scale` >= 0 or at integral part when `scale` < 0. - >>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect() + >>> spark.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect() [Row(r=2.0)] """ sc = SparkContext._active_spark_context @@ -494,7 +494,7 @@ def bround(col, scale=0): def shiftLeft(col, numBits): """Shift the given value numBits left. - >>> sqlContext.createDataFrame([(21,)], ['a']).select(shiftLeft('a', 1).alias('r')).collect() + >>> spark.createDataFrame([(21,)], ['a']).select(shiftLeft('a', 1).alias('r')).collect() [Row(r=42)] """ sc = SparkContext._active_spark_context @@ -505,7 +505,7 @@ def shiftLeft(col, numBits): def shiftRight(col, numBits): """Shift the given value numBits right. - >>> sqlContext.createDataFrame([(42,)], ['a']).select(shiftRight('a', 1).alias('r')).collect() + >>> spark.createDataFrame([(42,)], ['a']).select(shiftRight('a', 1).alias('r')).collect() [Row(r=21)] """ sc = SparkContext._active_spark_context @@ -517,7 +517,7 @@ def shiftRight(col, numBits): def shiftRightUnsigned(col, numBits): """Unsigned shift the given value numBits right. - >>> df = sqlContext.createDataFrame([(-42,)], ['a']) + >>> df = spark.createDataFrame([(-42,)], ['a']) >>> df.select(shiftRightUnsigned('a', 1).alias('r')).collect() [Row(r=9223372036854775787)] """ @@ -575,7 +575,7 @@ def greatest(*cols): Returns the greatest value of the list of column names, skipping null values. This function takes at least 2 parameters. It will return null iff all parameters are null. - >>> df = sqlContext.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) + >>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) >>> df.select(greatest(df.a, df.b, df.c).alias("greatest")).collect() [Row(greatest=4)] """ @@ -591,7 +591,7 @@ def least(*cols): Returns the least value of the list of column names, skipping null values. This function takes at least 2 parameters. It will return null iff all parameters are null. - >>> df = sqlContext.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) + >>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) >>> df.select(least(df.a, df.b, df.c).alias("least")).collect() [Row(least=1)] """ @@ -647,7 +647,7 @@ def log(arg1, arg2=None): def log2(col): """Returns the base-2 logarithm of the argument. - >>> sqlContext.createDataFrame([(4,)], ['a']).select(log2('a').alias('log2')).collect() + >>> spark.createDataFrame([(4,)], ['a']).select(log2('a').alias('log2')).collect() [Row(log2=2.0)] """ sc = SparkContext._active_spark_context @@ -660,7 +660,7 @@ def conv(col, fromBase, toBase): """ Convert a number in a string column from one base to another. - >>> df = sqlContext.createDataFrame([("010101",)], ['n']) + >>> df = spark.createDataFrame([("010101",)], ['n']) >>> df.select(conv(df.n, 2, 16).alias('hex')).collect() [Row(hex=u'15')] """ @@ -673,7 +673,7 @@ def factorial(col): """ Computes the factorial of the given value. - >>> df = sqlContext.createDataFrame([(5,)], ['n']) + >>> df = spark.createDataFrame([(5,)], ['n']) >>> df.select(factorial(df.n).alias('f')).collect() [Row(f=120)] """ @@ -765,7 +765,7 @@ def date_format(date, format): NOTE: Use when ever possible specialized functions like `year`. These benefit from a specialized implementation. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(date_format('a', 'MM/dd/yyy').alias('date')).collect() [Row(date=u'04/08/2015')] """ @@ -778,7 +778,7 @@ def year(col): """ Extract the year of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(year('a').alias('year')).collect() [Row(year=2015)] """ @@ -791,7 +791,7 @@ def quarter(col): """ Extract the quarter of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(quarter('a').alias('quarter')).collect() [Row(quarter=2)] """ @@ -804,7 +804,7 @@ def month(col): """ Extract the month of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(month('a').alias('month')).collect() [Row(month=4)] """ @@ -817,7 +817,7 @@ def dayofmonth(col): """ Extract the day of the month of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(dayofmonth('a').alias('day')).collect() [Row(day=8)] """ @@ -830,7 +830,7 @@ def dayofyear(col): """ Extract the day of the year of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(dayofyear('a').alias('day')).collect() [Row(day=98)] """ @@ -843,7 +843,7 @@ def hour(col): """ Extract the hours of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df.select(hour('a').alias('hour')).collect() [Row(hour=13)] """ @@ -856,7 +856,7 @@ def minute(col): """ Extract the minutes of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df.select(minute('a').alias('minute')).collect() [Row(minute=8)] """ @@ -869,7 +869,7 @@ def second(col): """ Extract the seconds of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df.select(second('a').alias('second')).collect() [Row(second=15)] """ @@ -882,7 +882,7 @@ def weekofyear(col): """ Extract the week number of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(weekofyear(df.a).alias('week')).collect() [Row(week=15)] """ @@ -895,7 +895,7 @@ def date_add(start, days): """ Returns the date that is `days` days after `start` - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['d']) >>> df.select(date_add(df.d, 1).alias('d')).collect() [Row(d=datetime.date(2015, 4, 9))] """ @@ -908,7 +908,7 @@ def date_sub(start, days): """ Returns the date that is `days` days before `start` - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['d']) >>> df.select(date_sub(df.d, 1).alias('d')).collect() [Row(d=datetime.date(2015, 4, 7))] """ @@ -921,7 +921,7 @@ def datediff(end, start): """ Returns the number of days from `start` to `end`. - >>> df = sqlContext.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2']) + >>> df = spark.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2']) >>> df.select(datediff(df.d2, df.d1).alias('diff')).collect() [Row(diff=32)] """ @@ -934,7 +934,7 @@ def add_months(start, months): """ Returns the date that is `months` months after `start` - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['d']) >>> df.select(add_months(df.d, 1).alias('d')).collect() [Row(d=datetime.date(2015, 5, 8))] """ @@ -947,7 +947,7 @@ def months_between(date1, date2): """ Returns the number of months between date1 and date2. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['t', 'd']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['t', 'd']) >>> df.select(months_between(df.t, df.d).alias('months')).collect() [Row(months=3.9495967...)] """ @@ -960,7 +960,7 @@ def to_date(col): """ Converts the column of StringType or TimestampType into DateType. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_date(df.t).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] """ @@ -975,7 +975,7 @@ def trunc(date, format): :param format: 'year', 'YYYY', 'yy' or 'month', 'mon', 'mm' - >>> df = sqlContext.createDataFrame([('1997-02-28',)], ['d']) + >>> df = spark.createDataFrame([('1997-02-28',)], ['d']) >>> df.select(trunc(df.d, 'year').alias('year')).collect() [Row(year=datetime.date(1997, 1, 1))] >>> df.select(trunc(df.d, 'mon').alias('month')).collect() @@ -993,7 +993,7 @@ def next_day(date, dayOfWeek): Day of the week parameter is case insensitive, and accepts: "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". - >>> df = sqlContext.createDataFrame([('2015-07-27',)], ['d']) + >>> df = spark.createDataFrame([('2015-07-27',)], ['d']) >>> df.select(next_day(df.d, 'Sun').alias('date')).collect() [Row(date=datetime.date(2015, 8, 2))] """ @@ -1006,7 +1006,7 @@ def last_day(date): """ Returns the last day of the month which the given date belongs to. - >>> df = sqlContext.createDataFrame([('1997-02-10',)], ['d']) + >>> df = spark.createDataFrame([('1997-02-10',)], ['d']) >>> df.select(last_day(df.d).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] """ @@ -1045,7 +1045,7 @@ def from_utc_timestamp(timestamp, tz): """ Assumes given timestamp is UTC and converts to given timezone. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(from_utc_timestamp(df.t, "PST").alias('t')).collect() [Row(t=datetime.datetime(1997, 2, 28, 2, 30))] """ @@ -1058,7 +1058,7 @@ def to_utc_timestamp(timestamp, tz): """ Assumes given timestamp is in given timezone and converts to UTC. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_utc_timestamp(df.t, "PST").alias('t')).collect() [Row(t=datetime.datetime(1997, 2, 28, 18, 30))] """ @@ -1087,7 +1087,7 @@ def window(timeColumn, windowDuration, slideDuration=None, startTime=None): The output column will be a struct called 'window' by default with the nested columns 'start' and 'end', where 'start' and 'end' will be of `TimestampType`. - >>> df = sqlContext.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val") + >>> df = spark.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val") >>> w = df.groupBy(window("date", "5 seconds")).agg(sum("val").alias("sum")) >>> w.select(w.window.start.cast("string").alias("start"), ... w.window.end.cast("string").alias("end"), "sum").collect() @@ -1124,7 +1124,7 @@ def crc32(col): Calculates the cyclic redundancy check value (CRC32) of a binary column and returns the value as a bigint. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(crc32('a').alias('crc32')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(crc32('a').alias('crc32')).collect() [Row(crc32=2743272264)] """ sc = SparkContext._active_spark_context @@ -1136,7 +1136,7 @@ def crc32(col): def md5(col): """Calculates the MD5 digest and returns the value as a 32 character hex string. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect() [Row(hash=u'902fbdd2b1df0c4f70b4a5d23525e932')] """ sc = SparkContext._active_spark_context @@ -1149,7 +1149,7 @@ def md5(col): def sha1(col): """Returns the hex string result of SHA-1. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect() [Row(hash=u'3c01bdbb26f358bab27f267924aa2c9a03fcfdb8')] """ sc = SparkContext._active_spark_context @@ -1179,7 +1179,7 @@ def sha2(col, numBits): def hash(*cols): """Calculates the hash code of given columns, and returns the result as a int column. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect() [Row(hash=-757602832)] """ sc = SparkContext._active_spark_context @@ -1215,7 +1215,7 @@ def concat(*cols): """ Concatenates multiple input string columns together into a single string column. - >>> df = sqlContext.createDataFrame([('abcd','123')], ['s', 'd']) + >>> df = spark.createDataFrame([('abcd','123')], ['s', 'd']) >>> df.select(concat(df.s, df.d).alias('s')).collect() [Row(s=u'abcd123')] """ @@ -1230,7 +1230,7 @@ def concat_ws(sep, *cols): Concatenates multiple input string columns together into a single string column, using the given separator. - >>> df = sqlContext.createDataFrame([('abcd','123')], ['s', 'd']) + >>> df = spark.createDataFrame([('abcd','123')], ['s', 'd']) >>> df.select(concat_ws('-', df.s, df.d).alias('s')).collect() [Row(s=u'abcd-123')] """ @@ -1268,7 +1268,7 @@ def format_number(col, d): :param col: the column name of the numeric value to be formatted :param d: the N decimal places - >>> sqlContext.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect() + >>> spark.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect() [Row(v=u'5.0000')] """ sc = SparkContext._active_spark_context @@ -1284,7 +1284,7 @@ def format_string(format, *cols): :param col: the column name of the numeric value to be formatted :param d: the N decimal places - >>> df = sqlContext.createDataFrame([(5, "hello")], ['a', 'b']) + >>> df = spark.createDataFrame([(5, "hello")], ['a', 'b']) >>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect() [Row(v=u'5 hello')] """ @@ -1301,7 +1301,7 @@ def instr(str, substr): NOTE: The position is not zero based, but 1 based index, returns 0 if substr could not be found in str. - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(instr(df.s, 'b').alias('s')).collect() [Row(s=2)] """ @@ -1317,7 +1317,7 @@ def substring(str, pos, len): returns the slice of byte array that starts at `pos` in byte and is of length `len` when str is Binary type - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(substring(df.s, 1, 2).alias('s')).collect() [Row(s=u'ab')] """ @@ -1334,7 +1334,7 @@ def substring_index(str, delim, count): returned. If count is negative, every to the right of the final delimiter (counting from the right) is returned. substring_index performs a case-sensitive match when searching for delim. - >>> df = sqlContext.createDataFrame([('a.b.c.d',)], ['s']) + >>> df = spark.createDataFrame([('a.b.c.d',)], ['s']) >>> df.select(substring_index(df.s, '.', 2).alias('s')).collect() [Row(s=u'a.b')] >>> df.select(substring_index(df.s, '.', -3).alias('s')).collect() @@ -1349,7 +1349,7 @@ def substring_index(str, delim, count): def levenshtein(left, right): """Computes the Levenshtein distance of the two given strings. - >>> df0 = sqlContext.createDataFrame([('kitten', 'sitting',)], ['l', 'r']) + >>> df0 = spark.createDataFrame([('kitten', 'sitting',)], ['l', 'r']) >>> df0.select(levenshtein('l', 'r').alias('d')).collect() [Row(d=3)] """ @@ -1370,7 +1370,7 @@ def locate(substr, str, pos=0): :param str: a Column of StringType :param pos: start position (zero based) - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(locate('b', df.s, 1).alias('s')).collect() [Row(s=2)] """ @@ -1384,7 +1384,7 @@ def lpad(col, len, pad): """ Left-pad the string column to width `len` with `pad`. - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(lpad(df.s, 6, '#').alias('s')).collect() [Row(s=u'##abcd')] """ @@ -1398,7 +1398,7 @@ def rpad(col, len, pad): """ Right-pad the string column to width `len` with `pad`. - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(rpad(df.s, 6, '#').alias('s')).collect() [Row(s=u'abcd##')] """ @@ -1412,7 +1412,7 @@ def repeat(col, n): """ Repeats a string column n times, and returns it as a new string column. - >>> df = sqlContext.createDataFrame([('ab',)], ['s',]) + >>> df = spark.createDataFrame([('ab',)], ['s',]) >>> df.select(repeat(df.s, 3).alias('s')).collect() [Row(s=u'ababab')] """ @@ -1428,7 +1428,7 @@ def split(str, pattern): NOTE: pattern is a string represent the regular expression. - >>> df = sqlContext.createDataFrame([('ab12cd',)], ['s',]) + >>> df = spark.createDataFrame([('ab12cd',)], ['s',]) >>> df.select(split(df.s, '[0-9]+').alias('s')).collect() [Row(s=[u'ab', u'cd'])] """ @@ -1441,7 +1441,7 @@ def split(str, pattern): def regexp_extract(str, pattern, idx): """Extract a specific(idx) group identified by a java regex, from the specified string column. - >>> df = sqlContext.createDataFrame([('100-200',)], ['str']) + >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_extract('str', '(\d+)-(\d+)', 1).alias('d')).collect() [Row(d=u'100')] """ @@ -1455,7 +1455,7 @@ def regexp_extract(str, pattern, idx): def regexp_replace(str, pattern, replacement): """Replace all substrings of the specified string value that match regexp with rep. - >>> df = sqlContext.createDataFrame([('100-200',)], ['str']) + >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_replace('str', '(\\d+)', '--').alias('d')).collect() [Row(d=u'-----')] """ @@ -1469,7 +1469,7 @@ def regexp_replace(str, pattern, replacement): def initcap(col): """Translate the first letter of each word to upper case in the sentence. - >>> sqlContext.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect() + >>> spark.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect() [Row(v=u'Ab Cd')] """ sc = SparkContext._active_spark_context @@ -1482,7 +1482,7 @@ def soundex(col): """ Returns the SoundEx encoding for a string - >>> df = sqlContext.createDataFrame([("Peters",),("Uhrbach",)], ['name']) + >>> df = spark.createDataFrame([("Peters",),("Uhrbach",)], ['name']) >>> df.select(soundex(df.name).alias("soundex")).collect() [Row(soundex=u'P362'), Row(soundex=u'U612')] """ @@ -1509,7 +1509,7 @@ def hex(col): """Computes hex value of the given column, which could be StringType, BinaryType, IntegerType or LongType. - >>> sqlContext.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() + >>> spark.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() [Row(hex(a)=u'414243', hex(b)=u'3')] """ sc = SparkContext._active_spark_context @@ -1523,7 +1523,7 @@ def unhex(col): """Inverse of hex. Interprets each pair of characters as a hexadecimal number and converts to the byte representation of number. - >>> sqlContext.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect() + >>> spark.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect() [Row(unhex(a)=bytearray(b'ABC'))] """ sc = SparkContext._active_spark_context @@ -1535,7 +1535,7 @@ def unhex(col): def length(col): """Calculates the length of a string or binary expression. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(length('a').alias('length')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(length('a').alias('length')).collect() [Row(length=3)] """ sc = SparkContext._active_spark_context @@ -1550,7 +1550,7 @@ def translate(srcCol, matching, replace): The translate will happen when any character in the string matching with the character in the `matching`. - >>> sqlContext.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123")\ + >>> spark.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123")\ .alias('r')).collect() [Row(r=u'1a2s3ae')] """ @@ -1608,7 +1608,7 @@ def array_contains(col, value): :param col: name of column containing array :param value: value to check for in array - >>> df = sqlContext.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) + >>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) >>> df.select(array_contains(df.data, "a")).collect() [Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)] """ @@ -1621,7 +1621,7 @@ def explode(col): """Returns a new row for each element in the given array or map. >>> from pyspark.sql import Row - >>> eDF = sqlContext.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})]) + >>> eDF = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})]) >>> eDF.select(explode(eDF.intlist).alias("anInt")).collect() [Row(anInt=1), Row(anInt=2), Row(anInt=3)] @@ -1648,7 +1648,7 @@ def get_json_object(col, path): :param path: path to the json object to extract >>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')] - >>> df = sqlContext.createDataFrame(data, ("key", "jstring")) + >>> df = spark.createDataFrame(data, ("key", "jstring")) >>> df.select(df.key, get_json_object(df.jstring, '$.f1').alias("c0"), \ get_json_object(df.jstring, '$.f2').alias("c1") ).collect() [Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)] @@ -1667,7 +1667,7 @@ def json_tuple(col, *fields): :param fields: list of fields to extract >>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')] - >>> df = sqlContext.createDataFrame(data, ("key", "jstring")) + >>> df = spark.createDataFrame(data, ("key", "jstring")) >>> df.select(df.key, json_tuple(df.jstring, 'f1', 'f2')).collect() [Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)] """ @@ -1683,7 +1683,7 @@ def size(col): :param col: name of column or expression - >>> df = sqlContext.createDataFrame([([1, 2, 3],),([1],),([],)], ['data']) + >>> df = spark.createDataFrame([([1, 2, 3],),([1],),([],)], ['data']) >>> df.select(size(df.data)).collect() [Row(size(data)=3), Row(size(data)=1), Row(size(data)=0)] """ @@ -1698,7 +1698,7 @@ def sort_array(col, asc=True): :param col: name of column or expression - >>> df = sqlContext.createDataFrame([([2, 1, 3],),([1],),([],)], ['data']) + >>> df = spark.createDataFrame([([2, 1, 3],),([1],),([],)], ['data']) >>> df.select(sort_array(df.data).alias('r')).collect() [Row(r=[1, 2, 3]), Row(r=[1]), Row(r=[])] >>> df.select(sort_array(df.data, asc=False).alias('r')).collect() @@ -1775,18 +1775,21 @@ def udf(f, returnType=StringType()): def _test(): import doctest - from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext + from pyspark.sql import Row, SparkSession import pyspark.sql.functions globs = pyspark.sql.functions.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.functions tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = SQLContext(sc) + globs['spark'] = spark globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.functions, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index ee734cb439287..6987af69cfb50 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -195,13 +195,15 @@ def pivot(self, pivot_col, values=None): def _test(): import doctest - from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext + from pyspark.sql import Row, SparkSession import pyspark.sql.group globs = pyspark.sql.group.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.group tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) @@ -216,7 +218,7 @@ def _test(): (failure_count, test_count) = doctest.testmod( pyspark.sql.group, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) From d207716451f722c899b3845ee454f1e16c094125 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 23 May 2016 21:07:14 -0700 Subject: [PATCH 105/143] [SPARK-15485][SQL][DOCS] Spark SQL Configuration #### What changes were proposed in this pull request? So far, the page Configuration in the official documentation does not have a section for Spark SQL. http://spark.apache.org/docs/latest/configuration.html For Spark users, the information and default values of these public configuration parameters are very useful. This PR is to add this missing section to the configuration.html. rxin yhuai marmbrus #### How was this patch tested? Below is the generated webpage. screenshot 2016-05-23 11 35 57 screenshot 2016-05-23 11 37 38 screenshot 2016-05-23 11 36 11 screenshot 2016-05-23 11 36 18 Author: gatorsmile Closes #13263 from gatorsmile/configurationSQL. --- docs/configuration.md | 42 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index d23f0fe1a1c46..d6471a8cc7873 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1480,6 +1480,48 @@ Apart from these, the following properties are also available, and may be useful +#### Spark SQL +Running the SET -v command will show the entire list of the SQL configuration. + +
+
+ +{% highlight scala %} +// spark is an existing SparkSession +spark.sql("SET -v").show(numRows = 200, truncate = false) +{% endhighlight %} + +
+ +
+ +{% highlight java %} +// spark is an existing SparkSession +spark.sql("SET -v").show(200, false); +{% endhighlight %} +
+ +
+ +{% highlight python %} +# spark is an existing SparkSession +spark.sql("SET -v").show(n=200, truncate=False) +{% endhighlight %} + +
+ +
+ +{% highlight r %} +# sqlContext is an existing sqlContext. +properties <- sql(sqlContext, "SET -v") +showDF(properties, numRows = 200, truncate = FALSE) +{% endhighlight %} + +
+
+ + #### Spark Streaming From fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 23 May 2016 21:12:34 -0700 Subject: [PATCH 106/143] [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB ## What changes were proposed in this pull request? This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method. ## How was this patch tested? Added new tests Author: Kazuaki Ishizaki Closes #13243 from kiszk/SPARK-15285. --- .../expressions/objects/objects.scala | 32 +++++++++++++---- .../spark/sql/DataFrameComplexTypeSuite.scala | 35 +++++++++++++++++++ 2 files changed, 61 insertions(+), 6 deletions(-) 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 2f2323fa3a25f..cc094f2a96a8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -232,27 +232,47 @@ case class NewInstance( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) - val argGen = arguments.map(_.genCode(ctx)) - val argString = argGen.map(_.value).mkString(", ") + val argIsNulls = ctx.freshName("argIsNulls") + ctx.addMutableState("boolean[]", argIsNulls, + s"$argIsNulls = new boolean[${arguments.size}];") + val argValues = arguments.zipWithIndex.map { case (e, i) => + val argValue = ctx.freshName("argValue") + ctx.addMutableState(ctx.javaType(e.dataType), argValue, "") + argValue + } + + val argCodes = arguments.zipWithIndex.map { case (e, i) => + val expr = e.genCode(ctx) + expr.code + s""" + $argIsNulls[$i] = ${expr.isNull}; + ${argValues(i)} = ${expr.value}; + """ + } + val argCode = ctx.splitExpressions(ctx.INPUT_ROW, argCodes) val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) var isNull = ev.isNull val setIsNull = if (propagateNull && arguments.nonEmpty) { - s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" + s""" + boolean $isNull = false; + for (int idx = 0; idx < ${arguments.length}; idx++) { + if ($argIsNulls[idx]) { $isNull = true; break; } + } + """ } else { isNull = "false" "" } val constructorCall = outer.map { gen => - s"""${gen.value}.new ${cls.getSimpleName}($argString)""" + s"""${gen.value}.new ${cls.getSimpleName}(${argValues.mkString(", ")})""" }.getOrElse { - s"new $className($argString)" + s"new $className(${argValues.mkString(", ")})" } val code = s""" - ${argGen.map(_.code).mkString("\n")} + $argCode ${outer.map(_.code).getOrElse("")} $setIsNull final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 72f676e6225ee..07fbaba0f819c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -58,4 +58,39 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { val nullIntRow = df.selectExpr("i[1]").collect()(0) assert(nullIntRow == org.apache.spark.sql.Row(null)) } + + test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { + val ds100_5 = Seq(S100_5()).toDS() + ds100_5.rdd.count + } } + +case class S100( + s1: String = "1", s2: String = "2", s3: String = "3", s4: String = "4", + s5: String = "5", s6: String = "6", s7: String = "7", s8: String = "8", + s9: String = "9", s10: String = "10", s11: String = "11", s12: String = "12", + s13: String = "13", s14: String = "14", s15: String = "15", s16: String = "16", + s17: String = "17", s18: String = "18", s19: String = "19", s20: String = "20", + s21: String = "21", s22: String = "22", s23: String = "23", s24: String = "24", + s25: String = "25", s26: String = "26", s27: String = "27", s28: String = "28", + s29: String = "29", s30: String = "30", s31: String = "31", s32: String = "32", + s33: String = "33", s34: String = "34", s35: String = "35", s36: String = "36", + s37: String = "37", s38: String = "38", s39: String = "39", s40: String = "40", + s41: String = "41", s42: String = "42", s43: String = "43", s44: String = "44", + s45: String = "45", s46: String = "46", s47: String = "47", s48: String = "48", + s49: String = "49", s50: String = "50", s51: String = "51", s52: String = "52", + s53: String = "53", s54: String = "54", s55: String = "55", s56: String = "56", + s57: String = "57", s58: String = "58", s59: String = "59", s60: String = "60", + s61: String = "61", s62: String = "62", s63: String = "63", s64: String = "64", + s65: String = "65", s66: String = "66", s67: String = "67", s68: String = "68", + s69: String = "69", s70: String = "70", s71: String = "71", s72: String = "72", + s73: String = "73", s74: String = "74", s75: String = "75", s76: String = "76", + s77: String = "77", s78: String = "78", s79: String = "79", s80: String = "80", + s81: String = "81", s82: String = "82", s83: String = "83", s84: String = "84", + s85: String = "85", s86: String = "86", s87: String = "87", s88: String = "88", + s89: String = "89", s90: String = "90", s91: String = "91", s92: String = "92", + s93: String = "93", s94: String = "94", s95: String = "95", s96: String = "96", + s97: String = "97", s98: String = "98", s99: String = "99", s100: String = "100") + +case class S100_5( + s1: S100 = S100(), s2: S100 = S100(), s3: S100 = S100(), s4: S100 = S100(), s5: S100 = S100()) From de726b0d533158d3ca08841bd6976bcfa26ca79d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 23 May 2016 21:43:11 -0700 Subject: [PATCH 107/143] Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB" This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf. --- .../expressions/objects/objects.scala | 32 ++++------------- .../spark/sql/DataFrameComplexTypeSuite.scala | 35 ------------------- 2 files changed, 6 insertions(+), 61 deletions(-) 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 cc094f2a96a8a..2f2323fa3a25f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -232,47 +232,27 @@ case class NewInstance( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) - val argIsNulls = ctx.freshName("argIsNulls") - ctx.addMutableState("boolean[]", argIsNulls, - s"$argIsNulls = new boolean[${arguments.size}];") - val argValues = arguments.zipWithIndex.map { case (e, i) => - val argValue = ctx.freshName("argValue") - ctx.addMutableState(ctx.javaType(e.dataType), argValue, "") - argValue - } - - val argCodes = arguments.zipWithIndex.map { case (e, i) => - val expr = e.genCode(ctx) - expr.code + s""" - $argIsNulls[$i] = ${expr.isNull}; - ${argValues(i)} = ${expr.value}; - """ - } - val argCode = ctx.splitExpressions(ctx.INPUT_ROW, argCodes) + val argGen = arguments.map(_.genCode(ctx)) + val argString = argGen.map(_.value).mkString(", ") val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) var isNull = ev.isNull val setIsNull = if (propagateNull && arguments.nonEmpty) { - s""" - boolean $isNull = false; - for (int idx = 0; idx < ${arguments.length}; idx++) { - if ($argIsNulls[idx]) { $isNull = true; break; } - } - """ + s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" } else { isNull = "false" "" } val constructorCall = outer.map { gen => - s"""${gen.value}.new ${cls.getSimpleName}(${argValues.mkString(", ")})""" + s"""${gen.value}.new ${cls.getSimpleName}($argString)""" }.getOrElse { - s"new $className(${argValues.mkString(", ")})" + s"new $className($argString)" } val code = s""" - $argCode + ${argGen.map(_.code).mkString("\n")} ${outer.map(_.code).getOrElse("")} $setIsNull final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 07fbaba0f819c..72f676e6225ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -58,39 +58,4 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { val nullIntRow = df.selectExpr("i[1]").collect()(0) assert(nullIntRow == org.apache.spark.sql.Row(null)) } - - test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { - val ds100_5 = Seq(S100_5()).toDS() - ds100_5.rdd.count - } } - -case class S100( - s1: String = "1", s2: String = "2", s3: String = "3", s4: String = "4", - s5: String = "5", s6: String = "6", s7: String = "7", s8: String = "8", - s9: String = "9", s10: String = "10", s11: String = "11", s12: String = "12", - s13: String = "13", s14: String = "14", s15: String = "15", s16: String = "16", - s17: String = "17", s18: String = "18", s19: String = "19", s20: String = "20", - s21: String = "21", s22: String = "22", s23: String = "23", s24: String = "24", - s25: String = "25", s26: String = "26", s27: String = "27", s28: String = "28", - s29: String = "29", s30: String = "30", s31: String = "31", s32: String = "32", - s33: String = "33", s34: String = "34", s35: String = "35", s36: String = "36", - s37: String = "37", s38: String = "38", s39: String = "39", s40: String = "40", - s41: String = "41", s42: String = "42", s43: String = "43", s44: String = "44", - s45: String = "45", s46: String = "46", s47: String = "47", s48: String = "48", - s49: String = "49", s50: String = "50", s51: String = "51", s52: String = "52", - s53: String = "53", s54: String = "54", s55: String = "55", s56: String = "56", - s57: String = "57", s58: String = "58", s59: String = "59", s60: String = "60", - s61: String = "61", s62: String = "62", s63: String = "63", s64: String = "64", - s65: String = "65", s66: String = "66", s67: String = "67", s68: String = "68", - s69: String = "69", s70: String = "70", s71: String = "71", s72: String = "72", - s73: String = "73", s74: String = "74", s75: String = "75", s76: String = "76", - s77: String = "77", s78: String = "78", s79: String = "79", s80: String = "80", - s81: String = "81", s82: String = "82", s83: String = "83", s84: String = "84", - s85: String = "85", s86: String = "86", s87: String = "87", s88: String = "88", - s89: String = "89", s90: String = "90", s91: String = "91", s92: String = "92", - s93: String = "93", s94: String = "94", s95: String = "95", s96: String = "96", - s97: String = "97", s98: String = "98", s99: String = "99", s100: String = "100") - -case class S100_5( - s1: S100 = S100(), s2: S100 = S100(), s3: S100 = S100(), s4: S100 = S100(), s5: S100 = S100()) From d642b273544bb77ef7f584326aa2d214649ac61b Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 23 May 2016 23:29:15 -0700 Subject: [PATCH 108/143] [SPARK-15397][SQL] fix string udf locate as hive ## What changes were proposed in this pull request? in hive, `locate("aa", "aaa", 0)` would yield 0, `locate("aa", "aaa", 1)` would yield 1 and `locate("aa", "aaa", 2)` would yield 2, while in Spark, `locate("aa", "aaa", 0)` would yield 1, `locate("aa", "aaa", 1)` would yield 2 and `locate("aa", "aaa", 2)` would yield 0. This results from the different understanding of the third parameter in udf `locate`. It means the starting index and starts from 1, so when we use 0, the return would always be 0. ## How was this patch tested? tested with modified `StringExpressionsSuite` and `StringFunctionsSuite` Author: Daoyuan Wang Closes #13186 from adrian-wang/locate. --- R/pkg/R/functions.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- python/pyspark/sql/functions.py | 2 +- .../expressions/stringExpressions.scala | 19 +++++++++++++------ .../expressions/StringExpressionsSuite.scala | 16 +++++++++------- .../spark/sql/StringFunctionsSuite.scala | 10 +++++----- 6 files changed, 30 insertions(+), 21 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 4a0bdf331523e..2665d1d477802 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2226,7 +2226,7 @@ setMethod("window", signature(x = "Column"), #' @export #' @examples \dontrun{locate('b', df$c, 1)} setMethod("locate", signature(substr = "character", str = "Column"), - function(substr, str, pos = 0) { + function(substr, str, pos = 1) { jc <- callJStatic("org.apache.spark.sql.functions", "locate", substr, str@jc, as.integer(pos)) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 6a99b43e5aa59..b2d769f2acd0c 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1152,7 +1152,7 @@ test_that("string operators", { l2 <- list(list(a = "aaads")) df2 <- createDataFrame(sqlContext, l2) expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) - expect_equal(collect(select(df2, locate("aa", df2$a, 1)))[1, 1], 2) + expect_equal(collect(select(df2, locate("aa", df2$a, 2)))[1, 1], 2) expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") # nolint expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") # nolint diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 1f15eec645382..64b8bc442dad0 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1359,7 +1359,7 @@ def levenshtein(left, right): @since(1.5) -def locate(substr, str, pos=0): +def locate(substr, str, pos=1): """ Locate the position of the first occurrence of substr in a string column, after position pos. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 78e846d3f580e..44ff7fda8ef49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -494,7 +494,7 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) extends TernaryExpression with ImplicitCastInputTypes { def this(substr: Expression, str: Expression) = { - this(substr, str, Literal(0)) + this(substr, str, Literal(1)) } override def children: Seq[Expression] = substr :: str :: start :: Nil @@ -516,9 +516,14 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) if (l == null) { null } else { - l.asInstanceOf[UTF8String].indexOf( - r.asInstanceOf[UTF8String], - s.asInstanceOf[Int]) + 1 + val sVal = s.asInstanceOf[Int] + if (sVal < 1) { + 0 + } else { + l.asInstanceOf[UTF8String].indexOf( + r.asInstanceOf[UTF8String], + s.asInstanceOf[Int] - 1) + 1 + } } } } @@ -537,8 +542,10 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) if (!${substrGen.isNull}) { ${strGen.code} if (!${strGen.isNull}) { - ${ev.value} = ${strGen.value}.indexOf(${substrGen.value}, - ${startGen.value}) + 1; + if (${startGen.value} > 0) { + ${ev.value} = ${strGen.value}.indexOf(${substrGen.value}, + ${startGen.value} - 1) + 1; + } } else { ${ev.isNull} = true; } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index c09c64fd6bf62..29bf15bf524b9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -508,16 +508,18 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val s2 = 'b.string.at(1) val s3 = 'c.string.at(2) val s4 = 'd.int.at(3) - val row1 = create_row("aaads", "aa", "zz", 1) - val row2 = create_row(null, "aa", "zz", 0) - val row3 = create_row("aaads", null, "zz", 0) - val row4 = create_row(null, null, null, 0) + val row1 = create_row("aaads", "aa", "zz", 2) + val row2 = create_row(null, "aa", "zz", 1) + val row3 = create_row("aaads", null, "zz", 1) + val row4 = create_row(null, null, null, 1) checkEvaluation(new StringLocate(Literal("aa"), Literal("aaads")), 1, row1) - checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(1)), 2, row1) - checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(2)), 0, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(0)), 0, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(1)), 1, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(2)), 2, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(3)), 0, row1) checkEvaluation(new StringLocate(Literal("de"), Literal("aaads")), 0, row1) - checkEvaluation(StringLocate(Literal("de"), Literal("aaads"), 1), 0, row1) + checkEvaluation(StringLocate(Literal("de"), Literal("aaads"), 2), 0, row1) checkEvaluation(new StringLocate(s2, s1), 1, row1) checkEvaluation(StringLocate(s2, s1, s4), 2, row1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index c7b95c2683d81..1de2d9b5adab1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -189,15 +189,15 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { } test("string locate function") { - val df = Seq(("aaads", "aa", "zz", 1)).toDF("a", "b", "c", "d") + val df = Seq(("aaads", "aa", "zz", 2)).toDF("a", "b", "c", "d") checkAnswer( - df.select(locate("aa", $"a"), locate("aa", $"a", 1)), - Row(1, 2)) + df.select(locate("aa", $"a"), locate("aa", $"a", 2), locate("aa", $"a", 0)), + Row(1, 2, 0)) checkAnswer( - df.selectExpr("locate(b, a)", "locate(b, a, d)"), - Row(1, 2)) + df.selectExpr("locate(b, a)", "locate(b, a, d)", "locate(b, a, 3)"), + Row(1, 2, 0)) } test("string padding functions") { From 6075f5b4d8e98483d26c31576f58e2229024b4f4 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 24 May 2016 10:02:10 +0200 Subject: [PATCH 109/143] [SPARK-15442][ML][PYSPARK] Add 'relativeError' param to PySpark QuantileDiscretizer This PR adds the `relativeError` param to PySpark's `QuantileDiscretizer` to match Scala. Also cleaned up a duplication of `numBuckets` where the param is both a class and instance attribute (I removed the instance attr to match the style of params throughout `ml`). Finally, cleaned up the docs for `QuantileDiscretizer` to reflect that it now uses `approxQuantile`. ## How was this patch tested? A little doctest and built API docs locally to check HTML doc generation. Author: Nick Pentreath Closes #13228 from MLnick/SPARK-15442-py-relerror-param. --- .../ml/feature/QuantileDiscretizer.scala | 13 +++-- python/pyspark/ml/feature.py | 51 +++++++++++++------ 2 files changed, 44 insertions(+), 20 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 5a6daa06ef546..61483590cde38 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -50,13 +50,13 @@ private[feature] trait QuantileDiscretizerBase extends Params /** * Relative error (see documentation for * [[org.apache.spark.sql.DataFrameStatFunctions.approxQuantile approxQuantile]] for description) - * Must be a number in [0, 1]. + * Must be in the range [0, 1]. * default: 0.001 * @group param */ val relativeError = new DoubleParam(this, "relativeError", "The relative target precision " + - "for approxQuantile", - ParamValidators.inRange(0.0, 1.0)) + "for the approximate quantile algorithm used to generate buckets. " + + "Must be in the range [0, 1].", ParamValidators.inRange(0.0, 1.0)) setDefault(relativeError -> 0.001) /** @group getParam */ @@ -66,8 +66,11 @@ private[feature] trait QuantileDiscretizerBase extends Params /** * :: Experimental :: * `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned - * categorical features. The bin ranges are chosen by taking a sample of the data and dividing it - * into roughly equal parts. The lower and upper bin bounds will be -Infinity and +Infinity, + * categorical features. The number of bins can be set using the `numBuckets` parameter. + * The bin ranges are chosen using an approximate algorithm (see the documentation for + * [[org.apache.spark.sql.DataFrameStatFunctions.approxQuantile approxQuantile]] + * for a detailed description). The precision of the approximation can be controlled with the + * `relativeError` parameter. The lower and upper bin bounds will be `-Infinity` and `+Infinity`, * covering all real values. */ @Experimental diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 93745c70c4fe6..eb555cb940931 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1177,16 +1177,20 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav .. note:: Experimental `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned - categorical features. The bin ranges are chosen by taking a sample of the data and dividing it - into roughly equal parts. The lower and upper bin bounds will be -Infinity and +Infinity, - covering all real values. This attempts to find numBuckets partitions based on a sample of data, - but it may find fewer depending on the data sample values. + categorical features. The number of bins can be set using the :py:attr:`numBuckets` parameter. + The bin ranges are chosen using an approximate algorithm (see the documentation for + :py:meth:`~.DataFrameStatFunctions.approxQuantile` for a detailed description). + The precision of the approximation can be controlled with the + :py:attr:`relativeError` parameter. + The lower and upper bin bounds will be `-Infinity` and `+Infinity`, covering all real values. >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) >>> qds = QuantileDiscretizer(numBuckets=2, - ... inputCol="values", outputCol="buckets", seed=123) + ... inputCol="values", outputCol="buckets", seed=123, relativeError=0.01) >>> qds.getSeed() 123 + >>> qds.getRelativeError() + 0.01 >>> bucketizer = qds.fit(df) >>> splits = bucketizer.getSplits() >>> splits[0] @@ -1205,32 +1209,35 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav .. versionadded:: 2.0.0 """ - # a placeholder to make it appear in the generated doc numBuckets = Param(Params._dummy(), "numBuckets", "Maximum number of buckets (quantiles, or " + - "categories) into which data points are grouped. Must be >= 2. Default 2.", + "categories) into which data points are grouped. Must be >= 2.", typeConverter=TypeConverters.toInt) + relativeError = Param(Params._dummy(), "relativeError", "The relative target precision for " + + "the approximate quantile algorithm used to generate buckets. " + + "Must be in the range [0, 1].", + typeConverter=TypeConverters.toFloat) + @keyword_only - def __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None): + def __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, relativeError=0.001): """ - __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None) + __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, relativeError=0.001) """ super(QuantileDiscretizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.QuantileDiscretizer", self.uid) - self.numBuckets = Param(self, "numBuckets", - "Maximum number of buckets (quantiles, or " + - "categories) into which data points are grouped. Must be >= 2.") - self._setDefault(numBuckets=2) + self._setDefault(numBuckets=2, relativeError=0.001) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("2.0.0") - def setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None): + def setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, + relativeError=0.001): """ - setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None) + setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, \ + relativeError=0.001) Set the params for the QuantileDiscretizer """ kwargs = self.setParams._input_kwargs @@ -1250,6 +1257,20 @@ def getNumBuckets(self): """ return self.getOrDefault(self.numBuckets) + @since("2.0.0") + def setRelativeError(self, value): + """ + Sets the value of :py:attr:`relativeError`. + """ + return self._set(relativeError=value) + + @since("2.0.0") + def getRelativeError(self): + """ + Gets the value of relativeError or its default value. + """ + return self.getOrDefault(self.relativeError) + def _create_model(self, java_model): """ Private method to convert the java_model to a Python model. From c24b6b679c3efa053f7de19be73eb36dc70d9930 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 24 May 2016 09:43:39 -0700 Subject: [PATCH 110/143] [SPARK-11753][SQL][TEST-HADOOP2.2] Make allowNonNumericNumbers option work ## What changes were proposed in this pull request? Jackson suppprts `allowNonNumericNumbers` option to parse non-standard non-numeric numbers such as "NaN", "Infinity", "INF". Currently used Jackson version (2.5.3) doesn't support it all. This patch upgrades the library and make the two ignored tests in `JsonParsingOptionsSuite` passed. ## How was this patch tested? `JsonParsingOptionsSuite`. Author: Liang-Chi Hsieh Author: Liang-Chi Hsieh Closes #9759 from viirya/fix-json-nonnumric. --- dev/deps/spark-deps-hadoop-2.2 | 11 ++-- dev/deps/spark-deps-hadoop-2.3 | 11 ++-- dev/deps/spark-deps-hadoop-2.4 | 11 ++-- dev/deps/spark-deps-hadoop-2.6 | 11 ++-- dev/deps/spark-deps-hadoop-2.7 | 11 ++-- pom.xml | 8 ++- python/pyspark/sql/readwriter.py | 3 + .../apache/spark/sql/DataFrameReader.scala | 2 + .../datasources/json/JacksonParser.scala | 28 +++++---- .../json/JsonParsingOptionsSuite.scala | 59 ++++++++++++++----- 10 files changed, 102 insertions(+), 53 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 91b333fcae4fe..a9068da8b2096 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -72,12 +72,13 @@ hk2-utils-2.4.0-b34.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 +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar janino-2.7.8.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar @@ -127,7 +128,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 61ed4c0889b8e..7e60a313ae8fe 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -74,12 +74,13 @@ hk2-utils-2.4.0-b34.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 +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar @@ -134,7 +135,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index fb014921765f9..70d33b4f4812c 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -74,12 +74,13 @@ hk2-utils-2.4.0-b34.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 +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar @@ -134,7 +135,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 0baf4e84fff0a..a80f6bc2a4061 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -80,13 +80,14 @@ htrace-core-3.0.4.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 +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.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-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar @@ -142,7 +143,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8be218cd68d91..c0b53f73cd499 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -80,13 +80,14 @@ htrace-core-3.1.0-incubating.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 +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.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-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar @@ -143,7 +144,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/pom.xml b/pom.xml index e778f77fa3151..f28aa14adc55f 100644 --- a/pom.xml +++ b/pom.xml @@ -160,7 +160,7 @@ ${scala.version} org.scala-lang 1.9.13 - 2.5.3 + 2.7.3 1.1.2.4 1.1.2 1.2.0-incubating @@ -180,6 +180,7 @@ 4.5.2-1 1.1 2.52.0 + 2.8 ${java.home} @@ -1825,6 +1826,11 @@ antlr4-runtime ${antlr4.version} + + com.thoughtworks.paranamer + paranamer + ${paranamer.version} + diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 855c9d666f0bc..6f788cf50c7d7 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -193,6 +193,9 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, 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 allowNonNumericNumbers: allows using non-numeric numbers such as "NaN", "Infinity", + "-Infinity", "INF", "-INF", which are convertd to floating + point numbers, ``true``. :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character using backslash quoting mechanism. If None is set, it uses the default value, ``false``. 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 57a2091fe8c77..0fed9171a8bf3 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 @@ -293,6 +293,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * *
  • `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers * (e.g. 00012)
  • + *
  • `allowNonNumericNumbers` (default `true`): allows using non-numeric numbers such as "NaN", + * "Infinity", "-Infinity", "INF", "-INF", which are convertd to floating point numbers.
  • *
  • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala index aeee2600a19ee..cafca323187d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala @@ -129,13 +129,15 @@ object JacksonParser extends Logging { case (VALUE_STRING, FloatType) => // Special case handling for NaN and Infinity. val value = parser.getText - val lowerCaseValue = value.toLowerCase() - if (lowerCaseValue.equals("nan") || - lowerCaseValue.equals("infinity") || - lowerCaseValue.equals("-infinity") || - lowerCaseValue.equals("inf") || - lowerCaseValue.equals("-inf")) { + if (value.equals("NaN") || + value.equals("Infinity") || + value.equals("+Infinity") || + value.equals("-Infinity")) { value.toFloat + } else if (value.equals("+INF") || value.equals("INF")) { + Float.PositiveInfinity + } else if (value.equals("-INF")) { + Float.NegativeInfinity } else { throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.") } @@ -146,13 +148,15 @@ object JacksonParser extends Logging { case (VALUE_STRING, DoubleType) => // Special case handling for NaN and Infinity. val value = parser.getText - val lowerCaseValue = value.toLowerCase() - if (lowerCaseValue.equals("nan") || - lowerCaseValue.equals("infinity") || - lowerCaseValue.equals("-infinity") || - lowerCaseValue.equals("inf") || - lowerCaseValue.equals("-inf")) { + if (value.equals("NaN") || + value.equals("Infinity") || + value.equals("+Infinity") || + value.equals("-Infinity")) { value.toDouble + } else if (value.equals("+INF") || value.equals("INF")) { + Double.PositiveInfinity + } else if (value.equals("-INF")) { + Double.NegativeInfinity } else { throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index c31dffedbdf67..2aab955c1ecbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.json import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} /** * Test cases for various [[JSONOptions]]. @@ -93,23 +94,51 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSQLContext { assert(df.first().getLong(0) == 18) } - // The following two tests are not really working - need to look into Jackson's - // JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS. - ignore("allowNonNumericNumbers off") { - val str = """{"age": NaN}""" - val rdd = spark.sparkContext.parallelize(Seq(str)) - val df = spark.read.json(rdd) - - assert(df.schema.head.name == "_corrupt_record") + test("allowNonNumericNumbers off") { + // non-quoted non-numeric numbers don't work if allowNonNumericNumbers is off. + var testCases: Seq[String] = Seq("""{"age": NaN}""", """{"age": Infinity}""", + """{"age": +Infinity}""", """{"age": -Infinity}""", """{"age": INF}""", + """{"age": +INF}""", """{"age": -INF}""") + testCases.foreach { str => + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.option("allowNonNumericNumbers", "false").json(rdd) + + assert(df.schema.head.name == "_corrupt_record") + } + + // quoted non-numeric numbers should still work even allowNonNumericNumbers is off. + testCases = Seq("""{"age": "NaN"}""", """{"age": "Infinity"}""", """{"age": "+Infinity"}""", + """{"age": "-Infinity"}""", """{"age": "INF"}""", """{"age": "+INF"}""", + """{"age": "-INF"}""") + val tests: Seq[Double => Boolean] = Seq(_.isNaN, _.isPosInfinity, _.isPosInfinity, + _.isNegInfinity, _.isPosInfinity, _.isPosInfinity, _.isNegInfinity) + val schema = StructType(StructField("age", DoubleType, true) :: Nil) + + testCases.zipWithIndex.foreach { case (str, idx) => + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.option("allowNonNumericNumbers", "false").schema(schema).json(rdd) + + assert(df.schema.head.name == "age") + assert(tests(idx)(df.first().getDouble(0))) + } } - ignore("allowNonNumericNumbers on") { - val str = """{"age": NaN}""" - val rdd = spark.sparkContext.parallelize(Seq(str)) - val df = spark.read.option("allowNonNumericNumbers", "true").json(rdd) - - assert(df.schema.head.name == "age") - assert(df.first().getDouble(0).isNaN) + test("allowNonNumericNumbers on") { + val testCases: Seq[String] = Seq("""{"age": NaN}""", """{"age": Infinity}""", + """{"age": +Infinity}""", """{"age": -Infinity}""", """{"age": +INF}""", + """{"age": -INF}""", """{"age": "NaN"}""", """{"age": "Infinity"}""", + """{"age": "-Infinity"}""") + val tests: Seq[Double => Boolean] = Seq(_.isNaN, _.isPosInfinity, _.isPosInfinity, + _.isNegInfinity, _.isPosInfinity, _.isNegInfinity, _.isNaN, _.isPosInfinity, + _.isNegInfinity, _.isPosInfinity, _.isNegInfinity) + val schema = StructType(StructField("age", DoubleType, true) :: Nil) + testCases.zipWithIndex.foreach { case (str, idx) => + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.option("allowNonNumericNumbers", "true").schema(schema).json(rdd) + + assert(df.schema.head.name == "age") + assert(tests(idx)(df.first().getDouble(0))) + } } test("allowBackslashEscapingAnyCharacter off") { From f8763b80ecd9968566018396c8cdc1851e7f8a46 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 24 May 2016 10:08:14 -0700 Subject: [PATCH 111/143] [SPARK-13135] [SQL] Don't print expressions recursively in generated code ## What changes were proposed in this pull request? This PR is an up-to-date and a little bit improved version of #11019 of rxin for - (1) preventing recursive printing of expressions in generated code. Since the major function of this PR is indeed the above, he should be credited for the work he did. In addition to #11019, this PR improves the followings in code generation. - (2) Improve multiline comment indentation. - (3) Reduce the number of empty lines (mainly consecutive empty lines). - (4) Remove all space characters on empty lines. **Example** ```scala spark.range(1, 1000).select('id+1+2+3, 'id+4+5+6) ``` **Before** ``` Generated code: /* 001 */ public Object generate(Object[] references) { ... /* 005 */ /** /* 006 */ * Codegend pipeline for /* 007 */ * Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 008 */ * +- Range 1, 1, 8, 999, [id#0L] /* 009 */ */ ... /* 075 */ // PRODUCE: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 076 */ /* 077 */ // PRODUCE: Range 1, 1, 8, 999, [id#0L] /* 078 */ /* 079 */ // initialize Range ... /* 092 */ // CONSUME: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 093 */ /* 094 */ // CONSUME: WholeStageCodegen /* 095 */ /* 096 */ // (((input[0, bigint, false] + 1) + 2) + 3) /* 097 */ // ((input[0, bigint, false] + 1) + 2) /* 098 */ // (input[0, bigint, false] + 1) ... /* 107 */ // (((input[0, bigint, false] + 4) + 5) + 6) /* 108 */ // ((input[0, bigint, false] + 4) + 5) /* 109 */ // (input[0, bigint, false] + 4) ... /* 126 */ } ``` **After** ``` Generated code: /* 001 */ public Object generate(Object[] references) { ... /* 005 */ /** /* 006 */ * Codegend pipeline for /* 007 */ * Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 008 */ * +- Range 1, 1, 8, 999, [id#0L] /* 009 */ */ ... /* 075 */ // PRODUCE: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 076 */ // PRODUCE: Range 1, 1, 8, 999, [id#0L] /* 077 */ // initialize Range ... /* 090 */ // CONSUME: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 091 */ // CONSUME: WholeStageCodegen /* 092 */ // (((input[0, bigint, false] + 1) + 2) + 3) ... /* 101 */ // (((input[0, bigint, false] + 4) + 5) + 6) ... /* 118 */ } ``` ## How was this patch tested? Pass the Jenkins tests and see the result of the following command manually. ```scala scala> spark.range(1, 1000).select('id+1+2+3, 'id+4+5+6).queryExecution.debug.codegen() ``` Author: Dongjoon Hyun Author: Reynold Xin Author: Dongjoon Hyun Closes #13192 from dongjoon-hyun/SPARK-13135. --- .../expressions/codegen/CodeFormatter.scala | 27 ++++++++++-- .../codegen/GenerateMutableProjection.scala | 3 +- .../codegen/GenerateOrdering.scala | 3 +- .../codegen/GeneratePredicate.scala | 3 +- .../codegen/GenerateSafeProjection.scala | 3 +- .../codegen/GenerateUnsafeProjection.scala | 3 +- .../codegen/GenerateUnsafeRowJoiner.scala | 2 +- .../codegen/CodeFormatterSuite.scala | 43 +++++++++++++++++++ .../sql/execution/WholeStageCodegenExec.scala | 4 +- .../columnar/GenerateColumnAccessor.scala | 3 +- 10 files changed, 82 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index c7410925da6c8..855ae6432da28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -40,7 +40,7 @@ object CodeFormatter { var lastLine: String = "dummy" input.split('\n').foreach { l => val line = l.trim() - val skip = line == "" && (lastLine == "" || lastLine.endsWith("{")) + val skip = line == "" && (lastLine == "" || lastLine.endsWith("{") || lastLine.endsWith("*/")) if (!skip) { code.append(line) code.append("\n") @@ -49,6 +49,24 @@ object CodeFormatter { } code.result() } + + def stripOverlappingComments(codeAndComment: CodeAndComment): CodeAndComment = { + val code = new StringBuilder + val map = codeAndComment.comment + var lastLine: String = "dummy" + codeAndComment.body.split('\n').foreach { l => + val line = l.trim() + val skip = lastLine.startsWith("/*") && lastLine.endsWith("*/") && + line.startsWith("/*") && line.endsWith("*/") && + map(lastLine).substring(3).contains(map(line).substring(3)) + if (!skip) { + code.append(line) + code.append("\n") + } + lastLine = line + } + new CodeAndComment(code.result().trim(), map) + } } private class CodeFormatter { @@ -100,8 +118,11 @@ private class CodeFormatter { indentString } code.append(f"/* ${currentLine}%03d */ ") - code.append(thisLineIndent) - code.append(line) + if (line.trim().length > 0) { + code.append(thisLineIndent) + if (inCommentBlock && line.startsWith("*") || line.startsWith("*/")) code.append(" ") + code.append(line) + } code.append("\n") indentLevel = newIndentLevel indentString = " " * (indentSize * newIndentLevel) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 1305289e783c6..0f82d2e613c73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -133,7 +133,8 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP } """ - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) 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 1c53d62a5e98a..c10829d4f14f3 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,8 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR } }""" - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index ef44e6b46b798..106bb27964cab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -61,7 +61,8 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool } }""" - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") val p = CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index 214dc40641f85..b891f94673752 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -181,7 +181,8 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] } """ - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 102f276e9b3ff..5efba4b3a6087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -390,7 +390,8 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } """ - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index 4dc1678ff67bf..4aa5ec82471ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -193,7 +193,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | } |} """.stripMargin - val code = new CodeAndComment(codeBody, Map.empty) + val code = CodeFormatter.stripOverlappingComments(new CodeAndComment(codeBody, Map.empty)) logDebug(s"SpecificUnsafeRowJoiner($schema1, $schema2):\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala index 6022f2dbbe948..76afc2e8ecbac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala @@ -36,6 +36,22 @@ class CodeFormatterSuite extends SparkFunSuite { } } + test("removing overlapping comments") { + val code = new CodeAndComment( + """/*project_c4*/ + |/*project_c3*/ + |/*project_c2*/ + """.stripMargin, + Map( + "/*project_c4*/" -> "// (((input[0, bigint, false] + 1) + 2) + 3))", + "/*project_c3*/" -> "// ((input[0, bigint, false] + 1) + 2)", + "/*project_c2*/" -> "// (input[0, bigint, false] + 1)" + )) + + val reducedCode = CodeFormatter.stripOverlappingComments(code) + assert(reducedCode.body === "/*project_c4*/") + } + testCase("basic example") { """class A { |blahblah; @@ -147,4 +163,31 @@ class CodeFormatterSuite extends SparkFunSuite { |/* 006 */ } """.stripMargin } + + // scalastyle:off whitespace.end.of.line + testCase("reduce empty lines") { + CodeFormatter.stripExtraNewLines( + """class A { + | + | + | /*** comment1 */ + | + | class body; + | + | + | if (c) {duh;} + | else {boo;} + |}""".stripMargin) + }{ + """ + |/* 001 */ class A { + |/* 002 */ /*** comment1 */ + |/* 003 */ class body; + |/* 004 */ + |/* 005 */ if (c) {duh;} + |/* 006 */ else {boo;} + |/* 007 */ } + """.stripMargin + } + // scalastyle:on whitespace.end.of.line } 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 2a1ce735b74ea..908e22de73bdc 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 @@ -333,8 +333,8 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co """.trim // try to compile, helpful for debug - val cleanedSource = - new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments()) + val cleanedSource = CodeFormatter.stripOverlappingComments( + new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments())) logDebug(s"\n${CodeFormatter.format(cleanedSource)}") CodeGenerator.compile(cleanedSource) 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 e0b48119f61dc..1041bab9d5d6c 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,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera } }""" - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] From 695d9a0fd461070ee2684b2210fb69d0b6ed1a95 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 24 May 2016 10:10:41 -0700 Subject: [PATCH 112/143] [SPARK-15433] [PYSPARK] PySpark core test should not use SerDe from PythonMLLibAPI ## What changes were proposed in this pull request? Currently PySpark core test uses the `SerDe` from `PythonMLLibAPI` which includes many MLlib things. It should use `SerDeUtil` instead. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13214 from viirya/pycore-use-serdeutil. --- .../main/scala/org/apache/spark/api/python/SerDeUtil.scala | 2 +- python/pyspark/tests.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 1c632ebdf9257..6e4eab4b805c1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -137,7 +137,7 @@ private[spark] object SerDeUtil extends Logging { * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by * PySpark. */ - private[spark] def javaToPython(jRDD: JavaRDD[_]): JavaRDD[Array[Byte]] = { + def javaToPython(jRDD: JavaRDD[_]): JavaRDD[Array[Byte]] = { jRDD.rdd.mapPartitions { iter => new AutoBatchedPickler(iter) } } diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 97ea39dde05fa..222c5ca5f497f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -960,13 +960,13 @@ def test_multiple_python_java_RDD_conversions(self): ] data_rdd = self.sc.parallelize(data) data_java_rdd = data_rdd._to_java_object_rdd() - data_python_rdd = self.sc._jvm.SerDe.javaToPython(data_java_rdd) + data_python_rdd = self.sc._jvm.SerDeUtil.javaToPython(data_java_rdd) converted_rdd = RDD(data_python_rdd, self.sc) self.assertEqual(2, converted_rdd.count()) # conversion between python and java RDD threw exceptions data_java_rdd = converted_rdd._to_java_object_rdd() - data_python_rdd = self.sc._jvm.SerDe.javaToPython(data_java_rdd) + data_python_rdd = self.sc._jvm.SerDeUtil.javaToPython(data_java_rdd) converted_rdd = RDD(data_python_rdd, self.sc) self.assertEqual(2, converted_rdd.count()) From a313a5ae74ae4e7686283657ba56076222317595 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 24 May 2016 10:26:55 -0700 Subject: [PATCH 113/143] [SPARK-15405][YARN] Remove unnecessary upload of config archive. We only need one copy of it. The client code that was uploading the second copy just needs to be modified to update the metadata in the cache, so that the AM knows where to find the configuration. Tested by running app on YARN and verifying in the logs only one archive is uploaded. Author: Marcelo Vanzin Closes #13232 from vanzin/SPARK-15405. --- .../scala/org/apache/spark/deploy/yarn/Client.scala | 9 +++++---- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 12 ++++++++---- 2 files changed, 13 insertions(+), 8 deletions(-) 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 3f6d7b28b5403..a12391d08180d 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 @@ -591,10 +591,11 @@ private[spark] class Client( copyFileToRemote(destDir, localConfArchive, replication, force = true, destName = Some(LOCALIZED_CONF_ARCHIVE)) - val (_, confLocalizedPath) = distribute(createConfArchive().toURI().getPath(), - resType = LocalResourceType.ARCHIVE, - destName = Some(LOCALIZED_CONF_DIR)) - require(confLocalizedPath != null) + // Manually add the config archive to the cache manager so that the AM is launched with + // the proper files set up. + distCacheMgr.addResource( + remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE, + LOCALIZED_CONF_DIR, statCache, appMasterOnly = false) // Clear the cache-related entries from the configuration to avoid them polluting the // UI's environment page. This works for client mode; for cluster mode, this is handled diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index a408c48d1d231..0a4f291e25fb0 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -144,9 +144,16 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll .set(SPARK_JARS, Seq(SPARK)) .set("spark.yarn.dist.jars", ADDED) val client = createClient(sparkConf, args = Array("--jar", USER)) + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), + any(classOf[Path]), anyShort(), anyBoolean(), any()) val tempDir = Utils.createTempDir() try { + // Because we mocked "copyFileToRemote" above to avoid having to create fake local files, + // we need to create a fake config archive in the temp dir to avoid having + // prepareLocalResources throw an exception. + new FileOutputStream(new File(tempDir, LOCALIZED_CONF_ARCHIVE)).close() + client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil) sparkConf.get(APP_JAR) should be (Some(USER)) @@ -384,10 +391,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll conf: Configuration = new Configuration(), args: Array[String] = Array()): Client = { val clientArgs = new ClientArguments(args) - val client = spy(new Client(clientArgs, conf, sparkConf)) - doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), - any(classOf[Path]), anyShort(), anyBoolean(), any()) - client + spy(new Client(clientArgs, conf, sparkConf)) } private def classpath(client: Client): Array[String] = { From 784cc07d1675eb9e0a387673cf86874e1bfc10f9 Mon Sep 17 00:00:00 2001 From: wangyang Date: Tue, 24 May 2016 11:03:12 -0700 Subject: [PATCH 114/143] [SPARK-15388][SQL] Fix spark sql CREATE FUNCTION with hive 1.2.1 ## What changes were proposed in this pull request? spark.sql("CREATE FUNCTION myfunc AS 'com.haizhi.bdp.udf.UDFGetGeoCode'") throws "org.apache.hadoop.hive.ql.metadata.HiveException:MetaException(message:NoSuchObjectException(message:Function default.myfunc does not exist))" with hive 1.2.1. I think it is introduced by pr #12853. Fixing it by catching Exception (not NoSuchObjectException) and string matching. ## How was this patch tested? added a unit test and also tested it manually Author: wangyang Closes #13177 from wangyang1992/fixCreateFunc2. --- .../apache/spark/sql/hive/client/HiveShim.scala | 14 ++++++++++++-- .../spark/sql/hive/client/VersionsSuite.scala | 1 + 2 files changed, 13 insertions(+), 2 deletions(-) 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 78713c3f0bace..9df4a26d55a27 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -24,6 +24,7 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JS import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf @@ -42,7 +43,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{IntegralType, StringType} -import org.apache.spark.util.CausedBy /** @@ -480,11 +480,21 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { try { Option(hive.getFunction(db, name)).map(fromHiveFunction) } catch { - case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + case NonFatal(e) if isCausedBy(e, s"$name does not exist") => None } } + private def isCausedBy(e: Throwable, matchMassage: String): Boolean = { + if (e.getMessage.contains(matchMassage)) { + true + } else if (e.getCause != null) { + isCausedBy(e.getCause, matchMassage) + } else { + false + } + } + override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { hive.getFunctions(db, pattern).asScala } 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 d46c4e7b2b50b..8ae4535f4ebf4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -440,6 +440,7 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.getFunctionOption("default", "func2").isEmpty) } else { assert(client.getFunctionOption("default", "func2").isDefined) + assert(client.getFunctionOption("default", "the_func_not_exists").isEmpty) } } From be99a99fe7976419d727c0cc92e872aa4af58bf1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 24 May 2016 11:09:54 -0700 Subject: [PATCH 115/143] [MINOR][CORE][TEST] Update obsolete `takeSample` test case. ## What changes were proposed in this pull request? This PR fixes some obsolete comments and assertion in `takeSample` testcase of `RDDSuite.scala`. ## How was this patch tested? This fixes the testcase only. Author: Dongjoon Hyun Closes #13260 from dongjoon-hyun/SPARK-15481. --- .../scala/org/apache/spark/rdd/RDDSuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) 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 979fb426c9482..a4992fe8ac556 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -678,27 +678,26 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } { val sample = data.takeSample(withReplacement = true, num = 20) - assert(sample.size === 20) // Got exactly 100 elements - assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.size === 20) // Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } { val sample = data.takeSample(withReplacement = true, num = n) - assert(sample.size === n) // Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n) // Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, n, seed) - assert(sample.size === n) // Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n) // Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, 2 * n, seed) - assert(sample.size === 2 * n) // Got exactly 200 elements - // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.size === 2 * n) // Got exactly 2 * n elements + // Chance of getting all distinct elements is still quite low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } } From 20900e5feced76e87f0a12823d0e3f07e082105f Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 24 May 2016 11:34:06 -0700 Subject: [PATCH 116/143] [SPARK-15502][DOC][ML][PYSPARK] add guide note that ALS only supports integer ids This PR adds a note to clarify that the ML API for ALS only supports integers for user/item ids, and that other types for these columns can be used but the ids must fall within integer range. (Refer [SPARK-14891](https://issues.apache.org/jira/browse/SPARK-14891)). Also cleaned up a reference to `mllib` in the ML doc. ## How was this patch tested? Built and viewed User Guide doc locally. Author: Nick Pentreath Closes #13278 from MLnick/SPARK-15502-als-int-id-doc-note. --- docs/ml-collaborative-filtering.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md index bd3d527d9a0e3..8bd75f3bcf7a7 100644 --- a/docs/ml-collaborative-filtering.md +++ b/docs/ml-collaborative-filtering.md @@ -29,6 +29,10 @@ following parameters: *baseline* confidence in preference observations (defaults to 1.0). * *nonnegative* specifies whether or not to use nonnegative constraints for least squares (defaults to `false`). +**Note:** The DataFrame-based API for ALS currently only supports integers for user and item ids. +Other numeric types are supported for the user and item id columns, +but the ids must be within the integer value range. + ### Explicit vs. implicit feedback The standard approach to matrix factorization based collaborative filtering treats @@ -36,7 +40,7 @@ the entries in the user-item matrix as *explicit* preferences given by the user for example, users giving ratings to movies. It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, -clicks, purchases, likes, shares etc.). The approach used in `spark.mllib` to deal with such data is taken +clicks, purchases, likes, shares etc.). The approach used in `spark.ml` to deal with such data is taken from [Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). Essentially, instead of trying to model the matrix of ratings directly, this approach treats the data as numbers representing the *strength* in observations of user actions (such as the number of clicks, From e631b819fe348729aab062207a452b8f1d1511bd Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 24 May 2016 14:27:39 -0700 Subject: [PATCH 117/143] [SPARK-15458][SQL][STREAMING] Disable schema inference for streaming datasets on file streams ## What changes were proposed in this pull request? If the user relies on the schema to be inferred in file streams can break easily for multiple reasons - accidentally running on a directory which has no data - schema changing underneath - on restart, the query will infer schema again, and may unexpectedly infer incorrect schema, as the file in the directory may be different at the time of the restart. To avoid these complicated scenarios, for Spark 2.0, we are going to disable schema inferencing by default with a config, so that user is forced to consider explicitly what is the schema it wants, rather than the system trying to infer it and run into weird corner cases. In this PR, I introduce a SQLConf that determines whether schema inference for file streams is allowed or not. It is disabled by default. ## How was this patch tested? Updated unit tests that test error behavior with and without schema inference enabled. Author: Tathagata Das Closes #13238 from tdas/SPARK-15458. --- .../execution/datasources/DataSource.scala | 11 + .../apache/spark/sql/internal/SQLConf.scala | 7 + .../sql/streaming/FileStreamSourceSuite.scala | 239 +++++++++++------- 3 files changed, 166 insertions(+), 91 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 e5dd4d81d6779..d0853f67b97ec 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 @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{CalendarIntervalType, StructType} import org.apache.spark.util.Utils @@ -186,6 +187,16 @@ case class DataSource( val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") }) + val isSchemaInferenceEnabled = sparkSession.conf.get(SQLConf.STREAMING_SCHEMA_INFERENCE) + val isTextSource = providingClass == classOf[text.DefaultSource] + // If the schema inference is disabled, only text sources require schema to be specified + if (!isSchemaInferenceEnabled && !isTextSource && userSpecifiedSchema.isEmpty) { + throw new IllegalArgumentException( + "Schema must be specified when creating a streaming source DataFrame. " + + "If some files already exist in the directory, then depending on the file format " + + "you may be able to create a static DataFrame on that directory with " + + "'spark.read.load(directory)' and infer schema from it.") + } SourceInfo(s"FileSource[$path]", inferFileFormatSchema(format)) case _ => 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 f3064eb6ac6d6..b91518acce33a 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 @@ -516,6 +516,13 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(60 * 1000L) // 10 minutes + val STREAMING_SCHEMA_INFERENCE = + SQLConfigBuilder("spark.sql.streaming.schemaInference") + .internal() + .doc("Whether file-based streaming sources will infer its own schema") + .booleanConf + .createWithDefault(false) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } 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 c97304c0ec1c1..1d784f1f4ee85 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 @@ -23,6 +23,7 @@ import java.util.UUID import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -165,19 +166,36 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { .collect { case s @ StreamingRelation(dataSource, _, _) => s.schema }.head } + // ============= Basic parameter exists tests ================ + test("FileStreamSource schema: no path") { - val e = intercept[IllegalArgumentException] { - createFileStreamSourceAndGetSchema(format = None, path = None, schema = None) + def testError(): Unit = { + val e = intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema(format = None, path = None, schema = None) + } + assert(e.getMessage.contains("path")) // reason is path, not schema } - assert("'path' is not specified" === e.getMessage) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { testError() } + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { testError() } } - test("FileStreamSource schema: path doesn't exist") { - intercept[AnalysisException] { + test("FileStreamSource schema: path doesn't exist, no schema") { + val e = intercept[IllegalArgumentException] { createFileStreamSourceAndGetSchema(format = None, path = Some("/a/b/c"), schema = None) } + assert(e.getMessage.toLowerCase.contains("schema")) // reason is schema absence, not the path + } + + test("FileStreamSource schema: path doesn't exist, with schema") { + val userSchema = new StructType().add(new StructField("value", IntegerType)) + val schema = createFileStreamSourceAndGetSchema( + format = None, path = Some("/a/b/c"), schema = Some(userSchema)) + assert(schema === userSchema) } + + // =============== Text file stream schema tests ================ + test("FileStreamSource schema: text, no existing files, no schema") { withTempDir { src => val schema = createFileStreamSourceAndGetSchema( @@ -205,13 +223,19 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== Parquet file stream schema tests ================ + test("FileStreamSource schema: parquet, no existing files, no schema") { withTempDir { src => - val e = intercept[AnalysisException] { - createFileStreamSourceAndGetSchema( - format = Some("parquet"), path = Some(new File(src, "1").getCanonicalPath), schema = None) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val e = intercept[AnalysisException] { + createFileStreamSourceAndGetSchema( + format = Some("parquet"), + path = Some(new File(src, "1").getCanonicalPath), + schema = None) + } + assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } - assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } } @@ -220,9 +244,21 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { Seq("a", "b", "c").toDS().as("userColumn").toDF().write .mode(org.apache.spark.sql.SaveMode.Overwrite) .parquet(src.getCanonicalPath) - val schema = createFileStreamSourceAndGetSchema( - format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) - assert(schema === new StructType().add("value", StringType)) + + // Without schema inference, should throw error + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { + intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema( + format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) + } + } + + // With schema inference, should infer correct schema + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val schema = createFileStreamSourceAndGetSchema( + format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("value", StringType)) + } } } @@ -237,22 +273,39 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== JSON file stream schema tests ================ + test("FileStreamSource schema: json, no existing files, no schema") { withTempDir { src => - val e = intercept[AnalysisException] { - createFileStreamSourceAndGetSchema( - format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + + val e = intercept[AnalysisException] { + createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + } + assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } - assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } } test("FileStreamSource schema: json, existing files, no schema") { withTempDir { src => - stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c': '3'}") - val schema = createFileStreamSourceAndGetSchema( - format = Some("json"), path = Some(src.getCanonicalPath), schema = None) - assert(schema === new StructType().add("c", StringType)) + + // Without schema inference, should throw error + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { + intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + } + } + + // With schema inference, should infer correct schema + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c': '3'}") + val schema = createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("c", StringType)) + } } } @@ -266,6 +319,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== Text file stream tests ================ + test("read from text files") { withTempDirs { case (src, tmp) => val textStream = createFileStream("text", src.getCanonicalPath) @@ -284,6 +339,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== JSON file stream tests ================ + test("read from json files") { withTempDirs { case (src, tmp) => val fileStream = createFileStream("json", src.getCanonicalPath, Some(valueSchema)) @@ -313,74 +370,82 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { test("read from json files with inferring schema") { withTempDirs { case (src, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { - // Add a file so that we can infer its schema - stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") - val fileStream = createFileStream("json", src.getCanonicalPath) - assert(fileStream.schema === StructType(Seq(StructField("c", StringType)))) + val fileStream = createFileStream("json", src.getCanonicalPath) + assert(fileStream.schema === StructType(Seq(StructField("c", StringType)))) - // FileStreamSource should infer the column "c" - val filtered = fileStream.filter($"c" contains "keep") + // FileStreamSource should infer the column "c" + val filtered = fileStream.filter($"c" contains "keep") - testStream(filtered)( - AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), - CheckAnswer("keep2", "keep3", "keep5", "keep6") - ) + testStream(filtered)( + AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6") + ) + } } } test("reading from json files inside partitioned directory") { withTempDirs { case (baseSrc, tmp) => - val src = new File(baseSrc, "type=X") - src.mkdirs() + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val src = new File(baseSrc, "type=X") + src.mkdirs() - // Add a file so that we can infer its schema - stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") - val fileStream = createFileStream("json", src.getCanonicalPath) + val fileStream = createFileStream("json", src.getCanonicalPath) - // FileStreamSource should infer the column "c" - val filtered = fileStream.filter($"c" contains "keep") + // FileStreamSource should infer the column "c" + val filtered = fileStream.filter($"c" contains "keep") - testStream(filtered)( - AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), - CheckAnswer("keep2", "keep3", "keep5", "keep6") - ) + testStream(filtered)( + AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6") + ) + } } } test("reading from json files with changing schema") { withTempDirs { case (src, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { - // Add a file so that we can infer its schema - stringToFile(new File(src, "existing"), "{'k': 'value0'}") + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'k': 'value0'}") - val fileStream = createFileStream("json", src.getCanonicalPath) + val fileStream = createFileStream("json", src.getCanonicalPath) - // FileStreamSource should infer the column "k" - assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) + // FileStreamSource should infer the column "k" + assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) - // After creating DF and before starting stream, add data with different schema - // Should not affect the inferred schema any more - stringToFile(new File(src, "existing2"), "{'k': 'value1', 'v': 'new'}") + // After creating DF and before starting stream, add data with different schema + // Should not affect the inferred schema any more + stringToFile(new File(src, "existing2"), "{'k': 'value1', 'v': 'new'}") - testStream(fileStream)( + testStream(fileStream)( - // Should not pick up column v in the file added before start - AddTextFileData("{'k': 'value2'}", src, tmp), - CheckAnswer("value0", "value1", "value2"), + // Should not pick up column v in the file added before start + AddTextFileData("{'k': 'value2'}", src, tmp), + CheckAnswer("value0", "value1", "value2"), - // Should read data in column k, and ignore v - AddTextFileData("{'k': 'value3', 'v': 'new'}", src, tmp), - CheckAnswer("value0", "value1", "value2", "value3"), + // Should read data in column k, and ignore v + AddTextFileData("{'k': 'value3', 'v': 'new'}", src, tmp), + CheckAnswer("value0", "value1", "value2", "value3"), - // Should ignore rows that do not have the necessary k column - AddTextFileData("{'v': 'value4'}", src, tmp), - CheckAnswer("value0", "value1", "value2", "value3", null)) + // Should ignore rows that do not have the necessary k column + AddTextFileData("{'v': 'value4'}", src, tmp), + CheckAnswer("value0", "value1", "value2", "value3", null)) + } } } + // =============== Parquet file stream tests ================ + test("read from parquet files") { withTempDirs { case (src, tmp) => val fileStream = createFileStream("parquet", src.getCanonicalPath, Some(valueSchema)) @@ -402,49 +467,39 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { test("read from parquet files with changing schema") { withTempDirs { case (src, tmp) => - // Add a file so that we can infer its schema - AddParquetFileData.writeToFile(Seq("value0").toDF("k"), src, tmp) - - val fileStream = createFileStream("parquet", src.getCanonicalPath) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { - // FileStreamSource should infer the column "k" - assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) + // Add a file so that we can infer its schema + AddParquetFileData.writeToFile(Seq("value0").toDF("k"), src, tmp) - // After creating DF and before starting stream, add data with different schema - // Should not affect the inferred schema any more - AddParquetFileData.writeToFile(Seq(("value1", 0)).toDF("k", "v"), src, tmp) + val fileStream = createFileStream("parquet", src.getCanonicalPath) - testStream(fileStream)( - // Should not pick up column v in the file added before start - AddParquetFileData(Seq("value2").toDF("k"), src, tmp), - CheckAnswer("value0", "value1", "value2"), + // FileStreamSource should infer the column "k" + assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) - // Should read data in column k, and ignore v - AddParquetFileData(Seq(("value3", 1)).toDF("k", "v"), src, tmp), - CheckAnswer("value0", "value1", "value2", "value3"), + // After creating DF and before starting stream, add data with different schema + // Should not affect the inferred schema any more + AddParquetFileData.writeToFile(Seq(("value1", 0)).toDF("k", "v"), src, tmp) - // Should ignore rows that do not have the necessary k column - AddParquetFileData(Seq("value5").toDF("v"), src, tmp), - CheckAnswer("value0", "value1", "value2", "value3", null) - ) - } - } + testStream(fileStream)( + // Should not pick up column v in the file added before start + AddParquetFileData(Seq("value2").toDF("k"), src, tmp), + CheckAnswer("value0", "value1", "value2"), - test("file stream source without schema") { - withTempDir { src => - // Only "text" doesn't need a schema - createFileStream("text", src.getCanonicalPath) + // Should read data in column k, and ignore v + AddParquetFileData(Seq(("value3", 1)).toDF("k", "v"), src, tmp), + CheckAnswer("value0", "value1", "value2", "value3"), - // Both "json" and "parquet" require a schema if no existing file to infer - intercept[AnalysisException] { - createFileStream("json", src.getCanonicalPath) - } - intercept[AnalysisException] { - createFileStream("parquet", src.getCanonicalPath) + // Should ignore rows that do not have the necessary k column + AddParquetFileData(Seq("value5").toDF("v"), src, tmp), + CheckAnswer("value0", "value1", "value2", "value3", null) + ) } } } + // =============== file stream globbing tests ================ + test("read new files in nested directories with globbing") { withTempDirs { case (dir, tmp) => @@ -518,6 +573,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== other tests ================ + test("fault tolerance") { withTempDirs { case (src, tmp) => val fileStream = createFileStream("text", src.getCanonicalPath) From f08bf587b1913c6cc8ecb34c45331cf4750961c9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 24 May 2016 18:55:23 -0700 Subject: [PATCH 118/143] [SPARK-15512][CORE] repartition(0) should raise IllegalArgumentException ## What changes were proposed in this pull request? Previously, SPARK-8893 added the constraints on positive number of partitions for repartition/coalesce operations in general. This PR adds one missing part for that and adds explicit two testcases. **Before** ```scala scala> sc.parallelize(1 to 5).coalesce(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> sc.parallelize(1 to 5).repartition(0).collect() res1: Array[Int] = Array() // empty scala> spark.sql("select 1").coalesce(0) res2: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int] scala> spark.sql("select 1").coalesce(0).collect() java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. scala> spark.sql("select 1").repartition(0) res3: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int] scala> spark.sql("select 1").repartition(0).collect() res4: Array[org.apache.spark.sql.Row] = Array() // empty ``` **After** ```scala scala> sc.parallelize(1 to 5).coalesce(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> sc.parallelize(1 to 5).repartition(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> spark.sql("select 1").coalesce(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> spark.sql("select 1").repartition(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... ``` ## How was this patch tested? Pass the Jenkins tests with new testcases. Author: Dongjoon Hyun Closes #13282 from dongjoon-hyun/SPARK-15512. --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 + core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 8 ++++++++ .../catalyst/plans/logical/basicLogicalOperators.scala | 1 + .../spark/sql/catalyst/plans/logical/partitioning.scala | 2 +- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 8 ++++++++ 6 files changed, 27 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e6db9b3eec8ea..e251421c48fca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -437,6 +437,7 @@ abstract class RDD[T: ClassTag]( partitionCoalescer: Option[PartitionCoalescer] = Option.empty) (implicit ord: Ordering[T] = null) : RDD[T] = withScope { + require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") if (shuffle) { /** Distributes elements evenly across output partitions, starting from a random partition. */ val distributePartition = (index: Int, items: Iterator[T]) => { 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 a4992fe8ac556..ad56715656c85 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -276,6 +276,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { test("repartitioned RDDs") { val data = sc.parallelize(1 to 1000, 10) + intercept[IllegalArgumentException] { + data.repartition(0) + } + // Coalesce partitions val repartitioned1 = data.repartition(2) assert(repartitioned1.partitions.size == 2) @@ -329,6 +333,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { test("coalesced RDDs") { val data = sc.parallelize(1 to 10, 10) + intercept[IllegalArgumentException] { + data.coalesce(0) + } + val coalesced1 = data.coalesce(2) assert(coalesced1.collect().toList === (1 to 10).toList) assert(coalesced1.glom().collect().map(_.toList).toList === 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 ca0096eeb2087..0a9250b71fb61 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 @@ -734,6 +734,7 @@ case class Distinct(child: LogicalPlan) extends UnaryNode { */ case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan) extends UnaryNode { + require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") override def output: Seq[Attribute] = child.output } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index a5bdee1b854ce..28cbce8748fcd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -43,7 +43,7 @@ case class RepartitionByExpression( child: LogicalPlan, numPartitions: Option[Int] = None) extends RedistributeData { numPartitions match { - case Some(n) => require(n > 0, "numPartitions must be greater than 0.") + case Some(n) => require(n > 0, s"Number of partitions ($n) must be positive.") case None => // Ok } } 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 f573abf859610..0614747352c11 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 @@ -259,12 +259,20 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("repartition") { + intercept[IllegalArgumentException] { + testData.select('key).repartition(0) + } + checkAnswer( testData.select('key).repartition(10).select('key), testData.select('key).collect().toSeq) } test("coalesce") { + intercept[IllegalArgumentException] { + testData.select('key).coalesce(0) + } + assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) checkAnswer( 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 0ffbd6db12be6..05de79eb2f1d0 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 @@ -81,6 +81,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val data = (1 to 100).map(i => ClassData(i.toString, i)) val ds = data.toDS() + intercept[IllegalArgumentException] { + ds.coalesce(0) + } + + intercept[IllegalArgumentException] { + ds.repartition(0) + } + assert(ds.repartition(10).rdd.partitions.length == 10) checkDataset( ds.repartition(10), From 14494da87bdf057d2d2f796b962a4d8bc4747d31 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 24 May 2016 20:55:47 -0700 Subject: [PATCH 119/143] [SPARK-15518] Rename various scheduler backend for consistency ## What changes were proposed in this pull request? This patch renames various scheduler backends to make them consistent: - LocalScheduler -> LocalSchedulerBackend - AppClient -> StandaloneAppClient - AppClientListener -> StandaloneAppClientListener - SparkDeploySchedulerBackend -> StandaloneSchedulerBackend - CoarseMesosSchedulerBackend -> MesosCoarseGrainedSchedulerBackend - MesosSchedulerBackend -> MesosFineGrainedSchedulerBackend ## How was this patch tested? Updated test cases to reflect the name change. Author: Reynold Xin Closes #13288 from rxin/SPARK-15518. --- .../scala/org/apache/spark/SparkContext.scala | 23 +++++----- ...Client.scala => StandaloneAppClient.scala} | 11 +++-- ...cala => StandaloneAppClientListener.scala} | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 11 +++-- ...scala => StandaloneSchedulerBackend.scala} | 15 +++--- ... MesosCoarseGrainedSchedulerBackend.scala} | 46 ++++++++++--------- ...=> MesosFineGrainedSchedulerBackend.scala} | 34 +++++++------- .../mesos/MesosSchedulerBackendUtil.scala | 2 +- ...la => LocalSchedulerBackendEndpoint.scala} | 18 ++++---- .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- .../SparkContextSchedulerCreationSuite.scala | 33 +++++++------ .../spark/deploy/client/AppClientSuite.scala | 4 +- ...sCoarseGrainedSchedulerBackendSuite.scala} | 12 +++-- ...sosFineGrainedSchedulerBackendSuite.scala} | 15 +++--- 14 files changed, 123 insertions(+), 107 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/client/{AppClient.scala => StandaloneAppClient.scala} (97%) rename core/src/main/scala/org/apache/spark/deploy/client/{AppClientListener.scala => StandaloneAppClientListener.scala} (96%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/{SparkDeploySchedulerBackend.scala => StandaloneSchedulerBackend.scala} (94%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{CoarseMesosSchedulerBackend.scala => MesosCoarseGrainedSchedulerBackend.scala} (93%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{MesosSchedulerBackend.scala => MesosFineGrainedSchedulerBackend.scala} (93%) rename core/src/main/scala/org/apache/spark/scheduler/local/{LocalBackend.scala => LocalSchedulerBackendEndpoint.scala} (87%) rename core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/{CoarseMesosSchedulerBackendSuite.scala => MesosCoarseGrainedSchedulerBackendSuite.scala} (96%) rename core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/{MesosSchedulerBackendSuite.scala => MesosFineGrainedSchedulerBackendSuite.scala} (95%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 351024bea47e6..36aa3becb4fc3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -56,10 +56,9 @@ import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, - SparkDeploySchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.scheduler.local.LocalBackend +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend} +import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump import org.apache.spark.ui.{ConsoleProgressBar, SparkUI} @@ -2426,7 +2425,7 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(sc.getConf, scheduler, 1) + val backend = new LocalSchedulerBackendEndpoint(sc.getConf, scheduler, 1) scheduler.initialize(backend) (backend, scheduler) @@ -2438,7 +2437,7 @@ object SparkContext extends Logging { throw new SparkException(s"Asked to run locally with $threadCount threads") } val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(sc.getConf, scheduler, threadCount) + val backend = new LocalSchedulerBackendEndpoint(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) @@ -2448,14 +2447,14 @@ object SparkContext extends Logging { // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) - val backend = new LocalBackend(sc.getConf, scheduler, threadCount) + val backend = new LocalSchedulerBackendEndpoint(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) + val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) (backend, scheduler) @@ -2472,9 +2471,9 @@ object SparkContext extends Logging { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt, sc.conf) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) + val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) - backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { + backend.shutdownCallback = (backend: StandaloneSchedulerBackend) => { localCluster.stop() } (backend, scheduler) @@ -2484,9 +2483,9 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc) val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", defaultValue = true) val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, sc, mesosUrl, sc.env.securityManager) + new MesosCoarseGrainedSchedulerBackend(scheduler, sc, mesosUrl, sc.env.securityManager) } else { - new MesosSchedulerBackend(scheduler, sc, mesosUrl) + new MesosFineGrainedSchedulerBackend(scheduler, sc, mesosUrl) } scheduler.initialize(backend) (backend, scheduler) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala rename to core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index 43b17e5d49bf1..a9df732df93ca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -32,17 +32,18 @@ import org.apache.spark.rpc._ import org.apache.spark.util.{RpcUtils, ThreadUtils} /** - * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, - * an app description, and a listener for cluster events, and calls back the listener when various - * events occur. + * Interface allowing applications to speak with a Spark standalone cluster manager. + * + * Takes a master URL, an app description, and a listener for cluster events, and calls + * back the listener when various events occur. * * @param masterUrls Each url should look like spark://host:port. */ -private[spark] class AppClient( +private[spark] class StandaloneAppClient( rpcEnv: RpcEnv, masterUrls: Array[String], appDescription: ApplicationDescription, - listener: AppClientListener, + listener: StandaloneAppClientListener, conf: SparkConf) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala rename to core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala index 94506a0cbb27c..370b16ce4213a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala @@ -24,7 +24,7 @@ package org.apache.spark.deploy.client * * Users of this API should *not* block inside the callback methods. */ -private[spark] trait AppClientListener { +private[spark] trait StandaloneAppClientListener { def connected(appId: String): Unit /** Disconnection may be a temporary state, as we fail over to a new Master. */ 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 371fb8602f785..01e85ca405587 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -33,21 +33,22 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality +import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. - * It can also work with a local setup by using a LocalBackend and setting isLocal to true. - * It handles common logic, like determining a scheduling order across jobs, waking up to launch - * speculative tasks, etc. + * It can also work with a local setup by using a [[LocalSchedulerBackendEndpoint]] and setting + * isLocal to true. It handles common logic, like determining a scheduling order across jobs, waking + * up to launch speculative tasks, etc. * * Clients should first call initialize() and start(), then submit task sets through the * runTasks method. * - * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple + * THREADING: [[SchedulerBackend]]s and task-submitting clients can call this class from multiple * threads, so it needs locks in public API methods to maintain its state. In addition, some - * SchedulerBackends synchronize on themselves when they want to send events here, and then + * [[SchedulerBackend]]s synchronize on themselves when they want to send events here, and then * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 85d002011d64c..8382fbe9ddb80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -21,28 +21,31 @@ import java.util.concurrent.Semaphore import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} -import org.apache.spark.deploy.client.{AppClient, AppClientListener} +import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} import org.apache.spark.internal.Logging import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ import org.apache.spark.util.Utils -private[spark] class SparkDeploySchedulerBackend( +/** + * A [[SchedulerBackend]] implementation for Spark's standalone cluster manager. + */ +private[spark] class StandaloneSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, masters: Array[String]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with AppClientListener + with StandaloneAppClientListener with Logging { - private var client: AppClient = null + private var client: StandaloneAppClient = null private var stopping = false private val launcherBackend = new LauncherBackend() { override protected def onStopRequest(): Unit = stop(SparkAppHandle.State.KILLED) } - @volatile var shutdownCallback: SparkDeploySchedulerBackend => Unit = _ + @volatile var shutdownCallback: StandaloneSchedulerBackend => Unit = _ @volatile private var appId: String = _ private val registrationBarrier = new Semaphore(0) @@ -100,7 +103,7 @@ private[spark] class SparkDeploySchedulerBackend( } val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor, initialExecutorLimit) - client = new AppClient(sc.env.rpcEnv, masters, appDesc, this, conf) + client = new StandaloneAppClient(sc.env.rpcEnv, masters, appDesc, this, conf) client.start() launcherBackend.setState(SparkAppHandle.State.SUBMITTED) waitForRegistration() 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/MesosCoarseGrainedSchedulerBackend.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 2c5be1f528894..e88e4ad4750d7 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/MesosCoarseGrainedSchedulerBackend.scala @@ -25,13 +25,12 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.{Buffer, HashMap, HashSet} -import org.apache.mesos.{Scheduler => MScheduler, SchedulerDriver} import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskState} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient -import org.apache.spark.rpc.{RpcEndpointAddress} +import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -43,16 +42,16 @@ import org.apache.spark.util.Utils * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable * latency. * - * Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to - * remove this. + * Unfortunately this has a bit of duplication from [[MesosFineGrainedSchedulerBackend]], + * but it seems hard to remove this. */ -private[spark] class CoarseMesosSchedulerBackend( +private[spark] class MesosCoarseGrainedSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String, securityManager: SecurityManager) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with MScheduler + with org.apache.mesos.Scheduler with MesosSchedulerUtils { val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures @@ -149,7 +148,7 @@ private[spark] class CoarseMesosSchedulerBackend( super.start() val driver = createSchedulerDriver( master, - CoarseMesosSchedulerBackend.this, + MesosCoarseGrainedSchedulerBackend.this, sc.sparkUser, sc.appName, sc.conf, @@ -239,9 +238,10 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} + override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + override def registered( + d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { appId = frameworkId.getValue mesosExternalShuffleClient.foreach(_.init(appId)) logInfo("Registered as framework ID " + appId) @@ -252,15 +252,15 @@ private[spark] class CoarseMesosSchedulerBackend( totalCoresAcquired >= maxCores * minRegisteredRatio } - override def disconnected(d: SchedulerDriver) {} + override def disconnected(d: org.apache.mesos.SchedulerDriver) {} - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} + override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {} /** * Method called by Mesos to offer resources on slaves. We respond by launching an executor, * unless we've already launched more than we wanted to. */ - override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { stateLock.synchronized { if (stopCalled) { logDebug("Ignoring offers during shutdown") @@ -282,7 +282,8 @@ private[spark] class CoarseMesosSchedulerBackend( } } - private def declineUnmatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { + private def declineUnmatchedOffers( + d: org.apache.mesos.SchedulerDriver, offers: Buffer[Offer]): Unit = { offers.foreach { offer => declineOffer(d, offer, Some("unmet constraints"), Some(rejectOfferDurationForUnmetConstraints)) @@ -290,7 +291,7 @@ private[spark] class CoarseMesosSchedulerBackend( } private def declineOffer( - d: SchedulerDriver, + d: org.apache.mesos.SchedulerDriver, offer: Offer, reason: Option[String] = None, refuseSeconds: Option[Long] = None): Unit = { @@ -319,7 +320,8 @@ private[spark] class CoarseMesosSchedulerBackend( * @param d SchedulerDriver * @param offers Mesos offers that match attribute constraints */ - private def handleMatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { + private def handleMatchedOffers( + d: org.apache.mesos.SchedulerDriver, offers: Buffer[Offer]): Unit = { val tasks = buildMesosTasks(offers) for (offer <- offers) { val offerAttributes = toAttributeMap(offer.getAttributesList) @@ -440,7 +442,7 @@ private[spark] class CoarseMesosSchedulerBackend( math.min(offerCPUs, maxCores - totalCoresAcquired)) } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { val taskId = status.getTaskId.getValue val slaveId = status.getSlaveId.getValue val state = TaskState.fromMesos(status.getState) @@ -498,7 +500,7 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def error(d: SchedulerDriver, message: String) { + override def error(d: org.apache.mesos.SchedulerDriver, message: String) { logError(s"Mesos error: $message") scheduler.error(message) } @@ -538,14 +540,15 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + override def frameworkMessage( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} /** * Called when a slave is lost or a Mesos task finished. Updates local view on * what tasks are running. It also notifies the driver that an executor was removed. */ private def executorTerminated( - d: SchedulerDriver, + d: org.apache.mesos.SchedulerDriver, slaveId: String, taskId: String, reason: String): Unit = { @@ -555,11 +558,12 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID): Unit = { + override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = { logInfo(s"Mesos slave lost: ${slaveId.getValue}") } - override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { + override def executorLost( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { logInfo("Mesos executor lost: %s".format(e.getValue)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index 1a94aee2ca30c..e08dc3b5957bb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -23,7 +23,6 @@ import java.util.{ArrayList => JArrayList, Collections, List => JList} import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, HashSet} -import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.protobuf.ByteString @@ -38,12 +37,12 @@ import org.apache.spark.util.Utils * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks * from multiple apps can run on different cores) and in time (a core can switch ownership). */ -private[spark] class MesosSchedulerBackend( +private[spark] class MesosFineGrainedSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String) extends SchedulerBackend - with MScheduler + with org.apache.mesos.Scheduler with MesosSchedulerUtils { // Stores the slave ids that has launched a Mesos executor. @@ -74,7 +73,7 @@ private[spark] class MesosSchedulerBackend( classLoader = Thread.currentThread.getContextClassLoader val driver = createSchedulerDriver( master, - MesosSchedulerBackend.this, + MesosFineGrainedSchedulerBackend.this, sc.sparkUser, sc.appName, sc.conf, @@ -175,9 +174,10 @@ private[spark] class MesosSchedulerBackend( execArgs } - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} + override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + override def registered( + d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { inClassLoader() { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) @@ -195,9 +195,9 @@ private[spark] class MesosSchedulerBackend( } } - override def disconnected(d: SchedulerDriver) {} + override def disconnected(d: org.apache.mesos.SchedulerDriver) {} - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} + override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {} private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = { val builder = new StringBuilder @@ -216,7 +216,7 @@ private[spark] class MesosSchedulerBackend( * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ - override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { inClassLoader() { // Fail first on offers with unmet constraints val (offersMatchingConstraints, offersNotMatchingConstraints) = @@ -355,7 +355,7 @@ private[spark] class MesosSchedulerBackend( (taskInfo, finalResources.asJava) } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { inClassLoader() { val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) @@ -373,7 +373,7 @@ private[spark] class MesosSchedulerBackend( } } - override def error(d: SchedulerDriver, message: String) { + override def error(d: org.apache.mesos.SchedulerDriver, message: String) { inClassLoader() { logError("Mesos error: " + message) markErr() @@ -391,7 +391,8 @@ private[spark] class MesosSchedulerBackend( mesosDriver.reviveOffers() } - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + override def frameworkMessage( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} /** * Remove executor associated with slaveId in a thread safe manner. @@ -403,7 +404,8 @@ private[spark] class MesosSchedulerBackend( } } - private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { + private def recordSlaveLost( + d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { inClassLoader() { logInfo("Mesos slave lost: " + slaveId.getValue) removeExecutor(slaveId.getValue, reason.toString) @@ -411,12 +413,12 @@ private[spark] class MesosSchedulerBackend( } } - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { + override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID) { recordSlaveLost(d, slaveId, SlaveLost()) } - override def executorLost(d: SchedulerDriver, executorId: ExecutorID, - slaveId: SlaveID, status: Int) { + override def executorLost( + d: org.apache.mesos.SchedulerDriver, executorId: ExecutorID, slaveId: SlaveID, status: Int) { logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue, slaveId.getValue)) recordSlaveLost(d, slaveId, ExecutorExited(status, exitCausedByApp = true)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala index 1b7ac172defb9..05b2b08944098 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -25,7 +25,7 @@ import org.apache.spark.internal.Logging /** * A collection of utility functions which can be used by both the - * MesosSchedulerBackend and the CoarseMesosSchedulerBackend. + * MesosSchedulerBackend and the [[MesosFineGrainedSchedulerBackend]]. */ private[mesos] object MesosSchedulerBackendUtil extends Logging { /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala similarity index 87% rename from core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala index 3473ef21b39a4..ee0658837997f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala @@ -39,15 +39,15 @@ private case class KillTask(taskId: Long, interruptThread: Boolean) private case class StopExecutor() /** - * Calls to LocalBackend are all serialized through LocalEndpoint. Using an RpcEndpoint makes the - * calls on LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend - * and the TaskSchedulerImpl. + * Calls to [[LocalSchedulerBackendEndpoint]] are all serialized through LocalEndpoint. Using an + * RpcEndpoint makes the calls on [[LocalSchedulerBackendEndpoint]] asynchronous, which is necessary + * to prevent deadlock between [[LocalSchedulerBackendEndpoint]] and the [[TaskSchedulerImpl]]. */ private[spark] class LocalEndpoint( override val rpcEnv: RpcEnv, userClassPath: Seq[URL], scheduler: TaskSchedulerImpl, - executorBackend: LocalBackend, + executorBackend: LocalSchedulerBackendEndpoint, private val totalCores: Int) extends ThreadSafeRpcEndpoint with Logging { @@ -91,11 +91,11 @@ private[spark] class LocalEndpoint( } /** - * LocalBackend is used when running a local version of Spark where the executor, backend, and - * master all run in the same JVM. It sits behind a TaskSchedulerImpl and handles launching tasks - * on a single Executor (created by the LocalBackend) running locally. + * Used when running a local version of Spark where the executor, backend, and master all run in + * the same JVM. It sits behind a [[TaskSchedulerImpl]] and handles launching tasks on a single + * Executor (created by the [[LocalSchedulerBackendEndpoint]]) running locally. */ -private[spark] class LocalBackend( +private[spark] class LocalSchedulerBackendEndpoint( conf: SparkConf, scheduler: TaskSchedulerImpl, val totalCores: Int) @@ -124,7 +124,7 @@ private[spark] class LocalBackend( override def start() { val rpcEnv = SparkEnv.get.rpcEnv val executorEndpoint = new LocalEndpoint(rpcEnv, userClassPath, scheduler, this, totalCores) - localEndpoint = rpcEnv.setupEndpoint("LocalBackendEndpoint", executorEndpoint) + localEndpoint = rpcEnv.setupEndpoint("LocalSchedulerBackendEndpoint", executorEndpoint) listenerBus.post(SparkListenerExecutorAdded( System.currentTimeMillis, executorEndpoint.localExecutorId, diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 1adc90ab1e9dd..81b94b57219db 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -242,8 +242,8 @@ class HeartbeatReceiverSuite } private def getTrackedExecutors: Map[String, Long] = { - // We may receive undesired SparkListenerExecutorAdded from LocalBackend, so exclude it from - // the map. See SPARK-10800. + // We may receive undesired SparkListenerExecutorAdded from LocalSchedulerBackend, + // so exclude it from the map. See SPARK-10800. heartbeatReceiver.invokePrivate(_executorLastSeen()). filterKeys(_ != SparkContext.DRIVER_IDENTIFIER) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 213d70f4e5840..11be40abca099 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -21,10 +21,10 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.scheduler.local.LocalBackend -import org.apache.spark.util.Utils +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend} +import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint + class SparkContextSchedulerCreationSuite extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging { @@ -58,7 +58,7 @@ class SparkContextSchedulerCreationSuite test("local") { val sched = createTaskScheduler("local") sched.backend match { - case s: LocalBackend => assert(s.totalCores === 1) + case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 1) case _ => fail() } } @@ -66,7 +66,8 @@ class SparkContextSchedulerCreationSuite test("local-*") { val sched = createTaskScheduler("local[*]") sched.backend match { - case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case s: LocalSchedulerBackendEndpoint => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } } @@ -75,7 +76,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) sched.backend match { - case s: LocalBackend => assert(s.totalCores === 5) + case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 5) case _ => fail() } } @@ -84,7 +85,8 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[* ,2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case s: LocalSchedulerBackendEndpoint => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } } @@ -93,7 +95,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[4, 2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalBackend => assert(s.totalCores === 4) + case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 4) case _ => fail() } } @@ -117,14 +119,14 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local", "client", conf) sched.backend match { - case s: LocalBackend => assert(s.defaultParallelism() === 16) + case s: LocalSchedulerBackendEndpoint => assert(s.defaultParallelism() === 16) case _ => fail() } } test("local-cluster") { createTaskScheduler("local-cluster[3, 14, 1024]").backend match { - case s: SparkDeploySchedulerBackend => // OK + case s: StandaloneSchedulerBackend => // OK case _ => fail() } } @@ -143,19 +145,20 @@ class SparkContextSchedulerCreationSuite } test("mesos fine-grained") { - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) + testMesos("mesos://localhost:1234", classOf[MesosFineGrainedSchedulerBackend], coarse = false) } test("mesos coarse-grained") { - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) + testMesos("mesos://localhost:1234", classOf[MesosCoarseGrainedSchedulerBackend], coarse = true) } test("mesos with zookeeper") { testMesos("mesos://zk://localhost:1234,localhost:2345", - classOf[MesosSchedulerBackend], coarse = false) + classOf[MesosFineGrainedSchedulerBackend], coarse = false) } test("mesos with zookeeper and Master URL starting with zk://") { - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) + testMesos("zk://localhost:1234,localhost:2345", + classOf[MesosFineGrainedSchedulerBackend], coarse = false) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index 7b46f9101d89b..f6ef9d15ddee2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -166,7 +166,7 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd } /** Application Listener to collect events */ - private class AppClientCollector extends AppClientListener with Logging { + private class AppClientCollector extends StandaloneAppClientListener with Logging { val connectedIdList = new ConcurrentLinkedQueue[String]() @volatile var disconnectedCount: Int = 0 val deadReasonList = new ConcurrentLinkedQueue[String]() @@ -208,7 +208,7 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd List(), Map(), Seq(), Seq(), Seq()) private val desc = new ApplicationDescription("AppClientSuite", Some(1), 512, cmd, "ignored") val listener = new AppClientCollector - val client = new AppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) + val client = new StandaloneAppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) } } 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/MesosCoarseGrainedSchedulerBackendSuite.scala similarity index 96% rename from core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 15d59e7052912..7f21d4c623afc 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/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.TaskSchedulerImpl -class CoarseMesosSchedulerBackendSuite extends SparkFunSuite +class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with BeforeAndAfter { @@ -44,7 +44,7 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite private var sparkConf: SparkConf = _ private var driver: SchedulerDriver = _ private var taskScheduler: TaskSchedulerImpl = _ - private var backend: CoarseMesosSchedulerBackend = _ + private var backend: MesosCoarseGrainedSchedulerBackend = _ private var externalShuffleClient: MesosExternalShuffleClient = _ private var driverEndpoint: RpcEndpointRef = _ @@ -230,7 +230,8 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) val securityManager = mock[SecurityManager] - val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master", securityManager) { + val backend = new MesosCoarseGrainedSchedulerBackend( + taskScheduler, sc, "master", securityManager) { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, @@ -323,10 +324,11 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite taskScheduler: TaskSchedulerImpl, driver: SchedulerDriver, shuffleClient: MesosExternalShuffleClient, - endpoint: RpcEndpointRef): CoarseMesosSchedulerBackend = { + endpoint: RpcEndpointRef): MesosCoarseGrainedSchedulerBackend = { val securityManager = mock[SecurityManager] - val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master", securityManager) { + val backend = new MesosCoarseGrainedSchedulerBackend( + taskScheduler, sc, "master", securityManager) { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index 7d6b7bde68253..41693b1191a3c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -40,7 +40,8 @@ import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.ExecutorInfo -class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { +class MesosFineGrainedSchedulerBackendSuite + extends SparkFunSuite with LocalSparkContext with MockitoSugar { test("weburi is set in created scheduler driver") { val conf = new SparkConf @@ -56,7 +57,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val driver = mock[SchedulerDriver] when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") { + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, @@ -96,7 +97,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val resources = Arrays.asList( mesosSchedulerBackend.createResource("cpus", 4), @@ -127,7 +128,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val resources = Arrays.asList( mesosSchedulerBackend.createResource("cpus", 4), @@ -163,7 +164,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(sc.conf).thenReturn(conf) when(sc.listenerBus).thenReturn(listenerBus) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val (execInfo, _) = backend.createExecutorInfo( Arrays.asList(backend.createResource("cpus", 4)), "mockExecutor") @@ -222,7 +223,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(sc.conf).thenReturn(new SparkConf) when(sc.listenerBus).thenReturn(listenerBus) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val minMem = backend.executorMemory(sc) val minCpu = 4 @@ -333,7 +334,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val mesosOffers = new java.util.ArrayList[Offer] mesosOffers.add(offer) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1) expectedWorkerOffers.append(new WorkerOffer( From 4acababcaba567c85f3be0d5e939d99119b82d1d Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 24 May 2016 20:58:20 -0700 Subject: [PATCH 120/143] [SPARK-15365][SQL] When table size statistics are not available from metastore, we should fallback to HDFS ## What changes were proposed in this pull request? Currently if a table is used in join operation we rely on Metastore returned size to calculate if we can convert the operation to Broadcast join. This optimization only kicks in for table's that have the statistics available in metastore. Hive generally rolls over to HDFS if the statistics are not available directly from metastore and this seems like a reasonable choice to adopt given the optimization benefit of using broadcast joins. ## How was this patch tested? I have executed queries locally to test. Author: Parth Brahmbhatt Closes #13150 from Parth-Brahmbhatt/SPARK-15365. --- .../apache/spark/sql/internal/SQLConf.scala | 9 ++++ .../spark/sql/hive/MetastoreRelation.scala | 33 +++++++++--- .../spark/sql/hive/StatisticsSuite.scala | 50 ++++++++++++++++++- 3 files changed, 83 insertions(+), 9 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 b91518acce33a..4efefdacabf22 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 @@ -113,6 +113,13 @@ object SQLConf { .longConf .createWithDefault(10L * 1024 * 1024) + val ENABLE_FALL_BACK_TO_HDFS_FOR_STATS = + SQLConfigBuilder("spark.sql.enableFallBackToHdfsForStats") + .doc("If the table statistics are not available from table metadata enable fall back to hdfs." + + " This is useful in determining if a table is small enough to use auto broadcast joins.") + .booleanConf + .createWithDefault(false) + val DEFAULT_SIZE_IN_BYTES = SQLConfigBuilder("spark.sql.defaultSizeInBytes") .internal() .doc("The default table size used in query planning. By default, it is set to a larger " + @@ -603,6 +610,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def autoBroadcastJoinThreshold: Long = getConf(AUTO_BROADCASTJOIN_THRESHOLD) + def fallBackToHdfsForStatsEnabled: Boolean = getConf(ENABLE_FALL_BACK_TO_HDFS_FOR_STATS) + def preferSortMergeJoin: Boolean = getConf(PREFER_SORTMERGEJOIN) def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 1671228fd9b42..9c820144aee12 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -17,9 +17,12 @@ package org.apache.spark.sql.hive +import java.io.IOException + import scala.collection.JavaConverters._ import com.google.common.base.Objects +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.FieldSchema @@ -114,17 +117,31 @@ private[hive] case class MetastoreRelation( val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) // TODO: check if this estimate is valid for tables after partition pruning. // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be - // relatively cheap if parameters for the table are populated into the metastore. An - // alternative would be going through Hadoop's FileSystem API, which can be expensive if a lot - // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, - // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. + // relatively cheap if parameters for the table are populated into the metastore. + // Besides `totalSize`, there are also `numFiles`, `numRows`, `rawDataSize` keys + // (see StatsSetupConst in Hive) that we can look at in the future. BigInt( // When table is external,`totalSize` is always zero, which will influence join strategy // so when `totalSize` is zero, use `rawDataSize` instead - // if the size is still less than zero, we use default size - Option(totalSize).map(_.toLong).filter(_ > 0) - .getOrElse(Option(rawDataSize).map(_.toLong).filter(_ > 0) - .getOrElse(sparkSession.sessionState.conf.defaultSizeInBytes))) + // if the size is still less than zero, we try to get the file size from HDFS. + // given this is only needed for optimization, if the HDFS call fails we return the default. + if (totalSize != null && totalSize.toLong > 0L) { + totalSize.toLong + } else if (rawDataSize != null && rawDataSize.toLong > 0) { + rawDataSize.toLong + } else if (sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled) { + try { + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val fs: FileSystem = hiveQlTable.getPath.getFileSystem(hadoopConf) + fs.getContentSummary(hiveQlTable.getPath).getLength + } catch { + case e: IOException => + logWarning("Failed to get table size from hdfs.", e) + sparkSession.sessionState.conf.defaultSizeInBytes + } + } else { + sparkSession.sessionState.conf.defaultSizeInBytes + }) } ) 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 1a7b6c0112279..f8e00a35a31e2 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.io.{File, PrintWriter} + import scala.reflect.ClassTag import org.apache.spark.sql.{QueryTest, Row} @@ -25,8 +27,9 @@ import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils -class StatisticsSuite extends QueryTest with TestHiveSingleton { +class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { import hiveContext.sql test("parse analyze commands") { @@ -68,6 +71,51 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { classOf[AnalyzeTableCommand]) } + test("MetastoreRelations fallback to HDFS for size estimation") { + val enableFallBackToHdfsForStats = hiveContext.conf.fallBackToHdfsForStatsEnabled + try { + withTempDir { tempDir => + + // EXTERNAL OpenCSVSerde table pointing to LOCATION + + val file1 = new File(tempDir + "/data1") + val writer1 = new PrintWriter(file1) + writer1.write("1,2") + writer1.close() + + val file2 = new File(tempDir + "/data2") + val writer2 = new PrintWriter(file2) + writer2.write("1,2") + writer2.close() + + sql( + s"""CREATE EXTERNAL TABLE csv_table(page_id INT, impressions INT) + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' + WITH SERDEPROPERTIES ( + \"separatorChar\" = \",\", + \"quoteChar\" = \"\\\"\", + \"escapeChar\" = \"\\\\\") + LOCATION '$tempDir' + """) + + hiveContext.setConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS, true) + + val relation = hiveContext.sessionState.catalog.lookupRelation(TableIdentifier("csv_table")) + .asInstanceOf[MetastoreRelation] + + val properties = relation.hiveQlTable.getParameters + assert(properties.get("totalSize").toLong <= 0, "external table totalSize must be <= 0") + assert(properties.get("rawDataSize").toLong <= 0, "external table rawDataSize must be <= 0") + + val sizeInBytes = relation.statistics.sizeInBytes + assert(sizeInBytes === BigInt(file1.length() + file2.length())) + } + } finally { + hiveContext.setConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS, enableFallBackToHdfsForStats) + sql("DROP TABLE csv_table ") + } + } + ignore("analyze MetastoreRelations") { def queryTotalSize(tableName: String): BigInt = hiveContext.sessionState.catalog.lookupRelation( From 50b660d725269dc0c11e0d350ddd7fc8b19539a0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 24 May 2016 21:23:39 -0700 Subject: [PATCH 121/143] [SPARK-15498][TESTS] fix slow tests ## What changes were proposed in this pull request? This PR fixes 3 slow tests: 1. `ParquetQuerySuite.read/write wide table`: This is not a good unit test as it runs more than 5 minutes. This PR removes it and add a new regression test in `CodeGenerationSuite`, which is more "unit". 2. `ParquetQuerySuite.returning batch for wide table`: reduce the threshold and use smaller data size. 3. `DatasetSuite.SPARK-14554: Dataset.map may generate wrong java code for wide table`: Improve `CodeFormatter.format`(introduced at https://github.com/apache/spark/pull/12979) can dramatically speed this it up. ## How was this patch tested? N/A Author: Wenchen Fan Closes #13273 from cloud-fan/test. --- .../expressions/codegen/CodeFormatter.scala | 48 +++++--- .../expressions/codegen/CodeGenerator.scala | 22 +--- .../expressions/CodeGenerationSuite.scala | 14 +++ .../codegen/CodeFormatterSuite.scala | 107 ++++++++++++------ .../parquet/ParquetQuerySuite.scala | 16 +-- 5 files changed, 128 insertions(+), 79 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index 855ae6432da28..05b7c96e44c02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import org.apache.commons.lang3.StringUtils +import java.util.regex.Matcher /** * An utility class that indents a block of code based on the curly braces and parentheses. @@ -26,13 +26,17 @@ import org.apache.commons.lang3.StringUtils * Written by Matei Zaharia. */ object CodeFormatter { + val commentHolder = """\/\*(.+?)\*\/""".r + def format(code: CodeAndComment): String = { - new CodeFormatter().addLines( - StringUtils.replaceEach( - code.body, - code.comment.keys.toArray, - code.comment.values.toArray) - ).result + val formatter = new CodeFormatter + code.body.split("\n").foreach { line => + val commentReplaced = commentHolder.replaceAllIn( + line.trim, + m => code.comment.get(m.group(1)).map(Matcher.quoteReplacement).getOrElse(m.group(0))) + formatter.addLine(commentReplaced) + } + formatter.result() } def stripExtraNewLines(input: String): String = { @@ -53,16 +57,28 @@ object CodeFormatter { def stripOverlappingComments(codeAndComment: CodeAndComment): CodeAndComment = { val code = new StringBuilder val map = codeAndComment.comment + + def getComment(line: String): Option[String] = { + if (line.startsWith("/*") && line.endsWith("*/")) { + map.get(line.substring(2, line.length - 2)) + } else { + None + } + } + var lastLine: String = "dummy" codeAndComment.body.split('\n').foreach { l => val line = l.trim() - val skip = lastLine.startsWith("/*") && lastLine.endsWith("*/") && - line.startsWith("/*") && line.endsWith("*/") && - map(lastLine).substring(3).contains(map(line).substring(3)) + + val skip = getComment(lastLine).zip(getComment(line)).exists { + case (lastComment, currentComment) => + lastComment.substring(3).contains(currentComment.substring(3)) + } + if (!skip) { - code.append(line) - code.append("\n") + code.append(line).append("\n") } + lastLine = line } new CodeAndComment(code.result().trim(), map) @@ -117,8 +133,9 @@ private class CodeFormatter { } else { indentString } - code.append(f"/* ${currentLine}%03d */ ") + code.append(f"/* ${currentLine}%03d */") if (line.trim().length > 0) { + code.append(" ") // add a space after the line number comment. code.append(thisLineIndent) if (inCommentBlock && line.startsWith("*") || line.startsWith("*/")) code.append(" ") code.append(line) @@ -129,10 +146,5 @@ private class CodeFormatter { currentLine += 1 } - private def addLines(code: String): CodeFormatter = { - code.split('\n').foreach(s => addLine(s.trim())) - this - } - private def result(): String = code.result() } 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 8b74d606dbb26..2706c38d9e0a1 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 @@ -717,28 +717,18 @@ class CodegenContext { */ def getPlaceHolderToComments(): collection.Map[String, String] = placeHolderToComments - /** - * Register a multi-line comment and return the corresponding place holder - */ - private def registerMultilineComment(text: String): String = { - val placeHolder = s"/*${freshName("c")}*/" - val comment = text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") - placeHolderToComments += (placeHolder -> comment) - placeHolder - } - /** * Register a comment and return the corresponding place holder */ def registerComment(text: String): String = { - if (text.contains("\n") || text.contains("\r")) { - registerMultilineComment(text) + val name = freshName("c") + val comment = if (text.contains("\n") || text.contains("\r")) { + text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") } else { - val placeHolder = s"/*${freshName("c")}*/" - val safeComment = s"// $text" - placeHolderToComments += (placeHolder -> safeComment) - placeHolder + s"// $text" } + placeHolderToComments += (name -> comment) + s"/*$name*/" } } 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 db34d12e286fa..8ffe39084c7f0 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 @@ -22,6 +22,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.objects.CreateExternalRow import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -137,6 +138,19 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("SPARK-14224: split wide external row creation into blocks due to JVM code size limit") { + val length = 5000 + val schema = StructType(Seq.fill(length)(StructField("int", IntegerType))) + val expressions = Seq(CreateExternalRow(Seq.fill(length)(Literal(1)), schema)) + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val expected = Seq(Row.fromSeq(Seq.fill(length)(1))) + + if (!checkResult(actual, expected)) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + } + } + test("test generated safe and unsafe projection") { val schema = new StructType(Array( StructField("a", StringType, true), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala index 76afc2e8ecbac..bc5a8f078234a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala @@ -23,9 +23,10 @@ import org.apache.spark.sql.catalyst.util._ class CodeFormatterSuite extends SparkFunSuite { - def testCase(name: String)(input: String)(expected: String): Unit = { + def testCase(name: String)( + input: String, comment: Map[String, String] = Map.empty)(expected: String): Unit = { test(name) { - val sourceCode = new CodeAndComment(input, Map.empty) + val sourceCode = new CodeAndComment(input.trim, comment) if (CodeFormatter.format(sourceCode).trim !== expected.trim) { fail( s""" @@ -43,9 +44,9 @@ class CodeFormatterSuite extends SparkFunSuite { |/*project_c2*/ """.stripMargin, Map( - "/*project_c4*/" -> "// (((input[0, bigint, false] + 1) + 2) + 3))", - "/*project_c3*/" -> "// ((input[0, bigint, false] + 1) + 2)", - "/*project_c2*/" -> "// (input[0, bigint, false] + 1)" + "project_c4" -> "// (((input[0, bigint, false] + 1) + 2) + 3))", + "project_c3" -> "// ((input[0, bigint, false] + 1) + 2)", + "project_c2" -> "// (input[0, bigint, false] + 1)" )) val reducedCode = CodeFormatter.stripOverlappingComments(code) @@ -53,9 +54,11 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("basic example") { - """class A { + """ + |class A { |blahblah; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ class A { @@ -65,11 +68,13 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("nested example") { - """class A { + """ + |class A { | if (c) { |duh; |} - |}""".stripMargin + |} + """.stripMargin } { """ |/* 001 */ class A { @@ -81,9 +86,11 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("single line") { - """class A { + """ + |class A { | if (c) {duh;} - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ class A { @@ -93,9 +100,11 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("if else on the same line") { - """class A { + """ + |class A { | if (c) {duh;} else {boo;} - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ class A { @@ -105,10 +114,12 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("function calls") { - """foo( + """ + |foo( |a, |b, - |c)""".stripMargin + |c) + """.stripMargin }{ """ |/* 001 */ foo( @@ -119,10 +130,12 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("single line comments") { - """// This is a comment about class A { { { ( ( + """ + |// This is a comment about class A { { { ( ( |class A { |class body; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ // This is a comment about class A { { { ( ( @@ -133,10 +146,12 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("single line comments /* */ ") { - """/** This is a comment about class A { { { ( ( */ + """ + |/** This is a comment about class A { { { ( ( */ |class A { |class body; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ /** This is a comment about class A { { { ( ( */ @@ -147,12 +162,14 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("multi-line comments") { - """ /* This is a comment about + """ + | /* This is a comment about |class A { |class body; ...*/ |class A { |class body; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ /* This is a comment about @@ -164,30 +181,56 @@ class CodeFormatterSuite extends SparkFunSuite { """.stripMargin } - // scalastyle:off whitespace.end.of.line testCase("reduce empty lines") { CodeFormatter.stripExtraNewLines( - """class A { + """ + |class A { | | - | /*** comment1 */ + | /* + | * multi + | * line + | * comment + | */ | | class body; | | | if (c) {duh;} | else {boo;} - |}""".stripMargin) + |} + """.stripMargin.trim) }{ """ |/* 001 */ class A { - |/* 002 */ /*** comment1 */ - |/* 003 */ class body; - |/* 004 */ - |/* 005 */ if (c) {duh;} - |/* 006 */ else {boo;} - |/* 007 */ } + |/* 002 */ /* + |/* 003 */ * multi + |/* 004 */ * line + |/* 005 */ * comment + |/* 006 */ */ + |/* 007 */ class body; + |/* 008 */ + |/* 009 */ if (c) {duh;} + |/* 010 */ else {boo;} + |/* 011 */ } + """.stripMargin + } + + testCase("comment place holder")( + """ + |/*c1*/ + |class A + |/*c2*/ + |class B + |/*c1*//*c2*/ + """.stripMargin, Map("c1" -> "/*abc*/", "c2" -> "/*xyz*/") + ) { + """ + |/* 001 */ /*abc*/ + |/* 002 */ class A + |/* 003 */ /*xyz*/ + |/* 004 */ class B + |/* 005 */ /*abc*//*xyz*/ """.stripMargin } - // scalastyle:on whitespace.end.of.line } 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 725e14c0fb2af..0a2fb0ef50661 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 @@ -581,21 +581,11 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext assert(CatalystReadSupport.expandUDT(schema) === expected) } - test("read/write wide table") { - withTempPath { dir => - val path = dir.getCanonicalPath - - val df = spark.range(1000).select(Seq.tabulate(1000) {i => ('id + i).as(s"c$i")} : _*) - df.write.mode(SaveMode.Overwrite).parquet(path) - checkAnswer(spark.read.parquet(path), df) - } - } - test("returning batch for wide table") { - withSQLConf("spark.sql.codegen.maxFields" -> "100") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "10") { withTempPath { dir => val path = dir.getCanonicalPath - val df = spark.range(100).select(Seq.tabulate(110) {i => ('id + i).as(s"c$i")} : _*) + val df = spark.range(10).select(Seq.tabulate(11) {i => ('id + i).as(s"c$i")} : _*) df.write.mode(SaveMode.Overwrite).parquet(path) // donot return batch, because whole stage codegen is disabled for wide table (>200 columns) @@ -605,7 +595,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext checkAnswer(df2, df) // return batch - val columns = Seq.tabulate(90) {i => s"c$i"} + val columns = Seq.tabulate(9) {i => s"c$i"} val df3 = df2.selectExpr(columns : _*) assert( df3.queryExecution.sparkPlan.find(_.isInstanceOf[BatchedDataSourceScanExec]).isDefined, From c9c1c0e54d34773ac2cf5457fe5925559ece36c7 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 24 May 2016 22:01:40 -0700 Subject: [PATCH 122/143] [SPARK-15508][STREAMING][TESTS] Fix flaky test: JavaKafkaStreamSuite.testKafkaStream ## What changes were proposed in this pull request? `JavaKafkaStreamSuite.testKafkaStream` assumes when `sent.size == result.size`, the contents of `sent` and `result` should be same. However, that's not true. The content of `result` may not be the final content. This PR modified the test to always retry the assertions even if the contents of `sent` and `result` are not same. Here is the failure in Jenkins: http://spark-tests.appspot.com/tests/org.apache.spark.streaming.kafka.JavaKafkaStreamSuite/testKafkaStream ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13281 from zsxwing/flaky-kafka-test. --- .../streaming/kafka/JavaKafkaStreamSuite.java | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 868df64e8c944..98fe38e826afb 100644 --- a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -122,14 +122,23 @@ public void call(JavaPairRDD rdd) { ssc.start(); long startTime = System.currentTimeMillis(); - boolean sizeMatches = false; - while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { - sizeMatches = sent.size() == result.size(); + AssertionError lastError = null; + while (System.currentTimeMillis() - startTime < 20000) { + try { + Assert.assertEquals(sent.size(), result.size()); + for (Map.Entry e : sent.entrySet()) { + Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); + } + return; + } catch (AssertionError e) { + lastError = e; + } Thread.sleep(200); } - Assert.assertEquals(sent.size(), result.size()); - for (Map.Entry e : sent.entrySet()) { - Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); + if (lastError != null) { + throw lastError; + } else { + Assert.fail("timeout"); } } } From cd9f16906cabd012b7676eb0f524e68a9cbe4db1 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 24 May 2016 22:20:00 -0700 Subject: [PATCH 123/143] [SPARK-15412][PYSPARK][SPARKR][DOCS] Improve linear isotonic regression pydoc & doc build insturctions ## What changes were proposed in this pull request? PySpark: Add links to the predictors from the models in regression.py, improve linear and isotonic pydoc in minor ways. User guide / R: Switch the installed package list to be enough to build the R docs on a "fresh" install on ubuntu and add sudo to match the rest of the commands. User Guide: Add a note about using gem2.0 for systems with both 1.9 and 2.0 (e.g. some ubuntu but maybe more). ## How was this patch tested? built pydocs locally, tested new user build instructions Author: Holden Karau Closes #13199 from holdenk/SPARK-15412-improve-linear-isotonic-regression-pydoc. --- docs/README.md | 4 +++- python/pyspark/ml/regression.py | 30 +++++++++++++++++------------- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/docs/README.md b/docs/README.md index bcea93e1f3b6d..8b515e187379c 100644 --- a/docs/README.md +++ b/docs/README.md @@ -20,8 +20,10 @@ installed. Also install the following libraries: $ sudo pip install Pygments # Following is needed only for generating API docs $ sudo pip install sphinx - $ Rscript -e 'install.packages(c("knitr", "devtools"), repos="http://cran.stat.ucla.edu/")' + $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "roxygen2", "testthat"), repos="http://cran.stat.ucla.edu/")' ``` +(Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0) + ## Generating the Documentation HTML We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 8f58594a66360..1b7af7ef597c2 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -48,11 +48,15 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction The learning objective is to minimize the squared error, with regularization. The specific squared error loss function used is: L = 1/2n ||A coefficients - y||^2^ - This support multiple types of regularization: - - none (a.k.a. ordinary least squares) - - L2 (ridge regression) - - L1 (Lasso) - - L2 + L1 (elastic net) + This supports multiple types of regularization: + + * none (a.k.a. ordinary least squares) + + * L2 (ridge regression) + + * L1 (Lasso) + + * L2 + L1 (elastic net) >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([ @@ -128,7 +132,7 @@ class LinearRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by LinearRegression. + Model fitted by :class:`LinearRegression`. .. versionadded:: 1.4.0 """ @@ -503,13 +507,13 @@ class IsotonicRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by IsotonicRegression. + Model fitted by :class:`IsotonicRegression`. """ @property def boundaries(self): """ - Model boundaries. + Boundaries in increasing order for which predictions are known. """ return self._call_java("boundaries") @@ -769,7 +773,7 @@ class DecisionTreeRegressionModel(DecisionTreeModel, JavaMLWritable, JavaMLReada """ .. note:: Experimental - Model fitted by DecisionTreeRegressor. + Model fitted by :class:`DecisionTreeRegressor`. .. versionadded:: 1.4.0 """ @@ -887,7 +891,7 @@ class RandomForestRegressionModel(TreeEnsembleModels, JavaMLWritable, JavaMLRead """ .. note:: Experimental - Model fitted by RandomForestRegressor. + Model fitted by :class:`RandomForestRegressor`. .. versionadded:: 1.4.0 """ @@ -1021,7 +1025,7 @@ class GBTRegressionModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by GBTRegressor. + Model fitted by :class:`GBTRegressor`. .. versionadded:: 1.4.0 """ @@ -1190,7 +1194,7 @@ class AFTSurvivalRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by AFTSurvivalRegression. + Model fitted by :class:`AFTSurvivalRegression`. .. versionadded:: 1.6.0 """ @@ -1380,7 +1384,7 @@ class GeneralizedLinearRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable """ .. note:: Experimental - Model fitted by GeneralizedLinearRegression. + Model fitted by :class:`GeneralizedLinearRegression`. .. versionadded:: 2.0.0 """ From 9082b7968ad952e05fc6f4feb499febef6aa45a7 Mon Sep 17 00:00:00 2001 From: Krishna Kalyan Date: Tue, 24 May 2016 22:21:52 -0700 Subject: [PATCH 124/143] [SPARK-12071][DOC] Document the behaviour of NA in R ## What changes were proposed in this pull request? Under Upgrading From SparkR 1.5.x to 1.6.x section added the information, SparkSQL converts `NA` in R to `null`. ## How was this patch tested? Document update, no tests. Author: Krishna Kalyan Closes #13268 from krishnakalyan3/spark-12071-1. --- docs/sparkr.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sparkr.md b/docs/sparkr.md index 9b5eaa1ec7232..6b2ca6d6a55f4 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -386,6 +386,7 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma ## Upgrading From SparkR 1.5.x to 1.6.x - Before Spark 1.6.0, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. + - SparkSQL converts `NA` in R to `null` and vice-versa. ## Upgrading From SparkR 1.6.x to 2.0 From 4f27b8dd58a66fca7ddd4c239e02b90c34b1cebd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 May 2016 19:17:53 +0200 Subject: [PATCH 125/143] [SPARK-15436][SQL] Remove DescribeFunction and ShowFunctions ## What changes were proposed in this pull request? This patch removes the last two commands defined in the catalyst module: DescribeFunction and ShowFunctions. They were unnecessary since the parser could just generate DescribeFunctionCommand and ShowFunctionsCommand directly. ## How was this patch tested? Created a new SparkSqlParserSuite. Author: Reynold Xin Closes #13292 from rxin/SPARK-15436. --- .../sql/catalyst/parser/AstBuilder.scala | 32 +-------- .../sql/catalyst/plans/logical/Command.scala | 25 +++++++ .../sql/catalyst/plans/logical/commands.scala | 55 --------------- .../analysis/UnsupportedOperationsSuite.scala | 9 ++- .../sql/catalyst/parser/PlanParserSuite.scala | 30 +++----- .../spark/sql/execution/SparkSqlParser.scala | 33 ++++++++- .../spark/sql/execution/SparkStrategies.scala | 6 -- .../sql/execution/SparkSqlParserSuite.scala | 68 +++++++++++++++++++ .../execution/command/DDLCommandSuite.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 6 +- 10 files changed, 145 insertions(+), 121 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala 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 a13c03a529f37..3473feec3209c 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 @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} @@ -81,37 +82,6 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * ******************************************************************************************** */ protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree) - /** - * Create a plan for a SHOW FUNCTIONS command. - */ - override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { - import ctx._ - if (qualifiedName != null) { - val name = visitFunctionName(qualifiedName) - ShowFunctions(name.database, Some(name.funcName)) - } else if (pattern != null) { - ShowFunctions(None, Some(string(pattern))) - } else { - ShowFunctions(None, None) - } - } - - /** - * Create a plan for a DESCRIBE FUNCTION command. - */ - override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) { - import ctx._ - val functionName = - if (describeFuncName.STRING() != null) { - FunctionIdentifier(string(describeFuncName.STRING()), database = None) - } else if (describeFuncName.qualifiedName() != null) { - visitFunctionName(describeFuncName.qualifiedName) - } else { - FunctionIdentifier(describeFuncName.getText, database = None) - } - DescribeFunction(functionName, EXTENDED != null) - } - /** * Create a top-level plan with Common Table Expressions. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala new file mode 100644 index 0000000000000..75a5b10d9ed04 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala @@ -0,0 +1,25 @@ +/* + * 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.plans.logical + +/** + * A logical node that represents a non-query command to be executed by the system. For example, + * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are + * eagerly executed. + */ +trait Command diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala deleted file mode 100644 index 0ec3ff3c25a00..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.plans.logical - -import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.types.StringType - -/** - * A logical node that represents a non-query command to be executed by the system. For example, - * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are - * eagerly executed. - */ -trait Command - -/** - * Returned for the "DESCRIBE FUNCTION [EXTENDED] functionName" command. - * - * @param functionName The function to be described. - * @param isExtended True if "DESCRIBE FUNCTION EXTENDED" is used. Otherwise, false. - */ -private[sql] case class DescribeFunction( - functionName: FunctionIdentifier, - isExtended: Boolean) extends LogicalPlan with Command { - - override def children: Seq[LogicalPlan] = Seq.empty - override val output: Seq[Attribute] = Seq( - AttributeReference("function_desc", StringType, nullable = false)()) -} - -/** - * Returned for the "SHOW FUNCTIONS" command, which will list all of the - * registered function list. - */ -private[sql] case class ShowFunctions( - db: Option[String], pattern: Option[String]) extends LogicalPlan with Command { - override def children: Seq[LogicalPlan] = Seq.empty - override val output: Seq[Attribute] = Seq( - AttributeReference("function", StringType, nullable = false)()) -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 674277bdbe15d..aaeee0f2a41c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -29,6 +28,12 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.IntegerType +/** A dummy command for testing unsupported operations. */ +case class DummyCommand() extends LogicalPlan with Command { + override def output: Seq[Attribute] = Nil + override def children: Seq[LogicalPlan] = Nil +} + class UnsupportedOperationsSuite extends SparkFunSuite { val attribute = AttributeReference("a", IntegerType, nullable = true)() @@ -70,7 +75,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite { // Commands assertNotSupportedInStreamingPlan( "commmands", - DescribeFunction(FunctionIdentifier("func", database = None), true), + DummyCommand(), outputMode = Append, expectedMsgs = "commands" :: Nil) 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 5811d32cd9e62..a6fad2d8a0398 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 @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.catalyst.parser import org.apache.spark.sql.Row @@ -24,17 +25,21 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.IntegerType - +/** + * Parser test cases for rules defined in [[CatalystSqlParser]] / [[AstBuilder]]. + * + * There is also SparkSqlParserSuite in sql/core module for parser rules defined in sql/core module. + */ class PlanParserSuite extends PlanTest { import CatalystSqlParser._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ - def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { + private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { comparePlans(parsePlan(sqlCommand), plan) } - def intercept(sqlCommand: String, messages: String*): Unit = { + private def intercept(sqlCommand: String, messages: String*): Unit = { val e = intercept[ParseException](parsePlan(sqlCommand)) messages.foreach { message => assert(e.message.contains(message)) @@ -53,25 +58,6 @@ class PlanParserSuite extends PlanTest { intercept("EXPLAIN formatted SELECT 1", "Unsupported SQL statement") } - test("show functions") { - assertEqual("show functions", ShowFunctions(None, None)) - 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", "Unsupported function name") - } - - test("describe function") { - assertEqual("describe function bar", - DescribeFunction(FunctionIdentifier("bar", database = None), isExtended = false)) - assertEqual("describe function extended bar", - DescribeFunction(FunctionIdentifier("bar", database = None), isExtended = true)) - assertEqual("describe function foo.bar", - DescribeFunction(FunctionIdentifier("bar", database = Option("foo")), isExtended = false)) - assertEqual("describe function extended f.bar", - DescribeFunction(FunctionIdentifier("bar", database = Option("f")), isExtended = true)) - } - test("set operations") { val a = table("a").select(star()) val b = table("b").select(star()) 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 f85d6062e8d35..57f534cd9eb30 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 @@ -24,7 +24,7 @@ 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 +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ @@ -493,6 +493,37 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { DescribeDatabaseCommand(ctx.identifier.getText, ctx.EXTENDED != null) } + /** + * Create a plan for a DESCRIBE FUNCTION command. + */ + override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) { + import ctx._ + val functionName = + if (describeFuncName.STRING() != null) { + FunctionIdentifier(string(describeFuncName.STRING()), database = None) + } else if (describeFuncName.qualifiedName() != null) { + visitFunctionName(describeFuncName.qualifiedName) + } else { + FunctionIdentifier(describeFuncName.getText, database = None) + } + DescribeFunctionCommand(functionName, EXTENDED != null) + } + + /** + * Create a plan for a SHOW FUNCTIONS command. + */ + override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { + import ctx._ + if (qualifiedName != null) { + val name = visitFunctionName(qualifiedName) + ShowFunctionsCommand(name.database, Some(name.funcName)) + } else if (pattern != null) { + ShowFunctionsCommand(None, Some(string(pattern))) + } else { + ShowFunctionsCommand(None, None) + } + } + /** * Create a [[CreateFunctionCommand]] command. * 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 c46cecc71f37e..e40525287a0a1 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 @@ -416,12 +416,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { c.child) ExecutedCommandExec(cmd) :: Nil - case logical.ShowFunctions(db, pattern) => - ExecutedCommandExec(ShowFunctionsCommand(db, pattern)) :: Nil - - case logical.DescribeFunction(function, extended) => - ExecutedCommandExec(DescribeFunctionCommand(function, extended)) :: Nil - case _ => Nil } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala new file mode 100644 index 0000000000000..e2858bb475401 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -0,0 +1,68 @@ +/* + * 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 org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.command.{DescribeFunctionCommand, ShowFunctionsCommand} +import org.apache.spark.sql.internal.SQLConf + +/** + * Parser test cases for rules defined in [[SparkSqlParser]]. + * + * See [[org.apache.spark.sql.catalyst.parser.PlanParserSuite]] for rules + * defined in the Catalyst module. + */ +class SparkSqlParserSuite extends PlanTest { + + private lazy val parser = new SparkSqlParser(new SQLConf) + + private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { + comparePlans(parser.parsePlan(sqlCommand), plan) + } + + private def intercept(sqlCommand: String, messages: String*): Unit = { + val e = intercept[ParseException](parser.parsePlan(sqlCommand)) + messages.foreach { message => + assert(e.message.contains(message)) + } + } + + test("show functions") { + assertEqual("show functions", ShowFunctionsCommand(None, None)) + assertEqual("show functions foo", ShowFunctionsCommand(None, Some("foo"))) + assertEqual("show functions foo.bar", ShowFunctionsCommand(Some("foo"), Some("bar"))) + assertEqual("show functions 'foo\\\\.*'", ShowFunctionsCommand(None, Some("foo\\.*"))) + intercept("show functions foo.bar.baz", "Unsupported function name") + } + + test("describe function") { + assertEqual("describe function bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = false)) + assertEqual("describe function extended bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = true)) + assertEqual("describe function foo.bar", + DescribeFunctionCommand( + FunctionIdentifier("bar", database = Option("foo")), isExtended = false)) + assertEqual("describe function extended f.bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = Option("f")), isExtended = true)) + } + +} 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 eab1f55712f6f..850fca585216d 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructType} // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { - private val parser = new SparkSqlParser(new SQLConf) + private lazy val parser = new SparkSqlParser(new SQLConf) private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { val e = intercept[ParseException] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index b12f3aafefb8b..65d53debd67cf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.SQLBuilder import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExplainCommand, SetCommand, ShowColumnsCommand} +import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable} import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} @@ -414,8 +414,8 @@ abstract class HiveComparisonTest // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && - (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && - (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive From b120fba6ae26186b3fa0dfbb1637046f4e76c2b0 Mon Sep 17 00:00:00 2001 From: Lukasz Date: Wed, 25 May 2016 10:24:21 -0700 Subject: [PATCH 126/143] [SPARK-9044] Fix "Storage" tab in UI so that it reflects RDD name change. ## What changes were proposed in this pull request? 1. Making 'name' field of RDDInfo mutable. 2. In StorageListener: catching the fact that RDD's name was changed and updating it in RDDInfo. ## How was this patch tested? 1. Manual verification - the 'Storage' tab now behaves as expected. 2. The commit also contains a new unit test which verifies this. Author: Lukasz Closes #13264 from lgieron/SPARK-9044. --- .../org/apache/spark/storage/RDDInfo.scala | 2 +- .../apache/spark/ui/storage/StorageTab.scala | 2 +- .../spark/ui/storage/StorageTabSuite.scala | 17 +++++++++++++++++ 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 083d78b59ebee..e5abbf745cc41 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.Utils @DeveloperApi class RDDInfo( val id: Int, - val name: String, + var name: String, val numPartitions: Int, var storageLevel: StorageLevel, val parentIds: Seq[Int], diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 50095831b4a53..c212362557be6 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -59,7 +59,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val rddInfos = stageSubmitted.stageInfo.rddInfos - rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } + rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info).name = info.name } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index 7d77deeb60618..411a0ddebeb77 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -179,6 +179,23 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { assert(storageListener.rddInfoList.size === 2) } + test("verify StorageTab still contains a renamed RDD") { + val rddInfo = new RDDInfo(0, "original_name", 1, memOnly, Seq(4)) + val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo), Seq.empty, "details") + bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + val blockUpdateInfos1 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(0, 1), memOnly, 100L, 0L)) + postUpdateBlocks(bus, blockUpdateInfos1) + assert(storageListener.rddInfoList.size == 1) + + val newName = "new_name" + val rddInfoRenamed = new RDDInfo(0, newName, 1, memOnly, Seq(4)) + val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfoRenamed), Seq.empty, "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) + assert(storageListener.rddInfoList.size == 1) + assert(storageListener.rddInfoList.head.name == newName) + } + private def postUpdateBlocks( bus: SparkListenerBus, blockUpdateInfos: Seq[BlockUpdatedInfo]): Unit = { blockUpdateInfos.foreach { blockUpdateInfo => From 01e7b9c85bb84924e279021f9748774dce9702c8 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Wed, 25 May 2016 10:46:51 -0700 Subject: [PATCH 127/143] [SPARK-15345][SQL][PYSPARK] SparkSession's conf doesn't take effect when this already an existing SparkContext ## What changes were proposed in this pull request? Override the existing SparkContext is the provided SparkConf is different. PySpark part hasn't been fixed yet, will do that after the first round of review to ensure this is the correct approach. ## How was this patch tested? Manually verify it in spark-shell. rxin Please help review it, I think this is a very critical issue for spark 2.0 Author: Jeff Zhang Closes #13160 from zjffdu/SPARK-15345. --- .../main/scala/org/apache/spark/SparkContext.scala | 3 +++ .../scala/org/apache/spark/sql/SparkSession.scala | 14 ++++++++++++-- .../spark/sql/SparkSessionBuilderSuite.scala | 14 +++++++++++++- 3 files changed, 28 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 36aa3becb4fc3..5018eb38d91c0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2254,6 +2254,9 @@ object SparkContext extends Logging { if (activeContext.get() == null) { setActiveContext(new SparkContext(config), allowMultipleContexts = false) } + if (config.getAll.nonEmpty) { + logWarning("Use an existing SparkContext, some configuration may not take effect.") + } activeContext.get() } } 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 5c87c844185c5..86c97b92dfc3d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -636,7 +636,7 @@ object SparkSession { /** * Builder for [[SparkSession]]. */ - class Builder { + class Builder extends Logging { private[this] val options = new scala.collection.mutable.HashMap[String, String] @@ -753,6 +753,9 @@ object SparkSession { var session = activeThreadSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } + if (options.nonEmpty) { + logWarning("Use an existing SparkSession, some configuration may not take effect.") + } return session } @@ -762,6 +765,9 @@ object SparkSession { session = defaultSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } + if (options.nonEmpty) { + logWarning("Use an existing SparkSession, some configuration may not take effect.") + } return session } @@ -774,7 +780,11 @@ object SparkSession { val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } - SparkContext.getOrCreate(sparkConf) + val sc = SparkContext.getOrCreate(sparkConf) + // maybe this is an existing SparkContext, update its SparkConf which maybe used + // by SparkSession + options.foreach { case (k, v) => sc.conf.set(k, v) } + sc } session = new SparkSession(sparkContext) options.foreach { case (k, v) => session.conf.set(k, v) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index ec6a2b3575869..786956df8a555 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} /** * Test cases for the builder pattern of [[SparkSession]]. @@ -90,4 +90,16 @@ class SparkSessionBuilderSuite extends SparkFunSuite { assert(newSession != activeSession) newSession.stop() } + + test("create SparkContext first then SparkSession") { + sparkContext.stop() + val conf = new SparkConf().setAppName("test").setMaster("local").set("key1", "value1") + val sparkContext2 = new SparkContext(conf) + val session = SparkSession.builder().config("key2", "value2").getOrCreate() + assert(session.conf.get("key1") == "value1") + assert(session.conf.get("key2") == "value2") + assert(session.sparkContext.conf.get("key1") == "value1") + assert(session.sparkContext.conf.get("key2") == "value2") + session.stop() + } } From 8239fdcb9b54ab6d13c31ad9916b8334dd1462c2 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 25 May 2016 10:49:11 -0700 Subject: [PATCH 128/143] [SPARK-15520][SQL] SparkSession builder in python should also allow overriding confs of existing sessions ## What changes were proposed in this pull request? This fixes the python SparkSession builder to allow setting confs correctly. This was a leftover TODO from https://github.com/apache/spark/pull/13200. ## How was this patch tested? Python doc tests. cc andrewor14 Author: Eric Liang Closes #13289 from ericl/spark-15520. --- python/pyspark/sql/session.py | 35 ++++++++++++++++++++++++----------- 1 file changed, 24 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 241947537fa29..52e7f3d348ab9 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -138,24 +138,37 @@ 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. - This method first checks whether there is a valid thread-local SparkSession, - and if yes, return that one. It then checks whether there is a valid global - default SparkSession, and if yes, return that one. If no valid global default - SparkSession exists, the method creates a new SparkSession and assigns the - newly created SparkSession as the global default. + This method first checks whether there is a valid global default SparkSession, and if + yes, return that one. If no valid global default SparkSession exists, the method + creates a new SparkSession and assigns the newly created SparkSession as the global + default. + + >>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() + >>> s1.conf.get("k1") == "v1" + True In case an existing SparkSession is returned, the config options specified in this builder will be applied to the existing SparkSession. + + >>> s2 = SparkSession.builder.config("k2", "v2").getOrCreate() + >>> s1.conf.get("k1") == s2.conf.get("k1") + True + >>> s1.conf.get("k2") == s2.conf.get("k2") + True """ with self._lock: - from pyspark.conf import SparkConf from pyspark.context import SparkContext - from pyspark.sql.context import SQLContext - sparkConf = SparkConf() + from pyspark.conf import SparkConf + session = SparkSession._instantiatedContext + if session is None: + sparkConf = SparkConf() + for key, value in self._options.items(): + sparkConf.set(key, value) + sc = SparkContext.getOrCreate(sparkConf) + session = SparkSession(sc) for key, value in self._options.items(): - sparkConf.set(key, value) - sparkContext = SparkContext.getOrCreate(sparkConf) - return SQLContext.getOrCreate(sparkContext).sparkSession + session.conf.set(key, value) + return session builder = Builder() From d6d3e50719b01005aa0e77349fc9a6ff88fecce3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 25 May 2016 10:51:33 -0700 Subject: [PATCH 129/143] [MINOR][CORE] Fix a HadoopRDD log message and remove unused imports in rdd files. ## What changes were proposed in this pull request? This PR fixes the following typos in log message and comments of `HadoopRDD.scala`. Also, this removes unused imports. ```scala - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + ... - // since its not removed yet + // since it's not removed yet ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13294 from dongjoon-hyun/minor_rdd_fix_log_message. --- core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 5 ++--- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 - .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala | 1 - 5 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index be0cb175f5340..41832e8354741 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.hadoop.conf.{ Configurable, Configuration } +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.JobContextImpl diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index b22134af45b30..515fd6f4e278c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -43,7 +43,6 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.Logging import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} @@ -70,7 +69,7 @@ private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: Inp val envVars: Map[String, String] = if (inputSplit.value.isInstanceOf[FileSplit]) { val is: FileSplit = inputSplit.value.asInstanceOf[FileSplit] // map_input_file is deprecated in favor of mapreduce_map_input_file but set both - // since its not removed yet + // since it's not removed yet Map("map_input_file" -> is.getPath().toString(), "mapreduce_map_input_file" -> is.getPath().toString()) } else { @@ -335,7 +334,7 @@ class HadoopRDD[K, V]( override def persist(storageLevel: StorageLevel): this.type = { if (storageLevel.deserialized) { - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + " Use a map transformation to make copies of the records.") } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index ad7c2216a042f..189dc7b331337 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.task.{JobContextImpl, TaskAttemptContextImpl} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.Logging import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.storage.StorageLevel 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 3b12448d63933..a7142376017d5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -40,7 +40,7 @@ import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} +import org.apache.spark.executor.OutputMetrics import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 45616856fd7a5..49625b7042d94 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -31,7 +31,6 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.util.control.NonFatal import org.apache.spark.{Partition, SparkEnv, TaskContext} import org.apache.spark.util.Utils From 02c8072eea72425e89256347e1f373a3e76e6eba Mon Sep 17 00:00:00 2001 From: lfzCarlosC Date: Wed, 25 May 2016 10:53:53 -0700 Subject: [PATCH 130/143] [MINOR][MLLIB][STREAMING][SQL] Fix typos fixed typos for source code for components [mllib] [streaming] and [SQL] None and obvious. Author: lfzCarlosC Closes #13298 from lfzCarlosC/master. --- mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala | 2 +- .../apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala | 2 +- .../sql/catalyst/plans/logical/basicLogicalOperators.scala | 2 +- .../apache/spark/sql/execution/vectorized/ColumnVector.java | 2 +- .../spark/sql/execution/streaming/state/StateStoreSuite.scala | 2 +- sql/hive-thriftserver/if/TCLIService.thrift | 4 ++-- .../org/apache/hive/service/ServiceStateChangeListener.java | 2 +- .../org/apache/hive/service/cli/operation/SQLOperation.java | 2 +- .../hive/service/cli/session/HiveSessionHookContext.java | 2 +- .../scala/org/apache/spark/sql/hive/HiveSessionState.scala | 2 +- .../spark/streaming/util/WriteAheadLogRecordHandle.java | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index 9457c6e9e35f2..bb4b37ef21a84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -204,7 +204,7 @@ private object IDFModel { * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector * * @param idf an IDF vector - * @param v a term frequence vector + * @param v a term frequency vector * @return a TF-IDF vector */ def transform(idf: Vector, v: Vector): Vector = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala index 9748fbf2c97b6..c3de5d75f4f7d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala @@ -45,7 +45,7 @@ import org.apache.spark.rdd.RDD * many elements are in each partition. Once these three values have been returned for every * partition, we can collect and operate locally. Locally, we can now adjust each distance by the * appropriate constant (the cumulative sum of number of elements in the prior partitions divided by - * thedata set size). Finally, we take the maximum absolute value, and this is the statistic. + * the data set size). Finally, we take the maximum absolute value, and this is the statistic. */ private[stat] object KolmogorovSmirnovTest extends Logging { 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 0a9250b71fb61..8b7e21b67982d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -591,7 +591,7 @@ case class Expand( } // This operator can reuse attributes (for example making them null when doing a roll up) so - // the contraints of the child may no longer be valid. + // the constraints of the child may no longer be valid. override protected def validConstraints: Set[Expression] = Set.empty[Expression] } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index e7dccd1b2261f..3f94255256699 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -504,7 +504,7 @@ public ColumnarBatch.Row getStruct(int rowId) { /** * Returns a utility object to get structs. - * provided to keep API compabilitity with InternalRow for code generation + * provided to keep API compatibility with InternalRow for code generation */ public ColumnarBatch.Row getStruct(int rowId, int size) { resultStruct.rowId = rowId; 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 f8f8bc7d6ff84..984b84fd13fbd 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 @@ -188,7 +188,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth provider.getStore(-1) } - // Prepare some data in the stoer + // Prepare some data in the store val store = provider.getStore(0) put(store, "a", 1) assert(store.commit() === 1) diff --git a/sql/hive-thriftserver/if/TCLIService.thrift b/sql/hive-thriftserver/if/TCLIService.thrift index baf583fb3ecd4..7cd6fa37cec37 100644 --- a/sql/hive-thriftserver/if/TCLIService.thrift +++ b/sql/hive-thriftserver/if/TCLIService.thrift @@ -661,7 +661,7 @@ union TGetInfoValue { // The function returns general information about the data source // using the same keys as ODBC. struct TGetInfoReq { - // The sesssion to run this request against + // The session to run this request against 1: required TSessionHandle sessionHandle 2: required TGetInfoType infoType @@ -1032,7 +1032,7 @@ enum TFetchOrientation { FETCH_PRIOR, // Return the rowset at the given fetch offset relative - // to the curren rowset. + // to the current rowset. // NOT SUPPORTED FETCH_RELATIVE, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java index 16ad9a991e921..d1aadad04cf67 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java @@ -29,7 +29,7 @@ public interface ServiceStateChangeListener { * have changed state before this callback is invoked. * * This operation is invoked on the thread that initiated the state change, - * while the service itself in in a sychronized section. + * while the service itself in in a synchronized section. *
      *
    1. Any long-lived operation here will prevent the service state * change from completing in a timely manner.
    2. 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 a35405484a765..5014cedd870b6 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 @@ -456,7 +456,7 @@ private SerDe getSerDe() throws SQLException { private HiveConf getConfigForOperation() throws HiveSQLException { HiveConf sqlOperationConf = getParentSession().getHiveConf(); if (!getConfOverlay().isEmpty() || shouldRunAsync()) { - // clone the partent session config for this query + // clone the parent session config for this query sqlOperationConf = new HiveConf(sqlOperationConf); // apply overlay query specific settings, if any 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 0a10dba8b479d..c56a107d42466 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 @@ -22,7 +22,7 @@ /** * HiveSessionHookContext. * Interface passed to the HiveServer2 session hook execution. This enables - * the hook implementation to accesss session config, user and session handle + * the hook implementation to access session config, user and session handle */ public interface HiveSessionHookContext { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 46579ecd85caa..081d85acb9084 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -143,7 +143,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) * converted to a data source table, using the data source set by spark.sql.sources.default. * The table in CTAS statement will be converted when it meets any of the following conditions: * - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or - * a Storage Hanlder (STORED BY), and the value of hive.default.fileformat in hive-site.xml + * a Storage Handler (STORED BY), and the value of hive.default.fileformat in hive-site.xml * is either TextFile or SequenceFile. * - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe * is specified (no ROW FORMAT SERDE clause). diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java index 662889e779fb2..3c5cc7e2cae1b 100644 --- a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java @@ -23,7 +23,7 @@ * This abstract class represents a handle that refers to a record written in a * {@link org.apache.spark.streaming.util.WriteAheadLog WriteAheadLog}. * It must contain all the information necessary for the record to be read and returned by - * an implemenation of the WriteAheadLog class. + * an implementation of the WriteAheadLog class. * * @see org.apache.spark.streaming.util.WriteAheadLog */ From 1cb347fbc446092b478ae0578fc7d1b0626a9294 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 25 May 2016 20:41:53 +0200 Subject: [PATCH 131/143] [SPARK-15500][DOC][ML][PYSPARK] Remove default value in Param doc field in ALS Remove "Default: MEMORY_AND_DISK" from `Param` doc field in ALS storage level params. This fixes up the output of `explainParam(s)` so that default values are not displayed twice. We can revisit in the case that [SPARK-15130](https://issues.apache.org/jira/browse/SPARK-15130) moves ahead with adding defaults in some way to PySpark param doc fields. Tests N/A. Author: Nick Pentreath Closes #13277 from MLnick/SPARK-15500-als-remove-default-storage-param. --- .../main/scala/org/apache/spark/ml/recommendation/ALS.scala | 4 ++-- python/pyspark/ml/recommendation.py | 6 ++---- 2 files changed, 4 insertions(+), 6 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 f257382d2205c..8dc7437d4747c 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 @@ -180,7 +180,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w * @group expertParam */ val intermediateStorageLevel = new Param[String](this, "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'. Default: 'MEMORY_AND_DISK'.", + "StorageLevel for intermediate datasets. Cannot be 'NONE'.", (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE") /** @group expertGetParam */ @@ -194,7 +194,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w * @group expertParam */ val finalStorageLevel = new Param[String](this, "finalStorageLevel", - "StorageLevel for ALS model factors. Default: 'MEMORY_AND_DISK'.", + "StorageLevel for ALS model factors.", (s: String) => Try(StorageLevel.fromString(s)).isSuccess) /** @group expertGetParam */ diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index bac2a3029fd71..1778bfe938b1e 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -120,12 +120,10 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha "whether to use nonnegative constraint for least squares", typeConverter=TypeConverters.toBoolean) intermediateStorageLevel = Param(Params._dummy(), "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'. " + - "Default: 'MEMORY_AND_DISK'.", + "StorageLevel for intermediate datasets. Cannot be 'NONE'.", typeConverter=TypeConverters.toString) finalStorageLevel = Param(Params._dummy(), "finalStorageLevel", - "StorageLevel for ALS model factors. " + - "Default: 'MEMORY_AND_DISK'.", + "StorageLevel for ALS model factors.", typeConverter=TypeConverters.toString) @keyword_only From 4b88067416ce922ae15a1445cf953fb9b5c43427 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 25 May 2016 12:02:07 -0700 Subject: [PATCH 132/143] [SPARK-15483][SQL] IncrementalExecution should use extra strategies. ## What changes were proposed in this pull request? Extra strategies does not work for streams because `IncrementalExecution` uses modified planner with stateful operations but it does not include extra strategies. This pr fixes `IncrementalExecution` to include extra strategies to use them. ## How was this patch tested? I added a test to check if extra strategies work for streams. Author: Takuya UESHIN Closes #13261 from ueshin/issues/SPARK-15483. --- .../streaming/IncrementalExecution.scala | 3 ++- .../apache/spark/sql/streaming/StreamSuite.scala | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) 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 8b96f65bc31ac..fe5f36e1cdeeb 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 @@ -36,7 +36,8 @@ class IncrementalExecution private[sql]( extends QueryExecution(sparkSession, logicalPlan) { // TODO: make this always part of planning. - val stateStrategy = sparkSession.sessionState.planner.StatefulAggregationStrategy :: Nil + val stateStrategy = sparkSession.sessionState.planner.StatefulAggregationStrategy +: + sparkSession.sessionState.experimentalMethods.extraStrategies // Modified planner with stateful operations. override def planner: SparkPlanner = 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 b742206b58137..ae89a6887a6d1 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 @@ -220,6 +220,21 @@ class StreamSuite extends StreamTest with SharedSQLContext { CheckOffsetLogLatestBatchId(2), CheckSinkLatestBatchId(2)) } + + test("insert an extraStrategy") { + try { + spark.experimental.extraStrategies = TestStrategy :: Nil + + val inputData = MemoryStream[(String, Int)] + val df = inputData.toDS().map(_._1).toDF("a") + + testStream(df)( + AddData(inputData, ("so slow", 1)), + CheckAnswer("so fast")) + } finally { + spark.experimental.extraStrategies = Nil + } + } } /** From c875d81a3de3f209b9eb03adf96b7c740b2c7b52 Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Wed, 25 May 2016 12:40:16 -0700 Subject: [PATCH 133/143] [SPARK-15493][SQL] default QuoteEscapingEnabled flag to true when writing CSV ## What changes were proposed in this pull request? Default QuoteEscapingEnabled flag to true when writing CSV and add an escapeQuotes option to be able to change this. See https://github.com/uniVocity/univocity-parsers/blob/f3eb2af26374940e60d91d1703bde54619f50c51/src/main/java/com/univocity/parsers/csv/CsvWriterSettings.java#L231-L247 This change is needed to be able to write RFC 4180 compatible CSV files (https://tools.ietf.org/html/rfc4180#section-2) https://issues.apache.org/jira/browse/SPARK-15493 ## How was this patch tested? Added a test that verifies the output is quoted correctly. Author: Jurriaan Pruis Closes #13267 from jurriaan/quote-escaping. --- 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/pyspark/sql/readwriter.py | 7 ++- sql/core/pom.xml | 2 +- .../apache/spark/sql/DataFrameWriter.scala | 3 ++ .../datasources/csv/CSVOptions.scala | 2 + .../execution/datasources/csv/CSVParser.scala | 1 + .../execution/datasources/csv/CSVSuite.scala | 51 +++++++++++++++++++ 11 files changed, 69 insertions(+), 7 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index a9068da8b2096..0ac1c00154838 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -159,7 +159,7 @@ 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.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 7e60a313ae8fe..fa35fa70516f3 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -167,7 +167,7 @@ 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.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 70d33b4f4812c..99dffa93bbcf7 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -167,7 +167,7 @@ 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.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index a80f6bc2a4061..a3bee36ce535a 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -175,7 +175,7 @@ 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.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index c0b53f73cd499..dbd7a8e0bf21c 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -176,7 +176,7 @@ 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.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 6f788cf50c7d7..73d2b81b6bd04 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -769,7 +769,7 @@ def text(self, path, compression=None): @since(2.0) def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, - header=None, nullValue=None): + header=None, nullValue=None, escapeQuotes=None): """Saves the content of the [[DataFrame]] in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -790,6 +790,9 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No 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 escapeQuotes: A flag indicating whether values containing quotes should always + be enclosed in quotes. If None is set, it uses the default value + ``true``, escaping all values containing a quote character. :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 @@ -810,6 +813,8 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No self.option("header", header) if nullValue is not None: self.option("nullValue", nullValue) + if escapeQuotes is not None: + self.option("escapeQuotes", nullValue) self._jwrite.csv(path) @since(1.5) diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 2ea980bf20dfe..b833b9369ec64 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -39,7 +39,7 @@ com.univocity univocity-parsers - 2.1.0 + 2.1.1 jar 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 6f5fb69ea3778..3aacce7d7f383 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 @@ -684,6 +684,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * the separator can be part of the value. *
    3. `escape` (default `\`): sets the single character used for escaping quotes inside * an already quoted value.
    4. + *
    5. `escapeQuotes` (default `true`): a flag indicating whether values containing + * quotes should always be enclosed in quotes. Default is to escape all values containing + * a quote character.
    6. *
    7. `header` (default `false`): writes the names of columns as the first line.
    8. *
    9. `nullValue` (default empty string): sets the string representation of a null value.
    10. *
    11. `compression` (default `null`): compression codec to use when saving to file. This can be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index e4fd09462fabd..9f4ce8358b045 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -111,6 +111,8 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str val maxCharsPerColumn = getInt("maxCharsPerColumn", 1000000) + val escapeQuotes = getBool("escapeQuotes", true) + val inputBufferSize = 128 val isCommentSet = this.comment != '\u0000' diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index 111995da9cbad..b06f12369dd0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -75,6 +75,7 @@ private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten writerSettings.setSkipEmptyLines(true) writerSettings.setQuoteAllFields(false) writerSettings.setHeaders(headers: _*) + writerSettings.setQuoteEscapingEnabled(params.escapeQuotes) private var buffer = new ByteArrayOutputStream() private var writer = new CsvWriter( 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 bae290776f6e1..ad7c05c12eb9a 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 @@ -366,6 +366,57 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } + test("save csv with quote escaping enabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escape", "\"") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "\"test \"\"quote\"\"\",123,\"it \"\"works\"\"!\",\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + + test("save csv with quote escaping disabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escapeQuotes", "false") + .option("escape", "\"") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "test \"quote\",123,it \"works\"!,\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + test("commented lines in CSV data") { val results = spark.read .format("csv") From 698ef762f80cf4c84bc7b7cf083aa97d44b87170 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 25 May 2016 13:57:25 -0700 Subject: [PATCH 134/143] [SPARK-14269][SCHEDULER] Eliminate unnecessary submitStage() call. ## What changes were proposed in this pull request? Currently a method `submitStage()` for waiting stages is called on every iteration of the event loop in `DAGScheduler` to submit all waiting stages, but most of them are not necessary because they are not related to Stage status. The case we should try to submit waiting stages is only when their parent stages are successfully completed. This elimination can improve `DAGScheduler` performance. ## How was this patch tested? Added some checks and other existing tests, and our projects. We have a project bottle-necked by `DAGScheduler`, having about 2000 stages. Before this patch the almost all execution time in `Driver` process was spent to process `submitStage()` of `dag-scheduler-event-loop` thread but after this patch the performance was improved as follows: | | total execution time | `dag-scheduler-event-loop` thread time | `submitStage()` | |--------|---------------------:|---------------------------------------:|----------------:| | Before | 760 sec | 710 sec | 667 sec | | After | 440 sec | 14 sec | 10 sec | Author: Takuya UESHIN Closes #12060 from ueshin/issues/SPARK-14269. --- .../apache/spark/scheduler/DAGScheduler.scala | 35 ++++++------------- .../spark/scheduler/DAGSchedulerSuite.scala | 7 ++++ 2 files changed, 17 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5291b663667ea..766e9797f9026 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -726,7 +726,6 @@ class DAGScheduler( reason = "as part of cancellation of all jobs")) activeJobs.clear() // These should already be empty by this point, jobIdToActiveJob.clear() // but just in case we lost track of some jobs... - submitWaitingStages() } /** @@ -752,23 +751,21 @@ class DAGScheduler( submitStage(stage) } } - submitWaitingStages() } /** * Check for waiting stages which are now eligible for resubmission. - * Ordinarily run on every iteration of the event loop. + * Submits stages that depend on the given parent stage. Called when the parent stage completes + * successfully. */ - private def submitWaitingStages() { - // TODO: We might want to run this less often, when we are sure that something has become - // runnable that wasn't before. - logTrace("Checking for newly runnable parent stages") + private def submitWaitingChildStages(parent: Stage) { + logTrace(s"Checking if any dependencies of $parent are now runnable") logTrace("running: " + runningStages) logTrace("waiting: " + waitingStages) logTrace("failed: " + failedStages) - val waitingStagesCopy = waitingStages.toArray - waitingStages.clear() - for (stage <- waitingStagesCopy.sortBy(_.firstJobId)) { + val childStages = waitingStages.filter(_.parents.contains(parent)).toArray + waitingStages --= childStages + for (stage <- childStages.sortBy(_.firstJobId)) { submitStage(stage) } } @@ -793,7 +790,6 @@ class DAGScheduler( } val jobIds = activeInGroup.map(_.jobId) jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) - submitWaitingStages() } private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { @@ -801,7 +797,6 @@ class DAGScheduler( // In that case, we wouldn't have the stage anymore in stageIdToStage. val stageAttemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1) listenerBus.post(SparkListenerTaskStart(task.stageId, stageAttemptId, taskInfo)) - submitWaitingStages() } private[scheduler] def handleTaskSetFailed( @@ -809,7 +804,6 @@ class DAGScheduler( reason: String, exception: Option[Throwable]): Unit = { stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason, exception) } - submitWaitingStages() } private[scheduler] def cleanUpAfterSchedulerStop() { @@ -832,7 +826,6 @@ class DAGScheduler( private[scheduler] def handleGetTaskResult(taskInfo: TaskInfo) { listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) - submitWaitingStages() } private[scheduler] def handleJobSubmitted(jobId: Int, @@ -871,8 +864,6 @@ class DAGScheduler( listenerBus.post( SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) submitStage(finalStage) - - submitWaitingStages() } private[scheduler] def handleMapStageSubmitted(jobId: Int, @@ -916,8 +907,6 @@ class DAGScheduler( if (finalStage.isAvailable) { markMapStageJobAsFinished(job, mapOutputTracker.getStatistics(dependency)) } - - submitWaitingStages() } /** Submits stage, but first recursively submits any missing parents. */ @@ -1073,6 +1062,8 @@ class DAGScheduler( s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})" } logDebug(debugString) + + submitWaitingChildStages(stage) } } @@ -1238,9 +1229,8 @@ class DAGScheduler( markMapStageJobAsFinished(job, stats) } } + submitWaitingChildStages(shuffleStage) } - - // Note: newly runnable stages will be submitted below when we submit waiting stages } } @@ -1315,7 +1305,6 @@ class DAGScheduler( // Unrecognized failure - also do nothing. If the task fails repeatedly, the TaskScheduler // will abort the job. } - submitWaitingStages() } /** @@ -1357,7 +1346,6 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } - submitWaitingStages() } private[scheduler] def handleExecutorAdded(execId: String, host: String) { @@ -1366,7 +1354,6 @@ class DAGScheduler( logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } - submitWaitingStages() } private[scheduler] def handleStageCancellation(stageId: Int) { @@ -1379,7 +1366,6 @@ class DAGScheduler( case None => logInfo("No active jobs to kill for Stage " + stageId) } - submitWaitingStages() } private[scheduler] def handleJobCancellation(jobId: Int, reason: String = "") { @@ -1389,7 +1375,6 @@ class DAGScheduler( failJobAndIndependentStages( jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason)) } - submitWaitingStages() } /** 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 088a476086217..fc75c9e022711 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -370,6 +370,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assert(mapStageB.parents === List(mapStageA)) assert(mapStageC.parents === List(mapStageA, mapStageB)) assert(finalStage.parents === List(mapStageC)) + + complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(1), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(3), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assertDataStructuresEmpty() } test("zero split job") { From 9c297df3d4d5fa4bbfdffdaad15f362586db384b Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Wed, 25 May 2016 14:29:14 -0700 Subject: [PATCH 135/143] [MINOR] [PYSPARK] [EXAMPLES] Changed examples to use SparkSession.sparkContext instead of _sc ## What changes were proposed in this pull request? Some PySpark examples need a SparkContext and get it by accessing _sc directly from the session. These examples should use the provided property `sparkContext` in `SparkSession` instead. ## How was this patch tested? Ran modified examples Author: Bryan Cutler Closes #13303 from BryanCutler/pyspark-session-sparkContext-MINOR. --- examples/src/main/python/als.py | 2 +- examples/src/main/python/avro_inputformat.py | 2 +- examples/src/main/python/parquet_inputformat.py | 2 +- examples/src/main/python/pi.py | 2 +- examples/src/main/python/transitive_closure.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 81562e20a927d..80290e7de9b06 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -67,7 +67,7 @@ def update(i, vec, mat, ratings): .appName("PythonALS")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index 3f65e8f79a65c..4422f9e7a9589 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -70,7 +70,7 @@ .appName("AvroKeyInputFormat")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext conf = None if len(sys.argv) == 3: diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index 2f09f4d5730e8..29a1ac274eccf 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -53,7 +53,7 @@ .appName("ParquetInputFormat")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext parquet_rdd = sc.newAPIHadoopFile( path, diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 5db03e4a21091..b39d710540b1b 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -32,7 +32,7 @@ .appName("PythonPi")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * partitions diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 37c41dcd03dc6..d88ea94e41c27 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -46,7 +46,7 @@ def generateGraph(): .appName("PythonTransitiveClosure")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 tc = sc.parallelize(generateGraph(), partitions).cache() From 589cce93c821ac28e9090a478f6e7465398b7c30 Mon Sep 17 00:00:00 2001 From: Gio Borje Date: Wed, 25 May 2016 16:52:31 -0500 Subject: [PATCH 136/143] Log warnings for numIterations * miniBatchFraction < 1.0 ## What changes were proposed in this pull request? Add a warning log for the case that `numIterations * miniBatchFraction <1.0` during gradient descent. If the product of those two numbers is less than `1.0`, then not all training examples will be used during optimization. To put this concretely, suppose that `numExamples = 100`, `miniBatchFraction = 0.2` and `numIterations = 3`. Then, 3 iterations will occur each sampling approximately 6 examples each. In the best case, each of the 6 examples are unique; hence 18/100 examples are used. This may be counter-intuitive to most users and led to the issue during the development of another Spark ML model: https://github.com/zhengruifeng/spark-libFM/issues/11. If a user actually does not require the training data set, it would be easier and more intuitive to use `RDD.sample`. ## How was this patch tested? `build/mvn -DskipTests clean package` build succeeds Author: Gio Borje Closes #13265 from Hydrotoast/master. --- .../apache/spark/mllib/optimization/GradientDescent.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index a67ea836e5681..735e780909ef2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -197,6 +197,11 @@ object GradientDescent extends Logging { "< 1.0 can be unstable because of the stochasticity in sampling.") } + if (numIterations * miniBatchFraction < 1.0) { + logWarning("Not all examples will be used if numIterations * miniBatchFraction < 1.0: " + + s"numIterations=$numIterations and miniBatchFraction=$miniBatchFraction") + } + val stochasticLossHistory = new ArrayBuffer[Double](numIterations) // Record previous weight and current one to calculate solution vector difference From ee682fe293b47988056b540ee46ca49861309982 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 May 2016 15:08:39 -0700 Subject: [PATCH 137/143] [SPARK-15534][SPARK-15535][SQL] Truncate table fixes ## What changes were proposed in this pull request? Two changes: - When things fail, `TRUNCATE TABLE` just returns nothing. Instead, we should throw exceptions. - Remove `TRUNCATE TABLE ... COLUMN`, which was never supported by either Spark or Hive. ## How was this patch tested? Jenkins. Author: Andrew Or Closes #13302 from andrewor14/truncate-table. --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 3 +-- .../apache/spark/sql/execution/SparkSqlParser.scala | 7 +------ .../apache/spark/sql/execution/command/tables.scala | 7 ++++--- .../spark/sql/hive/execution/HiveCommandSuite.scala | 12 ------------ 4 files changed, 6 insertions(+), 23 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 403191af5e5be..b0e71c7e7c7d1 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -115,8 +115,7 @@ statement | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData - | TRUNCATE TABLE tableIdentifier partitionSpec? - (COLUMNS identifierList)? #truncateTable + | TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration 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 57f534cd9eb30..cfebfc6a5ce71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -368,17 +368,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * For example: * {{{ * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] - * [COLUMNS (col1, col2)] * }}} */ override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { - if (ctx.identifierList != null) { - throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) - } TruncateTableCommand( visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ) + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } /** 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 13e63a1befb20..bef4c9222c296 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -278,7 +278,7 @@ case class LoadDataCommand( * * The syntax of this command is: * {{{ - * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ case class TruncateTableCommand( @@ -288,9 +288,10 @@ case class TruncateTableCommand( 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.") + throw new AnalysisException(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.") + throw new AnalysisException( + s"Operation not allowed: TRUNCATE TABLE on temporary tables: '$tableName'") } else { val locations = if (partitionSpec.isDefined) { catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) 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 df62ba08b8018..6f374d713bfc8 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 @@ -289,10 +289,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto 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]) @@ -320,10 +316,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto 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'"), @@ -332,10 +324,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto 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'"), From 527499b624e743583fe0f93ea0b487031891ac3a Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 25 May 2016 15:35:38 -0700 Subject: [PATCH 138/143] [SPARK-15525][SQL][BUILD] Upgrade ANTLR4 SBT plugin ## What changes were proposed in this pull request? The ANTLR4 SBT plugin has been moved from its own repo to one on bintray. The version was also changed from `0.7.10` to `0.7.11`. The latter actually broke our build (ihji has fixed this by also adding `0.7.10` and others to the bin-tray repo). This PR upgrades the SBT-ANTLR4 plugin and ANTLR4 to their most recent versions (`0.7.11`/`4.5.3`). I have also removed a few obsolete build configurations. ## How was this patch tested? Manually running SBT/Maven builds. Author: Herman van Hovell Closes #13299 from hvanhovell/SPARK-15525. --- 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 +- pom.xml | 7 +------ project/plugins.sbt | 5 +---- 7 files changed, 7 insertions(+), 15 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 0ac1c00154838..0d6b18e8b72d8 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -3,7 +3,7 @@ RoaringBitmap-0.5.11.jar ST4-4.0.4.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index fa35fa70516f3..a3597f4ec70e5 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 99dffa93bbcf7..3ca44c5abc8f0 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index a3bee36ce535a..01e75517081d5 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index dbd7a8e0bf21c..402fd05376ee5 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/pom.xml b/pom.xml index f28aa14adc55f..3fa0eeb5f0387 100644 --- a/pom.xml +++ b/pom.xml @@ -177,7 +177,7 @@ 3.5.2 1.3.9 0.9.2 - 4.5.2-1 + 4.5.3 1.1 2.52.0 2.8 @@ -1952,11 +1952,6 @@ - - org.antlr - antlr3-maven-plugin - 3.5.2 - org.antlr antlr4-maven-plugin diff --git a/project/plugins.sbt b/project/plugins.sbt index 44ec3a12ae709..4578b56247aa9 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -20,7 +20,4 @@ libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3" libraryDependencies += "org.ow2.asm" % "asm-commons" % "5.0.3" -// TODO I am not sure we want such a dep. -resolvers += "simplytyped" at "http://simplytyped.github.io/repo/releases" - -addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.10") +addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.11") From 06ed1fa3e45adfc11b0f615cb8b97c99fadc735f Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Wed, 25 May 2016 19:10:57 -0700 Subject: [PATCH 139/143] [SPARK-15533][SQL] Deprecate Dataset.explode ## What changes were proposed in this pull request? This patch deprecates `Dataset.explode` and documents appropriate workarounds to use `flatMap()` or `functions.explode()` instead. ## How was this patch tested? N/A Author: Sameer Agarwal Closes #13312 from sameeragarwal/deprecate. --- .../scala/org/apache/spark/sql/Dataset.scala | 33 ++++++++++++------- 1 file changed, 22 insertions(+), 11 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 78a167eef2e4e..e5140fcf1337e 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 @@ -1556,30 +1556,33 @@ class Dataset[T] private[sql]( } /** - * :: Experimental :: * (Scala-specific) Returns a new [[Dataset]] where each row has been expanded to zero or more * rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of * the input row are implicitly joined with each row that is output by the function. * - * The following example uses this function to count the number of books which contain - * a given word: + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()` or `flatMap()`. The following example uses these alternatives to count + * the number of books that contain a given word: * * {{{ * case class Book(title: String, words: String) * val ds: Dataset[Book] * - * case class Word(word: String) - * val allWords = ds.explode('words) { - * case Row(words: String) => words.split(" ").map(Word(_)) - * } + * val allWords = ds.select('title, explode(split('words, " ")).as("word")) * * val bookCountPerWord = allWords.groupBy("word").agg(countDistinct("title")) * }}} * + * Using `flatMap()` this can similarly be exploded as: + * + * {{{ + * ds.flatMap(_.words.split(" ")) + * }}} + * * @group untypedrel * @since 2.0.0 */ - @Experimental + @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { val elementSchema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] @@ -1596,19 +1599,27 @@ class Dataset[T] private[sql]( } /** - * :: Experimental :: * (Scala-specific) Returns a new [[Dataset]] where a single column has been expanded to zero * or more rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. All * columns of the input row are implicitly joined with each value that is output by the function. * + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()`: + * + * {{{ + * ds.select(explode(split('words, " ")).as("word")) + * }}} + * + * or `flatMap()`: + * * {{{ - * ds.explode("words", "word") {words: String => words.split(" ")} + * ds.flatMap(_.words.split(" ")) * }}} * * @group untypedrel * @since 2.0.0 */ - @Experimental + @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") def explode[A, B : TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) : DataFrame = { val dataType = ScalaReflection.schemaFor[B].dataType From 06bae8af17d9478c889d206a4556a697b5d629e7 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Wed, 25 May 2016 21:08:03 -0700 Subject: [PATCH 140/143] [SPARK-15439][SPARKR] Failed to run unit test in SparkR ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) There are some failures when running SparkR unit tests. In this PR, I fixed two of these failures in test_context.R and test_sparkSQL.R The first one is due to different masked name. I added missed names in the expected arrays. The second one is because one PR removed the logic of a previous fix of missing subset method. The file privilege issue is still there. I am debugging it. SparkR shell can run the test case successfully. test_that("pipeRDD() on RDDs", { actual <- collect(pipeRDD(rdd, "more")) When using run-test script, it complains no such directories as below: cannot open file '/tmp/Rtmp4FQbah/filee2273f9d47f7': No such file or directory ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually test it Author: wm624@hotmail.com Closes #13284 from wangmiao1981/R. --- R/pkg/R/DataFrame.R | 6 +++++- R/pkg/inst/tests/testthat/test_context.R | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 0c2a194483b0f..f719173607fd9 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1445,7 +1445,11 @@ setMethod("[", signature(x = "SparkDataFrame"), #' } setMethod("subset", signature(x = "SparkDataFrame"), function(x, subset, select, drop = F, ...) { - x[subset, select, drop = drop] + if (missing(subset)) { + x[, select, drop = drop, ...] + } else { + x[subset, select, drop = drop, ...] + } }) #' Select diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 0e5e15c0a96c9..95258babbf101 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -27,6 +27,11 @@ test_that("Check masked functions", { namesOfMasked <- c("describe", "cov", "filter", "lag", "na.omit", "predict", "sd", "var", "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", "summary", "transform", "drop", "window", "as.data.frame") + namesOfMaskedCompletely <- c("cov", "filter", "sample") + if (as.numeric(R.version$major) == 3 && as.numeric(R.version$minor) > 2) { + namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) + namesOfMaskedCompletely <- c("endsWith", "startsWith", namesOfMaskedCompletely) + } expect_equal(length(maskedBySparkR), length(namesOfMasked)) expect_equal(sort(maskedBySparkR), sort(namesOfMasked)) # above are those reported as masked when `library(SparkR)` @@ -36,7 +41,6 @@ test_that("Check masked functions", { any(grepl("=\"ANY\"", capture.output(showMethods(x)[-1]))) })) maskedCompletely <- masked[!funcHasAny] - namesOfMaskedCompletely <- c("cov", "filter", "sample") expect_equal(length(maskedCompletely), length(namesOfMaskedCompletely)) expect_equal(sort(maskedCompletely), sort(namesOfMaskedCompletely)) }) From dfc9fc02ccbceb09213c394177d54b9ca56b6f24 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 26 May 2016 00:29:09 -0500 Subject: [PATCH 141/143] [SPARK-10372] [CORE] basic test framework for entire spark scheduler This is a basic framework for testing the entire scheduler. The tests this adds aren't very interesting -- the point of this PR is just to setup the framework, to keep the initial change small, but it can be built upon to test more features (eg., speculation, killing tasks, blacklisting, etc.). Author: Imran Rashid Closes #8559 from squito/SPARK-10372-scheduler-integs. --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/TaskResult.scala | 2 +- ...che.spark.scheduler.ExternalClusterManager | 3 +- .../scheduler/BlacklistIntegrationSuite.scala | 130 ++++ .../spark/scheduler/DAGSchedulerSuite.scala | 15 +- .../ExternalClusterManagerSuite.scala | 25 +- .../scheduler/SchedulerIntegrationSuite.scala | 564 ++++++++++++++++++ 7 files changed, 728 insertions(+), 13 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 766e9797f9026..a2eadbcbd660a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1396,7 +1396,7 @@ class DAGScheduler( stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) - logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) + logInfo(s"$stage (${stage.name}) failed in $serviceTime s due to ${errorMessage.get}") } outputCommitCoordinator.stageEnd(stage.id) 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 80f2bf41224b5..77fda6fcff959 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -59,7 +59,7 @@ private[spark] class DirectTaskResult[T]( val numUpdates = in.readInt if (numUpdates == 0) { - accumUpdates = null + accumUpdates = Seq() } else { val _accumUpdates = new ArrayBuffer[AccumulatorV2[_, _]] for (i <- 0 until numUpdates) { diff --git a/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager index 3c570ffd8f566..757c6d2296aff 100644 --- a/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager +++ b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -1 +1,2 @@ -org.apache.spark.scheduler.DummyExternalClusterManager \ No newline at end of file +org.apache.spark.scheduler.DummyExternalClusterManager +org.apache.spark.scheduler.MockExternalClusterManager \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala new file mode 100644 index 0000000000000..6c9d4fb6f3bcc --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -0,0 +1,130 @@ +/* + * 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.scheduler + +import scala.concurrent.Await +import scala.concurrent.duration._ + +import org.apache.spark._ + +class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ + + val badHost = "host-0" + + /** + * This backend just always fails if the task is executed on a bad host, but otherwise succeeds + * all tasks. + */ + def badHostBackend(): Unit = { + val task = backend.beginTask() + val host = backend.executorIdToExecutor(task.executorId).host + if (host == badHost) { + backend.taskFailed(task, new RuntimeException("I'm a bad host!")) + } else { + backend.taskSuccess(task, 42) + } + } + + // Test demonstrating the issue -- without a config change, the scheduler keeps scheduling + // according to locality preferences, and so the job fails + testScheduler("If preferred node is bad, without blacklist job will fail") { + val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) + withBackend(badHostBackend _) { + val jobFuture = submit(rdd, (0 until 10).toArray) + val duration = Duration(1, SECONDS) + Await.ready(jobFuture, duration) + } + assert(results.isEmpty) + assertDataStructuresEmpty(noFailure = false) + } + + // even with the blacklist turned on, if maxTaskFailures is not more than the number + // of executors on the bad node, then locality preferences will lead to us cycling through + // the executors on the bad node, and still failing the job + testScheduler( + "With blacklist on, job will still fail if there are too many bad executors on bad host", + extraConfs = Seq( + // just set this to something much longer than the test duration + ("spark.scheduler.executorTaskBlacklistTime", "10000000") + ) + ) { + val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) + withBackend(badHostBackend _) { + val jobFuture = submit(rdd, (0 until 10).toArray) + val duration = Duration(3, SECONDS) + Await.ready(jobFuture, duration) + } + assert(results.isEmpty) + assertDataStructuresEmpty(noFailure = false) + } + + // Here we run with the blacklist on, and maxTaskFailures high enough that we'll eventually + // schedule on a good node and succeed the job + testScheduler( + "Bad node with multiple executors, job will still succeed with the right confs", + extraConfs = Seq( + // just set this to something much longer than the test duration + ("spark.scheduler.executorTaskBlacklistTime", "10000000"), + // this has to be higher than the number of executors on the bad host + ("spark.task.maxFailures", "5"), + // just to avoid this test taking too long + ("spark.locality.wait", "10ms") + ) + ) { + val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) + withBackend(badHostBackend _) { + val jobFuture = submit(rdd, (0 until 10).toArray) + val duration = Duration(1, SECONDS) + Await.ready(jobFuture, duration) + } + assert(results === (0 until 10).map { _ -> 42 }.toMap) + assertDataStructuresEmpty(noFailure = true) + } + +} + +class MultiExecutorMockBackend( + conf: SparkConf, + taskScheduler: TaskSchedulerImpl) extends MockBackend(conf, taskScheduler) { + + val nHosts = conf.getInt("spark.testing.nHosts", 5) + val nExecutorsPerHost = conf.getInt("spark.testing.nExecutorsPerHost", 4) + val nCoresPerExecutor = conf.getInt("spark.testing.nCoresPerExecutor", 2) + + override val executorIdToExecutor: Map[String, ExecutorTaskStatus] = { + (0 until nHosts).flatMap { hostIdx => + val hostName = "host-" + hostIdx + (0 until nExecutorsPerHost).map { subIdx => + val executorId = (hostIdx * nExecutorsPerHost + subIdx).toString + executorId -> + ExecutorTaskStatus(host = hostName, executorId = executorId, nCoresPerExecutor) + } + }.toMap + } + + override def defaultParallelism(): Int = nHosts * nExecutorsPerHost * nCoresPerExecutor +} + +class MockRDDWithLocalityPrefs( + sc: SparkContext, + numPartitions: Int, + shuffleDeps: Seq[ShuffleDependency[Int, Int, Nothing]], + val preferredLoc: String) extends MockRDD(sc, numPartitions, shuffleDeps) { + override def getPreferredLocations(split: Partition): Seq[String] = { + Seq(preferredLoc) + } +} 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 fc75c9e022711..b6765f064524f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -98,6 +98,8 @@ class DAGSchedulerSuiteDummyException extends Exception class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeouts { + import DAGSchedulerSuite._ + val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() @@ -2034,12 +2036,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou } } - private def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = - MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) - - private def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) - private def assertDataStructuresEmpty(): Unit = { assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) @@ -2079,5 +2075,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou } CompletionEvent(task, reason, result, accumUpdates ++ extraAccumUpdates, taskInfo) } +} + +object DAGSchedulerSuite { + def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) + def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) } 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 59c1b359a780a..e87cebf0cf358 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -27,12 +27,24 @@ class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { val conf = new SparkConf().setMaster("myclusterManager"). setAppName("testcm").set("spark.driver.allowMultipleContexts", "true") sc = new SparkContext(conf) - // check if the scheduler components are created - assert(sc.schedulerBackend.isInstanceOf[DummySchedulerBackend]) - assert(sc.taskScheduler.isInstanceOf[DummyTaskScheduler]) + // check if the scheduler components are created and initialized + sc.schedulerBackend match { + case dummy: DummySchedulerBackend => assert(dummy.initialized) + case other => fail(s"wrong scheduler backend: ${other}") + } + sc.taskScheduler match { + case dummy: DummyTaskScheduler => assert(dummy.initialized) + case other => fail(s"wrong task scheduler: ${other}") + } } } +/** + * Super basic ExternalClusterManager, just to verify ExternalClusterManagers can be configured. + * + * Note that if you want a special ClusterManager for tests, you are probably much more interested + * in [[MockExternalClusterManager]] and the corresponding [[SchedulerIntegrationSuite]] + */ private class DummyExternalClusterManager extends ExternalClusterManager { def canCreate(masterURL: String): Boolean = masterURL == "myclusterManager" @@ -44,11 +56,15 @@ private class DummyExternalClusterManager extends ExternalClusterManager { masterURL: String, scheduler: TaskScheduler): SchedulerBackend = new DummySchedulerBackend() - def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {} + def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[DummyTaskScheduler].initialized = true + backend.asInstanceOf[DummySchedulerBackend].initialized = true + } } private class DummySchedulerBackend extends SchedulerBackend { + var initialized = false def start() {} def stop() {} def reviveOffers() {} @@ -56,6 +72,7 @@ private class DummySchedulerBackend extends SchedulerBackend { } private class DummyTaskScheduler extends TaskScheduler { + var initialized = false override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start(): Unit = {} diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala new file mode 100644 index 0000000000000..02aa5caa731ff --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -0,0 +1,564 @@ +/* + * 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.scheduler + +import java.util.Properties +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.{Duration, SECONDS} +import scala.reflect.ClassTag + +import org.scalactic.TripleEquals +import org.scalatest.Assertions.AssertionsHelper + +import org.apache.spark._ +import org.apache.spark.TaskState._ +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.util.{CallSite, Utils} + +/** + * Tests for the entire scheduler code -- DAGScheduler, TaskSchedulerImpl, TaskSets, + * TaskSetManagers. + * + * Test cases are configured by providing a set of jobs to submit, and then simulating interaction + * with spark's executors via a mocked backend (eg., task completion, task failure, executors + * disconnecting, etc.). + */ +abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends SparkFunSuite + with LocalSparkContext { + + var taskScheduler: TestTaskScheduler = null + var scheduler: DAGScheduler = null + var backend: T = _ + + override def beforeEach(): Unit = { + if (taskScheduler != null) { + taskScheduler.runningTaskSets.clear() + } + results.clear() + failure = null + super.beforeEach() + } + + override def afterEach(): Unit = { + super.afterEach() + taskScheduler.stop() + backend.stop() + scheduler.stop() + } + + def setupScheduler(conf: SparkConf): Unit = { + conf.setAppName(this.getClass().getSimpleName()) + val backendClassName = implicitly[ClassTag[T]].runtimeClass.getName() + conf.setMaster(s"mock[${backendClassName}]") + sc = new SparkContext(conf) + backend = sc.schedulerBackend.asInstanceOf[T] + taskScheduler = sc.taskScheduler.asInstanceOf[TestTaskScheduler] + taskScheduler.initialize(sc.schedulerBackend) + scheduler = new DAGScheduler(sc, taskScheduler) + taskScheduler.setDAGScheduler(scheduler) + } + + def testScheduler(name: String)(testBody: => Unit): Unit = { + testScheduler(name, Seq())(testBody) + } + + def testScheduler(name: String, extraConfs: Seq[(String, String)])(testBody: => Unit): Unit = { + test(name) { + val conf = new SparkConf() + extraConfs.foreach{ case (k, v) => conf.set(k, v)} + setupScheduler(conf) + testBody + } + } + + val results = new HashMap[Int, Any]() + var failure: Throwable = _ + + /** + * When we submit dummy Jobs, this is the compute function we supply. + */ + private val jobComputeFunc: (TaskContext, scala.Iterator[_]) => Any = { + (context: TaskContext, it: Iterator[(_)]) => + throw new RuntimeException("jobComputeFunc shouldn't get called in this mock") + } + + /** Submits a job to the scheduler, and returns a future which does a bit of error handling. */ + protected def submit( + rdd: RDD[_], + partitions: Array[Int], + func: (TaskContext, Iterator[_]) => _ = jobComputeFunc): Future[Any] = { + val waiter: JobWaiter[Any] = scheduler.submitJob(rdd, func, partitions.toSeq, CallSite("", ""), + (index, res) => results(index) = res, new Properties()) + import scala.concurrent.ExecutionContext.Implicits.global + waiter.completionFuture.recover { case ex => + failure = ex + } + } + + protected def assertDataStructuresEmpty(noFailure: Boolean = true): Unit = { + if (noFailure) { + if (failure != null) { + // if there is a job failure, it can be a bit hard to tease the job failure msg apart + // from the test failure msg, so we do a little extra formatting + val msg = + raw""" + | There was a failed job. + | ----- Begin Job Failure Msg ----- + | ${Utils.exceptionString(failure)} + | ----- End Job Failure Msg ---- + """. + stripMargin + fail(msg) + } + // When a job fails, we terminate before waiting for all the task end events to come in, + // so there might still be a running task set. So we only check these conditions + // when the job succeeds + assert(taskScheduler.runningTaskSets.isEmpty) + assert(!backend.hasTasks) + } + assert(scheduler.activeJobs.isEmpty) + } + + /** + * Looks at all shuffleMapOutputs that are dependencies of the given RDD, and makes sure + * they are all registered + */ + def assertMapOutputAvailable(targetRdd: MockRDD): Unit = { + val shuffleIds = targetRdd.shuffleDeps.map{_.shuffleId} + val nParts = targetRdd.numPartitions + for { + shuffleId <- shuffleIds + reduceIdx <- (0 until nParts) + } { + val statuses = taskScheduler.mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceIdx) + // really we should have already thrown an exception rather than fail either of these + // asserts, but just to be extra defensive let's double check the statuses are OK + assert(statuses != null) + assert(statuses.nonEmpty) + } + } + + /** models a stage boundary with a single dependency, like a shuffle */ + def shuffle(nParts: Int, input: MockRDD): MockRDD = { + val partitioner = new HashPartitioner(nParts) + val shuffleDep = new ShuffleDependency[Int, Int, Nothing](input, partitioner) + new MockRDD(sc, nParts, List(shuffleDep)) + } + + /** models a stage boundary with multiple dependencies, like a join */ + def join(nParts: Int, inputs: MockRDD*): MockRDD = { + val partitioner = new HashPartitioner(nParts) + val shuffleDeps = inputs.map { inputRDD => + new ShuffleDependency[Int, Int, Nothing](inputRDD, partitioner) + } + new MockRDD(sc, nParts, shuffleDeps) + } + + /** + * Helper which makes it a little easier to setup a test, which starts a mock backend in another + * thread, responding to tasks with your custom function. You also supply the "body" of your + * test, where you submit jobs to your backend, wait for them to complete, then check + * whatever conditions you want. Note that this is *not* safe to all bad backends -- + * in particular, your `backendFunc` has to return quickly, it can't throw errors, (instead + * it should send back the right TaskEndReason) + */ + def withBackend[T](backendFunc: () => Unit)(testBody: => T): T = { + val backendContinue = new AtomicBoolean(true) + val backendThread = new Thread("mock backend thread") { + override def run(): Unit = { + while (backendContinue.get()) { + if (backend.hasTasksWaitingToRun) { + backendFunc() + } else { + Thread.sleep(10) + } + } + } + } + try { + backendThread.start() + testBody + } finally { + backendContinue.set(false) + backendThread.join() + } + } + +} + +/** + * Helper for running a backend in integration tests, does a bunch of the book-keeping + * so individual tests can focus on just responding to tasks. Individual tests will use + * [[beginTask]], [[taskSuccess]], and [[taskFailed]]. + */ +private[spark] abstract class MockBackend( + conf: SparkConf, + val taskScheduler: TaskSchedulerImpl) extends SchedulerBackend with Logging { + + /** + * Test backends should call this to get a task that has been assigned to them by the scheduler. + * Each task should be responded to with either [[taskSuccess]] or [[taskFailed]]. + */ + def beginTask(): TaskDescription = { + synchronized { + val toRun = assignedTasksWaitingToRun.remove(assignedTasksWaitingToRun.size - 1) + runningTasks += toRun + toRun + } + } + + /** + * Tell the scheduler the task completed successfully, with the given result. Also + * updates some internal state for this mock. + */ + def taskSuccess(task: TaskDescription, result: Any): Unit = { + val ser = env.serializer.newInstance() + val resultBytes = ser.serialize(result) + val directResult = new DirectTaskResult(resultBytes, Seq()) // no accumulator updates + taskUpdate(task, TaskState.FINISHED, directResult) + } + + /** + * Tell the scheduler the task failed, with the given state and result (probably ExceptionFailure + * or FetchFailed). Also updates some internal state for this mock. + */ + def taskFailed(task: TaskDescription, exc: Exception): Unit = { + taskUpdate(task, TaskState.FAILED, new ExceptionFailure(exc, Seq())) + } + + def taskFailed(task: TaskDescription, reason: TaskFailedReason): Unit = { + taskUpdate(task, TaskState.FAILED, reason) + } + + def taskUpdate(task: TaskDescription, state: TaskState, result: Any): Unit = { + val ser = env.serializer.newInstance() + val resultBytes = ser.serialize(result) + // statusUpdate is safe to call from multiple threads, its protected inside taskScheduler + taskScheduler.statusUpdate(task.taskId, state, resultBytes) + if (TaskState.isFinished(state)) { + synchronized { + runningTasks -= task + executorIdToExecutor(task.executorId).freeCores += taskScheduler.CPUS_PER_TASK + freeCores += taskScheduler.CPUS_PER_TASK + } + reviveOffers() + } + } + + // protected by this + private val assignedTasksWaitingToRun = new ArrayBuffer[TaskDescription](10000) + // protected by this + private val runningTasks = ArrayBuffer[TaskDescription]() + + def hasTasks: Boolean = synchronized { + assignedTasksWaitingToRun.nonEmpty || runningTasks.nonEmpty + } + + def hasTasksWaitingToRun: Boolean = { + assignedTasksWaitingToRun.nonEmpty + } + + override def start(): Unit = {} + + override def stop(): Unit = {} + + val env = SparkEnv.get + + /** Accessed by both scheduling and backend thread, so should be protected by this. */ + var freeCores: Int = _ + + /** + * Accessed by both scheduling and backend thread, so should be protected by this. + * Most likely the only thing that needs to be protected are the inidividual ExecutorTaskStatus, + * but for simplicity in this mock just lock the whole backend. + */ + def executorIdToExecutor: Map[String, ExecutorTaskStatus] + + private def generateOffers(): Seq[WorkerOffer] = { + executorIdToExecutor.values.filter { exec => + exec.freeCores > 0 + }.map { exec => + WorkerOffer(executorId = exec.executorId, host = exec.host, + cores = exec.freeCores) + }.toSeq + } + + /** + * This is called by the scheduler whenever it has tasks it would like to schedule. It gets + * called in the scheduling thread, not the backend thread. + */ + override def reviveOffers(): Unit = { + val offers: Seq[WorkerOffer] = generateOffers() + val newTasks = taskScheduler.resourceOffers(offers).flatten + synchronized { + newTasks.foreach { task => + executorIdToExecutor(task.executorId).freeCores -= taskScheduler.CPUS_PER_TASK + } + freeCores -= newTasks.size * taskScheduler.CPUS_PER_TASK + assignedTasksWaitingToRun ++= newTasks + } + } + + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = { + // We have to implement this b/c of SPARK-15385. + // Its OK for this to be a no-op, because even if a backend does implement killTask, + // it really can only be "best-effort" in any case, and the scheduler should be robust to that. + // And in fact its reasonably simulating a case where a real backend finishes tasks in between + // the time when the scheduler sends the msg to kill tasks, and the backend receives the msg. + } +} + +/** + * A very simple mock backend that can just run one task at a time. + */ +private[spark] class SingleCoreMockBackend( + conf: SparkConf, + taskScheduler: TaskSchedulerImpl) extends MockBackend(conf, taskScheduler) { + + val cores = 1 + + override def defaultParallelism(): Int = conf.getInt("spark.default.parallelism", cores) + + freeCores = cores + val localExecutorId = SparkContext.DRIVER_IDENTIFIER + val localExecutorHostname = "localhost" + + override val executorIdToExecutor: Map[String, ExecutorTaskStatus] = Map( + localExecutorId -> new ExecutorTaskStatus(localExecutorHostname, localExecutorId, freeCores) + ) +} + +case class ExecutorTaskStatus(host: String, executorId: String, var freeCores: Int) + +class MockRDD( + sc: SparkContext, + val numPartitions: Int, + val shuffleDeps: Seq[ShuffleDependency[Int, Int, Nothing]] +) extends RDD[(Int, Int)](sc, shuffleDeps) with Serializable { + + MockRDD.validate(numPartitions, shuffleDeps) + + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getPartitions: Array[Partition] = { + (0 until numPartitions).map(i => new Partition { + override def index: Int = i + }).toArray + } + override def getPreferredLocations(split: Partition): Seq[String] = Nil + override def toString: String = "MockRDD " + id +} + +object MockRDD extends AssertionsHelper with TripleEquals { + /** + * make sure all the shuffle dependencies have a consistent number of output partitions + * (mostly to make sure the test setup makes sense, not that Spark itself would get this wrong) + */ + def validate(numPartitions: Int, dependencies: Seq[ShuffleDependency[_, _, _]]): Unit = { + dependencies.foreach { dependency => + val partitioner = dependency.partitioner + assert(partitioner != null) + assert(partitioner.numPartitions === numPartitions) + } + } +} + +/** Simple cluster manager that wires up our mock backend. */ +private class MockExternalClusterManager extends ExternalClusterManager { + + val MOCK_REGEX = """mock\[(.*)\]""".r + def canCreate(masterURL: String): Boolean = MOCK_REGEX.findFirstIn(masterURL).isDefined + + def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = { + new TestTaskScheduler(sc) + } + + def createSchedulerBackend( + sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + masterURL match { + case MOCK_REGEX(backendClassName) => + val backendClass = Utils.classForName(backendClassName) + val ctor = backendClass.getConstructor(classOf[SparkConf], classOf[TaskSchedulerImpl]) + ctor.newInstance(sc.getConf, scheduler).asInstanceOf[SchedulerBackend] + } + } + + def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} + +/** TaskSchedulerImpl that just tracks a tiny bit more state to enable checks in tests. */ +class TestTaskScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { + /** Set of TaskSets the DAGScheduler has requested executed. */ + val runningTaskSets = HashSet[TaskSet]() + + override def submitTasks(taskSet: TaskSet): Unit = { + runningTaskSets += taskSet + super.submitTasks(taskSet) + } + + override def taskSetFinished(manager: TaskSetManager): Unit = { + runningTaskSets -= manager.taskSet + super.taskSetFinished(manager) + } +} + +/** + * Some very basic tests just to demonstrate the use of the test framework (and verify that it + * works). + */ +class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCoreMockBackend] { + + /** + * Very simple one stage job. Backend successfully completes each task, one by one + */ + testScheduler("super simple job") { + def runBackend(): Unit = { + val task = backend.beginTask() + backend.taskSuccess(task, 42) + } + withBackend(runBackend _) { + val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + val duration = Duration(1, SECONDS) + Await.ready(jobFuture, duration) + } + assert(results === (0 until 10).map { _ -> 42 }.toMap) + assertDataStructuresEmpty() + } + + /** + * 5 stage job, diamond dependencies. + * + * a ----> b ----> d --> result + * \--> c --/ + * + * Backend successfully completes each task + */ + testScheduler("multi-stage job") { + + def stageToOutputParts(stageId: Int): Int = { + stageId match { + case 0 => 10 + case 2 => 20 + case _ => 30 + } + } + + val a = new MockRDD(sc, 2, Nil) + val b = shuffle(10, a) + val c = shuffle(20, a) + val d = join(30, b, c) + + def runBackend(): Unit = { + val taskDescription = backend.beginTask() + val taskSet = taskScheduler.taskIdToTaskSetManager(taskDescription.taskId).taskSet + val task = taskSet.tasks(taskDescription.index) + + // make sure the required map output is available + task.stageId match { + case 1 => assertMapOutputAvailable(b) + case 3 => assertMapOutputAvailable(c) + case 4 => assertMapOutputAvailable(d) + case _ => // no shuffle map input, nothing to check + } + + (task.stageId, task.stageAttemptId, task.partitionId) match { + case (stage, 0, _) if stage < 4 => + backend.taskSuccess(taskDescription, + DAGSchedulerSuite.makeMapStatus("hostA", stageToOutputParts(stage))) + case (4, 0, partition) => + backend.taskSuccess(taskDescription, 4321 + partition) + } + } + withBackend(runBackend _) { + val jobFuture = submit(d, (0 until 30).toArray) + val duration = Duration(1, SECONDS) + Await.ready(jobFuture, duration) + } + assert(results === (0 until 30).map { idx => idx -> (4321 + idx) }.toMap) + assertDataStructuresEmpty() + } + + /** + * 2 stage job, with a fetch failure. Make sure that: + * (a) map output is available whenever we run stage 1 + * (b) we get a second attempt for stage 0 & stage 1 + */ + testScheduler("job with fetch failure") { + val input = new MockRDD(sc, 2, Nil) + val shuffledRdd = shuffle(10, input) + val shuffleId = shuffledRdd.shuffleDeps.head.shuffleId + + val stageToAttempts = new HashMap[Int, HashSet[Int]]() + + def runBackend(): Unit = { + val taskDescription = backend.beginTask() + val taskSet = taskScheduler.taskIdToTaskSetManager(taskDescription.taskId).taskSet + val task = taskSet.tasks(taskDescription.index) + stageToAttempts.getOrElseUpdate(task.stageId, new HashSet()) += task.stageAttemptId + + // make sure the required map output is available + task.stageId match { + case 1 => assertMapOutputAvailable(shuffledRdd) + case _ => // no shuffle map input, nothing to check + } + + (task.stageId, task.stageAttemptId, task.partitionId) match { + case (0, _, _) => + backend.taskSuccess(taskDescription, DAGSchedulerSuite.makeMapStatus("hostA", 10)) + case (1, 0, 0) => + val fetchFailed = FetchFailed( + DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored") + backend.taskFailed(taskDescription, fetchFailed) + case (1, _, partition) => + backend.taskSuccess(taskDescription, 42 + partition) + } + } + withBackend(runBackend _) { + val jobFuture = submit(shuffledRdd, (0 until 10).toArray) + val duration = Duration(1, SECONDS) + Await.ready(jobFuture, duration) + } + assert(results === (0 until 10).map { idx => idx -> (42 + idx) }.toMap) + assert(stageToAttempts === Map(0 -> Set(0, 1), 1 -> Set(0, 1))) + assertDataStructuresEmpty() + } + + testScheduler("job failure after 4 attempts") { + def runBackend(): Unit = { + val task = backend.beginTask() + backend.taskFailed(task, new RuntimeException("test task failure")) + } + withBackend(runBackend _) { + val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + val duration = Duration(1, SECONDS) + Await.ready(jobFuture, duration) + failure.getMessage.contains("test task failure") + } + assert(results.isEmpty) + assertDataStructuresEmpty(noFailure = false) + } +} From 361ebc282b2d09dc6dcf21419a53c5c617b1b6bd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 May 2016 23:54:24 -0700 Subject: [PATCH 142/143] [SPARK-15543][SQL] Rename DefaultSources to make them more self-describing ## What changes were proposed in this pull request? This patch renames various DefaultSources to make their names more self-describing. The choice of "DefaultSource" was from the days when we did not have a good way to specify short names. They are now named: - LibSVMFileFormat - CSVFileFormat - JdbcRelationProvider - JsonFileFormat - ParquetFileFormat - TextFileFormat Backward compatibility is maintained through aliasing. ## How was this patch tested? Updated relevant test cases too. Author: Reynold Xin Closes #13311 from rxin/SPARK-15543. --- ...pache.spark.sql.sources.DataSourceRegister | 2 +- .../ml/source/libsvm/LibSVMRelation.scala | 8 +++- project/MimaExcludes.scala | 4 +- ...pache.spark.sql.sources.DataSourceRegister | 10 ++--- .../spark/sql/execution/ExistingRDD.scala | 2 +- .../execution/datasources/DataSource.scala | 45 ++++++++++++++----- ...efaultSource.scala => CSVFileFormat.scala} | 6 +-- ...ource.scala => JdbcRelationProvider.scala} | 2 +- ...SONRelation.scala => JsonFileFormat.scala} | 4 +- ...Relation.scala => ParquetFileFormat.scala} | 14 +++--- ...faultSource.scala => TextFileFormat.scala} | 2 +- .../datasources/json/JsonSuite.scala | 4 +- .../parquet/ParquetSchemaSuite.scala | 12 ++--- .../sql/sources/ResolvedDataSourceSuite.scala | 18 ++++---- .../sql/streaming/FileStreamSinkSuite.scala | 4 +- .../DataFrameReaderWriterSuite.scala | 1 - ...pache.spark.sql.sources.DataSourceRegister | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 ++++---- ...{OrcRelation.scala => OrcFileFormat.scala} | 8 +++- .../hive/orc/OrcHadoopFsRelationSuite.scala | 2 +- 20 files changed, 99 insertions(+), 69 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/{DefaultSource.scala => CSVFileFormat.scala} (96%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/{DefaultSource.scala => JdbcRelationProvider.scala} (96%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/{JSONRelation.scala => JsonFileFormat.scala} (98%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{ParquetRelation.scala => ParquetFileFormat.scala} (98%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/{DefaultSource.scala => TextFileFormat.scala} (98%) rename sql/core/src/test/scala/org/apache/spark/sql/streaming/{ => test}/DataFrameReaderWriterSuite.scala (99%) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/{OrcRelation.scala => OrcFileFormat.scala} (98%) diff --git a/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index f632dd603c449..a865cbe19b184 100644 --- a/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1 @@ -org.apache.spark.ml.source.libsvm.DefaultSource +org.apache.spark.ml.source.libsvm.LibSVMFileFormat 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 5ba768d551842..64ebf0c982346 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 @@ -90,7 +90,7 @@ private[libsvm] class LibSVMOutputWriter( * .load("data/mllib/sample_libsvm_data.txt") * * // Java - * DataFrame df = spark.read().format("libsvm") + * Dataset df = spark.read().format("libsvm") * .option("numFeatures, "780") * .load("data/mllib/sample_libsvm_data.txt"); * }}} @@ -105,9 +105,13 @@ private[libsvm] class LibSVMOutputWriter( * - "vectorType": feature vector type, "sparse" (default) or "dense". * * @see [[https://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/ LIBSVM datasets]] + * + * Note that this class is public for documentation purpose. Please don't use this class directly. + * Rather, use the data source API as illustrated above. */ +// If this is moved or renamed, please update DataSource's backwardCompatibilityMap. @Since("1.6.0") -class DefaultSource extends FileFormat with DataSourceRegister { +class LibSVMFileFormat extends FileFormat with DataSourceRegister { @Since("1.6.0") override def shortName(): String = "libsvm" diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4e99a0965780b..08c575aaeec78 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -68,7 +68,9 @@ object MimaExcludes { // SPARK-13664 Replace HadoopFsRelation with FileFormat ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.LibSVMRelation"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelationProvider"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache") + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache"), + // SPARK-15543 Rename DefaultSources to make them more self-describing + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.DefaultSource") ) ++ Seq( ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.SparkContext.emptyRDD"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.broadcast.HttpBroadcastFactory"), 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 ef9255794b6ab..9f8bb5d38fd6c 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 @@ -1,6 +1,6 @@ -org.apache.spark.sql.execution.datasources.csv.DefaultSource -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.datasources.csv.CSVFileFormat +org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider +org.apache.spark.sql.execution.datasources.json.JsonFileFormat +org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +org.apache.spark.sql.execution.datasources.text.TextFileFormat org.apache.spark.sql.execution.streaming.ConsoleSinkProvider 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 ec23a9c41a029..412f5fa87ee7b 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCo import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetSource} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.BaseRelation 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 d0853f67b97ec..dfe06478fccb2 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 @@ -30,6 +30,10 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider +import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -74,15 +78,34 @@ case class DataSource( lazy val sourceInfo = sourceSchema() /** A map to maintain backward compatibility in case we move data sources around. */ - private val backwardCompatibilityMap = Map( - "org.apache.spark.sql.jdbc" -> classOf[jdbc.DefaultSource].getCanonicalName, - "org.apache.spark.sql.jdbc.DefaultSource" -> classOf[jdbc.DefaultSource].getCanonicalName, - "org.apache.spark.sql.json" -> classOf[json.DefaultSource].getCanonicalName, - "org.apache.spark.sql.json.DefaultSource" -> classOf[json.DefaultSource].getCanonicalName, - "org.apache.spark.sql.parquet" -> classOf[parquet.DefaultSource].getCanonicalName, - "org.apache.spark.sql.parquet.DefaultSource" -> classOf[parquet.DefaultSource].getCanonicalName, - "com.databricks.spark.csv" -> classOf[csv.DefaultSource].getCanonicalName - ) + private val backwardCompatibilityMap: Map[String, String] = { + val jdbc = classOf[JdbcRelationProvider].getCanonicalName + val json = classOf[JsonFileFormat].getCanonicalName + val parquet = classOf[ParquetFileFormat].getCanonicalName + val csv = classOf[CSVFileFormat].getCanonicalName + val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat" + val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat" + + Map( + "org.apache.spark.sql.jdbc" -> jdbc, + "org.apache.spark.sql.jdbc.DefaultSource" -> jdbc, + "org.apache.spark.sql.execution.datasources.jdbc.DefaultSource" -> jdbc, + "org.apache.spark.sql.execution.datasources.jdbc" -> jdbc, + "org.apache.spark.sql.json" -> json, + "org.apache.spark.sql.json.DefaultSource" -> json, + "org.apache.spark.sql.execution.datasources.json" -> json, + "org.apache.spark.sql.execution.datasources.json.DefaultSource" -> json, + "org.apache.spark.sql.parquet" -> parquet, + "org.apache.spark.sql.parquet.DefaultSource" -> parquet, + "org.apache.spark.sql.execution.datasources.parquet" -> parquet, + "org.apache.spark.sql.execution.datasources.parquet.DefaultSource" -> parquet, + "org.apache.spark.sql.hive.orc.DefaultSource" -> orc, + "org.apache.spark.sql.hive.orc" -> orc, + "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm, + "org.apache.spark.ml.source.libsvm" -> libsvm, + "com.databricks.spark.csv" -> csv + ) + } /** * Class that were removed in Spark 2.0. Used to detect incompatibility libraries for Spark 2.0. @@ -188,7 +211,7 @@ case class DataSource( throw new IllegalArgumentException("'path' is not specified") }) val isSchemaInferenceEnabled = sparkSession.conf.get(SQLConf.STREAMING_SCHEMA_INFERENCE) - val isTextSource = providingClass == classOf[text.DefaultSource] + val isTextSource = providingClass == classOf[text.TextFileFormat] // If the schema inference is disabled, only text sources require schema to be specified if (!isSchemaInferenceEnabled && !isTextSource && userSpecifiedSchema.isEmpty) { throw new IllegalArgumentException( @@ -229,7 +252,7 @@ case class DataSource( providingClass.newInstance() match { case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, options, partitionColumns) - case parquet: parquet.DefaultSource => + case parquet: parquet.ParquetFileFormat => val caseInsensitiveOptions = new CaseInsensitiveMap(options) val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") 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/CSVFileFormat.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 057bde1a75f07..4d36b760568cc 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/CSVFileFormat.scala @@ -28,8 +28,6 @@ import org.apache.hadoop.mapreduce._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -38,7 +36,7 @@ import org.apache.spark.util.SerializableConfiguration /** * Provides access to CSV data from pure SQL statements. */ -class DefaultSource extends FileFormat with DataSourceRegister { +class CSVFileFormat extends FileFormat with DataSourceRegister { override def shortName(): String = "csv" @@ -46,7 +44,7 @@ class DefaultSource extends FileFormat with DataSourceRegister { override def hashCode(): Int = getClass.hashCode() - override def equals(other: Any): Boolean = other.isInstanceOf[DefaultSource] + override def equals(other: Any): Boolean = other.isInstanceOf[CSVFileFormat] override def inferSchema( sparkSession: SparkSession, 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/JdbcRelationProvider.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index 6609e5dee3af5..106ed1d440102 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/JdbcRelationProvider.scala @@ -22,7 +22,7 @@ import java.util.Properties import org.apache.spark.sql.SQLContext import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, RelationProvider} -class DefaultSource extends RelationProvider with DataSourceRegister { +class JdbcRelationProvider extends RelationProvider with DataSourceRegister { override def shortName(): String = "jdbc" 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/JsonFileFormat.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 4c97abed53ce6..35f247692ffdc 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/JsonFileFormat.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -class DefaultSource extends FileFormat with DataSourceRegister { +class JsonFileFormat extends FileFormat with DataSourceRegister { override def shortName(): String = "json" @@ -151,7 +151,7 @@ class DefaultSource extends FileFormat with DataSourceRegister { override def hashCode(): Int = getClass.hashCode() - override def equals(other: Any): Boolean = other.isInstanceOf[DefaultSource] + override def equals(other: Any): Boolean = other.isInstanceOf[JsonFileFormat] } private[json] class JsonOutputWriter( 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/ParquetFileFormat.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index cf5c8e94f468d..b47d41e1661f0 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/ParquetFileFormat.scala @@ -50,7 +50,7 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration -private[sql] class DefaultSource +private[sql] class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging @@ -62,7 +62,7 @@ private[sql] class DefaultSource override def hashCode(): Int = getClass.hashCode() - override def equals(other: Any): Boolean = other.isInstanceOf[DefaultSource] + override def equals(other: Any): Boolean = other.isInstanceOf[ParquetFileFormat] override def prepareWrite( sparkSession: SparkSession, @@ -141,7 +141,7 @@ private[sql] class DefaultSource // Should we merge schemas from all Parquet part-files? val shouldMergeSchemas = parameters - .get(ParquetRelation.MERGE_SCHEMA) + .get(ParquetFileFormat.MERGE_SCHEMA) .map(_.toBoolean) .getOrElse(sparkSession.conf.get(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED)) @@ -217,7 +217,7 @@ private[sql] class DefaultSource .orElse(filesByType.data.headOption) .toSeq } - ParquetRelation.mergeSchemasInParallel(filesToTouch, sparkSession) + ParquetFileFormat.mergeSchemasInParallel(filesToTouch, sparkSession) } case class FileTypes( @@ -543,7 +543,7 @@ private[sql] class ParquetOutputWriter( override def close(): Unit = recordWriter.close(context) } -private[sql] object ParquetRelation extends Logging { +private[sql] object ParquetFileFormat extends Logging { // Whether we should merge schemas collected from all Parquet part-files. private[sql] val MERGE_SCHEMA = "mergeSchema" @@ -822,9 +822,9 @@ private[sql] object ParquetRelation extends Logging { if (footers.isEmpty) { Iterator.empty } else { - var mergedSchema = ParquetRelation.readSchemaFromFooter(footers.head, converter) + var mergedSchema = ParquetFileFormat.readSchemaFromFooter(footers.head, converter) footers.tail.foreach { footer => - val schema = ParquetRelation.readSchemaFromFooter(footer, converter) + val schema = ParquetFileFormat.readSchemaFromFooter(footer, converter) try { mergedSchema = mergedSchema.merge(schema) } catch { case cause: SparkException => 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/TextFileFormat.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index f091615a9a714..d9525efe6d9be 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/TextFileFormat.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.SerializableConfiguration /** * A data source for reading text files. */ -class DefaultSource extends FileFormat with DataSourceRegister { +class TextFileFormat extends FileFormat with DataSourceRegister { override def shortName(): String = "text" 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 46213a22ed3d3..500d8ff55a9a8 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 @@ -1320,7 +1320,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { userSpecifiedSchema = None, partitionColumns = Array.empty[String], bucketSpec = None, - className = classOf[DefaultSource].getCanonicalName, + className = classOf[JsonFileFormat].getCanonicalName, options = Map("path" -> path)).resolveRelation() val d2 = DataSource( @@ -1328,7 +1328,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { userSpecifiedSchema = None, partitionColumns = Array.empty[String], bucketSpec = None, - className = classOf[DefaultSource].getCanonicalName, + className = classOf[JsonFileFormat].getCanonicalName, options = Map("path" -> path)).resolveRelation() assert(d1 === d2) }) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index c43b142de2cf1..6db649228210d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -375,7 +375,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { StructField("lowerCase", StringType), StructField("UPPERCase", DoubleType, nullable = false)))) { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("lowercase", StringType), StructField("uppercase", DoubleType, nullable = false))), @@ -390,7 +390,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { StructType(Seq( StructField("UPPERCase", DoubleType, nullable = false)))) { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("uppercase", DoubleType, nullable = false))), @@ -401,7 +401,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { // Metastore schema contains additional non-nullable fields. assert(intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("uppercase", DoubleType, nullable = false), StructField("lowerCase", BinaryType, nullable = false))), @@ -412,7 +412,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { // Conflicting non-nullable field names intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq(StructField("lower", StringType, nullable = false))), StructType(Seq(StructField("lowerCase", BinaryType)))) } @@ -426,7 +426,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { StructField("firstField", StringType, nullable = true), StructField("secondField", StringType, nullable = true), StructField("thirdfield", StringType, nullable = true)))) { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("firstfield", StringType, nullable = true), StructField("secondfield", StringType, nullable = true), @@ -439,7 +439,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { // Merge should fail if the Metastore contains any additional fields that are not // nullable. assert(intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("firstfield", StringType, nullable = true), StructField("secondfield", StringType, nullable = true), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 4f6df5441736e..320aaea1e4eeb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -27,37 +27,37 @@ class ResolvedDataSourceSuite extends SparkFunSuite { test("jdbc") { assert( getProvidingClass("jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) assert( getProvidingClass("org.apache.spark.sql.execution.datasources.jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) assert( getProvidingClass("org.apache.spark.sql.jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) } test("json") { assert( getProvidingClass("json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) assert( getProvidingClass("org.apache.spark.sql.execution.datasources.json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) assert( getProvidingClass("org.apache.spark.sql.json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) } test("parquet") { assert( getProvidingClass("parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) assert( getProvidingClass("org.apache.spark.sql.execution.datasources.parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) assert( getProvidingClass("org.apache.spark.sql.parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) } test("error message for unknown data sources") { 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 6238b74ffac56..f3262f772b160 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 @@ -41,7 +41,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { path.delete() val hadoopConf = spark.sparkContext.hadoopConfiguration - val fileFormat = new parquet.DefaultSource() + val fileFormat = new parquet.ParquetFileFormat() def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { val df = spark @@ -73,7 +73,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { path.delete() val hadoopConf = spark.sparkContext.hadoopConfiguration - val fileFormat = new parquet.DefaultSource() + val fileFormat = new parquet.ParquetFileFormat() def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { val df = spark 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/test/DataFrameReaderWriterSuite.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index e6c0ce95e7b57..288f6dc59741e 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/test/DataFrameReaderWriterSuite.scala @@ -101,7 +101,6 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { } class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { - import testImplicits._ private def newMetadataDir = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath diff --git a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 4a774fbf1fdf8..32aa13ff257a6 100644 --- a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1 @@ -org.apache.spark.sql.hive.orc.DefaultSource +org.apache.spark.sql.hive.orc.OrcFileFormat 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 86ab152402625..b377a20e3943b 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 @@ -32,8 +32,8 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.CreateTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} -import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetDefaultSource, ParquetRelation} -import org.apache.spark.sql.hive.orc.{DefaultSource => OrcDefaultSource} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ @@ -281,7 +281,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val inferredSchema = defaultSource.inferSchema(sparkSession, options, fileCatalog.allFiles()) inferredSchema.map { inferred => - ParquetRelation.mergeMetastoreParquetSchema(metastoreSchema, inferred) + ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, inferred) }.getOrElse(metastoreSchema) } else { defaultSource.inferSchema(sparkSession, options, fileCatalog.allFiles()).get @@ -348,13 +348,13 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } private def convertToParquetRelation(relation: MetastoreRelation): LogicalRelation = { - val defaultSource = new ParquetDefaultSource() - val fileFormatClass = classOf[ParquetDefaultSource] + val defaultSource = new ParquetFileFormat() + val fileFormatClass = classOf[ParquetFileFormat] val mergeSchema = sessionState.convertMetastoreParquetWithSchemaMerging val options = Map( - ParquetRelation.MERGE_SCHEMA -> mergeSchema.toString, - ParquetRelation.METASTORE_TABLE_NAME -> TableIdentifier( + ParquetFileFormat.MERGE_SCHEMA -> mergeSchema.toString, + ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier( relation.tableName, Some(relation.databaseName) ).unquotedString @@ -400,8 +400,8 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } private def convertToOrcRelation(relation: MetastoreRelation): LogicalRelation = { - val defaultSource = new OrcDefaultSource() - val fileFormatClass = classOf[OrcDefaultSource] + val defaultSource = new OrcFileFormat() + val fileFormatClass = classOf[OrcFileFormat] val options = Map[String, String]() convertToLogicalRelation(relation, options, defaultSource, fileFormatClass, "orc") 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/OrcFileFormat.scala similarity index 98% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 38f50c112a236..f1198179a0994 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/OrcFileFormat.scala @@ -42,7 +42,11 @@ import org.apache.spark.sql.sources.{Filter, _} import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -private[sql] class DefaultSource +/** + * [[FileFormat]] for reading ORC files. If this is moved or renamed, please update + * [[DataSource]]'s backwardCompatibilityMap. + */ +private[sql] class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable { override def shortName(): String = "orc" @@ -262,7 +266,7 @@ private[orc] case class OrcTableScan( // Figure out the actual schema from the ORC source (without partition columns) so that we // can pick the correct ordinals. Note that this assumes that all files have the same schema. - val orcFormat = new DefaultSource + val orcFormat = new OrcFileFormat val dataSchema = orcFormat .inferSchema(sparkSession, Map.empty, inputPaths) 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 0207b4e8c9775..5dfa58f673bf2 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.types._ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { import testImplicits._ - override val dataSourceName: String = classOf[DefaultSource].getCanonicalName + override val dataSourceName: String = classOf[OrcFileFormat].getCanonicalName // ORC does not play well with NullType and UDT. override protected def supportsDataType(dataType: DataType): Boolean = dataType match { From 53d4abe9e996e53c1bdcd5ac4cb8cbf08b9ec8b5 Mon Sep 17 00:00:00 2001 From: Bo Meng Date: Thu, 26 May 2016 00:22:47 -0700 Subject: [PATCH 143/143] [SPARK-15537][SQL] fix dir delete issue ## What changes were proposed in this pull request? For some of the test cases, e.g. `OrcSourceSuite`, it will create temp folders and temp files inside them. But after tests finish, the folders are not removed. This will cause lots of temp files created and space occupied, if we keep running the test cases. The reason is dir.delete() won't work if dir is not empty. We need to recursively delete the content before deleting the folder. ## How was this patch tested? Manually checked the temp folder to make sure the temp files were deleted. Author: Bo Meng Closes #13304 from bomeng/SPARK-15537. --- .../spark/sql/hive/orc/OrcQuerySuite.scala | 6 ------ .../spark/sql/hive/orc/OrcSourceSuite.scala | 18 +++--------------- 2 files changed, 3 insertions(+), 21 deletions(-) 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 f83b3a3de2e54..9771b2314a08d 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 @@ -53,12 +53,6 @@ case class Person(name: String, age: Int, contacts: Seq[Contact]) class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { - def getTempFilePath(prefix: String, suffix: String = ""): File = { - val tempFile = File.createTempFile(prefix, suffix) - tempFile.delete() - tempFile - } - test("Read/write All Types") { val data = (0 to 255).map { i => (s"$i", i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0) 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 6081d86f4478b..4cac334859ba1 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 @@ -25,6 +25,7 @@ import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils case class OrcData(intField: Int, stringField: String) @@ -37,14 +38,10 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA override def beforeAll(): Unit = { super.beforeAll() - orcTableAsDir = File.createTempFile("orctests", "sparksql") - orcTableAsDir.delete() - orcTableAsDir.mkdir() + orcTableAsDir = Utils.createTempDir("orctests", "sparksql") // Hack: to prepare orc data files using hive external tables - orcTableDir = File.createTempFile("orctests", "sparksql") - orcTableDir.delete() - orcTableDir.mkdir() + orcTableDir = Utils.createTempDir("orctests", "sparksql") import org.apache.spark.sql.hive.test.TestHive.implicits._ sparkContext @@ -68,15 +65,6 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA """.stripMargin) } - override def afterAll(): Unit = { - try { - orcTableDir.delete() - orcTableAsDir.delete() - } finally { - super.afterAll() - } - } - test("create temporary orc table") { checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10))
  • Property NameDefaultMeaning