From c40051932290db3a63f80324900a116019b1e589 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 10 Jun 2020 13:11:24 -0700 Subject: [PATCH 01/40] [SPARK-31956][SQL] Do not fail if there is no ambiguous self join ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/28695 , to fix the problem completely. The root cause is that, `df("col").as("name")` is not a column reference anymore, and should not have the special column metadata. However, this was broken in https://github.com/apache/spark/commit/ba7adc494923de8104ab37d412edd78afe540f45#diff-ac415c903887e49486ba542a65eec980L1050-L1053 This PR fixes the regression, by strip the special column metadata in `Column.name`, which is the behavior before https://github.com/apache/spark/pull/28326 . ### Why are the changes needed? Fix a regression. We shouldn't fail if there is no ambiguous self-join. ### Does this PR introduce _any_ user-facing change? Yes, the query in the test can run now. ### How was this patch tested? updated test Closes #28783 from cloud-fan/self-join. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 2 +- .../org/apache/spark/sql/DataFrameSelfJoinSuite.scala | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) 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 2144472937f9b..e6f7b1d723af6 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 @@ -1042,7 +1042,7 @@ class Column(val expr: Expression) extends Logging { * @since 2.0.0 */ def name(alias: String): Column = withExpr { - Alias(expr, alias)() + Alias(normalizedExpr(), alias)() } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala index fb58c9851224b..3b3b54f75da57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala @@ -204,7 +204,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { } } - test("SPARK-28344: don't fail as ambiguous self join when there is no join") { + test("SPARK-28344: don't fail if there is no ambiguous self join") { withSQLConf( SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true") { val df = Seq(1, 1, 2, 2).toDF("a") @@ -212,6 +212,11 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { checkAnswer( df.select(df("a").alias("x"), sum(df("a")).over(w)), Seq((1, 2), (1, 2), (2, 4), (2, 4)).map(Row.fromTuple)) + + val joined = df.join(spark.range(1)).select($"a") + checkAnswer( + joined.select(joined("a").alias("x"), sum(joined("a")).over(w)), + Seq((1, 2), (1, 2), (2, 4), (2, 4)).map(Row.fromTuple)) } } } From 2ab82fae57b4e1a5c219537d578e6dce9cbca1be Mon Sep 17 00:00:00 2001 From: William Hyun Date: Wed, 10 Jun 2020 14:42:45 -0700 Subject: [PATCH 02/40] [SPARK-31963][PYSPARK][SQL] Support both pandas 0.23 and 1.0 in serializers.py MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR aims to support both pandas 0.23 and 1.0. ### Why are the changes needed? ``` $ pip install pandas==0.23.2 $ python -c "import pandas.CategoricalDtype" Traceback (most recent call last):   File "", line 1, in ModuleNotFoundError: No module named 'pandas.CategoricalDtype' $ python -c "from pandas.api.types import CategoricalDtype" ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins. ``` $ pip freeze | grep pandas pandas==0.23.2 $ python/run-tests.py --python-executables python --modules pyspark-sql ... Tests passed in 359 seconds ``` Closes #28789 from williamhyun/williamhyun-patch-2. Authored-by: William Hyun Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/pandas/serializers.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index ff0b10a9306cf..15b9f2fdeb0f0 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -143,6 +143,10 @@ def _create_batch(self, series): import pandas as pd import pyarrow as pa from pyspark.sql.pandas.types import _check_series_convert_timestamps_internal + try: + from pandas import CategoricalDtype + except ImportError: + from pandas.api.types import CategoricalDtype # Make input conform to [(series1, type1), (series2, type2), ...] if not isinstance(series, (list, tuple)) or \ (len(series) == 2 and isinstance(series[1], pa.DataType)): @@ -154,7 +158,7 @@ def create_array(s, t): # Ensure timestamp series are in expected form for Spark internal representation if t is not None and pa.types.is_timestamp(t): s = _check_series_convert_timestamps_internal(s, self._timezone) - elif type(s.dtype) == pd.CategoricalDtype: + elif type(s.dtype) == CategoricalDtype: # Note: This can be removed once minimum pyarrow version is >= 0.16.1 s = s.astype(s.dtypes.categories.dtype) try: From 00d06cad564d5e3e5f78a687776d02fe0695a861 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 10 Jun 2020 15:54:07 -0700 Subject: [PATCH 03/40] [SPARK-31915][SQL][PYTHON] Resolve the grouping column properly per the case sensitivity in grouped and cogrouped pandas UDFs ### What changes were proposed in this pull request? This is another approach to fix the issue. See the previous try https://github.com/apache/spark/pull/28745. It was too invasive so I took more conservative approach. This PR proposes to resolve grouping attributes separately first so it can be properly referred when `FlatMapGroupsInPandas` and `FlatMapCoGroupsInPandas` are resolved without ambiguity. Previously, ```python from pyspark.sql.functions import * df = spark.createDataFrame([[1, 1]], ["column", "Score"]) pandas_udf("column integer, Score float", PandasUDFType.GROUPED_MAP) def my_pandas_udf(pdf): return pdf.assign(Score=0.5) df.groupby('COLUMN').apply(my_pandas_udf).show() ``` was failed as below: ``` pyspark.sql.utils.AnalysisException: "Reference 'COLUMN' is ambiguous, could be: COLUMN, COLUMN.;" ``` because the unresolved `COLUMN` in `FlatMapGroupsInPandas` doesn't know which reference to take from the child projection. After this fix, it resolves the child projection first with grouping keys and pass, to `FlatMapGroupsInPandas`, the attribute as a grouping key from the child projection that is positionally selected. ### Why are the changes needed? To resolve grouping keys correctly. ### Does this PR introduce _any_ user-facing change? Yes, ```python from pyspark.sql.functions import * df = spark.createDataFrame([[1, 1]], ["column", "Score"]) pandas_udf("column integer, Score float", PandasUDFType.GROUPED_MAP) def my_pandas_udf(pdf): return pdf.assign(Score=0.5) df.groupby('COLUMN').apply(my_pandas_udf).show() ``` ```python df1 = spark.createDataFrame([(1, 1)], ("column", "value")) df2 = spark.createDataFrame([(1, 1)], ("column", "value")) df1.groupby("COLUMN").cogroup( df2.groupby("COLUMN") ).applyInPandas(lambda r, l: r + l, df1.schema).show() ``` Before: ``` pyspark.sql.utils.AnalysisException: Reference 'COLUMN' is ambiguous, could be: COLUMN, COLUMN.; ``` ``` pyspark.sql.utils.AnalysisException: cannot resolve '`COLUMN`' given input columns: [COLUMN, COLUMN, value, value];; 'FlatMapCoGroupsInPandas ['COLUMN], ['COLUMN], (column#9L, value#10L, column#13L, value#14L), [column#22L, value#23L] :- Project [COLUMN#9L, column#9L, value#10L] : +- LogicalRDD [column#9L, value#10L], false +- Project [COLUMN#13L, column#13L, value#14L] +- LogicalRDD [column#13L, value#14L], false ``` After: ``` +------+-----+ |column|Score| +------+-----+ | 1| 0.5| +------+-----+ ``` ``` +------+-----+ |column|value| +------+-----+ | 2| 2| +------+-----+ ``` ### How was this patch tested? Unittests were added and manually tested. Closes #28777 from HyukjinKwon/SPARK-31915-another. Authored-by: HyukjinKwon Signed-off-by: Bryan Cutler --- .../sql/tests/test_pandas_cogrouped_map.py | 18 +++++++++++++++++- .../sql/tests/test_pandas_grouped_map.py | 10 ++++++++++ .../spark/sql/RelationalGroupedDataset.scala | 17 ++++++++++------- 3 files changed, 37 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py index 3ed9d2ac62fd3..c1cb30c3caa91 100644 --- a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py @@ -19,7 +19,7 @@ import sys from pyspark.sql.functions import array, explode, col, lit, udf, sum, pandas_udf, PandasUDFType -from pyspark.sql.types import DoubleType, StructType, StructField +from pyspark.sql.types import DoubleType, StructType, StructField, Row from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ pandas_requirement_message, pyarrow_requirement_message from pyspark.testing.utils import QuietTest @@ -193,6 +193,22 @@ def test_wrong_args(self): left.groupby('id').cogroup(right.groupby('id')) \ .applyInPandas(lambda: 1, StructType([StructField("d", DoubleType())])) + def test_case_insensitive_grouping_column(self): + # SPARK-31915: case-insensitive grouping column should work. + df1 = self.spark.createDataFrame([(1, 1)], ("column", "value")) + + row = df1.groupby("ColUmn").cogroup( + df1.groupby("COLUMN") + ).applyInPandas(lambda r, l: r + l, "column long, value long").first() + self.assertEquals(row.asDict(), Row(column=2, value=2).asDict()) + + df2 = self.spark.createDataFrame([(1, 1)], ("column", "value")) + + row = df1.groupby("ColUmn").cogroup( + df2.groupby("COLUMN") + ).applyInPandas(lambda r, l: r + l, "column long, value long").first() + self.assertEquals(row.asDict(), Row(column=2, value=2).asDict()) + @staticmethod def _test_with_key(left, right, isLeft): diff --git a/python/pyspark/sql/tests/test_pandas_grouped_map.py b/python/pyspark/sql/tests/test_pandas_grouped_map.py index ff53a0c6f2cf2..76119432662ba 100644 --- a/python/pyspark/sql/tests/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_grouped_map.py @@ -587,6 +587,16 @@ def f(key, pdf): # Check that all group and window_range values from udf matched expected self.assertTrue(all([r[0] for r in result])) + def test_case_insensitive_grouping_column(self): + # SPARK-31915: case-insensitive grouping column should work. + def my_pandas_udf(pdf): + return pdf.assign(score=0.5) + + df = self.spark.createDataFrame([[1, 1]], ["column", "score"]) + row = df.groupby('COLUMN').applyInPandas( + my_pandas_udf, schema="column integer, score float").first() + self.assertEquals(row.asDict(), Row(column=1, score=0.5).asDict()) + if __name__ == "__main__": from pyspark.sql.tests.test_pandas_grouped_map import * 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 b1ba7d4538732..c37d8eaa294bf 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 @@ -546,9 +546,10 @@ class RelationalGroupedDataset protected[sql]( case ne: NamedExpression => ne case other => Alias(other, other.toString)() } - val groupingAttributes = groupingNamedExpressions.map(_.toAttribute) val child = df.logicalPlan - val project = Project(groupingNamedExpressions ++ child.output, child) + val project = df.sparkSession.sessionState.executePlan( + Project(groupingNamedExpressions ++ child.output, child)).analyzed + val groupingAttributes = project.output.take(groupingNamedExpressions.length) val output = expr.dataType.asInstanceOf[StructType].toAttributes val plan = FlatMapGroupsInPandas(groupingAttributes, expr, output, project) @@ -583,14 +584,16 @@ class RelationalGroupedDataset protected[sql]( case other => Alias(other, other.toString)() } - val leftAttributes = leftGroupingNamedExpressions.map(_.toAttribute) - val rightAttributes = rightGroupingNamedExpressions.map(_.toAttribute) - val leftChild = df.logicalPlan val rightChild = r.df.logicalPlan - val left = Project(leftGroupingNamedExpressions ++ leftChild.output, leftChild) - val right = Project(rightGroupingNamedExpressions ++ rightChild.output, rightChild) + val left = df.sparkSession.sessionState.executePlan( + Project(leftGroupingNamedExpressions ++ leftChild.output, leftChild)).analyzed + val right = r.df.sparkSession.sessionState.executePlan( + Project(rightGroupingNamedExpressions ++ rightChild.output, rightChild)).analyzed + + val leftAttributes = left.output.take(leftGroupingNamedExpressions.length) + val rightAttributes = right.output.take(rightGroupingNamedExpressions.length) val output = expr.dataType.asInstanceOf[StructType].toAttributes val plan = FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, expr, output, left, right) From 4a25200cd74561152e42fb92b02e860f2d5c7d4e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 10 Jun 2020 17:21:03 -0700 Subject: [PATCH 04/40] Revert "[SPARK-31926][SQL][TEST-HIVE1.2] Fix concurrency issue for ThriftCLIService to getPortNumber" This reverts commit 02f32cfae4082696a6c013fb6beb150040be3228. --- project/SparkBuild.scala | 3 +- .../thriftserver/SharedThriftServer.scala | 46 ++++++------------- .../ThriftServerQueryTestSuite.scala | 3 -- .../ThriftServerWithSparkContextSuite.scala | 11 +---- .../cli/thrift/ThriftBinaryCLIService.java | 11 +---- .../service/cli/thrift/ThriftCLIService.java | 3 -- .../cli/thrift/ThriftHttpCLIService.java | 21 +++------ .../cli/thrift/ThriftBinaryCLIService.java | 11 +---- .../service/cli/thrift/ThriftCLIService.java | 3 -- .../cli/thrift/ThriftHttpCLIService.java | 21 +++------ 10 files changed, 29 insertions(+), 104 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1360d30fdd575..eb12f2f1f6ab7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -480,8 +480,7 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextInHttpSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextInBinarySuite", + "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" ) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index c9e41db52cd50..e002bc0117c8b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -33,8 +33,6 @@ trait SharedThriftServer extends SharedSparkSession { private var hiveServer2: HiveThriftServer2 = _ private var serverPort: Int = 0 - def mode: ServerMode.Value - override def beforeAll(): Unit = { super.beforeAll() // Retries up to 3 times with different port numbers if the server fails to start @@ -55,17 +53,11 @@ trait SharedThriftServer extends SharedSparkSession { } } - protected def jdbcUri: String = if (mode == ServerMode.http) { - s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice" - } else { - s"jdbc:hive2://localhost:$serverPort" - } - protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = - fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } + fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } val statements = connections.map(_.createStatement()) try { @@ -79,33 +71,21 @@ trait SharedThriftServer extends SharedSparkSession { private def startThriftServer(attempt: Int): Unit = { logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt") val sqlContext = spark.newSession().sqlContext - // Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could - // randomly pick any free port to use. + // Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use. // It's much more robust than set a random port generated by ourselves ahead sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) - - try { - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) - hiveServer2.getServices.asScala.foreach { - case t: ThriftCLIService => - serverPort = t.getPortNumber - logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt") - case _ => - } + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + hiveServer2.getServices.asScala.foreach { + case t: ThriftCLIService if t.getPortNumber != 0 => + serverPort = t.getPortNumber + logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt") + case _ => + } - // Wait for thrift server to be ready to serve the query, via executing simple query - // till the query succeeds. See SPARK-30345 for more details. - eventually(timeout(30.seconds), interval(1.seconds)) { - withJdbcStatement { _.execute("SELECT 1") } - } - } catch { - case e: Exception => - logError("Error start hive server with Context ", e) - if (hiveServer2 != null) { - hiveServer2.stop() - } + // Wait for thrift server to be ready to serve the query, via executing simple query + // till the query succeeds. See SPARK-30345 for more details. + eventually(timeout(30.seconds), interval(1.seconds)) { + withJdbcStatement { _.execute("SELECT 1") } } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 553f10a275bce..15cc3109da3f7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -54,9 +54,6 @@ import org.apache.spark.sql.types._ */ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer { - - override def mode: ServerMode.Value = ServerMode.binary - override protected def testFile(fileName: String): String = { val url = Thread.currentThread().getContextClassLoader.getResource(fileName) // Copy to avoid URISyntaxException during accessing the resources in `sql/core` diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index d6420dee41adb..3e1fce78ae71c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -trait ThriftServerWithSparkContextSuite extends SharedThriftServer { +class ThriftServerWithSparkContextSuite extends SharedThriftServer { test("SPARK-29911: Uncache cached tables when session closed") { val cacheManager = spark.sharedState.cacheManager @@ -42,12 +42,3 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } - - -class ThriftServerWithSparkContextInBinarySuite extends ThriftServerWithSparkContextSuite { - override def mode: ServerMode.Value = ServerMode.binary -} - -class ThriftServerWithSparkContextInHttpSuite extends ThriftServerWithSparkContextSuite { - override def mode: ServerMode.Value = ServerMode.http -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index 00bdf7e19126e..e1ee503b81209 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -46,7 +45,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - protected void initializeServer() { + public void run() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -101,14 +100,6 @@ protected void initializeServer() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { server.serve(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 783e5795aca76..8fce9d9383438 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -175,7 +175,6 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { - initializeServer(); new Thread(this).start(); isStarted = true; } @@ -634,8 +633,6 @@ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { return resp; } - protected abstract void initializeServer(); - @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index bd64c777c1d76..1099a00b67eb7 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.thrift.TCLIService.Iface; @@ -54,8 +53,13 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ @Override - protected void initializeServer() { + public void run() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -146,19 +150,6 @@ protected void initializeServer() { + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { httpServer.join(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index ce79e3c8228a6..a7de9c0f3d0d2 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -47,7 +46,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - protected void initializeServer() { + public void run() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -102,14 +101,6 @@ protected void initializeServer() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { server.serve(); } catch (Throwable t) { LOG.error( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index e46799a1c427d..d41c3b493bb47 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -176,7 +176,6 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { - initializeServer(); new Thread(this).start(); isStarted = true; } @@ -671,8 +670,6 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) return resp; } - protected abstract void initializeServer(); - @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index ab9ed5b1f371e..73d5f84476af0 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.rpc.thrift.TCLIService; @@ -55,8 +54,13 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ @Override - protected void initializeServer() { + public void run() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -147,19 +151,6 @@ protected void initializeServer() { + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { httpServer.join(); } catch (Throwable t) { LOG.error( From c7d45c0e0b8c077da8ed4a902503a6102becf255 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 10 Jun 2020 17:36:32 -0700 Subject: [PATCH 05/40] [SPARK-31935][SQL][TESTS][FOLLOWUP] Fix the test case for Hadoop2/3 ### What changes were proposed in this pull request? This PR updates the test case to accept Hadoop 2/3 error message correctly. ### Why are the changes needed? SPARK-31935(https://github.com/apache/spark/pull/28760) breaks Hadoop 3.2 UT because Hadoop 2 and Hadoop 3 have different exception messages. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the Jenkins with both Hadoop 2/3 or do the following manually. **Hadoop 2.7** ``` $ build/sbt "sql/testOnly *.FileBasedDataSourceSuite -- -z SPARK-31935" ... [info] All tests passed. ``` **Hadoop 3.2** ``` $ build/sbt "sql/testOnly *.FileBasedDataSourceSuite -- -z SPARK-31935" -Phadoop-3.2 ... [info] All tests passed. ``` Closes #28791 from dongjoon-hyun/SPARK-31935. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/FileBasedDataSourceSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index efc7cac6a5f21..d8157d3c779b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -849,15 +849,15 @@ class FileBasedDataSourceSuite extends QueryTest withTempDir { dir => val path = dir.getCanonicalPath val defaultFs = "nonexistFS://nonexistFS" - val expectMessage = "No FileSystem for scheme: nonexistFS" + val expectMessage = "No FileSystem for scheme nonexistFS" val message1 = intercept[java.io.IOException] { spark.range(10).write.option("fs.defaultFS", defaultFs).parquet(path) }.getMessage - assert(message1 == expectMessage) + assert(message1.filterNot(Set(':', '"').contains) == expectMessage) val message2 = intercept[java.io.IOException] { spark.read.option("fs.defaultFS", defaultFs).parquet(path) }.getMessage - assert(message2 == expectMessage) + assert(message2.filterNot(Set(':', '"').contains) == expectMessage) } } } From b7ef5294f17d54e7d90e36a4be02e8bd67200144 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 11 Jun 2020 10:26:40 +0900 Subject: [PATCH 06/40] [SPARK-31964][PYTHON] Use Pandas is_categorical on Arrow category type conversion ### What changes were proposed in this pull request? When using pyarrow to convert a Pandas categorical column, use `is_categorical` instead of trying to import `CategoricalDtype` ### Why are the changes needed? The import for `CategoricalDtype` had changed from Pandas 0.23 to 1.0 and pyspark currently tries both locations. Using `is_categorical` is a more stable API. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests Closes #28793 from BryanCutler/arrow-use-is_categorical-SPARK-31964. Authored-by: Bryan Cutler Signed-off-by: HyukjinKwon --- python/pyspark/sql/pandas/serializers.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 15b9f2fdeb0f0..42562e1fb9c46 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -143,10 +143,7 @@ def _create_batch(self, series): import pandas as pd import pyarrow as pa from pyspark.sql.pandas.types import _check_series_convert_timestamps_internal - try: - from pandas import CategoricalDtype - except ImportError: - from pandas.api.types import CategoricalDtype + from pandas.api.types import is_categorical # Make input conform to [(series1, type1), (series2, type2), ...] if not isinstance(series, (list, tuple)) or \ (len(series) == 2 and isinstance(series[1], pa.DataType)): @@ -158,7 +155,7 @@ def create_array(s, t): # Ensure timestamp series are in expected form for Spark internal representation if t is not None and pa.types.is_timestamp(t): s = _check_series_convert_timestamps_internal(s, self._timezone) - elif type(s.dtype) == CategoricalDtype: + elif is_categorical(s.dtype): # Note: This can be removed once minimum pyarrow version is >= 0.16.1 s = s.astype(s.dtypes.categories.dtype) try: From 22dda6e18e91c6db6fa8ff9fafaafe09a79db4ea Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 11 Jun 2020 03:29:12 +0000 Subject: [PATCH 07/40] [SPARK-31939][SQL][TEST-JAVA11] Fix Parsing day of year when year field pattern is missing ### What changes were proposed in this pull request? If a datetime pattern contains no year field, the day of year field should not be ignored if exists e.g. ``` spark-sql> select to_timestamp('31', 'DD'); 1970-01-01 00:00:00 spark-sql> select to_timestamp('31 30', 'DD dd'); 1970-01-30 00:00:00 spark.sql.legacy.timeParserPolicy legacy spark-sql> select to_timestamp('31', 'DD'); 1970-01-31 00:00:00 spark-sql> select to_timestamp('31 30', 'DD dd'); NULL ``` This PR only fixes some corner cases that use 'D' pattern to parse datetimes and there is w/o 'y'. ### Why are the changes needed? fix some corner cases ### Does this PR introduce _any_ user-facing change? yes, the day of year field will not be ignored ### How was this patch tested? add unit tests. Closes #28766 from yaooqinn/SPARK-31939. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../util/DateTimeFormatterHelper.scala | 26 ++++- .../catalyst/util/DateFormatterSuite.scala | 2 + .../util/DatetimeFormatterSuite.scala | 78 +++++++++++++ .../util/TimestampFormatterSuite.scala | 2 + .../inputs/datetime-parsing-invalid.sql | 20 ++++ .../inputs/datetime-parsing-legacy.sql | 2 + .../sql-tests/inputs/datetime-parsing.sql | 16 +++ .../results/datetime-parsing-invalid.sql.out | 110 ++++++++++++++++++ .../results/datetime-parsing-legacy.sql.out | 106 +++++++++++++++++ .../results/datetime-parsing.sql.out | 106 +++++++++++++++++ 10 files changed, 465 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-legacy.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 992a2b12a462f..5de06af6af12a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -39,6 +39,18 @@ trait DateTimeFormatterHelper { } } + private def verifyLocalDate( + accessor: TemporalAccessor, field: ChronoField, candidate: LocalDate): Unit = { + if (accessor.isSupported(field)) { + val actual = accessor.get(field) + val expected = candidate.get(field) + if (actual != expected) { + throw new DateTimeException(s"Conflict found: Field $field $actual differs from" + + s" $field $expected derived from $candidate") + } + } + } + protected def toLocalDate(accessor: TemporalAccessor): LocalDate = { val localDate = accessor.query(TemporalQueries.localDate()) // If all the date fields are specified, return the local date directly. @@ -48,9 +60,17 @@ trait DateTimeFormatterHelper { // later, and we should provide default values for missing fields. // To be compatible with Spark 2.4, we pick 1970 as the default value of year. val year = getOrDefault(accessor, ChronoField.YEAR, 1970) - val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1) - val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1) - LocalDate.of(year, month, day) + if (accessor.isSupported(ChronoField.DAY_OF_YEAR)) { + val dayOfYear = accessor.get(ChronoField.DAY_OF_YEAR) + val date = LocalDate.ofYearDay(year, dayOfYear) + verifyLocalDate(accessor, ChronoField.MONTH_OF_YEAR, date) + verifyLocalDate(accessor, ChronoField.DAY_OF_MONTH, date) + date + } else { + val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1) + val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1) + LocalDate.of(year, month, day) + } } private def toLocalTime(accessor: TemporalAccessor): LocalTime = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala index 4892deae92b3d..0a29d94dd984d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala @@ -31,6 +31,8 @@ class DateFormatterSuite extends DatetimeFormatterSuite { DateFormatter(pattern, UTC, isParsing) } + override protected def useDateFormatter: Boolean = true + test("parsing dates") { outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala index 31ff50fda1ad6..b78facd963338 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala @@ -17,15 +17,61 @@ package org.apache.spark.sql.catalyst.util +import java.time.DateTimeException + import org.scalatest.Matchers import org.apache.spark.{SparkFunSuite, SparkUpgradeException} import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC} trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers { import DateTimeFormatterHelper._ + import LegacyDateFormats._ def checkFormatterCreation(pattern: String, isParsing: Boolean): Unit + private def dateFormatter( + pattern: String, ldf: LegacyDateFormat = FAST_DATE_FORMAT): DateFormatter = { + DateFormatter(pattern, UTC, DateFormatter.defaultLocale, ldf, isParsing = true) + } + + private def timestampFormatter( + pattern: String, ldf: LegacyDateFormat = SIMPLE_DATE_FORMAT): TimestampFormatter = { + TimestampFormatter(pattern, UTC, legacyFormat = ldf, isParsing = true) + } + + protected def useDateFormatter: Boolean + + private def assertEqual(pattern: String, datetimeStr: String, expected: Long): Unit = { + if (useDateFormatter) { + assert(dateFormatter(pattern).parse(datetimeStr) === + DateTimeUtils.microsToEpochDays(expected, UTC)) + } else { + assert(timestampFormatter(pattern).parse(datetimeStr) === expected) + } + } + + private def assertError(pattern: String, datetimeStr: String, expectedMsg: String): Unit = { + if (useDateFormatter) { + LegacyDateFormats.values.foreach { ldf => + // The legacy DateFormatter is always lenient by default + val e = intercept[SparkUpgradeException](dateFormatter(pattern, ldf).parse(datetimeStr)) + assert(e.getCause.getMessage.contains(expectedMsg)) + } + } else { + // In strict mode, the legacy TimestampFormatter fails too + val e = intercept[DateTimeException](timestampFormatter(pattern).parse(datetimeStr)) + assert(e.getMessage.contains(expectedMsg)) + // In lenient mode, the legacy TimestampFormatter does not fail + Seq(FAST_DATE_FORMAT, LENIENT_SIMPLE_DATE_FORMAT).foreach { ldf => + val e = intercept[SparkUpgradeException] { + timestampFormatter(pattern, ldf).parse(datetimeStr) + } + assert(e.getCause.getMessage.contains(expectedMsg)) + } + } + } + test("explicitly forbidden datetime patterns") { Seq(true, false).foreach { isParsing => @@ -51,4 +97,36 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers pattern => intercept[SparkUpgradeException](checkFormatterCreation(pattern, true)) } } + + test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") { + // resolved to queryable LocaleDate or fail directly + assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29)) + assertError("yyyy-dd-DD", "2020-02-60", + "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29") + assertEqual("yyyy-MM-DD", "2020-02-60", date(2020, 2, 29)) + assertError("yyyy-MM-DD", "2020-03-60", + "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 2020-02-29") + assertEqual("yyyy-MM-dd-DD", "2020-02-29-60", date(2020, 2, 29)) + assertError("yyyy-MM-dd-DD", "2020-03-01-60", + "Field DayOfYear 61 differs from DayOfYear 60 derived from 2020-03-01") + assertEqual("yyyy-DDD", "2020-366", date(2020, 12, 31)) + assertError("yyyy-DDD", "2019-366", + "Invalid date 'DayOfYear 366' as '2019' is not a leap year") + + // unresolved and need to check manually(SPARK-31939 fixed) + assertEqual("DDD", "365", date(1970, 12, 31)) + assertError("DDD", "366", + "Invalid date 'DayOfYear 366' as '1970' is not a leap year") + assertEqual("MM-DD", "03-60", date(1970, 3)) + assertError("MM-DD", "02-60", + "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 1970-03-01") + assertEqual("MM-dd-DD", "02-28-59", date(1970, 2, 28)) + assertError("MM-dd-DD", "02-28-60", + "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 1970-03-01") + assertError("MM-dd-DD", "02-28-58", + "Field DayOfMonth 28 differs from DayOfMonth 27 derived from 1970-02-27") + assertEqual("dd-DD", "28-59", date(1970, 2, 28)) + assertError("dd-DD", "27-59", + "Field DayOfMonth 27 differs from DayOfMonth 28 derived from 1970-02-28") + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index e70f805b30f39..51286986b835c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -33,6 +33,8 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { TimestampFormatter(pattern, UTC, isParsing) } + override protected def useDateFormatter: Boolean = false + test("parsing timestamps using time zones") { val localDate = "2018-12-02T10:11:12.001234" val expectedMicros = Map( diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql new file mode 100644 index 0000000000000..0b313e5a0b9f4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql @@ -0,0 +1,20 @@ +--- TESTS FOR DATETIME PARSING FUNCTIONS WITH INVALID VALUES --- + +-- parsing invalid values with pattern 'D' +select to_timestamp('366', 'D'); +select to_timestamp('9', 'DD'); +-- in java 8 this case is invalid, but valid in java 11, disabled for jenkins +-- select to_timestamp('100', 'DD'); +select to_timestamp('366', 'DD'); +select to_timestamp('9', 'DDD'); +select to_timestamp('99', 'DDD'); +select to_timestamp('30-365', 'dd-DDD'); +select to_timestamp('11-365', 'MM-DDD'); +select to_timestamp('2019-366', 'yyyy-DDD'); +select to_timestamp('12-30-365', 'MM-dd-DDD'); +select to_timestamp('2020-01-365', 'yyyy-dd-DDD'); +select to_timestamp('2020-10-350', 'yyyy-MM-DDD'); +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD'); +-- add a special case to test csv, because the legacy formatter it uses is lenient then Spark should +-- throw SparkUpgradeException +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-legacy.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-legacy.sql new file mode 100644 index 0000000000000..ee1afe502ab79 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.timeParserPolicy=LEGACY +--IMPORT datetime-parsing.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql new file mode 100644 index 0000000000000..74866d9c6ffa1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql @@ -0,0 +1,16 @@ +--- TESTS FOR DATETIME PARSING FUNCTIONS --- + +-- parsing with pattern 'D' +select to_timestamp('9', 'D'); +select to_timestamp('300', 'D'); +select to_timestamp('09', 'DD'); +select to_timestamp('99', 'DD'); +select to_timestamp('009', 'DDD'); +select to_timestamp('365', 'DDD'); +select to_timestamp('31-365', 'dd-DDD'); +select to_timestamp('12-365', 'MM-DDD'); +select to_timestamp('2020-365', 'yyyy-DDD'); +select to_timestamp('12-31-365', 'MM-dd-DDD'); +select to_timestamp('2020-30-365', 'yyyy-dd-DDD'); +select to_timestamp('2020-12-350', 'yyyy-MM-DDD'); +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD'); diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out new file mode 100644 index 0000000000000..dcee0d4c270a3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -0,0 +1,110 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query +select to_timestamp('366', 'D') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('9', 'DD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('366', 'DD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('9', 'DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('99', 'DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '99' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('30-365', 'dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('11-365', 'MM-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-366', 'yyyy-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('12-30-365', 'MM-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-01-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-10-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-366' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out new file mode 100644 index 0000000000000..742e0839d919a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out @@ -0,0 +1,106 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query +select to_timestamp('9', 'D') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('300', 'D') +-- !query schema +struct +-- !query output +1970-10-27 00:00:00 + + +-- !query +select to_timestamp('09', 'DD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('99', 'DD') +-- !query schema +struct +-- !query output +1970-04-09 00:00:00 + + +-- !query +select to_timestamp('009', 'DDD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('365', 'DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('31-365', 'dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('12-365', 'MM-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-365', 'yyyy-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('12-31-365', 'MM-dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-30-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-12-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +2020-12-15 00:00:00 + + +-- !query +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-31 00:00:00 diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out new file mode 100644 index 0000000000000..742e0839d919a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out @@ -0,0 +1,106 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query +select to_timestamp('9', 'D') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('300', 'D') +-- !query schema +struct +-- !query output +1970-10-27 00:00:00 + + +-- !query +select to_timestamp('09', 'DD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('99', 'DD') +-- !query schema +struct +-- !query output +1970-04-09 00:00:00 + + +-- !query +select to_timestamp('009', 'DDD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('365', 'DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('31-365', 'dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('12-365', 'MM-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-365', 'yyyy-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('12-31-365', 'MM-dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-30-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-12-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +2020-12-15 00:00:00 + + +-- !query +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-31 00:00:00 From 5d7853750f083b7e4dbf673add62abc93eaa165b Mon Sep 17 00:00:00 2001 From: manuzhang Date: Thu, 11 Jun 2020 03:34:07 +0000 Subject: [PATCH 08/40] [SPARK-31942] Revert "[SPARK-31864][SQL] Adjust AQE skew join trigger condition ### What changes were proposed in this pull request? This reverts commit https://github.com/apache/spark/commit/b9737c3c228f465d332e41f1ea0cece2a5f7667e while keeping following changes * set default value of `spark.sql.adaptive.skewJoin.skewedPartitionFactor` to 5 * improve tests * remove unused imports ### Why are the changes needed? As discussed in https://github.com/apache/spark/pull/28669#issuecomment-641044531, revert SPARK-31864 for optimizing skew join to work for extremely clustered keys. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #28770 from manuzhang/spark-31942. Authored-by: manuzhang Signed-off-by: Wenchen Fan --- .../adaptive/OptimizeSkewedJoin.scala | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index b5d287ca7ac79..396c9c9d6b4e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import org.apache.commons.io.FileUtils -import org.apache.spark.{MapOutputTrackerMaster, SparkEnv} +import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ @@ -70,9 +70,9 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { size > conf.getConf(SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD) } - private def medianSize(sizes: Seq[Long]): Long = { - val numPartitions = sizes.length - val bytes = sizes.sorted + private def medianSize(stats: MapOutputStatistics): Long = { + val numPartitions = stats.bytesByPartitionId.length + val bytes = stats.bytesByPartitionId.sorted numPartitions match { case _ if (numPartitions % 2 == 0) => math.max((bytes(numPartitions / 2) + bytes(numPartitions / 2 - 1)) / 2, 1) @@ -163,16 +163,16 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { if supportedJoinTypes.contains(joinType) => assert(left.partitionsWithSizes.length == right.partitionsWithSizes.length) val numPartitions = left.partitionsWithSizes.length - // Use the median size of the actual (coalesced) partition sizes to detect skewed partitions. - val leftMedSize = medianSize(left.partitionsWithSizes.map(_._2)) - val rightMedSize = medianSize(right.partitionsWithSizes.map(_._2)) + // We use the median size of the original shuffle partitions to detect skewed partitions. + val leftMedSize = medianSize(left.mapStats) + val rightMedSize = medianSize(right.mapStats) logDebug( s""" |Optimizing skewed join. |Left side partitions size info: - |${getSizeInfo(leftMedSize, left.partitionsWithSizes.map(_._2))} + |${getSizeInfo(leftMedSize, left.mapStats.bytesByPartitionId)} |Right side partitions size info: - |${getSizeInfo(rightMedSize, right.partitionsWithSizes.map(_._2))} + |${getSizeInfo(rightMedSize, right.mapStats.bytesByPartitionId)} """.stripMargin) val canSplitLeft = canSplitLeftSide(joinType) val canSplitRight = canSplitRightSide(joinType) @@ -291,15 +291,17 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { private object ShuffleStage { def unapply(plan: SparkPlan): Option[ShuffleStageInfo] = plan match { case s: ShuffleQueryStageExec if s.mapStats.isDefined => - val sizes = s.mapStats.get.bytesByPartitionId + val mapStats = s.mapStats.get + val sizes = mapStats.bytesByPartitionId val partitions = sizes.zipWithIndex.map { case (size, i) => CoalescedPartitionSpec(i, i + 1) -> size } - Some(ShuffleStageInfo(s, partitions)) + Some(ShuffleStageInfo(s, mapStats, partitions)) case CustomShuffleReaderExec(s: ShuffleQueryStageExec, partitionSpecs) if s.mapStats.isDefined && partitionSpecs.nonEmpty => - val sizes = s.mapStats.get.bytesByPartitionId + val mapStats = s.mapStats.get + val sizes = mapStats.bytesByPartitionId val partitions = partitionSpecs.map { case spec @ CoalescedPartitionSpec(start, end) => var sum = 0L @@ -312,7 +314,7 @@ private object ShuffleStage { case other => throw new IllegalArgumentException( s"Expect CoalescedPartitionSpec but got $other") } - Some(ShuffleStageInfo(s, partitions)) + Some(ShuffleStageInfo(s, mapStats, partitions)) case _ => None } @@ -320,4 +322,5 @@ private object ShuffleStage { private case class ShuffleStageInfo( shuffleStage: ShuffleQueryStageExec, + mapStats: MapOutputStatistics, partitionsWithSizes: Seq[(CoalescedPartitionSpec, Long)]) From 76b5ed4ffaa82241944aeae0a0238cf8ee86e44a Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 10 Jun 2020 20:59:48 -0700 Subject: [PATCH 09/40] [SPARK-31935][SQL][TESTS][FOLLOWUP] Fix the test case for Hadoop2/3 ### What changes were proposed in this pull request? This PR updates the test case to accept Hadoop 2/3 error message correctly. ### Why are the changes needed? SPARK-31935(#28760) breaks Hadoop 3.2 UT because Hadoop 2 and Hadoop 3 have different exception messages. In https://github.com/apache/spark/pull/28791, there are two test suites missed the fix ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test Closes #28796 from gengliangwang/SPARK-31926-followup. Authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/datasources/DataSourceSuite.scala | 3 ++- .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala index 9345158fd07ed..aa9179195042b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala @@ -142,7 +142,8 @@ class DataSourceSuite extends SharedSparkSession with PrivateMethodTester { val message = intercept[java.io.IOException] { dataSource invokePrivate checkAndGlobPathIfNecessary(false, false) }.getMessage - assert(message.equals("No FileSystem for scheme: nonexistsFs")) + val expectMessage = "No FileSystem for scheme nonexistsFs" + assert(message.filterNot(Set(':', '"').contains) == expectMessage) } } 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 32dceaac7059c..7b16aebc531fb 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 @@ -536,11 +536,11 @@ class FileStreamSourceSuite extends FileStreamSourceTest { withTempDir { dir => val path = dir.getCanonicalPath val defaultFs = "nonexistFS://nonexistFS" - val expectMessage = "No FileSystem for scheme: nonexistFS" + val expectMessage = "No FileSystem for scheme nonexistFS" val message = intercept[java.io.IOException] { spark.readStream.option("fs.defaultFS", defaultFs).text(path) }.getMessage - assert(message == expectMessage) + assert(message.filterNot(Set(':', '"').contains) == expectMessage) } } From 56264fb5d3ad1a488be5e08feb2e0304d1c2ed6a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 10 Jun 2020 21:15:40 -0700 Subject: [PATCH 10/40] [SPARK-31965][TESTS][PYTHON] Move doctests related to Java function registration to test conditionally ### What changes were proposed in this pull request? This PR proposes to move the doctests in `registerJavaUDAF` and `registerJavaFunction` to the proper unittests that run conditionally when the test classes are present. Both tests are dependent on the test classes in JVM side, `test.org.apache.spark.sql.JavaStringLength` and `test.org.apache.spark.sql.MyDoubleAvg`. So if you run the tests against the plain `sbt package`, it fails as below: ``` ********************************************************************** File "/.../spark/python/pyspark/sql/udf.py", line 366, in pyspark.sql.udf.UDFRegistration.registerJavaFunction Failed example: spark.udf.registerJavaFunction( "javaStringLength", "test.org.apache.spark.sql.JavaStringLength", IntegerType()) Exception raised: Traceback (most recent call last): ... test.org.apache.spark.sql.JavaStringLength, please make sure it is on the classpath; ... 6 of 7 in pyspark.sql.udf.UDFRegistration.registerJavaFunction 2 of 4 in pyspark.sql.udf.UDFRegistration.registerJavaUDAF ***Test Failed*** 8 failures. ``` ### Why are the changes needed? In order to support to run the tests against the plain SBT build. See also https://spark.apache.org/developer-tools.html ### Does this PR introduce _any_ user-facing change? No, it's test-only. ### How was this patch tested? Manually tested as below: ```bash ./build/sbt -DskipTests -Phive-thriftserver clean package cd python ./run-tests --python-executable=python3 --testname="pyspark.sql.udf UserDefinedFunction" ./run-tests --python-executable=python3 --testname="pyspark.sql.tests.test_udf UDFTests" ``` ```bash ./build/sbt -DskipTests -Phive-thriftserver clean test:package cd python ./run-tests --python-executable=python3 --testname="pyspark.sql.udf UserDefinedFunction" ./run-tests --python-executable=python3 --testname="pyspark.sql.tests.test_udf UDFTests" ``` Closes #28795 from HyukjinKwon/SPARK-31965. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_udf.py | 28 ++++++++++++++++++++++++++++ python/pyspark/sql/udf.py | 14 +++++++++----- 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 2689b9c33d576..d673f7c15918f 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -21,6 +21,8 @@ import tempfile import unittest +import py4j + from pyspark import SparkContext from pyspark.sql import SparkSession, Column, Row from pyspark.sql.functions import UserDefinedFunction, udf @@ -357,6 +359,32 @@ def test_udf_registration_returns_udf(self): df.select(add_four("id").alias("plus_four")).collect() ) + @unittest.skipIf(not test_compiled, test_not_compiled_message) + def test_register_java_function(self): + self.spark.udf.registerJavaFunction( + "javaStringLength", "test.org.apache.spark.sql.JavaStringLength", IntegerType()) + [value] = self.spark.sql("SELECT javaStringLength('test')").first() + self.assertEqual(value, 4) + + self.spark.udf.registerJavaFunction( + "javaStringLength2", "test.org.apache.spark.sql.JavaStringLength") + [value] = self.spark.sql("SELECT javaStringLength2('test')").first() + self.assertEqual(value, 4) + + self.spark.udf.registerJavaFunction( + "javaStringLength3", "test.org.apache.spark.sql.JavaStringLength", "integer") + [value] = self.spark.sql("SELECT javaStringLength3('test')").first() + self.assertEqual(value, 4) + + @unittest.skipIf(not test_compiled, test_not_compiled_message) + def test_register_java_udaf(self): + self.spark.udf.registerJavaUDAF("javaUDAF", "test.org.apache.spark.sql.MyDoubleAvg") + df = self.spark.createDataFrame([(1, "a"), (2, "b"), (3, "a")], ["id", "name"]) + df.createOrReplaceTempView("df") + row = self.spark.sql( + "SELECT name, javaUDAF(id) as avg from df group by name order by name desc").first() + self.assertEqual(row.asDict(), Row(name='b', avg=102.0).asDict()) + def test_non_existed_udf(self): spark = self.spark self.assertRaisesRegexp(AnalysisException, "Can not load class non_existed_udf", diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 10546ecacc57f..da68583b04e1c 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -365,17 +365,20 @@ def registerJavaFunction(self, name, javaClassName, returnType=None): >>> from pyspark.sql.types import IntegerType >>> spark.udf.registerJavaFunction( ... "javaStringLength", "test.org.apache.spark.sql.JavaStringLength", IntegerType()) - >>> spark.sql("SELECT javaStringLength('test')").collect() + ... # doctest: +SKIP + >>> spark.sql("SELECT javaStringLength('test')").collect() # doctest: +SKIP [Row(javaStringLength(test)=4)] >>> spark.udf.registerJavaFunction( ... "javaStringLength2", "test.org.apache.spark.sql.JavaStringLength") - >>> spark.sql("SELECT javaStringLength2('test')").collect() + ... # doctest: +SKIP + >>> spark.sql("SELECT javaStringLength2('test')").collect() # doctest: +SKIP [Row(javaStringLength2(test)=4)] >>> spark.udf.registerJavaFunction( ... "javaStringLength3", "test.org.apache.spark.sql.JavaStringLength", "integer") - >>> spark.sql("SELECT javaStringLength3('test')").collect() + ... # doctest: +SKIP + >>> spark.sql("SELECT javaStringLength3('test')").collect() # doctest: +SKIP [Row(javaStringLength3(test)=4)] """ @@ -395,10 +398,11 @@ def registerJavaUDAF(self, name, javaClassName): :param javaClassName: fully qualified name of java class >>> spark.udf.registerJavaUDAF("javaUDAF", "test.org.apache.spark.sql.MyDoubleAvg") + ... # doctest: +SKIP >>> df = spark.createDataFrame([(1, "a"),(2, "b"), (3, "a")],["id", "name"]) >>> df.createOrReplaceTempView("df") - >>> spark.sql("SELECT name, javaUDAF(id) as avg from df group by name order by name desc") \ - .collect() + >>> q = "SELECT name, javaUDAF(id) as avg from df group by name order by name desc" + >>> spark.sql(q).collect() # doctest: +SKIP [Row(name=u'b', avg=102.0), Row(name=u'a', avg=102.0)] """ From 4afe2b1bc9ef190c0117e28da447871b90100622 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 10 Jun 2020 21:32:16 -0700 Subject: [PATCH 11/40] [SPARK-28199][SS][FOLLOWUP] Remove package private in class/object in sql.execution package ### What changes were proposed in this pull request? This PR proposes to remove package private in classes/objects in sql.execution package, as per SPARK-16964. ### Why are the changes needed? This is per post-hoc review comment, see https://github.com/apache/spark/pull/24996#discussion_r437126445 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/A Closes #28790 from HeartSaVioR/SPARK-28199-FOLLOWUP-apply-SPARK-16964. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/streaming/Triggers.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index f29970d5de192..ebd237bfc0c19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -50,17 +50,17 @@ private object Triggers { * A [[Trigger]] that processes only one batch of data in a streaming query then terminates * the query. */ -private[sql] case object OneTimeTrigger extends Trigger +case object OneTimeTrigger extends Trigger /** * A [[Trigger]] that runs a query periodically based on the processing time. If `interval` is 0, * the query will run as fast as possible. */ -private[sql] case class ProcessingTimeTrigger(intervalMs: Long) extends Trigger { +case class ProcessingTimeTrigger(intervalMs: Long) extends Trigger { Triggers.validate(intervalMs) } -private[sql] object ProcessingTimeTrigger { +object ProcessingTimeTrigger { import Triggers._ def apply(interval: String): ProcessingTimeTrigger = { @@ -84,11 +84,11 @@ private[sql] object ProcessingTimeTrigger { * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at * the specified interval. */ -private[sql] case class ContinuousTrigger(intervalMs: Long) extends Trigger { +case class ContinuousTrigger(intervalMs: Long) extends Trigger { Triggers.validate(intervalMs) } -private[sql] object ContinuousTrigger { +object ContinuousTrigger { import Triggers._ def apply(interval: String): ContinuousTrigger = { From 56d4f27cf66cda81940a0a9e7959c03fad2844fc Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 10 Jun 2020 21:56:35 -0700 Subject: [PATCH 12/40] [SPARK-31966][ML][TESTS][PYTHON] Increase the timeout for StreamingLogisticRegressionWithSGDTests.test_training_and_prediction ### What changes were proposed in this pull request? This is similar with https://github.com/apache/spark/commit/64cb6f7066134a0b9e441291992d2da73de5d918 The test `StreamingLogisticRegressionWithSGDTests.test_training_and_prediction` seems also flaky. This PR just increases the timeout to 3 mins too. The cause is very likely the time elapsed. See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123787/testReport/pyspark.mllib.tests.test_streaming_algorithms/StreamingLogisticRegressionWithSGDTests/test_training_and_prediction/ ``` Traceback (most recent call last): File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 330, in test_training_and_prediction eventually(condition, timeout=60.0) File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/testing/utils.py", line 90, in eventually % (timeout, lastValue)) AssertionError: Test failed due to timeout after 60 sec, with last condition returning: Latest errors: 0.67, 0.71, 0.78, 0.7, 0.75, 0.74, 0.73, 0.69, 0.62, 0.71, 0.69, 0.75, 0.72, 0.77, 0.71, 0.74, 0.76, 0.78, 0.7, 0.78, 0.8, 0.74, 0.77, 0.75, 0.76, 0.76, 0.75 ``` ### Why are the changes needed? To make PR builds more stable. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Jenkins will test them out. Closes #28798 from HyukjinKwon/SPARK-31966. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- python/pyspark/mllib/tests/test_streaming_algorithms.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/mllib/tests/test_streaming_algorithms.py b/python/pyspark/mllib/tests/test_streaming_algorithms.py index f57de83bae64d..2f35e07cdf71c 100644 --- a/python/pyspark/mllib/tests/test_streaming_algorithms.py +++ b/python/pyspark/mllib/tests/test_streaming_algorithms.py @@ -327,7 +327,7 @@ def condition(): return True return "Latest errors: " + ", ".join(map(lambda x: str(x), errors)) - eventually(condition, timeout=60.0) + eventually(condition, timeout=180.0) class StreamingLinearRegressionWithTests(MLLibStreamingTestCase): From 6fb9c80da129d0b43f9ff5b8be6ce8bad992a4ed Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 11 Jun 2020 06:39:14 +0000 Subject: [PATCH 13/40] [SPARK-31958][SQL] normalize special floating numbers in subquery ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23388 . https://github.com/apache/spark/pull/23388 has an issue: it doesn't handle subquery expressions and assumes they will be turned into joins. However, this is not true for non-correlated subquery expressions. This PR fixes this issue. It now doesn't skip `Subquery`, and subquery expressions will be handled by `OptimizeSubqueries`, which runs the optimizer with the subquery. Note that, correlated subquery expressions will be handled twice: once in `OptimizeSubqueries`, once later when it becomes join. This is OK as `NormalizeFloatingNumbers` is idempotent now. ### Why are the changes needed? fix a bug ### Does this PR introduce _any_ user-facing change? yes, see the newly added test. ### How was this patch tested? new test Closes #28785 from cloud-fan/normalize. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../optimizer/NormalizeFloatingNumbers.scala | 4 ---- .../org/apache/spark/sql/SQLQuerySuite.scala | 18 ++++++++++++++++++ 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala index 5f94af5ffe636..43738204c6704 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -56,10 +56,6 @@ import org.apache.spark.sql.types._ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan match { - // A subquery will be rewritten into join later, and will go through this rule - // eventually. Here we skip subquery, as we only need to run this rule once. - case _: Subquery => plan - case _ => plan transform { case w: Window if w.partitionSpec.exists(p => needNormalize(p)) => // Although the `windowExpressions` may refer to `partitionSpec` expressions, we don't need 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 f7a904169d6c3..a219b91627b2b 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 @@ -3503,6 +3503,24 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(sql("select CAST(-32768 as short) DIV CAST (-1 as short)"), Seq(Row(Short.MinValue.toLong * -1))) } + + test("normalize special floating numbers in subquery") { + withTempView("v1", "v2", "v3") { + Seq(-0.0).toDF("d").createTempView("v1") + Seq(0.0).toDF("d").createTempView("v2") + spark.range(2).createTempView("v3") + + // non-correlated subquery + checkAnswer(sql("SELECT (SELECT v1.d FROM v1 JOIN v2 ON v1.d = v2.d)"), Row(-0.0)) + // correlated subquery + checkAnswer( + sql( + """ + |SELECT id FROM v3 WHERE EXISTS + | (SELECT v1.d FROM v1 JOIN v2 ON v1.d = v2.d WHERE id > 0) + |""".stripMargin), Row(1)) + } + } } case class Foo(bar: Option[String]) From 912d45df7c6535336f72c971c90fecd11cfe87e9 Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Thu, 11 Jun 2020 22:03:40 +0900 Subject: [PATCH 14/40] [SPARK-31954][SQL] Delete duplicate testcase in HiveQuerySuite ### What changes were proposed in this pull request? remove duplicate test cases ### Why are the changes needed? improve test quality ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? No test Closes #28782 from GuoPhilipse/31954-delete-duplicate-testcase. Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Co-authored-by: GuoPhilipse Signed-off-by: HyukjinKwon --- ...ast #3-0-732ed232ac592c5e7f7c913a88874fd2} | 0 ...ast #4-0-6d2da5cfada03605834e38bc4075bc79} | 0 ...cast #4-0-732ed232ac592c5e7f7c913a88874fd2 | 1 - ...cast #8-0-6d2da5cfada03605834e38bc4075bc79 | 1 - .../sql/hive/execution/HiveQuerySuite.scala | 26 +++++++------------ 5 files changed, 10 insertions(+), 18 deletions(-) rename sql/hive/src/test/resources/golden/{timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 => timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2} (100%) rename sql/hive/src/test/resources/golden/{timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 => timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 rename to sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 rename to sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 deleted file mode 100644 index 5625e59da8873..0000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 +++ /dev/null @@ -1 +0,0 @@ -1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 deleted file mode 100644 index 1d94c8a014fb4..0000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 +++ /dev/null @@ -1 +0,0 @@ --1.2 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 63b985fbe4d32..b10a8cb8bf2bf 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 @@ -556,33 +556,27 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd assert(1 == res.getDouble(0)) } - createQueryTest("timestamp cast #2", - "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - - test("timestamp cast #3") { - val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head - assert(1200 == res.getInt(0)) + test("timestamp cast #2") { + val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + assert(-1 == res.get(0)) } - createQueryTest("timestamp cast #4", + createQueryTest("timestamp cast #3", "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + createQueryTest("timestamp cast #4", + "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + test("timestamp cast #5") { - val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head - assert(-1 == res.get(0)) + val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + assert(1200 == res.getInt(0)) } - createQueryTest("timestamp cast #6", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - - test("timestamp cast #7") { + test("timestamp cast #6") { val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) } - createQueryTest("timestamp cast #8", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") From 91cd06bd563f3dfa7c8ed7c426c3e87dfaa0af98 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 11 Jun 2020 14:16:12 +0000 Subject: [PATCH 15/40] [SPARK-8981][CORE][FOLLOW-UP] Clean up MDC properties after running a task ### What changes were proposed in this pull request? This PR is a followup of #26624. This PR cleans up MDC properties if the original value is empty. Besides, this PR adds a warning and ignore the value when the user tries to override the value of `taskName`. ### Why are the changes needed? Before this PR, running the following jobs: ``` sc.setLocalProperty("mdc.my", "ABC") sc.parallelize(1 to 100).count() sc.setLocalProperty("mdc.my", null) sc.parallelize(1 to 100).count() ``` there's still MDC value "ABC" in the log of the second count job even if we've unset the value. ### Does this PR introduce _any_ user-facing change? Yes, user will 1) no longer see the MDC values after unsetting the value; 2) see a warning if he/she tries to override the value of `taskName`. ### How was this patch tested? Tested Manaually. Closes #28756 from Ngone51/followup-8981. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/executor/Executor.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 45cec726c4ca7..93d1acdd2d156 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -401,9 +401,7 @@ private[spark] class Executor( } override def run(): Unit = { - setMDCForTask(taskName, mdcProperties) - threadId = Thread.currentThread.getId Thread.currentThread.setName(threadName) val threadMXBean = ManagementFactory.getThreadMXBean @@ -703,11 +701,11 @@ private[spark] class Executor( } private def setMDCForTask(taskName: String, mdc: Seq[(String, String)]): Unit = { + // make sure we run the task with the user-specified mdc properties only + MDC.clear() + mdc.foreach { case (key, value) => MDC.put(key, value) } + // avoid overriding the takName by the user MDC.put("taskName", taskName) - - mdc.foreach { case (key, value) => - MDC.put(key, value) - } } /** @@ -750,9 +748,7 @@ private[spark] class Executor( private[this] val takeThreadDump: Boolean = conf.get(TASK_REAPER_THREAD_DUMP) override def run(): Unit = { - setMDCForTask(taskRunner.taskName, taskRunner.mdcProperties) - val startTimeNs = System.nanoTime() def elapsedTimeNs = System.nanoTime() - startTimeNs def timeoutExceeded(): Boolean = killTimeoutNs > 0 && elapsedTimeNs > killTimeoutNs From 11d3a744e20fe403dd76e18d57963b6090a7c581 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 11 Jun 2020 10:13:45 -0700 Subject: [PATCH 16/40] [SPARK-31705][SQL] Push more possible predicates through Join via CNF conversion ### What changes were proposed in this pull request? This PR add a new rule to support push predicate through join by rewriting join condition to CNF(conjunctive normal form). The following example is the steps of this rule: 1. Prepare Table: ```sql CREATE TABLE x(a INT); CREATE TABLE y(b INT); ... SELECT * FROM x JOIN y ON ((a < 0 and a > b) or a > 10); ``` 2. Convert the join condition to CNF: ``` (a < 0 or a > 10) and (a > b or a > 10) ``` 3. Split conjunctive predicates Predicates ---| (a < 0 or a > 10) (a > b or a > 10) 4. Push predicate Table | Predicate --- | --- x | (a < 0 or a > 10) ### Why are the changes needed? Improve query performance. PostgreSQL, [Impala](https://issues.apache.org/jira/browse/IMPALA-9183) and Hive support this feature. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test and benchmark test. SQL | Before this PR | After this PR --- | --- | --- TPCDS 5T Q13 | 84s | 21s TPCDS 5T q85 | 66s | 34s TPCH 1T q19 | 37s | 32s Closes #28733 from gengliangwang/cnf. Lead-authored-by: Gengliang Wang Co-authored-by: Yuming Wang Signed-off-by: Gengliang Wang --- .../sql/catalyst/expressions/predicates.scala | 96 ++++++++++- .../sql/catalyst/optimizer/Optimizer.scala | 9 +- .../PushCNFPredicateThroughJoin.scala | 62 +++++++ .../apache/spark/sql/internal/SQLConf.scala | 15 ++ .../ConjunctiveNormalFormPredicateSuite.scala | 128 ++++++++++++++ .../optimizer/FilterPushdownSuite.scala | 162 +++++++++++++++++- 6 files changed, 468 insertions(+), 4 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 2c4f41f98ac20..c9b57367e0f44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.immutable.TreeSet +import scala.collection.mutable +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult @@ -95,7 +97,7 @@ object Predicate extends CodeGeneratorWithInterpretedFallback[Expression, BasePr } } -trait PredicateHelper { +trait PredicateHelper extends Logging { protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { condition match { case And(cond1, cond2) => @@ -198,6 +200,98 @@ trait PredicateHelper { case e: Unevaluable => false case e => e.children.forall(canEvaluateWithinJoin) } + + /** + * Convert an expression into conjunctive normal form. + * Definition and algorithm: https://en.wikipedia.org/wiki/Conjunctive_normal_form + * CNF can explode exponentially in the size of the input expression when converting [[Or]] + * clauses. Use a configuration [[SQLConf.MAX_CNF_NODE_COUNT]] to prevent such cases. + * + * @param condition to be converted into CNF. + * @return the CNF result as sequence of disjunctive expressions. If the number of expressions + * exceeds threshold on converting `Or`, `Seq.empty` is returned. + */ + def conjunctiveNormalForm(condition: Expression): Seq[Expression] = { + val postOrderNodes = postOrderTraversal(condition) + val resultStack = new mutable.Stack[Seq[Expression]] + val maxCnfNodeCount = SQLConf.get.maxCnfNodeCount + // Bottom up approach to get CNF of sub-expressions + while (postOrderNodes.nonEmpty) { + val cnf = postOrderNodes.pop() match { + case _: And => + val right = resultStack.pop() + val left = resultStack.pop() + left ++ right + case _: Or => + // For each side, there is no need to expand predicates of the same references. + // So here we can aggregate predicates of the same qualifier as one single predicate, + // for reducing the size of pushed down predicates and corresponding codegen. + val right = groupExpressionsByQualifier(resultStack.pop()) + val left = groupExpressionsByQualifier(resultStack.pop()) + // Stop the loop whenever the result exceeds the `maxCnfNodeCount` + if (left.size * right.size > maxCnfNodeCount) { + logInfo(s"As the result size exceeds the threshold $maxCnfNodeCount. " + + "The CNF conversion is skipped and returning Seq.empty now. To avoid this, you can " + + s"raise the limit ${SQLConf.MAX_CNF_NODE_COUNT.key}.") + return Seq.empty + } else { + for { x <- left; y <- right } yield Or(x, y) + } + case other => other :: Nil + } + resultStack.push(cnf) + } + if (resultStack.length != 1) { + logWarning("The length of CNF conversion result stack is supposed to be 1. There might " + + "be something wrong with CNF conversion.") + return Seq.empty + } + resultStack.top + } + + private def groupExpressionsByQualifier(expressions: Seq[Expression]): Seq[Expression] = { + expressions.groupBy(_.references.map(_.qualifier)).map(_._2.reduceLeft(And)).toSeq + } + + /** + * Iterative post order traversal over a binary tree built by And/Or clauses with two stacks. + * For example, a condition `(a And b) Or c`, the postorder traversal is + * (`a`,`b`, `And`, `c`, `Or`). + * Following is the complete algorithm. After step 2, we get the postorder traversal in + * the second stack. + * 1. Push root to first stack. + * 2. Loop while first stack is not empty + * 2.1 Pop a node from first stack and push it to second stack + * 2.2 Push the children of the popped node to first stack + * + * @param condition to be traversed as binary tree + * @return sub-expressions in post order traversal as a stack. + * The first element of result stack is the leftmost node. + */ + private def postOrderTraversal(condition: Expression): mutable.Stack[Expression] = { + val stack = new mutable.Stack[Expression] + val result = new mutable.Stack[Expression] + stack.push(condition) + while (stack.nonEmpty) { + val node = stack.pop() + node match { + case Not(a And b) => stack.push(Or(Not(a), Not(b))) + case Not(a Or b) => stack.push(And(Not(a), Not(b))) + case Not(Not(a)) => stack.push(a) + case a And b => + result.push(node) + stack.push(a) + stack.push(b) + case a Or b => + result.push(node) + stack.push(a) + stack.push(b) + case _ => + result.push(node) + } + } + result + } } @ExpressionDescription( 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 f1a307b1c2cc1..a1a7213664ac8 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 @@ -51,7 +51,8 @@ abstract class Optimizer(catalogManager: CatalogManager) override protected val blacklistedOnceBatches: Set[String] = Set( "PartitionPruning", - "Extract Python UDFs") + "Extract Python UDFs", + "Push CNF predicate through join") protected def fixedPoint = FixedPoint( @@ -118,7 +119,11 @@ abstract class Optimizer(catalogManager: CatalogManager) Batch("Infer Filters", Once, InferFiltersFromConstraints) :: Batch("Operator Optimization after Inferring Filters", fixedPoint, - rulesWithoutInferFiltersFromConstraints: _*) :: Nil + rulesWithoutInferFiltersFromConstraints: _*) :: + // Set strategy to Once to avoid pushing filter every time because we do not change the + // join condition. + Batch("Push CNF predicate through join", Once, + PushCNFPredicateThroughJoin) :: Nil } val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala new file mode 100644 index 0000000000000..f406b7d77ab63 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala @@ -0,0 +1,62 @@ +/* + * 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.expressions.{And, PredicateHelper} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * Try converting join condition to conjunctive normal form expression so that more predicates may + * be able to be pushed down. + * To avoid expanding the join condition, the join condition will be kept in the original form even + * when predicate pushdown happens. + */ +object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case j @ Join(left, right, joinType, Some(joinCondition), hint) => + val predicates = conjunctiveNormalForm(joinCondition) + if (predicates.isEmpty) { + j + } else { + val pushDownCandidates = predicates.filter(_.deterministic) + lazy val leftFilterConditions = + pushDownCandidates.filter(_.references.subsetOf(left.outputSet)) + lazy val rightFilterConditions = + pushDownCandidates.filter(_.references.subsetOf(right.outputSet)) + + lazy val newLeft = + leftFilterConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) + lazy val newRight = + rightFilterConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) + + joinType match { + case _: InnerLike | LeftSemi => + Join(newLeft, newRight, joinType, Some(joinCondition), hint) + case RightOuter => + Join(newLeft, right, RightOuter, Some(joinCondition), hint) + case LeftOuter | LeftAnti | ExistenceJoin(_) => + Join(left, newRight, joinType, Some(joinCondition), hint) + case FullOuter => j + case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") + case UsingJoin(_, _) => sys.error("Untransformed Using join node") + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 189740e313207..33f40b47d072b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -545,6 +545,19 @@ object SQLConf { .booleanConf .createWithDefault(true) + val MAX_CNF_NODE_COUNT = + buildConf("spark.sql.optimizer.maxCNFNodeCount") + .internal() + .doc("Specifies the maximum allowable number of conjuncts in the result of CNF " + + "conversion. If the conversion exceeds the threshold, an empty sequence is returned. " + + "For example, CNF conversion of (a && b) || (c && d) generates " + + "four conjuncts (a || c) && (a || d) && (b || c) && (b || d).") + .version("3.1.0") + .intConf + .checkValue(_ >= 0, + "The depth of the maximum rewriting conjunction normal form must be positive.") + .createWithDefault(128) + val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals") .internal() .doc("When true, string literals (including regex patterns) remain escaped in our SQL " + @@ -2874,6 +2887,8 @@ class SQLConf extends Serializable with Logging { def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED) + def maxCnfNodeCount: Int = getConf(MAX_CNF_NODE_COUNT) + def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS) def fileCompressionFactor: Double = getConf(FILE_COMPRESSION_FACTOR) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala new file mode 100644 index 0000000000000..b449ed5cc0d07 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.BooleanType + +class ConjunctiveNormalFormPredicateSuite extends SparkFunSuite with PredicateHelper with PlanTest { + private val a = AttributeReference("A", BooleanType)(exprId = ExprId(1)).withQualifier(Seq("ta")) + private val b = AttributeReference("B", BooleanType)(exprId = ExprId(2)).withQualifier(Seq("tb")) + private val c = AttributeReference("C", BooleanType)(exprId = ExprId(3)).withQualifier(Seq("tc")) + private val d = AttributeReference("D", BooleanType)(exprId = ExprId(4)).withQualifier(Seq("td")) + private val e = AttributeReference("E", BooleanType)(exprId = ExprId(5)).withQualifier(Seq("te")) + private val f = AttributeReference("F", BooleanType)(exprId = ExprId(6)).withQualifier(Seq("tf")) + private val g = AttributeReference("G", BooleanType)(exprId = ExprId(7)).withQualifier(Seq("tg")) + private val h = AttributeReference("H", BooleanType)(exprId = ExprId(8)).withQualifier(Seq("th")) + private val i = AttributeReference("I", BooleanType)(exprId = ExprId(9)).withQualifier(Seq("ti")) + private val j = AttributeReference("J", BooleanType)(exprId = ExprId(10)).withQualifier(Seq("tj")) + private val a1 = + AttributeReference("a1", BooleanType)(exprId = ExprId(11)).withQualifier(Seq("ta")) + private val a2 = + AttributeReference("a2", BooleanType)(exprId = ExprId(12)).withQualifier(Seq("ta")) + private val b1 = + AttributeReference("b1", BooleanType)(exprId = ExprId(12)).withQualifier(Seq("tb")) + + // Check CNF conversion with expected expression, assuming the input has non-empty result. + private def checkCondition(input: Expression, expected: Expression): Unit = { + val cnf = conjunctiveNormalForm(input) + assert(cnf.nonEmpty) + val result = cnf.reduceLeft(And) + assert(result.semanticEquals(expected)) + } + + test("Keep non-predicated expressions") { + checkCondition(a, a) + checkCondition(Literal(1), Literal(1)) + } + + test("Conversion of Not") { + checkCondition(!a, !a) + checkCondition(!(!a), a) + checkCondition(!(!(a && b)), a && b) + checkCondition(!(!(a || b)), a || b) + checkCondition(!(a || b), !a && !b) + checkCondition(!(a && b), !a || !b) + } + + test("Conversion of And") { + checkCondition(a && b, a && b) + checkCondition(a && b && c, a && b && c) + checkCondition(a && (b || c), a && (b || c)) + checkCondition((a || b) && c, (a || b) && c) + checkCondition(a && b && c && d, a && b && c && d) + } + + test("Conversion of Or") { + checkCondition(a || b, a || b) + checkCondition(a || b || c, a || b || c) + checkCondition(a || b || c || d, a || b || c || d) + checkCondition((a && b) || c, (a || c) && (b || c)) + checkCondition((a && b) || (c && d), (a || c) && (a || d) && (b || c) && (b || d)) + } + + test("More complex cases") { + checkCondition(a && !(b || c), a && !b && !c) + checkCondition((a && b) || !(c && d), (a || !c || !d) && (b || !c || !d)) + checkCondition(a || b || c && d, (a || b || c) && (a || b || d)) + checkCondition(a || (b && c || d), (a || b || d) && (a || c || d)) + checkCondition(a && !(b && c || d && e), a && (!b || !c) && (!d || !e)) + checkCondition(((a && b) || c) || (d || e), (a || c || d || e) && (b || c || d || e)) + + checkCondition( + (a && b && c) || (d && e && f), + (a || d) && (a || e) && (a || f) && (b || d) && (b || e) && (b || f) && + (c || d) && (c || e) && (c || f) + ) + } + + test("Aggregate predicate of same qualifiers to avoid expanding") { + checkCondition(((a && b && a1) || c), ((a && a1) || c) && (b ||c)) + checkCondition(((a && a1 && b) || c), ((a && a1) || c) && (b ||c)) + checkCondition(((b && d && a && a1) || c), ((a && a1) || c) && (b ||c) && (d || c)) + checkCondition(((b && a2 && d && a && a1) || c), ((a2 && a && a1) || c) && (b ||c) && (d || c)) + checkCondition(((b && d && a && a1 && b1) || c), + ((a && a1) || c) && ((b && b1) ||c) && (d || c)) + checkCondition((a && a1) || (b && b1), (a && a1) || (b && b1)) + checkCondition((a && a1 && c) || (b && b1), ((a && a1) || (b && b1)) && (c || (b && b1))) + } + + test("Return Seq.empty when exceeding MAX_CNF_NODE_COUNT") { + // The following expression contains 36 conjunctive sub-expressions in CNF + val input = (a && b && c) || (d && e && f) || (g && h && i && j) + // The following expression contains 9 conjunctive sub-expressions in CNF + val input2 = (a && b && c) || (d && e && f) + Seq(8, 9, 10, 35, 36, 37).foreach { maxCount => + withSQLConf(SQLConf.MAX_CNF_NODE_COUNT.key -> maxCount.toString) { + if (maxCount < 36) { + assert(conjunctiveNormalForm(input).isEmpty) + } else { + assert(conjunctiveNormalForm(input).nonEmpty) + } + if (maxCount < 9) { + assert(conjunctiveNormalForm(input2).isEmpty) + } else { + assert(conjunctiveNormalForm(input2).nonEmpty) + } + } + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 70e29dca46e9e..bb8f5f90f8508 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -25,12 +25,17 @@ 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._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, IntegerType} import org.apache.spark.unsafe.types.CalendarInterval class FilterPushdownSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { + + override protected val blacklistedOnceBatches: Set[String] = + Set("Push CNF predicate through join") + val batches = Batch("Subqueries", Once, EliminateSubqueryAliases) :: @@ -39,7 +44,9 @@ class FilterPushdownSuite extends PlanTest { PushPredicateThroughNonJoin, BooleanSimplification, PushPredicateThroughJoin, - CollapseProject) :: Nil + CollapseProject) :: + Batch("Push CNF predicate through join", Once, + PushCNFPredicateThroughJoin) :: Nil } val attrA = 'a.int @@ -51,6 +58,15 @@ class FilterPushdownSuite extends PlanTest { val testRelation1 = LocalRelation(attrD) + val simpleDisjunctivePredicate = + ("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11) + val expectedCNFPredicatePushDownResult = { + val left = testRelation.where(('a > 3 || 'a > 1)).subquery('x) + val right = testRelation.where('a > 13 || 'a > 11).subquery('y) + left.join(right, condition = Some("x.b".attr === "y.b".attr + && (("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11)))).analyze + } + // This test already passes. test("eliminate subqueries") { val originalQuery = @@ -1230,4 +1246,148 @@ class FilterPushdownSuite extends PlanTest { comparePlans(Optimize.execute(query.analyze), expected) } + + test("inner join: rewrite filter predicates to conjunctive normal form") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = x.join(y).where(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, expectedCNFPredicatePushDownResult) + } + + test("inner join: rewrite join predicates to conjunctive normal form") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = + x.join(y, condition = Some(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate))) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, expectedCNFPredicatePushDownResult) + } + + test("inner join: rewrite complex join predicates to conjunctive normal form") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val joinCondition = (("x.b".attr === "y.b".attr) + && ((("x.a".attr === 5) && ("y.a".attr >= 2) && ("y.a".attr <= 3)) + || (("x.a".attr === 2) && ("y.a".attr >= 1) && ("y.a".attr <= 14)) + || (("x.a".attr === 1) && ("y.a".attr >= 9) && ("y.a".attr <= 27)))) + + val originalQuery = x.join(y, condition = Some(joinCondition)) + val optimized = Optimize.execute(originalQuery.analyze) + val left = testRelation.where( + ('a === 5 || 'a === 2 || 'a === 1)).subquery('x) + val right = testRelation.where( + ('a >= 2 && 'a <= 3) || ('a >= 1 && 'a <= 14) || ('a >= 9 && 'a <= 27)).subquery('y) + val correctAnswer = left.join(right, condition = Some(joinCondition)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("inner join: rewrite join predicates(with NOT predicate) to conjunctive normal form") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = + x.join(y, condition = Some(("x.b".attr === "y.b".attr) + && Not(("x.a".attr > 3) + && ("x.a".attr < 2 || ("y.a".attr > 13)) || ("x.a".attr > 1) && ("y.a".attr > 11)))) + + val optimized = Optimize.execute(originalQuery.analyze) + val left = testRelation.where('a <= 3 || 'a >= 2).subquery('x) + val right = testRelation.subquery('y) + val correctAnswer = + left.join(right, condition = Some("x.b".attr === "y.b".attr + && (("x.a".attr <= 3) || (("x.a".attr >= 2) && ("y.a".attr <= 13))) + && (("x.a".attr <= 1) || ("y.a".attr <= 11)))) + .analyze + comparePlans(optimized, correctAnswer) + } + + test("left join: rewrite join predicates to conjunctive normal form") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = + x.join(y, joinType = LeftOuter, condition = Some(("x.b".attr === "y.b".attr) + && simpleDisjunctivePredicate)) + + val optimized = Optimize.execute(originalQuery.analyze) + val left = testRelation.subquery('x) + val right = testRelation.where('a > 13 || 'a > 11).subquery('y) + val correctAnswer = + left.join(right, joinType = LeftOuter, condition = Some("x.b".attr === "y.b".attr + && (("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11)))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("right join: rewrite join predicates to conjunctive normal form") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = + x.join(y, joinType = RightOuter, condition = Some(("x.b".attr === "y.b".attr) + && simpleDisjunctivePredicate)) + + val optimized = Optimize.execute(originalQuery.analyze) + val left = testRelation.where('a > 3 || 'a > 1).subquery('x) + val right = testRelation.subquery('y) + val correctAnswer = + left.join(right, joinType = RightOuter, condition = Some("x.b".attr === "y.b".attr + && (("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11)))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("inner join: rewrite to conjunctive normal form avoid generating too many predicates") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = + x.join(y, condition = Some(("x.b".attr === "y.b".attr) && ((("x.a".attr > 3) && + ("x.a".attr < 13) && ("y.c".attr <= 5)) || (("y.a".attr > 2) && ("y.c".attr < 1))))) + + val optimized = Optimize.execute(originalQuery.analyze) + val left = testRelation.subquery('x) + val right = testRelation.where('c <= 5 || ('a > 2 && 'c < 1)).subquery('y) + val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr && + ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) || + (("y.a".attr > 2) && ("y.c".attr < 1))))).analyze + + comparePlans(optimized, correctAnswer) + } + + test(s"Disable rewrite to CNF by setting ${SQLConf.MAX_CNF_NODE_COUNT.key}=0") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = + x.join(y, condition = Some(("x.b".attr === "y.b".attr) + && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) + || (("y.a".attr > 2) && ("y.c".attr < 1))))) + + Seq(0, 10).foreach { count => + withSQLConf(SQLConf.MAX_CNF_NODE_COUNT.key -> count.toString) { + val optimized = Optimize.execute(originalQuery.analyze) + val (left, right) = if (count == 0) { + (testRelation.subquery('x), testRelation.subquery('y)) + } else { + (testRelation.subquery('x), + testRelation.where('c <= 5 || ('a > 2 && 'c < 1)).subquery('y)) + } + val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr + && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) + || (("y.a".attr > 2) && ("y.c".attr < 1))))).analyze + + comparePlans(optimized, correctAnswer) + } + } + } } From b1adc3deee00058cba669534aee156dc7af243dc Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 11 Jun 2020 14:15:28 -0700 Subject: [PATCH 17/40] [SPARK-21117][SQL] Built-in SQL Function Support - WIDTH_BUCKET ### What changes were proposed in this pull request? This PR intends to add a build-in SQL function - `WIDTH_BUCKET`. It is the rework of #18323. Closes #18323 The other RDBMS references for `WIDTH_BUCKET`: - Oracle: https://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2137.htm#OLADM717 - PostgreSQL: https://www.postgresql.org/docs/current/functions-math.html - Snowflake: https://docs.snowflake.com/en/sql-reference/functions/width_bucket.html - Prestodb: https://prestodb.io/docs/current/functions/math.html - Teradata: https://docs.teradata.com/reader/kmuOwjp1zEYg98JsB8fu_A/Wa8vw69cGzoRyNULHZeudg - DB2: https://www.ibm.com/support/producthub/db2/docs/content/SSEPGG_11.5.0/com.ibm.db2.luw.sql.ref.doc/doc/r0061483.html?pos=2 ### Why are the changes needed? For better usability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit tests. Closes #28764 from maropu/SPARK-21117. Lead-authored-by: Takeshi Yamamuro Co-authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/mathExpressions.scala | 96 ++++++++++ .../sql-functions/sql-expression-schema.md | 3 +- .../resources/sql-tests/inputs/operators.sql | 14 ++ .../sql-tests/inputs/postgreSQL/numeric.sql | 76 +++++--- .../sql-tests/results/operators.sql.out | 98 +++++++++- .../results/postgreSQL/numeric.sql.out | 173 +++++++++++++++++- 7 files changed, 431 insertions(+), 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index e2559d4c07297..3989df5d29467 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -274,6 +274,7 @@ object FunctionRegistry { expression[Tan]("tan"), expression[Cot]("cot"), expression[Tanh]("tanh"), + expression[WidthBucket]("width_bucket"), expression[Add]("+"), expression[Subtract]("-"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index fe8ea2a3c6733..5c764956adf0a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -1325,3 +1325,99 @@ case class BRound(child: Expression, scale: Expression) with Serializable with ImplicitCastInputTypes { def this(child: Expression) = this(child, Literal(0)) } + +object WidthBucket { + + def computeBucketNumber(value: Double, min: Double, max: Double, numBucket: Long): jl.Long = { + if (numBucket <= 0 || numBucket == Long.MaxValue || jl.Double.isNaN(value) || min == max || + jl.Double.isNaN(min) || jl.Double.isInfinite(min) || + jl.Double.isNaN(max) || jl.Double.isInfinite(max)) { + return null + } + + val lower = Math.min(min, max) + val upper = Math.max(min, max) + + if (min < max) { + if (value < lower) { + 0L + } else if (value >= upper) { + numBucket + 1L + } else { + (numBucket.toDouble * (value - lower) / (upper - lower)).toLong + 1L + } + } else { // `min > max` case + if (value > upper) { + 0L + } else if (value <= lower) { + numBucket + 1L + } else { + (numBucket.toDouble * (upper - value) / (upper - lower)).toLong + 1L + } + } + } +} + +/** + * Returns the bucket number into which the value of this expression would fall + * after being evaluated. Note that input arguments must follow conditions listed below; + * otherwise, the method will return null. + * - `numBucket` must be greater than zero and be less than Long.MaxValue + * - `value`, `min`, and `max` cannot be NaN + * - `min` bound cannot equal `max` + * - `min` and `max` must be finite + * + * Note: If `minValue` > `maxValue`, a return value is as follows; + * if `value` > `minValue`, it returns 0. + * if `value` <= `maxValue`, it returns `numBucket` + 1. + * otherwise, it returns (`numBucket` * (`minValue` - `value`) / (`minValue` - `maxValue`)) + 1 + * + * @param value is the expression to compute a bucket number in the histogram + * @param minValue is the minimum value of the histogram + * @param maxValue is the maximum value of the histogram + * @param numBucket is the number of buckets + */ +@ExpressionDescription( + usage = """ + _FUNC_(value, min_value, max_value, num_bucket) - Returns the bucket number to which + `value` would be assigned in an equiwidth histogram with `num_bucket` buckets, + in the range `min_value` to `max_value`." + """, + examples = """ + Examples: + > SELECT _FUNC_(5.3, 0.2, 10.6, 5); + 3 + > SELECT _FUNC_(-2.1, 1.3, 3.4, 3); + 0 + > SELECT _FUNC_(8.1, 0.0, 5.7, 4); + 5 + > SELECT _FUNC_(-0.9, 5.2, 0.5, 2); + 3 + """, + since = "3.1.0") +case class WidthBucket( + value: Expression, + minValue: Expression, + maxValue: Expression, + numBucket: Expression) + extends QuaternaryExpression with ImplicitCastInputTypes with NullIntolerant { + + override def children: Seq[Expression] = Seq(value, minValue, maxValue, numBucket) + override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType, DoubleType, DoubleType, LongType) + override def dataType: DataType = LongType + override def nullable: Boolean = true + + override protected def nullSafeEval(input: Any, min: Any, max: Any, numBucket: Any): Any = { + WidthBucket.computeBucketNumber( + input.asInstanceOf[Double], + min.asInstanceOf[Double], + max.asInstanceOf[Double], + numBucket.asInstanceOf[Long]) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, (input, min, max, numBucket) => + "org.apache.spark.sql.catalyst.expressions.WidthBucket" + + s".computeBucketNumber($input, $min, $max, $numBucket)") + } +} diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index d245aa5a17345..1a9d5bb775a0a 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -1,6 +1,6 @@ ## Summary - - Number of queries: 337 + - Number of queries: 338 - Number of expressions that missing example: 34 - Expressions missing examples: and,string,tinyint,double,smallint,date,decimal,boolean,float,binary,bigint,int,timestamp,struct,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch ## Schema of Built-in Functions @@ -291,6 +291,7 @@ | org.apache.spark.sql.catalyst.expressions.Uuid | uuid | SELECT uuid() | struct | | org.apache.spark.sql.catalyst.expressions.WeekDay | weekday | SELECT weekday('2009-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.WeekOfYear | weekofyear | SELECT weekofyear('2008-02-20') | struct | +| org.apache.spark.sql.catalyst.expressions.WidthBucket | width_bucket | SELECT width_bucket(5.3, 0.2, 10.6, 5) | struct | | org.apache.spark.sql.catalyst.expressions.XxHash64 | xxhash64 | SELECT xxhash64('Spark', array(123), 2) | struct | | org.apache.spark.sql.catalyst.expressions.Year | year | SELECT year('2016-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | struct>> | diff --git a/sql/core/src/test/resources/sql-tests/inputs/operators.sql b/sql/core/src/test/resources/sql-tests/inputs/operators.sql index 20bf0eb15c5b2..c296fa5f7e87e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/operators.sql @@ -81,3 +81,17 @@ select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11); -- pmod select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null); select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)); + +-- width_bucket +select width_bucket(5.35, 0.024, 10.06, 5); +select width_bucket(5.35, 0.024, 10.06, 3 + 2); +select width_bucket('5.35', '0.024', '10.06', '5'); +select width_bucket(5.35, 0.024, 10.06, 2.5); +select width_bucket(5.35, 0.024, 10.06, 0.5); +select width_bucket(null, 0.024, 10.06, 5); +select width_bucket(5.35, null, 10.06, 5); +select width_bucket(5.35, 0.024, null, -5); +select width_bucket(5.35, 0.024, 10.06, null); +select width_bucket(5.35, 0.024, 10.06, -5); +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L); -- long max value +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L - 1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql index dbdb2cace0e0c..53f2aa41ae3fa 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql @@ -754,22 +754,22 @@ DROP TABLE ceil_floor_round; -- round((2.5 * 10 ^ i)::numeric, -i) -- FROM generate_series(-5,5) AS t(i); --- [SPARK-21117] Built-in SQL Function Support - WIDTH_BUCKET -- Testing for width_bucket(). For convenience, we test both the -- numeric and float8 versions of the function in this file. -- errors --- SELECT width_bucket(5.0, 3.0, 4.0, 0); --- SELECT width_bucket(5.0, 3.0, 4.0, -5); --- SELECT width_bucket(3.5, 3.0, 3.0, 888); --- SELECT width_bucket(5.0::float8, 3.0::float8, 4.0::float8, 0); --- SELECT width_bucket(5.0::float8, 3.0::float8, 4.0::float8, -5); --- SELECT width_bucket(3.5::float8, 3.0::float8, 3.0::float8, 888); --- SELECT width_bucket('NaN', 3.0, 4.0, 888); --- SELECT width_bucket(0::float8, 'NaN', 4.0::float8, 888); +SELECT width_bucket(5.0, 3.0, 4.0, 0); +SELECT width_bucket(5.0, 3.0, 4.0, -5); +SELECT width_bucket(3.5, 3.0, 3.0, 888); +SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0); +SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5); +SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888); +SELECT width_bucket('NaN', 3.0, 4.0, 888); +SELECT width_bucket(double(0), 'NaN', double(4.0), 888); -- normal operation -- CREATE TABLE width_bucket_test (operand_num numeric, operand_f8 float8); +CREATE TABLE width_bucket_test (operand_num decimal(30,15), operand_f8 double) USING parquet; -- COPY width_bucket_test (operand_num) FROM stdin; -- -5.2 @@ -795,28 +795,50 @@ DROP TABLE ceil_floor_round; -- UPDATE width_bucket_test SET operand_f8 = operand_num::float8; --- SELECT --- operand_num, --- width_bucket(operand_num, 0, 10, 5) AS wb_1, --- width_bucket(operand_f8, 0, 10, 5) AS wb_1f, --- width_bucket(operand_num, 10, 0, 5) AS wb_2, --- width_bucket(operand_f8, 10, 0, 5) AS wb_2f, --- width_bucket(operand_num, 2, 8, 4) AS wb_3, --- width_bucket(operand_f8, 2, 8, 4) AS wb_3f, --- width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, --- width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, --- width_bucket(operand_num, -25, 25, 10) AS wb_5, --- width_bucket(operand_f8, -25, 25, 10) AS wb_5f --- FROM width_bucket_test; +INSERT INTO width_bucket_test VALUES + (-5.2, -5.2), + (-0.0000000001, -0.0000000001), + (0.000000000001, 0.000000000001), + (1, 1), + (1.99999999999999, 1.99999999999999), + (2, 2), + (2.00000000000001, 2.00000000000001), + (3, 3), + (4, 4), + (4.5, 4.5), + (5, 5), + (5.5, 5.5), + (6, 6), + (7, 7), + (8, 8), + (9, 9), + (9.99999999999999, 9.99999999999999), + (10, 10), + (10.0000000000001, 10.0000000000001); + +SELECT + operand_num, + width_bucket(operand_num, 0, 10, 5) AS wb_1, + width_bucket(operand_f8, 0, 10, 5) AS wb_1f, + width_bucket(operand_num, 10, 0, 5) AS wb_2, + width_bucket(operand_f8, 10, 0, 5) AS wb_2f, + width_bucket(operand_num, 2, 8, 4) AS wb_3, + width_bucket(operand_f8, 2, 8, 4) AS wb_3f, + width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, + width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, + width_bucket(operand_num, -25, 25, 10) AS wb_5, + width_bucket(operand_f8, -25, 25, 10) AS wb_5f + FROM width_bucket_test + ORDER BY operand_num ASC; -- for float8 only, check positive and negative infinity: we require -- finite bucket bounds, but allow an infinite operand --- SELECT width_bucket(0.0::float8, 'Infinity'::float8, 5, 10); -- error --- SELECT width_bucket(0.0::float8, 5, '-Infinity'::float8, 20); -- error --- SELECT width_bucket('Infinity'::float8, 1, 10, 10), --- width_bucket('-Infinity'::float8, 1, 10, 10); +SELECT width_bucket(double(0.0), double('Infinity'), 5, 10); -- error +SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20); -- error +SELECT width_bucket(double('Infinity'), 1, 10, 10), + width_bucket(double('-Infinity'), 1, 10, 10); --- DROP TABLE width_bucket_test; +DROP TABLE width_bucket_test; -- [SPARK-28137] Missing Data Type Formatting Functions: TO_CHAR -- TO_CHAR() diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index 9accc57d0bf60..fc58bc784c82f 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 57 +-- Number of queries: 69 -- !query @@ -456,3 +456,99 @@ select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint) struct -- !query output NULL NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 5) +-- !query schema +struct +-- !query output +3 + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 3 + 2) +-- !query schema +struct +-- !query output +3 + + +-- !query +select width_bucket('5.35', '0.024', '10.06', '5') +-- !query schema +struct +-- !query output +3 + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 2.5) +-- !query schema +struct +-- !query output +2 + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(null, 0.024, 10.06, 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, null, 10.06, 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, null, -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L - 1) +-- !query schema +struct +-- !query output +4894746858139549697 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 7b7aeb4ec7934..3cade4f1f9558 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 577 +-- Number of queries: 592 -- !query @@ -4423,6 +4423,177 @@ struct<> +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(3.5, 3.0, 3.0, 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket('NaN', 3.0, 4.0, 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(0), 'NaN', double(4.0), 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +CREATE TABLE width_bucket_test (operand_num decimal(30,15), operand_f8 double) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO width_bucket_test VALUES + (-5.2, -5.2), + (-0.0000000001, -0.0000000001), + (0.000000000001, 0.000000000001), + (1, 1), + (1.99999999999999, 1.99999999999999), + (2, 2), + (2.00000000000001, 2.00000000000001), + (3, 3), + (4, 4), + (4.5, 4.5), + (5, 5), + (5.5, 5.5), + (6, 6), + (7, 7), + (8, 8), + (9, 9), + (9.99999999999999, 9.99999999999999), + (10, 10), + (10.0000000000001, 10.0000000000001) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + operand_num, + width_bucket(operand_num, 0, 10, 5) AS wb_1, + width_bucket(operand_f8, 0, 10, 5) AS wb_1f, + width_bucket(operand_num, 10, 0, 5) AS wb_2, + width_bucket(operand_f8, 10, 0, 5) AS wb_2f, + width_bucket(operand_num, 2, 8, 4) AS wb_3, + width_bucket(operand_f8, 2, 8, 4) AS wb_3f, + width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, + width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, + width_bucket(operand_num, -25, 25, 10) AS wb_5, + width_bucket(operand_f8, -25, 25, 10) AS wb_5f + FROM width_bucket_test + ORDER BY operand_num ASC +-- !query schema +struct +-- !query output +-5.200000000000000 0 0 6 6 0 0 0 0 4 4 +-0.000000000100000 0 0 6 6 0 0 0 0 5 5 +0.000000000001000 1 1 5 5 0 0 0 0 6 6 +1.000000000000000 1 1 5 5 0 0 0 0 6 6 +1.999999999999990 1 1 5 5 0 0 0 0 6 6 +2.000000000000000 2 2 5 5 1 1 0 0 6 6 +2.000000000000010 2 2 4 4 1 1 0 0 6 6 +3.000000000000000 2 2 4 4 1 1 0 0 6 6 +4.000000000000000 3 3 4 4 2 2 0 0 6 6 +4.500000000000000 3 3 3 3 2 2 0 0 6 6 +5.000000000000000 3 3 3 3 3 3 1 1 7 7 +5.500000000000000 3 3 3 3 3 3 21 21 7 7 +6.000000000000000 4 4 3 3 3 3 21 21 7 7 +7.000000000000000 4 4 2 2 4 4 21 21 7 7 +8.000000000000000 5 5 2 2 5 5 21 21 7 7 +9.000000000000000 5 5 1 1 5 5 21 21 7 7 +9.999999999999990 5 5 1 1 5 5 21 21 7 7 +10.000000000000000 6 6 1 1 5 5 21 21 8 8 +10.000000000000100 6 6 0 0 5 5 21 21 8 8 + + +-- !query +SELECT width_bucket(double(0.0), double('Infinity'), 5, 10) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double('Infinity'), 1, 10, 10), + width_bucket(double('-Infinity'), 1, 10, 10) +-- !query schema +struct +-- !query output +11 0 + + +-- !query +DROP TABLE width_bucket_test +-- !query schema +struct<> +-- !query output + + + -- !query CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet -- !query schema From 88a4e55fae19c2c87effba022e041206154b5246 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 11 Jun 2020 18:27:53 -0500 Subject: [PATCH 18/40] [SPARK-31765][WEBUI][TEST-MAVEN] Upgrade HtmlUnit >= 2.37.0 ### What changes were proposed in this pull request? This PR upgrades HtmlUnit. Selenium and Jetty also upgraded because of dependency. ### Why are the changes needed? Recently, a security issue which affects HtmlUnit is reported. https://nvd.nist.gov/vuln/detail/CVE-2020-5529 According to the report, arbitrary code can be run by malicious users. HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing testcases. Closes #28585 from sarutak/upgrade-htmlunit. Authored-by: Kousuke Saruta Signed-off-by: Sean Owen --- core/pom.xml | 2 +- .../scala/org/apache/spark/ui/JettyUtils.scala | 7 ++++++- .../org/apache/spark/ui/UISeleniumSuite.scala | 2 +- pom.xml | 14 +++++++++----- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 20 insertions(+), 11 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index b0f68880f1d8a..14b217d7fb22e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -334,7 +334,7 @@ org.seleniumhq.selenium - selenium-htmlunit-driver + htmlunit-driver test diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 4b4788f453243..f1962ef39fc06 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -23,6 +23,7 @@ import javax.servlet.DispatcherType import javax.servlet.http._ import scala.language.implicitConversions +import scala.util.Try import scala.xml.Node import org.eclipse.jetty.client.HttpClient @@ -500,7 +501,11 @@ private[spark] case class ServerInfo( threadPool match { case pool: QueuedThreadPool => // Workaround for SPARK-30385 to avoid Jetty's acceptor thread shrink. - pool.setIdleTimeout(0) + // As of Jetty 9.4.21, the implementation of + // QueuedThreadPool#setIdleTimeout is changed and IllegalStateException + // will be thrown if we try to set idle timeout after the server has started. + // But this workaround works for Jetty 9.4.28 by ignoring the exception. + Try(pool.setIdleTimeout(0)) case _ => } server.stop() diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 909056eab8c5a..ecfdf481f4f6c 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -24,6 +24,7 @@ import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import scala.io.Source import scala.xml.Node +import com.gargoylesoftware.css.parser.CSSParseException import com.gargoylesoftware.htmlunit.DefaultCssErrorHandler import org.json4s._ import org.json4s.jackson.JsonMethods @@ -33,7 +34,6 @@ import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser -import org.w3c.css.sac.CSSParseException import org.apache.spark._ import org.apache.spark.LocalSparkContext._ diff --git a/pom.xml b/pom.xml index b3f7b7db1a79a..8d552e08f0cd9 100644 --- a/pom.xml +++ b/pom.xml @@ -139,7 +139,7 @@ com.twitter 1.6.0 - 9.4.18.v20190429 + 9.4.28.v20200408 3.1.0 0.9.5 2.4.0 @@ -187,8 +187,8 @@ 0.12.0 4.7.1 1.1 - 2.52.0 - 2.22 + 3.141.59 + 2.40.0 @@ -591,12 +591,16 @@ io.netty netty + + net.bytebuddy + byte-buddy + org.seleniumhq.selenium - selenium-htmlunit-driver - ${selenium.version} + htmlunit-driver + ${htmlunit.version} test diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 7c5fcba9c2131..e4ef1467a960e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -162,7 +162,7 @@ org.seleniumhq.selenium - selenium-htmlunit-driver + htmlunit-driver test diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 1de2677d5ede5..5bf20b209aff7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -95,7 +95,7 @@ org.seleniumhq.selenium - selenium-htmlunit-driver + htmlunit-driver test diff --git a/streaming/pom.xml b/streaming/pom.xml index ea351d449481a..53b49dd320e94 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -109,7 +109,7 @@ org.seleniumhq.selenium - selenium-htmlunit-driver + htmlunit-driver test From b87a342c7dd51046fcbe323db640c825646fb8d4 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Fri, 12 Jun 2020 09:19:29 +0900 Subject: [PATCH 19/40] [SPARK-31916][SQL] StringConcat can lead to StringIndexOutOfBoundsException ### What changes were proposed in this pull request? A minor fix to fix the append method of StringConcat to cap the length at MAX_ROUNDED_ARRAY_LENGTH to make sure it does not overflow and cause StringIndexOutOfBoundsException Thanks to **Jeffrey Stokes** for reporting the issue and explaining the underlying problem in detail in the JIRA. ### Why are the changes needed? This fixes StringIndexOutOfBoundsException on an overflow. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added a test in StringsUtilSuite. Closes #28750 from dilipbiswal/SPARK-31916. Authored-by: Dilip Biswal Signed-off-by: Takeshi Yamamuro --- .../spark/sql/catalyst/util/StringUtils.scala | 6 +++- .../sql/catalyst/util/StringUtilsSuite.scala | 32 ++++++++++++++++++- 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index b42ae4e45366e..2a416d6c97584 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -123,7 +123,11 @@ object StringUtils extends Logging { val stringToAppend = if (available >= sLen) s else s.substring(0, available) strings.append(stringToAppend) } - length += sLen + + // Keeps the total length of appended strings. Note that we need to cap the length at + // `ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH`; otherwise, we will overflow + // length causing StringIndexOutOfBoundsException in the substring call above. + length = Math.min(length.toLong + sLen, ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH).toInt } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala index 67bc4bc81cb92..c68e89fc8a5ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.StringUtils._ +import org.apache.spark.sql.internal.SQLConf -class StringUtilsSuite extends SparkFunSuite { +class StringUtilsSuite extends SparkFunSuite with SQLHelper { test("escapeLikeRegex") { val expectedEscapedStrOne = "(?s)\\Qa\\E\\Qb\\E\\Qd\\E\\Qe\\E\\Qf\\E" @@ -98,4 +100,32 @@ class StringUtilsSuite extends SparkFunSuite { assert(checkLimit("1234567")) assert(checkLimit("1234567890")) } + + test("SPARK-31916: StringConcat doesn't overflow on many inputs") { + val concat = new StringConcat(maxLength = 100) + val stringToAppend = "Test internal index of StringConcat does not overflow with many " + + "append calls" + 0.to((Integer.MAX_VALUE / stringToAppend.length) + 1).foreach { _ => + concat.append(stringToAppend) + } + assert(concat.toString.length === 100) + } + + test("SPARK-31916: verify that PlanStringConcat's output shows the actual length of the plan") { + withSQLConf(SQLConf.MAX_PLAN_STRING_LENGTH.key -> "0") { + val concat = new PlanStringConcat() + 0.to(3).foreach { i => + concat.append(s"plan fragment $i") + } + assert(concat.toString === "Truncated plan of 60 characters") + } + + withSQLConf(SQLConf.MAX_PLAN_STRING_LENGTH.key -> "60") { + val concat = new PlanStringConcat() + 0.to(2).foreach { i => + concat.append(s"plan fragment $i") + } + assert(concat.toString === "plan fragment 0plan fragment 1... 15 more characters") + } + } } From 78f9043862c96f7b6169f6e2a5609e482b732f45 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 11 Jun 2020 22:50:36 -0700 Subject: [PATCH 20/40] [SPARK-31912][SQL][TESTS] Normalize all binary comparison expressions ### What changes were proposed in this pull request? This pr normalize all binary comparison expressions when comparing plans. ### Why are the changes needed? Improve test framework, otherwise this test will fail: ```scala test("SPARK-31912 Normalize all binary comparison expressions") { val original = testRelation .where('a === 'b && Literal(13) >= 'b).as("x") val optimized = testRelation .where(IsNotNull('a) && IsNotNull('b) && 'a === 'b && 'b <= 13 && 'a <= 13).as("x") comparePlans(Optimize.execute(original.analyze), optimized.analyze) } ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual test. Closes #28734 from wangyum/SPARK-31912. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- .../spark/sql/catalyst/plans/PlanTest.scala | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 9600357f43cc9..53bb0e3a527cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -104,8 +104,8 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => protected def normalizePlan(plan: LogicalPlan): LogicalPlan = { plan transform { case Filter(condition: Expression, child: LogicalPlan) => - Filter(splitConjunctivePredicates(condition).map(rewriteEqual).sortBy(_.hashCode()) - .reduce(And), child) + Filter(splitConjunctivePredicates(condition).map(rewriteBinaryComparison) + .sortBy(_.hashCode()).reduce(And), child) case sample: Sample => sample.copy(seed = 0L) case Join(left, right, joinType, condition, hint) if condition.isDefined => @@ -117,23 +117,26 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => } val newCondition = - splitConjunctivePredicates(condition.get).map(rewriteEqual).sortBy(_.hashCode()) - .reduce(And) + splitConjunctivePredicates(condition.get).map(rewriteBinaryComparison) + .sortBy(_.hashCode()).reduce(And) Join(left, right, newJoinType, Some(newCondition), hint) } } /** - * Rewrite [[EqualTo]] and [[EqualNullSafe]] operator to keep order. The following cases will be + * Rewrite [[BinaryComparison]] operator to keep order. The following cases will be * equivalent: * 1. (a = b), (b = a); * 2. (a <=> b), (b <=> a). + * 3. (a > b), (b < a) */ - private def rewriteEqual(condition: Expression): Expression = condition match { - case eq @ EqualTo(l: Expression, r: Expression) => - Seq(l, r).sortBy(_.hashCode()).reduce(EqualTo) - case eq @ EqualNullSafe(l: Expression, r: Expression) => - Seq(l, r).sortBy(_.hashCode()).reduce(EqualNullSafe) + private def rewriteBinaryComparison(condition: Expression): Expression = condition match { + case EqualTo(l, r) => Seq(l, r).sortBy(_.hashCode()).reduce(EqualTo) + case EqualNullSafe(l, r) => Seq(l, r).sortBy(_.hashCode()).reduce(EqualNullSafe) + case GreaterThan(l, r) if l.hashCode() > r.hashCode() => LessThan(r, l) + case LessThan(l, r) if l.hashCode() > r.hashCode() => GreaterThan(r, l) + case GreaterThanOrEqual(l, r) if l.hashCode() > r.hashCode() => LessThanOrEqual(r, l) + case LessThanOrEqual(l, r) if l.hashCode() > r.hashCode() => GreaterThanOrEqual(r, l) case _ => condition // Don't reorder. } From c259844df8b6690b752a1c67b241de2981cdb5fe Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 12 Jun 2020 06:17:31 +0000 Subject: [PATCH 21/40] [SPARK-31959][SQL][TEST-JAVA11] Fix Gregorian-Julian micros rebasing while switching standard time zone offset ### What changes were proposed in this pull request? Fix the bug in microseconds rebasing during transitions from one standard time zone offset to another one. In the PR, I propose to change the implementation of `rebaseGregorianToJulianMicros` which performs rebasing via local timestamps. In the case of overlapping: 1. Check that the original instant belongs to earlier or later instant of overlapped local timestamp. 2. If it is an earlier instant, take zone and DST offsets from the previous day otherwise 3. Set time zone offsets to Julian timestamp from the next day. Note: The fix assumes that transitions cannot happen more often than once per 2 days. ### Why are the changes needed? Current implementation handles timestamps overlapping only during daylight saving time but overlapping can happen also during transition from one standard time zone to another one. For example in the case of `Asia/Hong_Kong`, the time zone switched from `Japan Standard Time` (UTC+9) to `Hong Kong Time` (UTC+8) on _Sunday, 18 November, 1945 01:59:59 AM_. The changes allow to handle the special case as well. ### Does this PR introduce _any_ user-facing change? It might affect micros rebasing in before common era when not-optimised version of `rebaseGregorianToJulianMicros()` is used directly. ### How was this patch tested? 1. By existing tests in `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `TimestampFormatterSuite`. 2. Added new test to `RebaseDateTimeSuite` 3. Regenerated `gregorian-julian-rebase-micros.json` with the step of 30 minutes, and got the same JSON file. The JSON file isn't affected because previously it was generated with the step of 1 week. And the spike in diffs/switch points during 1 hour of timestamp overlapping wasn't detected. Closes #28787 from MaxGekk/HongKong-tz-1945. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/RebaseDateTime.scala | 26 ++++++++++---- .../catalyst/util/RebaseDateTimeSuite.scala | 36 +++++++++++++++++++ 2 files changed, 56 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index cc75340cd8fcd..c31dc624b0611 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -326,20 +326,34 @@ object RebaseDateTime { */ private[sql] def rebaseGregorianToJulianMicros(zoneId: ZoneId, micros: Long): Long = { val instant = microsToInstant(micros) - var ldt = instant.atZone(zoneId).toLocalDateTime + val zonedDateTime = instant.atZone(zoneId) + var ldt = zonedDateTime.toLocalDateTime if (ldt.isAfter(julianEndTs) && ldt.isBefore(gregorianStartTs)) { ldt = LocalDateTime.of(gregorianStartDate, ldt.toLocalTime) } val cal = new Calendar.Builder() - // `gregory` is a hybrid calendar that supports both - // the Julian and Gregorian calendar systems + // `gregory` is a hybrid calendar that supports both the Julian and Gregorian calendar systems .setCalendarType("gregory") .setDate(ldt.getYear, ldt.getMonthValue - 1, ldt.getDayOfMonth) .setTimeOfDay(ldt.getHour, ldt.getMinute, ldt.getSecond) - // Local time-line can overlaps, such as at an autumn daylight savings cutover. - // This setting selects the original local timestamp mapped to the given `micros`. - .set(Calendar.DST_OFFSET, zoneId.getRules.getDaylightSavings(instant).toMillis.toInt) .build() + // A local timestamp can have 2 instants in the cases of switching from: + // 1. Summer to winter time. + // 2. One standard time zone to another one. For example, Asia/Hong_Kong switched from JST + // to HKT on 18 November, 1945 01:59:59 AM. + // Below we check that the original `instant` is earlier or later instant. If it is an earlier + // instant, we take the standard and DST offsets of the previous day otherwise of the next one. + val trans = zoneId.getRules.getTransition(ldt) + if (trans != null && trans.isOverlap) { + val cloned = cal.clone().asInstanceOf[Calendar] + // Does the current offset belong to the offset before the transition. + // If so, we will take zone offsets from the previous day otherwise from the next day. + // This assumes that transitions cannot happen often than once per 2 days. + val shift = if (trans.getOffsetBefore == zonedDateTime.getOffset) -1 else 1 + cloned.add(Calendar.DAY_OF_MONTH, shift) + cal.set(Calendar.ZONE_OFFSET, cloned.get(Calendar.ZONE_OFFSET)) + cal.set(Calendar.DST_OFFSET, cloned.get(Calendar.DST_OFFSET)) + } millisToMicros(cal.getTimeInMillis) + ldt.get(ChronoField.MICRO_OF_SECOND) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index cb5f8e43d762f..e3af64a562e23 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -409,4 +409,40 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { } } } + + test("SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945") { + // The 'Asia/Hong_Kong' time zone switched from 'Japan Standard Time' (JST = UTC+9) + // to 'Hong Kong Time' (HKT = UTC+8). After Sunday, 18 November, 1945 01:59:59 AM, + // clocks were moved backward to become Sunday, 18 November, 1945 01:00:00 AM. + // In this way, the overlap happened w/o Daylight Saving Time. + val hkZid = getZoneId("Asia/Hong_Kong") + withDefaultTimeZone(hkZid) { + var expected = "1945-11-18 01:30:00.0" + var ldt = LocalDateTime.of(1945, 11, 18, 1, 30, 0) + var earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) + var laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) + if (earlierMicros + MICROS_PER_HOUR != laterMicros) { + // Old JDK might have an outdated time zone database. + // See https://bugs.openjdk.java.net/browse/JDK-8228469: "Hong Kong ... Its 1945 transition + // from JST to HKT was on 11-18 at 02:00, not 09-15 at 00:00" + expected = "1945-09-14 23:30:00.0" + ldt = LocalDateTime.of(1945, 9, 14, 23, 30, 0) + earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) + laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) + assert(earlierMicros + MICROS_PER_HOUR === laterMicros) + } + val rebasedEarlierMicros = rebaseGregorianToJulianMicros(hkZid, earlierMicros) + val rebasedLaterMicros = rebaseGregorianToJulianMicros(hkZid, laterMicros) + def toTsStr(micros: Long): String = toJavaTimestamp(micros).toString + assert(toTsStr(rebasedEarlierMicros) === expected) + assert(toTsStr(rebasedLaterMicros) === expected) + assert(rebasedEarlierMicros + MICROS_PER_HOUR === rebasedLaterMicros) + // Check optimized rebasing + assert(rebaseGregorianToJulianMicros(earlierMicros) === rebasedEarlierMicros) + assert(rebaseGregorianToJulianMicros(laterMicros) === rebasedLaterMicros) + // Check reverse rebasing + assert(rebaseJulianToGregorianMicros(rebasedEarlierMicros) === earlierMicros) + assert(rebaseJulianToGregorianMicros(rebasedLaterMicros) === laterMicros) + } + } } From ff89b1114319e783eb4f4187bf2583e5e21c64e4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 12 Jun 2020 16:54:55 +0900 Subject: [PATCH 22/40] [SPARK-31736][SQL] Nested column aliasing for RepartitionByExpression/Join ### What changes were proposed in this pull request? Currently we only push nested column pruning through a few operators such as LIMIT, SAMPLE, etc. This patch extends the feature to other operators including RepartitionByExpression, Join. ### Why are the changes needed? Currently nested column pruning only applied on a few operators. It limits the benefit of nested column pruning. Extending nested column pruning coverage to make this feature more generally applied through different queries. ### Does this PR introduce _any_ user-facing change? Yes. More SQL operators are covered by nested column pruning. ### How was this patch tested? Added unit test, end-to-end tests. Closes #28556 from viirya/others-column-pruning. Authored-by: Liang-Chi Hsieh Signed-off-by: HyukjinKwon --- .../optimizer/NestedColumnAliasing.scala | 29 +++-- .../optimizer/NestedColumnAliasingSuite.scala | 84 ++++++++++++++- .../datasources/SchemaPruningSuite.scala | 100 ++++++++++++++++++ 3 files changed, 197 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index c38a1189387d7..eb8022c8c8404 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -34,7 +34,8 @@ object NestedColumnAliasing { : Option[(Map[ExtractValue, Alias], Map[ExprId, Seq[Alias]])] = plan match { case Project(projectList, child) if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) => - getAliasSubMap(projectList) + val exprCandidatesToPrune = projectList ++ child.expressions + getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq) case plan if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(plan) => val exprCandidatesToPrune = plan.expressions @@ -53,11 +54,11 @@ object NestedColumnAliasing { case Project(projectList, child) => Project( getNewProjectList(projectList, nestedFieldToAlias), - replaceChildrenWithAliases(child, nestedFieldToAlias, attrToAliases)) + replaceWithAliases(child, nestedFieldToAlias, attrToAliases)) // The operators reaching here was already guarded by `canPruneOn`. case other => - replaceChildrenWithAliases(other, nestedFieldToAlias, attrToAliases) + replaceWithAliases(other, nestedFieldToAlias, attrToAliases) } /** @@ -73,9 +74,10 @@ object NestedColumnAliasing { } /** - * Return a plan with new children replaced with aliases. + * Return a plan with new children replaced with aliases, and expressions replaced with + * aliased attributes. */ - def replaceChildrenWithAliases( + def replaceWithAliases( plan: LogicalPlan, nestedFieldToAlias: Map[ExtractValue, Alias], attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { @@ -104,6 +106,8 @@ object NestedColumnAliasing { case _: LocalLimit => true case _: Repartition => true case _: Sample => true + case _: RepartitionByExpression => true + case _: Join => true case _ => false } @@ -202,7 +206,9 @@ object GeneratorNestedColumnAliasing { val exprsToPrune = projectList ++ g.generator.children NestedColumnAliasing.getAliasSubMap(exprsToPrune, g.qualifiedGeneratorOutput).map { case (nestedFieldToAlias, attrToAliases) => - val newChild = pruneGenerate(g, nestedFieldToAlias, attrToAliases) + // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. + val newChild = + NestedColumnAliasing.replaceWithAliases(g, nestedFieldToAlias, attrToAliases) Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) } @@ -215,21 +221,14 @@ object GeneratorNestedColumnAliasing { NestedColumnAliasing.getAliasSubMap( g.generator.children, g.requiredChildOutput).map { case (nestedFieldToAlias, attrToAliases) => - pruneGenerate(g, nestedFieldToAlias, attrToAliases) + // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. + NestedColumnAliasing.replaceWithAliases(g, nestedFieldToAlias, attrToAliases) } case _ => None } - private def pruneGenerate( - g: Generate, - nestedFieldToAlias: Map[ExtractValue, Alias], - attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { - // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. - NestedColumnAliasing.replaceChildrenWithAliases(g, nestedFieldToAlias, attrToAliases) - } - /** * This is a while-list for pruning nested fields at `Generator`. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index 30fdcf17f8d60..7b1735a6f04ab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -144,7 +144,6 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Pushing a single nested field projection - negative") { val ops = Seq( (input: LogicalPlan) => input.distribute('name)(1), - (input: LogicalPlan) => input.distribute($"name.middle")(1), (input: LogicalPlan) => input.orderBy('name.asc), (input: LogicalPlan) => input.orderBy($"name.middle".asc), (input: LogicalPlan) => input.sortBy('name.asc), @@ -342,6 +341,89 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized, expected) } + test("Nested field pruning through RepartitionByExpression") { + val query1 = contact + .distribute($"id")(1) + .select($"name.middle") + .analyze + val optimized1 = Optimize.execute(query1) + + val aliases1 = collectGeneratedAliases(optimized1) + + val expected1 = contact + .select('id, 'name.getField("middle").as(aliases1(0))) + .distribute($"id")(1) + .select($"${aliases1(0)}".as("middle")) + .analyze + comparePlans(optimized1, expected1) + + val query2 = contact + .distribute($"name.middle")(1) + .select($"name.middle") + .analyze + val optimized2 = Optimize.execute(query2) + + val aliases2 = collectGeneratedAliases(optimized2) + + val expected2 = contact + .select('name.getField("middle").as(aliases2(0))) + .distribute($"${aliases2(0)}")(1) + .select($"${aliases2(0)}".as("middle")) + .analyze + comparePlans(optimized2, expected2) + + val query3 = contact + .select($"name") + .distribute($"name")(1) + .select($"name.middle") + .analyze + val optimized3 = Optimize.execute(query3) + + comparePlans(optimized3, query3) + } + + test("Nested field pruning through Join") { + val department = LocalRelation( + 'depID.int, + 'personID.string) + + val query1 = contact.join(department, condition = Some($"id" === $"depID")) + .select($"name.middle") + .analyze + val optimized1 = Optimize.execute(query1) + + val aliases1 = collectGeneratedAliases(optimized1) + + val expected1 = contact.select('id, 'name.getField("middle").as(aliases1(0))) + .join(department.select('depID), condition = Some($"id" === $"depID")) + .select($"${aliases1(0)}".as("middle")) + .analyze + comparePlans(optimized1, expected1) + + val query2 = contact.join(department, condition = Some($"name.middle" === $"personID")) + .select($"name.first") + .analyze + val optimized2 = Optimize.execute(query2) + + val aliases2 = collectGeneratedAliases(optimized2) + + val expected2 = contact.select( + 'name.getField("first").as(aliases2(0)), + 'name.getField("middle").as(aliases2(1))) + .join(department.select('personID), condition = Some($"${aliases2(1)}" === $"personID")) + .select($"${aliases2(0)}".as("first")) + .analyze + comparePlans(optimized2, expected2) + + val contact2 = LocalRelation('name2.struct(name)) + val query3 = contact.select('name) + .join(contact2, condition = Some($"name" === $"name2")) + .select($"name.first") + .analyze + val optimized3 = Optimize.execute(query3) + comparePlans(optimized3, query3) + } + test("Nested field pruning for Aggregate") { def runTest(basePlan: LogicalPlan => LogicalPlan): Unit = { val query1 = basePlan(contact).groupBy($"id")(first($"name.first").as("first")).analyze diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 2f9e510752b02..8b859e951b9b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -51,6 +51,11 @@ abstract class SchemaPruningSuite relatives: Map[String, FullName] = Map.empty, employer: Employer = null, relations: Map[FullName, String] = Map.empty) + case class Department( + depId: Int, + depName: String, + contactId: Int, + employer: Employer) val janeDoe = FullName("Jane", "X.", "Doe") val johnDoe = FullName("John", "Y.", "Doe") @@ -58,6 +63,7 @@ abstract class SchemaPruningSuite val employer = Employer(0, Company("abc", "123 Business Street")) val employerWithNullCompany = Employer(1, null) + val employerWithNullCompany2 = Employer(2, null) val contacts = Contact(0, janeDoe, "123 Main Street", 1, friends = Array(susanSmith), @@ -66,6 +72,11 @@ abstract class SchemaPruningSuite Contact(1, johnDoe, "321 Wall Street", 3, relatives = Map("sister" -> janeDoe), employer = employerWithNullCompany, relations = Map(janeDoe -> "sister")) :: Nil + val departments = + Department(0, "Engineering", 0, employer) :: + Department(1, "Marketing", 1, employerWithNullCompany) :: + Department(2, "Operation", 4, employerWithNullCompany2) :: Nil + case class Name(first: String, last: String) case class BriefContact(id: Int, name: Name, address: String) @@ -350,6 +361,83 @@ abstract class SchemaPruningSuite checkAnswer(query, Row(0) :: Nil) } + testSchemaPruning("select one deep nested complex field after repartition by expression") { + val query1 = sql("select * from contacts") + .repartition(100, col("id")) + .where("employer.company.address is not null") + .selectExpr("employer.id as employer_id") + checkScan(query1, + "struct>>") + checkAnswer(query1, Row(0) :: Nil) + + val query2 = sql("select * from contacts") + .repartition(100, col("employer")) + .where("employer.company.address is not null") + .selectExpr("employer.id as employer_id") + checkScan(query2, + "struct>>") + checkAnswer(query2, Row(0) :: Nil) + + val query3 = sql("select * from contacts") + .repartition(100, col("employer.company")) + .where("employer.company.address is not null") + .selectExpr("employer.company as employer_company") + checkScan(query3, + "struct>>") + checkAnswer(query3, Row(Row("abc", "123 Business Street")) :: Nil) + + val query4 = sql("select * from contacts") + .repartition(100, col("employer.company.address")) + .where("employer.company.address is not null") + .selectExpr("employer.company.address as employer_company_addr") + checkScan(query4, + "struct>>") + checkAnswer(query4, Row("123 Business Street") :: Nil) + } + + testSchemaPruning("select one deep nested complex field after join") { + val query1 = sql("select contacts.name.middle from contacts, departments where " + + "contacts.id = departments.contactId") + checkScan(query1, + "struct>", + "struct") + checkAnswer(query1, Row("X.") :: Row("Y.") :: Nil) + + val query2 = sql("select contacts.name.middle from contacts, departments where " + + "contacts.employer = departments.employer") + checkScan(query2, + "struct," + + "employer:struct>>", + "struct>>") + checkAnswer(query2, Row("X.") :: Row("Y.") :: Nil) + + val query3 = sql("select contacts.employer.company.name from contacts, departments where " + + "contacts.employer = departments.employer") + checkScan(query3, + "struct>>", + "struct>>") + checkAnswer(query3, Row("abc") :: Row(null) :: Nil) + } + + testSchemaPruning("select one deep nested complex field after outer join") { + val query1 = sql("select departments.contactId, contacts.name.middle from departments " + + "left outer join contacts on departments.contactId = contacts.id") + checkScan(query1, + "struct", + "struct>") + checkAnswer(query1, Row(0, "X.") :: Row(1, "Y.") :: Row(4, null) :: Nil) + + val query2 = sql("select contacts.name.first, departments.employer.company.name " + + "from contacts right outer join departments on contacts.id = departments.contactId") + checkScan(query2, + "struct>", + "struct>>") + checkAnswer(query2, + Row("Jane", "abc") :: + Row("John", null) :: + Row(null, null) :: Nil) + } + testSchemaPruning("select nested field in aggregation function of Aggregate") { val query1 = sql("select count(name.first) from contacts group by name.last") checkScan(query1, "struct>") @@ -439,6 +527,7 @@ abstract class SchemaPruningSuite makeDataSourceFile(contacts, new File(path + "/contacts/p=1")) makeDataSourceFile(briefContacts, new File(path + "/contacts/p=2")) + makeDataSourceFile(departments, new File(path + "/departments")) // Providing user specified schema. Inferred schema from different data sources might // be different. @@ -451,6 +540,11 @@ abstract class SchemaPruningSuite spark.read.format(dataSourceName).schema(schema).load(path + "/contacts") .createOrReplaceTempView("contacts") + val departmentScahem = "`depId` INT,`depName` STRING,`contactId` INT, " + + "`employer` STRUCT<`id`: INT, `company`: STRUCT<`name`: STRING, `address`: STRING>>" + spark.read.format(dataSourceName).schema(departmentScahem).load(path + "/departments") + .createOrReplaceTempView("departments") + testThunk } } @@ -461,6 +555,7 @@ abstract class SchemaPruningSuite makeDataSourceFile(contactsWithDataPartitionColumn, new File(path + "/contacts/p=1")) makeDataSourceFile(briefContactsWithDataPartitionColumn, new File(path + "/contacts/p=2")) + makeDataSourceFile(departments, new File(path + "/departments")) // Providing user specified schema. Inferred schema from different data sources might // be different. @@ -473,6 +568,11 @@ abstract class SchemaPruningSuite spark.read.format(dataSourceName).schema(schema).load(path + "/contacts") .createOrReplaceTempView("contacts") + val departmentScahem = "`depId` INT,`depName` STRING,`contactId` INT, " + + "`employer` STRUCT<`id`: INT, `company`: STRUCT<`name`: STRING, `address`: STRING>>" + spark.read.format(dataSourceName).schema(departmentScahem).load(path + "/departments") + .createOrReplaceTempView("departments") + testThunk } } From d3a5e2963cca58cc464d6ed779bb9fb649a614a8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 12 Jun 2020 17:50:43 +0800 Subject: [PATCH 23/40] Revert "[SPARK-31860][BUILD] only push release tags on succes" This reverts commit 69ba9b662e2ace592380e3cc5de041031dec2254. --- dev/create-release/do-release.sh | 14 +++----------- dev/create-release/release-build.sh | 7 ++----- dev/create-release/release-tag.sh | 16 +++++++++++++--- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh index 64fba8a56affe..4f18a55a3bceb 100755 --- a/dev/create-release/do-release.sh +++ b/dev/create-release/do-release.sh @@ -17,8 +17,6 @@ # limitations under the License. # -set -e - SELF=$(cd $(dirname $0) && pwd) . "$SELF/release-util.sh" @@ -54,6 +52,9 @@ function should_build { if should_build "tag" && [ $SKIP_TAG = 0 ]; then run_silent "Creating release tag $RELEASE_TAG..." "tag.log" \ "$SELF/release-tag.sh" + echo "It may take some time for the tag to be synchronized to github." + echo "Press enter when you've verified that the new tag ($RELEASE_TAG) is available." + read else echo "Skipping tag creation for $RELEASE_TAG." fi @@ -78,12 +79,3 @@ if should_build "publish"; then else echo "Skipping publish step." fi - -if should_build "tag" && [ $SKIP_TAG = 0 ]; then - git push origin $RELEASE_TAG - if [[ $RELEASE_TAG != *"preview"* ]]; then - git push origin HEAD:$GIT_BRANCH - else - echo "It's preview release. We only push $RELEASE_TAG to remote." - fi -fi diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 66c51845cc1d0..e3bcb72ab5c6f 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -92,12 +92,9 @@ BASE_DIR=$(pwd) init_java init_maven_sbt -# Only clone repo fresh if not present, otherwise use checkout from the tag step -if [ ! -d spark ]; then - git clone "$ASF_REPO" -fi +rm -rf spark +git clone "$ASF_REPO" cd spark -git fetch git checkout $GIT_REF git_hash=`git rev-parse --short HEAD` echo "Checked out Spark git hash $git_hash" diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index e37aa27fc0aac..39856a9955955 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -25,7 +25,6 @@ function exit_with_usage { cat << EOF usage: $NAME Tags a Spark release on a particular branch. -You must push the tags after. Inputs are specified with the following environment variables: ASF_USERNAME - Apache Username @@ -106,8 +105,19 @@ sed -i".tmp7" 's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$R_NEXT_VERSION" git commit -a -m "Preparing development version $NEXT_VERSION" -cd .. -if is_dry_run; then +if ! is_dry_run; then + # Push changes + git push origin $RELEASE_TAG + if [[ $RELEASE_VERSION != *"preview"* ]]; then + git push origin HEAD:$GIT_BRANCH + else + echo "It's preview release. We only push $RELEASE_TAG to remote." + fi + + cd .. + rm -rf spark +else + cd .. mv spark spark.tag echo "Clone with version changes and tag available as spark.tag in the output directory." fi From 9b098f1eb91a5e9f488d573bfeea3f6bfd9b95b3 Mon Sep 17 00:00:00 2001 From: iRakson Date: Fri, 12 Jun 2020 10:27:31 -0500 Subject: [PATCH 24/40] [SPARK-30119][WEBUI] Support pagination for streaming tab ### What changes were proposed in this pull request? #28747 reverted #28439 due to some flaky test case. This PR fixes the flaky test and adds pagination support. ### Why are the changes needed? To support pagination for streaming tab ### Does this PR introduce _any_ user-facing change? Yes, Now streaming tab tables will be paginated. ### How was this patch tested? Manually. Closes #28748 from iRakson/fixstreamingpagination. Authored-by: iRakson Signed-off-by: Sean Owen --- .../org/apache/spark/ui/static/webui.js | 3 +- .../spark/streaming/ui/AllBatchesTable.scala | 282 ++++++++++-------- .../spark/streaming/ui/StreamingPage.scala | 113 +++++-- .../spark/streaming/UISeleniumSuite.scala | 39 ++- 4 files changed, 259 insertions(+), 178 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js index 4f8409ca2b7c2..bb3725650c667 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.js +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -87,7 +87,8 @@ $(function() { collapseTablePageLoad('collapse-aggregated-poolActiveStages','aggregated-poolActiveStages'); collapseTablePageLoad('collapse-aggregated-tasks','aggregated-tasks'); collapseTablePageLoad('collapse-aggregated-rdds','aggregated-rdds'); - collapseTablePageLoad('collapse-aggregated-activeBatches','aggregated-activeBatches'); + collapseTablePageLoad('collapse-aggregated-waitingBatches','aggregated-waitingBatches'); + collapseTablePageLoad('collapse-aggregated-runningBatches','aggregated-runningBatches'); collapseTablePageLoad('collapse-aggregated-completedBatches','aggregated-completedBatches'); collapseTablePageLoad('collapse-aggregated-runningExecutions','aggregated-runningExecutions'); collapseTablePageLoad('collapse-aggregated-completedExecutions','aggregated-completedExecutions'); diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala index 1e443f656734c..c0eec0e0b0a85 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala @@ -17,30 +17,41 @@ package org.apache.spark.streaming.ui -import scala.xml.Node - -import org.apache.spark.ui.{UIUtils => SparkUIUtils} +import java.net.URLEncoder +import java.nio.charset.StandardCharsets.UTF_8 +import javax.servlet.http.HttpServletRequest -private[ui] abstract class BatchTableBase(tableId: String, batchInterval: Long) { +import scala.xml.Node - protected def columns: Seq[Node] = { - Batch Time - Records - Scheduling Delay - {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "top")} - - Processing Time - {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "top")} - } +import org.apache.spark.ui.{PagedDataSource, PagedTable, UIUtils => SparkUIUtils} + +private[ui] class StreamingPagedTable( + request: HttpServletRequest, + tableTag: String, + batches: Seq[BatchUIData], + basePath: String, + subPath: String, + batchInterval: Long) extends PagedTable[BatchUIData] { + + private val(sortColumn, desc, pageSize) = getTableParameters(request, tableTag, "Batch Time") + private val parameterPath = s"$basePath/$subPath/?${getParameterOtherTable(request, tableTag)}" + private val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + + private val firstFailureReason: Option[String] = + if (!tableTag.equals("waitingBatches")) { + getFirstFailureReason(batches) + } else { + None + } /** * Return the first failure reason if finding in the batches. */ - protected def getFirstFailureReason(batches: Seq[BatchUIData]): Option[String] = { + private def getFirstFailureReason(batches: Seq[BatchUIData]): Option[String] = { batches.flatMap(_.outputOperations.flatMap(_._2.failureReason)).headOption } - protected def getFirstFailureTableCell(batch: BatchUIData): Seq[Node] = { + private def getFirstFailureTableCell(batch: BatchUIData): Seq[Node] = { val firstFailureReason = batch.outputOperations.flatMap(_._2.failureReason).headOption firstFailureReason.map { failureReason => val failureReasonForUI = UIUtils.createOutputOperationFailureForUI(failureReason) @@ -49,147 +60,154 @@ private[ui] abstract class BatchTableBase(tableId: String, batchInterval: Long) }.getOrElse(-) } - protected def baseRow(batch: BatchUIData): Seq[Node] = { - val batchTime = batch.batchTime.milliseconds - val formattedBatchTime = SparkUIUtils.formatBatchTime(batchTime, batchInterval) - val numRecords = batch.numRecords - val schedulingDelay = batch.schedulingDelay - val formattedSchedulingDelay = schedulingDelay.map(SparkUIUtils.formatDuration).getOrElse("-") - val processingTime = batch.processingDelay - val formattedProcessingTime = processingTime.map(SparkUIUtils.formatDuration).getOrElse("-") - val batchTimeId = s"batch-$batchTime" - - - - {formattedBatchTime} - - - {numRecords.toString} records - - {formattedSchedulingDelay} - - - {formattedProcessingTime} - - } - - private def batchTable: Seq[Node] = { - - - {columns} - - - {renderRows} - -
- } - - def toNodeSeq: Seq[Node] = { - batchTable - } - - protected def createOutputOperationProgressBar(batch: BatchUIData): Seq[Node] = { + private def createOutputOperationProgressBar(batch: BatchUIData): Seq[Node] = { { - SparkUIUtils.makeProgressBar( - started = batch.numActiveOutputOp, - completed = batch.numCompletedOutputOp, - failed = batch.numFailedOutputOp, - skipped = 0, - reasonToNumKilled = Map.empty, - total = batch.outputOperations.size) + SparkUIUtils.makeProgressBar( + started = batch.numActiveOutputOp, + completed = batch.numCompletedOutputOp, + failed = batch.numFailedOutputOp, + skipped = 0, + reasonToNumKilled = Map.empty, + total = batch.outputOperations.size) } } - /** - * Return HTML for all rows of this table. - */ - protected def renderRows: Seq[Node] -} + override def tableId: String = s"$tableTag-table" -private[ui] class ActiveBatchTable( - runningBatches: Seq[BatchUIData], - waitingBatches: Seq[BatchUIData], - batchInterval: Long) extends BatchTableBase("active-batches-table", batchInterval) { + override def tableCssClass: String = + "table table-bordered table-sm table-striped table-head-clickable table-cell-width-limited" - private val firstFailureReason = getFirstFailureReason(runningBatches) + override def pageSizeFormField: String = s"$tableTag.pageSize" - override protected def columns: Seq[Node] = super.columns ++ { - Output Ops: Succeeded/Total - Status ++ { - if (firstFailureReason.nonEmpty) { - Error - } else { - Nil - } - } - } + override def pageNumberFormField: String = s"$tableTag.page" - override protected def renderRows: Seq[Node] = { - // The "batchTime"s of "waitingBatches" must be greater than "runningBatches"'s, so display - // waiting batches before running batches - waitingBatches.flatMap(batch => {waitingBatchRow(batch)}) ++ - runningBatches.flatMap(batch => {runningBatchRow(batch)}) + override def pageLink(page: Int): String = { + parameterPath + + s"&$tableTag.sort=$encodedSortColumn" + + s"&$tableTag.desc=$desc" + + s"&$pageNumberFormField=$page" + + s"&$pageSizeFormField=$pageSize" + + s"#$tableTag" } - private def runningBatchRow(batch: BatchUIData): Seq[Node] = { - baseRow(batch) ++ createOutputOperationProgressBar(batch) ++ processing ++ { - if (firstFailureReason.nonEmpty) { - getFirstFailureTableCell(batch) - } else { - Nil + override def goButtonFormPath: String = + s"$parameterPath&$tableTag.sort=$encodedSortColumn&$tableTag.desc=$desc#$tableTag" + + override def dataSource: PagedDataSource[BatchUIData] = + new StreamingDataSource(batches, pageSize, sortColumn, desc) + + override def headers: Seq[Node] = { + // headers, sortable and tooltips + val headersAndCssClasses: Seq[(String, Boolean, Option[String])] = { + Seq( + ("Batch Time", true, None), + ("Records", true, None), + ("Scheduling Delay", true, Some("Time taken by Streaming scheduler to submit jobs " + + "of a batch")), + ("Processing Time", true, Some("Time taken to process all jobs of a batch"))) ++ { + if (tableTag.equals("completedBatches")) { + Seq( + ("Total Delay", true, Some("Total time taken to handle a batch")), + ("Output Ops: Succeeded/Total", false, None)) + } else { + Seq( + ("Output Ops: Succeeded/Total", false, None), + ("Status", false, None)) + } + } ++ { + if (firstFailureReason.nonEmpty) { + Seq(("Error", false, None)) + } else { + Nil + } } } + // check if sort column is a valid sortable column + isSortColumnValid(headersAndCssClasses, sortColumn) + + headerRow(headersAndCssClasses, desc, pageSize, sortColumn, parameterPath, tableTag, tableTag) } - private def waitingBatchRow(batch: BatchUIData): Seq[Node] = { - baseRow(batch) ++ createOutputOperationProgressBar(batch) ++ queued++ { - if (firstFailureReason.nonEmpty) { - // Waiting batches have not run yet, so must have no failure reasons. - - - } else { - Nil + override def row(batch: BatchUIData): Seq[Node] = { + val batchTime = batch.batchTime.milliseconds + val formattedBatchTime = SparkUIUtils.formatBatchTime(batchTime, batchInterval) + val numRecords = batch.numRecords + val schedulingDelay = batch.schedulingDelay + val formattedSchedulingDelay = schedulingDelay.map(SparkUIUtils.formatDuration).getOrElse("-") + val processingTime = batch.processingDelay + val formattedProcessingTime = processingTime.map(SparkUIUtils.formatDuration).getOrElse("-") + val batchTimeId = s"batch-$batchTime" + val totalDelay = batch.totalDelay + val formattedTotalDelay = totalDelay.map(SparkUIUtils.formatDuration).getOrElse("-") + + + + + {formattedBatchTime} + + + {numRecords.toString} records + {formattedSchedulingDelay} + {formattedProcessingTime} + { + if (tableTag.equals("completedBatches")) { + {formattedTotalDelay} ++ + createOutputOperationProgressBar(batch) ++ { + if (firstFailureReason.nonEmpty) { + getFirstFailureTableCell(batch) + } else { + Nil + } + } + } else if (tableTag.equals("runningBatches")) { + createOutputOperationProgressBar(batch) ++ + processing ++ { + if (firstFailureReason.nonEmpty) { + getFirstFailureTableCell(batch) + } else { + Nil + } + } + } else { + createOutputOperationProgressBar(batch) ++ + queued ++ { + if (firstFailureReason.nonEmpty) { + // Waiting batches have not run yet, so must have no failure reasons. + - + } else { + Nil + } + } + } } - } + } } -private[ui] class CompletedBatchTable(batches: Seq[BatchUIData], batchInterval: Long) - extends BatchTableBase("completed-batches-table", batchInterval) { +private[ui] class StreamingDataSource(info: Seq[BatchUIData], pageSize: Int, sortColumn: String, + desc: Boolean) extends PagedDataSource[BatchUIData](pageSize) { - private val firstFailureReason = getFirstFailureReason(batches) + private val data = info.sorted(ordering(sortColumn, desc)) - override protected def columns: Seq[Node] = super.columns ++ { - Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "top")} - Output Ops: Succeeded/Total ++ { - if (firstFailureReason.nonEmpty) { - Error - } else { - Nil - } - } - } + override protected def dataSize: Int = data.size - override protected def renderRows: Seq[Node] = { - batches.flatMap(batch => {completedBatchRow(batch)}) - } + override protected def sliceData(from: Int, to: Int): Seq[BatchUIData] = data.slice(from, to) - private def completedBatchRow(batch: BatchUIData): Seq[Node] = { - val totalDelay = batch.totalDelay - val formattedTotalDelay = totalDelay.map(SparkUIUtils.formatDuration).getOrElse("-") - - baseRow(batch) ++ { - - {formattedTotalDelay} - - } ++ createOutputOperationProgressBar(batch)++ { - if (firstFailureReason.nonEmpty) { - getFirstFailureTableCell(batch) - } else { - Nil - } + private def ordering(column: String, desc: Boolean): Ordering[BatchUIData] = { + val ordering: Ordering[BatchUIData] = column match { + case "Batch Time" => Ordering.by(_.batchTime.milliseconds) + case "Records" => Ordering.by(_.numRecords) + case "Scheduling Delay" => Ordering.by(_.schedulingDelay.getOrElse(Long.MaxValue)) + case "Processing Time" => Ordering.by(_.processingDelay.getOrElse(Long.MaxValue)) + case "Total Delay" => Ordering.by(_.totalDelay.getOrElse(Long.MaxValue)) + case unknownColumn => throw new IllegalArgumentException(s"Unknown Column: $unknownColumn") + } + if (desc) { + ordering.reverse + } else { + ordering } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 3bdf009dbce66..42d0e50a068ec 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -20,10 +20,12 @@ package org.apache.spark.streaming.ui import java.util.concurrent.TimeUnit import javax.servlet.http.HttpServletRequest +import scala.collection.mutable import scala.xml.{Node, Unparsed} import org.apache.spark.internal.Logging import org.apache.spark.ui.{GraphUIData, JsCollector, UIUtils => SparkUIUtils, WebUIPage} +import org.apache.spark.util.Utils /** * A helper class for "scheduling delay", "processing time" and "total delay" to generate data that @@ -86,7 +88,7 @@ private[ui] class StreamingPage(parent: StreamingTab) onClickTimelineFunc ++ basicInfo ++ listener.synchronized { generateStatTable() ++ - generateBatchListTables() + generateBatchListTables(request) } SparkUIUtils.headerSparkPage(request, "Streaming Statistics", content, parent) } @@ -432,50 +434,97 @@ private[ui] class StreamingPage(parent: StreamingTab) } - private def generateBatchListTables(): Seq[Node] = { + private def streamingTable(request: HttpServletRequest, batches: Seq[BatchUIData], + tableTag: String): Seq[Node] = { + val interval: Long = listener.batchDuration + val streamingPage = Option(request.getParameter(s"$tableTag.page")).map(_.toInt).getOrElse(1) + + try { + new StreamingPagedTable( + request, + tableTag, + batches, + SparkUIUtils.prependBaseUri(request, parent.basePath), + "streaming", + interval + ).table(streamingPage) + } catch { + case e @ (_: IllegalArgumentException | _: IndexOutOfBoundsException) => +
+

Error while rendering streaming table:

+
+            {Utils.exceptionString(e)}
+          
+
+ } + } + + private def generateBatchListTables(request: HttpServletRequest): Seq[Node] = { val runningBatches = listener.runningBatches.sortBy(_.batchTime.milliseconds).reverse val waitingBatches = listener.waitingBatches.sortBy(_.batchTime.milliseconds).reverse val completedBatches = listener.retainedCompletedBatches. sortBy(_.batchTime.milliseconds).reverse - val activeBatchesContent = { -
-
- -

- - Active Batches ({runningBatches.size + waitingBatches.size}) -

-
-
- {new ActiveBatchTable(runningBatches, waitingBatches, listener.batchDuration).toNodeSeq} + val content = mutable.ListBuffer[Node]() + + if (runningBatches.nonEmpty) { + content ++= +
+
+ +

+ + Running Batches ({runningBatches.size}) +

+
+
+ { streamingTable(request, runningBatches, "runningBatches") } +
-
} - val completedBatchesContent = { -
-
- -

- - Completed Batches (last {completedBatches.size} - out of {listener.numTotalCompletedBatches}) -

-
-
- {new CompletedBatchTable(completedBatches, listener.batchDuration).toNodeSeq} + if (waitingBatches.nonEmpty) { + content ++= +
+
+ +

+ + Waiting Batches ({waitingBatches.size}) +

+
+
+ { streamingTable(request, waitingBatches, "waitingBatches") } +
-
} - activeBatchesContent ++ completedBatchesContent + if (completedBatches.nonEmpty) { + content ++= +
+
+ +

+ + Completed Batches (last {completedBatches.size} + out of {listener.numTotalCompletedBatches}) +

+
+
+ { streamingTable(request, completedBatches, "completedBatches") } +
+
+
+ } + content } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index bdc9e9ee2aed1..7041e46f99b18 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -63,7 +63,7 @@ class UISeleniumSuite .setMaster("local") .setAppName("test") .set(UI_ENABLED, true) - val ssc = new StreamingContext(conf, Seconds(1)) + val ssc = new StreamingContext(conf, Milliseconds(100)) assert(ssc.sc.ui.isDefined, "Spark UI is not started!") ssc } @@ -104,7 +104,7 @@ class UISeleniumSuite find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None) } - eventually(timeout(10.seconds), interval(50.milliseconds)) { + eventually(timeout(10.seconds), interval(500.milliseconds)) { // check whether streaming page exists go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq @@ -125,24 +125,37 @@ class UISeleniumSuite // Check batch tables val h4Text = findAll(cssSelector("h4")).map(_.text).toSeq - h4Text.exists(_.matches("Active Batches \\(\\d+\\)")) should be (true) h4Text.exists(_.matches("Completed Batches \\(last \\d+ out of \\d+\\)")) should be (true) - findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be { - List("Batch Time", "Records", "Scheduling Delay (?)", "Processing Time (?)", - "Output Ops: Succeeded/Total", "Status") - } - findAll(cssSelector("""#completed-batches-table th""")).map(_.text).toSeq should be { - List("Batch Time", "Records", "Scheduling Delay (?)", "Processing Time (?)", - "Total Delay (?)", "Output Ops: Succeeded/Total") + val arrow = 0x25BE.toChar + findAll(cssSelector("""#completedBatches-table th""")).map(_.text).toList should be { + List(s"Batch Time $arrow", "Records", "Scheduling Delay", "Processing Time", + "Total Delay", "Output Ops: Succeeded/Total") } - val batchLinks = - findAll(cssSelector("""#completed-batches-table a""")).flatMap(_.attribute("href")).toSeq + val pageSize = 1 + val pagedTablePath = "/streaming/?completedBatches.sort=Batch+Time" + + "&completedBatches.desc=true&completedBatches.page=1" + + s"&completedBatches.pageSize=$pageSize#completedBatches" + + go to (sparkUI.webUrl.stripSuffix("/") + pagedTablePath) + val completedTableRows = findAll(cssSelector("""#completedBatches-table tr""")) + .map(_.text).toList + // header row + pagesize + completedTableRows.length should be (1 + pageSize) + + val sortedBatchTimePath = "/streaming/?&completedBatches.sort=Batch+Time" + + s"&completedBatches.desc=false&completedBatches.pageSize=$pageSize#completedBatches" + + // sort batches in ascending order of batch time + go to (sparkUI.webUrl.stripSuffix("/") + sortedBatchTimePath) + + val batchLinks = findAll(cssSelector("""#completedBatches-table td a""")) + .flatMap(_.attribute("href")).toSeq batchLinks.size should be >= 1 // Check a normal batch page - go to (batchLinks.last) // Last should be the first batch, so it will have some jobs + go to (batchLinks.head) // Head is the first batch, so it will have some jobs val summaryText = findAll(cssSelector("li strong")).map(_.text).toSeq summaryText should contain ("Batch Duration:") summaryText should contain ("Input data size:") From 28f131fc8a4bbabb2d0bca3c46418b78c8ba3e1d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 12 Jun 2020 11:06:52 -0700 Subject: [PATCH 25/40] [SPARK-31979] Release script should not fail when remove non-existing files ### What changes were proposed in this pull request? When removing non-existing files in the release script, do not fail. ### Why are the changes needed? This is to make the release script more robust, as we don't care if the files exist before we remove them. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? tested when cutting 3.0.0 RC Closes #28815 from cloud-fan/release. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index e3bcb72ab5c6f..eb972589a995e 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -164,7 +164,7 @@ fi DEST_DIR_NAME="$SPARK_PACKAGE_VERSION" git clean -d -f -x -rm .gitignore +rm -f .gitignore cd .. if [[ "$1" == "package" ]]; then @@ -174,9 +174,9 @@ if [[ "$1" == "package" ]]; then # For source release in v2.4+, exclude copy of binary license/notice if [[ $SPARK_VERSION > "2.4" ]]; then - rm spark-$SPARK_VERSION/LICENSE-binary - rm spark-$SPARK_VERSION/NOTICE-binary - rm -r spark-$SPARK_VERSION/licenses-binary + rm -f spark-$SPARK_VERSION/LICENSE-binary + rm -f spark-$SPARK_VERSION/NOTICE-binary + rm -rf spark-$SPARK_VERSION/licenses-binary fi tar cvzf spark-$SPARK_VERSION.tgz --exclude spark-$SPARK_VERSION/.git spark-$SPARK_VERSION From 78d08a8c38a75acd9a92fe0e8787eb32553ca7b1 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 13 Jun 2020 07:12:27 +0900 Subject: [PATCH 26/40] [SPARK-31950][SQL][TESTS] Extract SQL keywords from the SqlBase.g4 file ### What changes were proposed in this pull request? This PR intends to extract SQL reserved/non-reserved keywords from the ANTLR grammar file (`SqlBase.g4`) directly. This approach is based on the cloud-fan suggestion: https://github.com/apache/spark/pull/28779#issuecomment-642033217 ### Why are the changes needed? It is hard to maintain a full set of the keywords in `TableIdentifierParserSuite`, so it would be nice if we could extract them from the `SqlBase.g4` file directly. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #28802 from maropu/SPARK-31950-2. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 + .../parser/TableIdentifierParserSuite.scala | 432 +++++------------- 2 files changed, 110 insertions(+), 326 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 14b5fa3dbfda6..61aeafdc27f11 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 @@ -1009,6 +1009,7 @@ alterColumnAction // You can find the full keywords list by searching "Start of the keywords list" in this file. // The non-reserved keywords are listed below. Keywords not in this list are reserved keywords. ansiNonReserved +//--ANSI-NON-RESERVED-START : ADD | AFTER | ALTER @@ -1185,6 +1186,7 @@ ansiNonReserved | VIEW | VIEWS | WINDOW +//--ANSI-NON-RESERVED-END ; // When `SQL_standard_keyword_behavior=false`, there are 2 kinds of keywords in Spark SQL. @@ -1462,6 +1464,7 @@ nonReserved //============================ // Start of the keywords list //============================ +//--SPARK-KEYWORD-LIST-START ADD: 'ADD'; AFTER: 'AFTER'; ALL: 'ALL'; @@ -1714,6 +1717,7 @@ WHERE: 'WHERE'; WINDOW: 'WINDOW'; WITH: 'WITH'; YEAR: 'YEAR'; +//--SPARK-KEYWORD-LIST-END //============================ // End of the keywords list //============================ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index bd617bf7e3df6..04969e34fb841 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -16,9 +16,14 @@ */ package org.apache.spark.sql.catalyst.parser +import java.util.Locale + +import scala.collection.mutable + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.fileToString import org.apache.spark.sql.internal.SQLConf class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { @@ -285,334 +290,109 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "where", "with") - // All the keywords in `docs/sql-keywords.md` are listed below: - val allCandidateKeywords = Set( - "add", - "after", - "all", - "alter", - "analyze", - "and", - "anti", - "any", - "archive", - "array", - "as", - "asc", - "at", - "authorization", - "between", - "both", - "bucket", - "buckets", - "by", - "cache", - "cascade", - "case", - "cast", - "change", - "check", - "clear", - "cluster", - "clustered", - "codegen", - "collate", - "collection", - "column", - "columns", - "comment", - "commit", - "compact", - "compactions", - "compute", - "concatenate", - "constraint", - "cost", - "create", - "cross", - "cube", - "current", - "current_date", - "current_time", - "current_timestamp", - "current_user", - "data", - "database", - "databases", - "day", - "dbproperties", - "defined", - "delete", - "delimited", - "desc", - "describe", - "dfs", - "directories", - "directory", - "distinct", - "distribute", - "div", - "drop", - "else", - "end", - "escape", - "escaped", - "except", - "exchange", - "exists", - "explain", - "export", - "extended", - "external", - "extract", - "false", - "fetch", - "fields", - "fileformat", - "first", - "following", - "for", - "foreign", - "format", - "formatted", - "from", - "full", - "function", - "functions", - "global", - "grant", - "group", - "grouping", - "having", - "hour", - "if", - "ignore", - "import", - "in", - "index", - "indexes", - "inner", - "inpath", - "inputformat", - "insert", - "intersect", - "interval", - "into", - "is", - "items", - "join", - "keys", - "last", - "lateral", - "lazy", - "leading", - "left", - "like", - "limit", - "lines", - "list", - "load", - "local", - "location", - "lock", - "locks", - "logical", - "macro", - "map", - "minus", - "minute", - "month", - "msck", - "namespaces", - "natural", - "no", - "not", - "null", - "nulls", - "of", - "on", - "only", - "option", - "options", - "or", - "order", - "out", - "outer", - "outputformat", - "over", - "overlaps", - "overlay", - "overwrite", - "partition", - "partitioned", - "partitions", - "percent", - "pivot", - "placing", - "position", - "preceding", - "primary", - "principals", - "purge", - "query", - "range", - "recordreader", - "recordwriter", - "recover", - "reduce", - "references", - "refresh", - "rename", - "repair", - "replace", - "reset", - "restrict", - "revoke", - "right", - "rlike", - "role", - "roles", - "rollback", - "rollup", - "row", - "rows", - "schema", - "second", - "select", - "semi", - "separated", - "serde", - "serdeproperties", - "session_user", - "set", - "sets", - "show", - "skewed", - "some", - "sort", - "sorted", - "start", - "statistics", - "stored", - "stratify", - "struct", - "substr", - "substring", - "table", - "tables", - "tablesample", - "tblproperties", - "temporary", - "terminated", - "then", - "to", - "touch", - "trailing", - "transaction", - "transactions", - "transform", - "true", - "truncate", - "type", - "unarchive", - "unbounded", - "uncache", - "union", - "unique", - "unknown", - "unlock", - "unset", - "use", - "user", - "using", - "values", - "view", - "views", - "when", - "where", - "window", - "with", - "year") + private val sqlSyntaxDefs = { + val sqlBasePath = { + val sparkHome = { + assert(sys.props.contains("spark.test.home") || + sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not set.") + sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) + } + java.nio.file.Paths.get(sparkHome, "sql", "catalyst", "src", "main", "antlr4", "org", + "apache", "spark", "sql", "catalyst", "parser", "SqlBase.g4").toFile + } + fileToString(sqlBasePath).split("\n") + } - val reservedKeywordsInAnsiMode = Set( - "all", - "and", - "anti", - "any", - "as", - "authorization", - "both", - "case", - "cast", - "check", - "collate", - "column", - "constraint", - "create", - "cross", - "current_date", - "current_time", - "current_timestamp", - "current_user", - "day", - "distinct", - "else", - "end", - "escape", - "except", - "false", - "fetch", - "for", - "foreign", - "from", - "full", - "grant", - "group", - "having", - "hour", - "in", - "inner", - "intersect", - "into", - "join", - "is", - "leading", - "left", - "minute", - "month", - "natural", - "not", - "null", - "on", - "only", - "or", - "order", - "outer", - "overlaps", - "primary", - "references", - "right", - "select", - "semi", - "session_user", - "minus", - "second", - "some", - "table", - "then", - "to", - "trailing", - "union", - "unique", - "unknown", - "user", - "using", - "when", - "where", - "with", - "year") + private def parseAntlrGrammars[T](startTag: String, endTag: String) + (f: PartialFunction[String, Seq[T]]): Set[T] = { + val keywords = new mutable.ArrayBuffer[T] + val default = (_: String) => Nil + var startTagFound = false + var parseFinished = false + val lineIter = sqlSyntaxDefs.toIterator + while (!parseFinished && lineIter.hasNext) { + val line = lineIter.next() + if (line.trim.startsWith(startTag)) { + startTagFound = true + } else if (line.trim.startsWith(endTag)) { + parseFinished = true + } else if (startTagFound) { + f.applyOrElse(line, default).foreach { symbol => + keywords += symbol + } + } + } + assert(keywords.nonEmpty && startTagFound && parseFinished, "cannot extract keywords from " + + s"the `SqlBase.g4` file, so please check if the start/end tags (`$startTag` and `$endTag`) " + + "are placed correctly in the file.") + keywords.toSet + } - val nonReservedKeywordsInAnsiMode = allCandidateKeywords -- reservedKeywordsInAnsiMode + // If a symbol does not have the same string with its literal (e.g., `SETMINUS: 'MINUS';`), + // we need to map a symbol to actual literal strings. + val symbolsToExpandIntoDifferentLiterals = { + val kwDef = """([A-Z_]+):(.+);""".r + val keywords = parseAntlrGrammars( + "//--SPARK-KEYWORD-LIST-START", "//--SPARK-KEYWORD-LIST-END") { + case kwDef(symbol, literalDef) => + val splitDefs = literalDef.split("""\|""") + val hasMultipleLiterals = splitDefs.length > 1 + // The case where a symbol has multiple literal definitions, + // e.g., `DATABASES: 'DATABASES' | 'SCHEMAS';`. + if (hasMultipleLiterals) { + val literals = splitDefs.map(_.replaceAll("'", "").trim).toSeq + (symbol, literals) :: Nil + } else { + val literal = literalDef.replaceAll("'", "").trim + // The case where a symbol string and its literal string are different, + // e.g., `SETMINUS: 'MINUS';`. + if (symbol != literal) { + (symbol, literal :: Nil) :: Nil + } else { + Nil + } + } + } + keywords.toMap + } + + // All the SQL keywords defined in `SqlBase.g4` + val allCandidateKeywords = { + val kwDef = """([A-Z_]+):.+;""".r + val keywords = parseAntlrGrammars( + "//--SPARK-KEYWORD-LIST-START", "//--SPARK-KEYWORD-LIST-END") { + // Parses a pattern, e.g., `AFTER: 'AFTER';` + case kwDef(symbol) => + if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { + symbolsToExpandIntoDifferentLiterals(symbol) + } else { + symbol :: Nil + } + } + keywords + } + + val nonReservedKeywordsInAnsiMode = { + val kwDef = """\s*[\|:]\s*([A-Z_]+)\s*""".r + parseAntlrGrammars("//--ANSI-NON-RESERVED-START", "//--ANSI-NON-RESERVED-END") { + // Parses a pattern, e.g., ` | AFTER` + case kwDef(symbol) => + if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { + symbolsToExpandIntoDifferentLiterals(symbol) + } else { + symbol :: Nil + } + } + } + + val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode + + test("check # of reserved keywords") { + val numReservedKeywords = 78 + assert(reservedKeywordsInAnsiMode.size == numReservedKeywords, + s"The expected number of reserved keywords is $numReservedKeywords, but " + + s"${reservedKeywordsInAnsiMode.size} found.") + } test("table identifier") { // Regular names. From a620a2a7e53cbf05de07cdc1f32c43fe006e3e7a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 13 Jun 2020 07:26:37 +0900 Subject: [PATCH 27/40] [SPARK-31977][SQL] Returns the plan directly from NestedColumnAliasing ### What changes were proposed in this pull request? This proposes a minor refactoring to match `NestedColumnAliasing` to `GeneratorNestedColumnAliasing` so it returns the pruned plan directly. ```scala case p NestedColumnAliasing(nestedFieldToAlias, attrToAliases) => NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases) ``` vs ```scala case GeneratorNestedColumnAliasing(p) => p ``` ### Why are the changes needed? Just for readability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests should cover. Closes #28812 from HyukjinKwon/SPARK-31977. Authored-by: HyukjinKwon Signed-off-by: Takeshi Yamamuro --- .../optimizer/NestedColumnAliasing.scala | 19 ++++++++++++------- .../sql/catalyst/optimizer/Optimizer.scala | 3 +-- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index eb8022c8c8404..7f49d8db4852d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -30,16 +30,21 @@ import org.apache.spark.sql.types._ */ object NestedColumnAliasing { - def unapply(plan: LogicalPlan) - : Option[(Map[ExtractValue, Alias], Map[ExprId, Seq[Alias]])] = plan match { + def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { case Project(projectList, child) if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) => val exprCandidatesToPrune = projectList ++ child.expressions - getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq) + getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map { + case (nestedFieldToAlias, attrToAliases) => + NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases) + } - case plan if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(plan) => - val exprCandidatesToPrune = plan.expressions - getAliasSubMap(exprCandidatesToPrune, plan.producedAttributes.toSeq) + case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) => + val exprCandidatesToPrune = p.expressions + getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map { + case (nestedFieldToAlias, attrToAliases) => + NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases) + } case _ => None } @@ -47,7 +52,7 @@ object NestedColumnAliasing { /** * Replace nested columns to prune unused nested columns later. */ - def replaceToAliases( + private def replaceToAliases( plan: LogicalPlan, nestedFieldToAlias: Map[ExtractValue, Alias], attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match { 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 a1a7213664ac8..8e57e9737c73f 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 @@ -649,8 +649,7 @@ object ColumnPruning extends Rule[LogicalPlan] { // Can't prune the columns on LeafNode case p @ Project(_, _: LeafNode) => p - case p @ NestedColumnAliasing(nestedFieldToAlias, attrToAliases) => - NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases) + case NestedColumnAliasing(p) => p // for all other logical plans that inherits the output from it's children // Project over project is handled by the first case, skip it here. From f535004e14b197ceb1f2108a67b033c052d65bcb Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 12 Jun 2020 17:22:41 -0700 Subject: [PATCH 28/40] [SPARK-31967][UI] Downgrade to vis.js 4.21.0 to fix Jobs UI loading time regression ### What changes were proposed in this pull request? After #28192, the job list page becomes very slow. For example, after the following operation, the UI loading can take >40 sec. ``` (1 to 1000).foreach(_ => sc.parallelize(1 to 10).collect) ``` This is caused by a [performance issue of `vis-timeline`](https://github.com/visjs/vis-timeline/issues/379). The serious issue affects both branch-3.0 and branch-2.4 I tried a different version 4.21.0 from https://cdnjs.com/libraries/vis The infinite drawing issue seems also fixed if the zoom is disabled as default. ### Why are the changes needed? Fix the serious perf issue in web UI by falling back vis-timeline-graph2d to an ealier version. ### Does this PR introduce _any_ user-facing change? Yes, fix the UI perf regression ### How was this patch tested? Manual test Closes #28806 from gengliangwang/downgradeVis. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- .../ui/static/vis-timeline-graph2d.min.css | 3 +- .../static/vis-timeline-graph2d.min.css.map | 1 - .../ui/static/vis-timeline-graph2d.min.js | 70 ++++++------------- .../ui/static/vis-timeline-graph2d.min.js.map | 1 - dev/.rat-excludes | 2 - licenses-binary/LICENSE-vis-timeline.txt | 29 ++++---- licenses/LICENSE-vis-timeline.txt | 29 ++++---- 7 files changed, 49 insertions(+), 86 deletions(-) delete mode 100644 core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css.map delete mode 100644 core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js.map diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css index d22a9ac801891..eeacbd802d305 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css +++ b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css @@ -1,2 +1 @@ -.vis [class*=span]{min-height:0;width:auto}.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}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}input.vis-configuration.vis-config-range{-webkit-appearance:none;border:0 solid #fff;background-color:transparent;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,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(180deg,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr="#dedede",endColorstr="#c8c8c8",GradientType=0);border:1px solid #999;box-shadow:0 0 3px 0 #aaa;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,#385380);background:-o-linear-gradient(top,#3876c2 0,#385380 100%);background:-ms-linear-gradient(top,#3876c2 0,#385380 100%);background:linear-gradient(180deg,#3876c2 0,#385380);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr="#3876c2",endColorstr="#385380",GradientType=0);box-shadow:0 0 1px 0 #111927;margin-top:-7px}input.vis-configuration.vis-config-range:focus{outline:none}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,#c8c8c8 99%);background:-o-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:linear-gradient(180deg,#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,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(180deg,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr="#dedede",endColorstr="#c8c8c8",GradientType=0);border:1px solid #999;box-shadow:0 0 3px 0 #aaa;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:1px solid #fff;outline-offset:-1px}input.vis-configuration.vis-config-range::-ms-track{width:300px;height:5px;background:transparent;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) rgba(136,183,213,0) rgba(136,183,213,0) rgba(57,76,89,.85);border-width:8px;margin-top:-8px}.vis-configuration-popup:before{border-color:rgba(194,225,245,0) rgba(194,225,245,0) rgba(194,225,245,0) #f2faff;border-width:12px;margin-top:-12px}div.vis-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;z-index:5}.vis-panel{position:absolute;padding:0;margin:0;box-sizing:border-box}.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-left.vis-panel.vis-vertical-scroll,.vis-right.vis-panel.vis-vertical-scroll{height:100%;overflow-x:hidden;overflow-y:scroll}.vis-left.vis-panel.vis-vertical-scroll{direction:rtl}.vis-left.vis-panel.vis-vertical-scroll .vis-content,.vis-right.vis-panel.vis-vertical-scroll{direction:ltr}.vis-right.vis-panel.vis-vertical-scroll .vis-content{direction:rtl}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-top{border-left-style:solid;border-right-style:solid}.vis-background{overflow:hidden}.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-panel .vis-shadow.vis-top{top:-1px;left:0}.vis-panel .vis-shadow.vis-bottom{bottom:-1px;left:0}.vis-current-time{background-color:#ff7f6e;width:2px;z-index:1;pointer-events:none}.vis-rolling-mode-btn{height:40px;width:40px;position:absolute;top:7px;right:20px;border-radius:50%;font-size:28px;cursor:pointer;opacity:.8;color:#fff;font-weight:700;text-align:center;background:#3876c2}.vis-rolling-mode-btn:before{content:"\26F6"}.vis-rolling-mode-btn:hover{opacity:1}.vis-custom-time{background-color:#6e94ff;width:2px;cursor:move;z-index:1}.vis-custom-time>.vis-custom-time-marker{background-color:inherit;color:#fff;font-size:12px;white-space:nowrap;padding:3px 5px;top:0;cursor:auto;z-index:inherit}.vis-timeline{position:relative;border:1px solid #bfbfbf;overflow:hidden;padding:0;margin:0;box-sizing:border-box}.vis-loading-screen{width:100%;height:100%;position:absolute;top:0;left:0}.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}.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 hsla(0,0%,60.4%,.55)}.vis-legend-text{white-space:nowrap;display:inline-block}.vis-itemset{position:relative;padding:0;margin:0;box-sizing:border-box}.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-nesting-group{cursor:pointer}.vis-label.vis-nested-group.vis-group-level-unknown-but-gte1{background:#f5f5f5}.vis-label.vis-nested-group.vis-group-level-0{background-color:#fff}.vis-ltr .vis-label.vis-nested-group.vis-group-level-0 .vis-inner{padding-left:0}.vis-rtl .vis-label.vis-nested-group.vis-group-level-0 .vis-inner{padding-right:0}.vis-label.vis-nested-group.vis-group-level-1{background-color:rgba(0,0,0,.05)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-1 .vis-inner{padding-left:15px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-1 .vis-inner{padding-right:15px}.vis-label.vis-nested-group.vis-group-level-2{background-color:rgba(0,0,0,.1)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-2 .vis-inner{padding-left:30px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-2 .vis-inner{padding-right:30px}.vis-label.vis-nested-group.vis-group-level-3{background-color:rgba(0,0,0,.15)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-3 .vis-inner{padding-left:45px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-3 .vis-inner{padding-right:45px}.vis-label.vis-nested-group.vis-group-level-4{background-color:rgba(0,0,0,.2)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-4 .vis-inner{padding-left:60px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-4 .vis-inner{padding-right:60px}.vis-label.vis-nested-group.vis-group-level-5{background-color:rgba(0,0,0,.25)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-5 .vis-inner{padding-left:75px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-5 .vis-inner{padding-right:75px}.vis-label.vis-nested-group.vis-group-level-6{background-color:rgba(0,0,0,.3)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-6 .vis-inner{padding-left:90px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-6 .vis-inner{padding-right:90px}.vis-label.vis-nested-group.vis-group-level-7{background-color:rgba(0,0,0,.35)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-7 .vis-inner{padding-left:105px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-7 .vis-inner{padding-right:105px}.vis-label.vis-nested-group.vis-group-level-8{background-color:rgba(0,0,0,.4)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-8 .vis-inner{padding-left:120px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-8 .vis-inner{padding-right:120px}.vis-label.vis-nested-group.vis-group-level-9{background-color:rgba(0,0,0,.45)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-9 .vis-inner{padding-left:135px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-9 .vis-inner{padding-right:135px}.vis-label.vis-nested-group{background-color:rgba(0,0,0,.5)}.vis-ltr .vis-label.vis-nested-group .vis-inner{padding-left:150px}.vis-rtl .vis-label.vis-nested-group .vis-inner{padding-right:150px}.vis-group-level-unknown-but-gte1{border:1px solid red}.vis-label.vis-nesting-group:before{display:inline-block;width:15px}.vis-label.vis-nesting-group.expanded:before{content:"\25BC"}.vis-label.vis-nesting-group.collapsed:before{content:"\25B6"}.vis-rtl .vis-label.vis-nesting-group.collapsed:before{content:"\25C0"}.vis-ltr .vis-label:not(.vis-nesting-group):not(.vis-group-level-0){padding-left:15px}.vis-rtl .vis-label:not(.vis-nesting-group):not(.vis-group-level-0){padding-right:15px}.vis-overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-labelset{overflow:hidden}.vis-labelset,.vis-labelset .vis-label{position:relative;box-sizing:border-box}.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-group-is-dragging{background:rgba(0,0,0,.1)}.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-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-item{position:absolute;color:#1a1a1a;border-color:#97b0f8;border-width:1px;background-color:#d5ddf6;display:inline-block;z-index:1}.vis-item.vis-selected{border-color:#ffc200;background-color:#fff785;z-index:2}.vis-editable.vis-selected{cursor:move}.vis-item.vis-point.vis-selected{background-color:#fff785}.vis-item.vis-box{text-align:center;border-style:solid;border-radius:2px}.vis-item.vis-point{background:none}.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-visible-frame{white-space:nowrap}.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-onUpdateTime-tooltip{position:absolute;background:#4f81bd;color:#fff;width:200px;text-align:center;white-space:nowrap;padding:5px;border-radius:1px;transition:.4s;-o-transition:.4s;-moz-transition:.4s;-webkit-transition:.4s}.vis-item .vis-delete,.vis-item .vis-delete-rtl{position:absolute;top:0;width:24px;height:24px;box-sizing:border-box;padding:0 5px;cursor:pointer;-webkit-transition:background .2s linear;-moz-transition:background .2s linear;-ms-transition:background .2s linear;-o-transition:background .2s linear;transition:background .2s linear}.vis-item .vis-delete{right:-24px}.vis-item .vis-delete-rtl{left:-24px}.vis-item .vis-delete-rtl:after,.vis-item .vis-delete:after{content:"\00D7";color:red;font-family:arial,sans-serif;font-size:22px;font-weight:700;-webkit-transition:color .2s linear;-moz-transition:color .2s linear;-ms-transition:color .2s linear;-o-transition:color .2s linear;transition:color .2s linear}.vis-item .vis-delete-rtl:hover,.vis-item .vis-delete:hover{background:red}.vis-item .vis-delete-rtl:hover:after,.vis-item .vis-delete:hover:after{color:#fff}.vis-item .vis-drag-center{position:absolute;width:100%;height:100%;top:0;left:0;cursor:move}.vis-item.vis-range .vis-drag-left{left:-4px;cursor:w-resize}.vis-item.vis-range .vis-drag-left,.vis-item.vis-range .vis-drag-right{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0}.vis-item.vis-range .vis-drag-right{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-item.vis-cluster{vertical-align:center;text-align:center;border-style:solid;border-radius:2px}.vis-item.vis-cluster-line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis-item.vis-cluster-dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px} -/*# sourceMappingURL=vis-timeline-graph2d.min.css.map */ \ No newline at end of file +.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}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%);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%);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%)}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%);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:1px solid #fff;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}div.vis-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;z-index:5}.vis-current-time{background-color:#ff7f6e;width:2px;z-index:1;pointer-events:none}.vis-rolling-mode-btn{height:40px;width:40px;position:absolute;top:7px;right:20px;border-radius:50%;font-size:28px;cursor:pointer;opacity:.8;color:#fff;font-weight:700;text-align:center;background:#3876c2}.vis-rolling-mode-btn:before{content:"\26F6"}.vis-rolling-mode-btn:hover{opacity:1}.vis-custom-time{background-color:#6e94ff;width:2px;cursor:move;z-index:1}.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-item{position:absolute;color:#1a1a1a;border-color:#97b0f8;border-width:1px;background-color:#d5ddf6;display:inline-block;z-index:1}.vis-item.vis-selected{border-color:#ffc200;background-color:#fff785;z-index:2}.vis-editable.vis-selected{cursor:move}.vis-item.vis-point.vis-selected{background-color:#fff785}.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-visible-frame{white-space:nowrap}.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-onUpdateTime-tooltip{position:absolute;background:#4f81bd;color:#fff;width:200px;text-align:center;white-space:nowrap;padding:5px;border-radius:1px;transition:.4s;-o-transition:.4s;-moz-transition:.4s;-webkit-transition:.4s}.vis-item .vis-delete,.vis-item .vis-delete-rtl{position:absolute;top:0;width:24px;height:24px;box-sizing:border-box;padding:0 5px;cursor:pointer;-webkit-transition:background .2s linear;-moz-transition:background .2s linear;-ms-transition:background .2s linear;-o-transition:background .2s linear;transition:background .2s linear}.vis-item .vis-delete{right:-24px}.vis-item .vis-delete-rtl{left:-24px}.vis-item .vis-delete-rtl:after,.vis-item .vis-delete:after{content:"\00D7";color:red;font-family:arial,sans-serif;font-size:22px;font-weight:700;-webkit-transition:color .2s linear;-moz-transition:color .2s linear;-ms-transition:color .2s linear;-o-transition:color .2s linear;transition:color .2s linear}.vis-item .vis-delete-rtl:hover,.vis-item .vis-delete:hover{background:red}.vis-item .vis-delete-rtl:hover:after,.vis-item .vis-delete:hover:after{color:#fff}.vis-item .vis-drag-center{position:absolute;width:100%;height:100%;top:0;left:0;cursor:move}.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-itemset{position:relative;padding:0;margin:0;box-sizing:border-box}.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-nesting-group{cursor:pointer}.vis-nested-group{background:#f5f5f5}.vis-label.vis-nesting-group.expanded:before{content:"\25BC"}.vis-label.vis-nesting-group.collapsed-rtl:before{content:"\25C0"}.vis-label.vis-nesting-group.collapsed:before{content:"\25B6"}.vis-overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-labelset{position:relative;overflow:hidden;box-sizing:border-box}.vis-labelset .vis-label{position:relative;left:0;top:0;width:100%;color:#4d4d4d;box-sizing:border-box}.vis-labelset .vis-label{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-panel{position:absolute;padding:0;margin:0;box-sizing:border-box}.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-left.vis-panel.vis-vertical-scroll,.vis-right.vis-panel.vis-vertical-scroll{height:100%;overflow-x:hidden;overflow-y:scroll}.vis-left.vis-panel.vis-vertical-scroll{direction:rtl}.vis-left.vis-panel.vis-vertical-scroll .vis-content{direction:ltr}.vis-right.vis-panel.vis-vertical-scroll{direction:ltr}.vis-right.vis-panel.vis-vertical-scroll .vis-content{direction:rtl}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-top{border-left-style:solid;border-right-style:solid}.vis-background{overflow:hidden}.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-panel .vis-shadow.vis-top{top:-1px;left:0}.vis-panel .vis-shadow.vis-bottom{bottom:-1px;left:0}.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}.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-timeline{position:relative;border:1px solid #bfbfbf;overflow:hidden;padding:0;margin:0;box-sizing:border-box} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css.map b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css.map deleted file mode 100644 index 3c165a792d0f3..0000000000000 --- a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css.map +++ /dev/null @@ -1 +0,0 @@ -{"version":3,"sources":["bootstrap.css","activator.css","configuration.css","tooltip.css","panel.css","currenttime.css","customtime.css","timeline.css","pathStyles.css","dataaxis.css","itemset.css","labelset.css","timeaxis.css","item.css"],"names":[],"mappings":"AAEA,mBACE,YAAa,CACb,UACF,CCLA,cACE,iBAAkB,CAClB,KAAM,CACN,MAAO,CACP,UAAW,CACX,WAAY,CAGZ,UACF,CAEA,YACE,2BACF,CCbA,sBACI,iBAAiB,CACjB,aAAa,CACb,UAAU,CACV,cACJ,CAEA,8BACI,aAAa,CACb,WACJ,CAEA,oCACE,UAAW,CACX,UAAW,CACX,aACF,CAEA,kDACI,aAAa,CACb,WAAW,CACX,qBAAyB,CACzB,wBAAwB,CACxB,iBAAiB,CACjB,eAAe,CACf,SAAS,CACT,gBACJ,CAEA,wCACI,aAAa,CACb,WAAW,CACX,WAAW,CACX,qBAAsB,CACtB,gBAAgB,CAChB,wBAAyB,CACzB,wBAAwB,CACxB,iBAAiB,CACjB,eAAe,CACf,SAAS,CACT,gBAAgB,CAChB,cAAe,CACf,kBACJ,CAEA,8CACI,wBAAyB,CACzB,wBAAwB,CACxB,UACJ,CAEA,sCACI,aAAa,CACb,UAAU,CACV,WAAW,CACX,WAAW,CACX,qBAAsB,CACtB,gBACJ,CAGA,oDACI,SAAS,CACT,wBAAyB,CACzB,gBAAgB,CAChB,iBACJ,CACA,oDACI,SAAS,CACT,wBAAyB,CACzB,gBAAgB,CAChB,iBACJ,CACA,oDACI,SAAS,CACT,wBAAyB,CACzB,gBAAgB,CAChB,iBACJ,CAEA,wCACI,cAAc,CACd,eACJ,CAEA,uCACI,WAAW,CACX,WAAW,CACX,gBACJ,CAEA,qDACI,WACJ,CACA,qDACI,WACJ,CAEA,4CACI,OAAO,CACP,UAAU,CACV,WAAW,CACX,qBAAwB,CACxB,iBAAiB,CACjB,SAAW,CACX,QAAU,CACV,cACJ,CAEA,4CACI,SACJ,CAGA,8CACI,iBAAiB,CACjB,QAAQ,CACR,UAAU,CAEV,WAAW,CACX,QAAQ,CACR,mBACJ,CAEA,yCAEI,uBAAwB,CAGxB,mBAAuB,CACvB,4BAA8B,CAG9B,WAAY,CACZ,WACJ,CACA,wEACI,WAAY,CACZ,UAAW,CACX,kBAAmB,CACnB,0DAA+D,CAC/D,sGAA4G,CAC5G,2DAAiE,CACjE,wDAA4D,CAC5D,yDAA6D,CAC7D,wDAA+D,CAC/D,+GAAmH,CAEnH,qBAAyB,CACzB,yBAAmC,CACnC,iBACJ,CACA,+DACI,uBAAwB,CACxB,wBAAyB,CACzB,WAAY,CACZ,UAAW,CACX,iBAAkB,CAClB,kBAAmB,CACnB,2DAAgE,CAChE,uGAA6G,CAC7G,uDAAkE,CAClE,yDAA6D,CAC7D,0DAA8D,CAC9D,oDAAgE,CAChE,+GAAmH,CACnH,4BAAmC,CACnC,eACJ,CACA,+CACI,YACJ,CACA,8EACI,kBAAmB,CACnB,0DAA8D,CAC9D,sGAA4G,CAC5G,2DAAiE,CACjE,wDAA4D,CAC5D,yDAA6D,CAC7D,wDAA+D,CAC/D,+GACJ,CAEA,2DACI,WAAY,CACZ,WAAY,CACZ,kBAAmB,CACnB,0DAA+D,CAC/D,sGAA4G,CAC5G,2DAAiE,CACjE,wDAA4D,CAC5D,yDAA6D,CAC7D,wDAA+D,CAC/D,+GAAmH,CAEnH,qBAAyB,CACzB,yBAAmC,CACnC,iBACJ,CACA,2DACI,WAAY,CACZ,WAAY,CACZ,UAAW,CAEX,iBAAkB,CAClB,kBACJ,CAGA,wDACI,sBAAwB,CACxB,mBACJ,CAEA,oDACI,WAAY,CACZ,UAAW,CAGX,sBAAuB,CAGvB,wBAAyB,CACzB,kBAAmB,CAGnB,iBACJ,CACA,yDACI,eAAgB,CAChB,kBACJ,CACA,yDACI,eAAgB,CAChB,kBACJ,CACA,oDACI,WAAY,CACZ,WAAY,CACZ,UAAW,CACX,iBAAkB,CAClB,kBACJ,CACA,+DACI,eACJ,CACA,+DACI,eACJ,CAEA,yBACI,iBAAkB,CAClB,6BAAkC,CAClC,wBAAyB,CACzB,gBAAgB,CAChB,WAAW,CACX,WAAW,CACX,iBAAiB,CACjB,UAAc,CACd,cAAc,CACd,iBAAiB,CACjB,0CAA4C,CAC5C,uCAAyC,CACzC,kCACJ,CACA,+DACI,SAAU,CACV,OAAQ,CACR,wBAAyB,CACzB,WAAY,CACZ,QAAS,CACT,OAAQ,CACR,iBAAkB,CAClB,mBACJ,CAEA,+BAEI,2FAAyC,CACzC,gBAAiB,CACjB,eACJ,CACA,gCAEI,gFAA0B,CAC1B,iBAAkB,CAClB,gBACJ,CC/RA,gBACE,iBAAkB,CAClB,iBAAkB,CAClB,WAAY,CACZ,kBAAmB,CAEnB,mBAAoB,CACpB,cAAc,CACd,UAAa,CACb,wBAAyB,CAEzB,sBAAuB,CACvB,yBAA0B,CAC1B,iBAAkB,CAClB,wBAAyB,CAEzB,sCAA2C,CAC3C,mBAAoB,CAEpB,SACF,CCpBA,WACE,iBAAkB,CAElB,SAAU,CACV,QAAS,CAET,qBACF,CAEA,wGAKE,kBACF,CAEA,+DAGE,sBAAuB,CACvB,yBAA0B,CAC1B,eACF,CAEA,iFACE,WAAY,CACZ,iBAAkB,CAClB,iBACF,CAEA,wCACE,aACF,CAMA,8FACE,aACF,CAEA,sDACE,aACF,CAEA,+DAGE,uBAAwB,CACxB,wBACF,CAEA,gBACE,eACF,CAEA,wBACE,iBACF,CAEA,uBACE,iBAAkB,CAClB,UAAW,CACX,UAAW,CACX,kCAIF,CAEA,+BACE,QAAS,CACT,MACF,CAEA,kCACE,WAAY,CACZ,MACF,CChFA,kBACE,wBAAyB,CACzB,SAAU,CACV,SAAU,CACV,mBACF,CAEA,sBACE,WAAY,CACZ,UAAW,CACX,iBAAkB,CAClB,OAAQ,CACR,UAAW,CACX,iBAAkB,CAClB,cAAe,CACf,cAAe,CACf,UAAY,CACZ,UAAY,CACZ,eAAiB,CACjB,iBAAkB,CAClB,kBACF,CACA,6BACE,eACF,CAEA,4BACE,SACF,CC5BA,iBACE,wBAAyB,CACzB,SAAU,CACV,WAAY,CACZ,SACF,CAEA,yCACE,wBAAyB,CACzB,UAAY,CACZ,cAAe,CACf,kBAAmB,CACnB,eAAgB,CAChB,KAAQ,CACR,WAAe,CACf,eACF,CCfA,cACE,iBAAkB,CAClB,wBAAyB,CACzB,eAAgB,CAChB,SAAU,CACV,QAAS,CACT,qBACF,CAEA,oBACE,UAAW,CACX,WAAY,CACZ,iBAAkB,CAClB,KAAM,CACN,MACF,CChBA,kBACI,YAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,SAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,WACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,wBACI,eAAgB,CAChB,WACJ,CAGA,uBACI,eAAgB,CAChB,gBACJ,CAEA,yBACI,gBAAgB,CAChB,cACJ,CAGA,qCACI,gBAAgB,CAChB,eAAgB,CAChB,SAAa,CACb,cACJ,CAGA,2BACI,gBAAgB,CAChB,cAAc,CACd,SAAa,CACb,cACJ,CAEA,6BACI,eAAgB,CAChB,WACJ,CCxGA,kEACE,iBAAkB,CAClB,UAAW,CACX,QAAS,CACT,uBACF,CAEA,6DACE,oBACF,CAEA,6DACE,oBACF,CAGA,qCACE,UAAW,CACX,iBAAkB,CAClB,aAAc,CACd,kBACF,CAEA,iDACE,SAAU,CACV,QAAS,CACT,QAAS,CACT,iBAAkB,CAClB,UACF,CAGA,qCACE,iBAAkB,CAClB,UAAW,CACX,aAAc,CACd,kBACF,CAEA,iDACE,SAAU,CACV,QAAS,CACT,QAAS,CACT,iBAAkB,CAClB,UACF,CAEA,qCACE,iBAAkB,CAClB,aAAc,CACd,kBAAmB,CACnB,WAAY,CACZ,iBACF,CAEA,iDACE,SAAU,CACV,QAAS,CACT,iBAAkB,CAClB,UACF,CAEA,8CACE,QAAS,CACT,iCAAkC,CAClC,8BAA+B,CAC/B,6BAA8B,CAC9B,4BAA6B,CAC7B,4BAA6B,CAC7B,gCAAiC,CACjC,6BAA8B,CAC9B,4BAA6B,CAC7B,2BAA4B,CAC5B,wBACF,CAEA,+CACE,QAAS,CACT,qCAAsC,CACtC,kCAAmC,CACnC,iCAAkC,CAClC,gCAAiC,CACjC,6BAA8B,CAC9B,+BAAgC,CAChC,4BAA6B,CAC7B,2BAA4B,CAC5B,0BAA2B,CAC3B,uBACF,CAEA,YACE,sCAA2C,CAC3C,WAAY,CACZ,wBAAyB,CACzB,4CACF,CAEA,iBAEE,kBAAmB,CACnB,oBACF,CCrGA,aACE,iBAAkB,CAClB,SAAU,CACV,QAAS,CAET,qBACF,CAEA,0DAEE,iBAAkB,CAClB,UAAW,CACX,WAAY,CACZ,gBACF,CAEA,UACE,iBAAkB,CAClB,UAAW,CACX,QAAS,CACT,MAAO,CACP,SACF,CAEA,2BACE,iBAAkB,CAClB,qBAAsB,CACtB,+BACF,CAEA,sCACE,kBACF,CAEA,mBACE,cACF,CAEA,6DACE,kBACF,CACA,8CACE,qBACF,CACA,kEACE,cACF,CACA,kEACE,eACF,CACA,8CACE,gCACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,+BACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,gCACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,+BACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,gCACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,+BACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,gCACF,CACA,kEACE,kBACF,CACA,kEACE,mBACF,CACA,8CACE,+BACF,CACA,kEACE,kBACF,CACA,kEACE,mBACF,CACA,8CACE,gCACF,CACA,kEACE,kBACF,CACA,kEACE,mBACF,CAGA,4BACE,+BACF,CACA,gDACE,kBACF,CACA,gDACE,mBACF,CAEA,kCACE,oBACF,CAGA,oCAEE,oBAAqB,CACrB,UACF,CACA,6CACE,eACF,CACA,8CACE,eACF,CACA,uDACE,eACF,CAEA,oEACE,iBACF,CACA,oEACE,kBACF,CAEA,aACE,iBAAkB,CAClB,KAAM,CACN,MAAO,CACP,UAAW,CACX,WAAY,CACZ,UACF,CCjLA,cAGE,eAGF,CAEA,uCAPE,iBAAkB,CAIlB,qBAWF,CARA,yBAEE,MAAO,CACP,KAAM,CACN,UAAW,CACX,aAAc,CAMd,+BAHF,CAMA,mCACE,cACF,CAEA,uBACE,yBACF,CAEA,oCACE,kBACF,CAEA,oCACE,oBAAqB,CACrB,WACF,CAEA,+CACE,SACF,CC1CA,eACE,iBAAkB,CAClB,eACF,CAEA,8BACE,KAAM,CACN,MAAO,CACP,UACF,CAEA,8BACE,iBAAkB,CAClB,KAAM,CACN,MAAO,CACP,UAAW,CACX,WACF,CAEA,yBACE,iBAAkB,CAClB,aAAc,CACd,WAAY,CACZ,eAAgB,CAChB,qBAAsB,CAEtB,kBACF,CAEA,qCACE,iBAAkB,CAClB,cAAe,CACf,eAAgB,CAChB,aAAc,CACd,cAAe,CACf,iBACF,CAEA,sCACE,iBAAkB,CAClB,qBACF,CAEA,0CACE,iBAAkB,CAClB,sBACF,CAEA,mCACE,oBACF,CAEA,mCACE,oBACF,CCrDA,UACE,iBAAkB,CAClB,aAAc,CACd,oBAAqB,CACrB,gBAAiB,CACjB,wBAAyB,CACzB,oBAAqB,CACrB,SAEF,CAEA,uBACE,oBAAqB,CACrB,wBAAyB,CAGzB,SACF,CAEA,2BACE,WACF,CAEA,iCACE,wBACF,CAEA,kBACE,iBAAkB,CAClB,kBAAmB,CACnB,iBACF,CAEA,oBACE,eACF,CAEA,kBACE,iBAAkB,CAClB,SAAU,CACV,gBAAiB,CACjB,kBAAmB,CACnB,iBACF,CAEA,oBACE,kBAAmB,CACnB,iBAAkB,CAClB,qBACF,CAEA,yBACE,WAAY,CACZ,qCAA0C,CAC1C,qBAAsB,CACtB,SAAU,CACV,QACF,CAEA,6BACE,iBAAkB,CAClB,UAAW,CACX,WAAY,CACZ,SAAU,CACV,QAAS,CACT,eACF,CAEA,wBACE,kBACF,CAEA,sCACE,iBAAkB,CAClB,oBACF,CAEA,2CACE,iBAAkB,CAClB,oBACF,CAEA,mBACE,SAAU,CACV,iBAAkB,CAClB,OAAQ,CACR,qBAAsB,CACtB,uBACF,CAEA,4BACE,kBAAmB,CACnB,qBAAsB,CACtB,WACF,CAEA,oCACE,iBAAkB,CAClB,kBAAmB,CACnB,UAAY,CACZ,WAAY,CACZ,iBAAkB,CAClB,kBAAmB,CACnB,WAAY,CACZ,iBAAkB,CAClB,cAAgB,CAChB,iBAAmB,CACnB,mBAAqB,CACrB,sBACF,CAEA,gDACE,iBAAkB,CAClB,KAAQ,CACR,UAAW,CACX,WAAY,CACZ,qBAAsB,CACtB,aAAgB,CAChB,cAAe,CAEf,wCAA0C,CAC1C,qCAAuC,CACvC,oCAAsC,CACtC,mCAAqC,CACrC,gCACF,CAEA,sBACE,WACF,CAEA,0BACE,UACF,CAEA,4DACE,eAAgB,CAChB,SAAU,CACV,4BAA8B,CAC9B,cAAe,CACf,eAAiB,CAEjB,mCAAqC,CACrC,gCAAkC,CAClC,+BAAiC,CACjC,8BAAgC,CAChC,2BACF,CAEA,4DACE,cACF,CAEA,wEACE,UACF,CAEA,2BACE,iBAAkB,CAClB,UAAW,CACX,WAAY,CACZ,KAAM,CACN,MAAS,CACT,WACF,CAEA,mCAOE,SAAU,CAEV,eACF,CAEA,uEAXE,iBAAkB,CAClB,UAAW,CACX,aAAc,CACd,aAAc,CACd,WAAY,CACZ,KAgBF,CAVA,oCAOE,UAAW,CAEX,eACF,CAEA,iGAEE,WACF,CAEA,sBACE,qBAAsB,CACtB,iBAAkB,CAClB,kBAAmB,CACnB,iBACF,CAEA,2BACE,SAAU,CACV,iBAAkB,CAClB,OAAQ,CACR,qBAAsB,CACtB,uBACF,CAEA,0BACE,iBAAkB,CAClB,SAAU,CACV,gBAAiB,CACjB,kBAAmB,CACnB,iBACF","file":"vis-timeline-graph2d.min.css","sourcesContent":["/* override some bootstrap styles screwing up the timelines css */\n\n.vis [class*=\"span\"] {\n min-height: 0;\n width: auto;\n}\n",".vis .overlay {\n position: absolute;\n top: 0;\n left: 0;\n width: 100%;\n height: 100%;\n\n /* Must be displayed above for example selected Timeline items */\n z-index: 10;\n}\n\n.vis-active {\n box-shadow: 0 0 10px #86d5f8;\n}\n","div.vis-configuration {\n position:relative;\n display:block;\n float:left;\n font-size:12px;\n}\n\ndiv.vis-configuration-wrapper {\n display:block;\n width:700px;\n}\n\ndiv.vis-configuration-wrapper::after {\n clear: both;\n content: \"\";\n display: block;\n}\n\ndiv.vis-configuration.vis-config-option-container{\n display:block;\n width:495px;\n background-color: #ffffff;\n border:2px solid #f7f8fa;\n border-radius:4px;\n margin-top:20px;\n left:10px;\n padding-left:5px;\n}\n\ndiv.vis-configuration.vis-config-button{\n display:block;\n width:495px;\n height:25px;\n vertical-align: middle;\n line-height:25px;\n background-color: #f7f8fa;\n border:2px solid #ceced0;\n border-radius:4px;\n margin-top:20px;\n left:10px;\n padding-left:5px;\n cursor: pointer;\n margin-bottom:30px;\n}\n\ndiv.vis-configuration.vis-config-button.hover{\n background-color: #4588e6;\n border:2px solid #214373;\n color:#ffffff;\n}\n\ndiv.vis-configuration.vis-config-item{\n display:block;\n float:left;\n width:495px;\n height:25px;\n vertical-align: middle;\n line-height:25px;\n}\n\n\ndiv.vis-configuration.vis-config-item.vis-config-s2{\n left:10px;\n background-color: #f7f8fa;\n padding-left:5px;\n border-radius:3px;\n}\ndiv.vis-configuration.vis-config-item.vis-config-s3{\n left:20px;\n background-color: #e4e9f0;\n padding-left:5px;\n border-radius:3px;\n}\ndiv.vis-configuration.vis-config-item.vis-config-s4{\n left:30px;\n background-color: #cfd8e6;\n padding-left:5px;\n border-radius:3px;\n}\n\ndiv.vis-configuration.vis-config-header{\n font-size:18px;\n font-weight: bold;\n}\n\ndiv.vis-configuration.vis-config-label{\n width:120px;\n height:25px;\n line-height: 25px;\n}\n\ndiv.vis-configuration.vis-config-label.vis-config-s3{\n width:110px;\n}\ndiv.vis-configuration.vis-config-label.vis-config-s4{\n width:100px;\n}\n\ndiv.vis-configuration.vis-config-colorBlock{\n top:1px;\n width:30px;\n height:19px;\n border:1px solid #444444;\n border-radius:2px;\n padding:0px;\n margin:0px;\n cursor:pointer;\n}\n\ninput.vis-configuration.vis-config-checkbox {\n left:-5px;\n}\n\n\ninput.vis-configuration.vis-config-rangeinput{\n position:relative;\n top:-5px;\n width:60px;\n /*height:13px;*/\n padding:1px;\n margin:0;\n pointer-events:none;\n}\n\ninput.vis-configuration.vis-config-range{\n /*removes default webkit styles*/\n -webkit-appearance: none;\n\n /*fix for FF unable to apply focus style bug */\n border: 0px solid white;\n background-color:rgba(0,0,0,0);\n\n /*required for proper track sizing in FF*/\n width: 300px;\n height:20px;\n}\ninput.vis-configuration.vis-config-range::-webkit-slider-runnable-track {\n width: 300px;\n height: 5px;\n background: #dedede; /* Old browsers */\n background: -moz-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#dedede), color-stop(99%,#c8c8c8)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* IE10+ */\n background: linear-gradient(to bottom, #dedede 0%,#c8c8c8 99%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8',GradientType=0 ); /* IE6-9 */\n\n border: 1px solid #999999;\n box-shadow: #aaaaaa 0px 0px 3px 0px;\n border-radius: 3px;\n}\ninput.vis-configuration.vis-config-range::-webkit-slider-thumb {\n -webkit-appearance: none;\n border: 1px solid #14334b;\n height: 17px;\n width: 17px;\n border-radius: 50%;\n background: #3876c2; /* Old browsers */\n background: -moz-linear-gradient(top, #3876c2 0%, #385380 100%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#3876c2), color-stop(100%,#385380)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #3876c2 0%,#385380 100%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #3876c2 0%,#385380 100%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #3876c2 0%,#385380 100%); /* IE10+ */\n background: linear-gradient(to bottom, #3876c2 0%,#385380 100%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#3876c2', endColorstr='#385380',GradientType=0 ); /* IE6-9 */\n box-shadow: #111927 0px 0px 1px 0px;\n margin-top: -7px;\n}\ninput.vis-configuration.vis-config-range:focus {\n outline: none;\n}\ninput.vis-configuration.vis-config-range:focus::-webkit-slider-runnable-track {\n background: #9d9d9d; /* Old browsers */\n background: -moz-linear-gradient(top, #9d9d9d 0%, #c8c8c8 99%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#9d9d9d), color-stop(99%,#c8c8c8)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #9d9d9d 0%,#c8c8c8 99%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #9d9d9d 0%,#c8c8c8 99%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #9d9d9d 0%,#c8c8c8 99%); /* IE10+ */\n background: linear-gradient(to bottom, #9d9d9d 0%,#c8c8c8 99%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#9d9d9d', endColorstr='#c8c8c8',GradientType=0 ); /* IE6-9 */\n}\n\ninput.vis-configuration.vis-config-range::-moz-range-track {\n width: 300px;\n height: 10px;\n background: #dedede; /* Old browsers */\n background: -moz-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#dedede), color-stop(99%,#c8c8c8)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* IE10+ */\n background: linear-gradient(to bottom, #dedede 0%,#c8c8c8 99%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8',GradientType=0 ); /* IE6-9 */\n\n border: 1px solid #999999;\n box-shadow: #aaaaaa 0px 0px 3px 0px;\n border-radius: 3px;\n}\ninput.vis-configuration.vis-config-range::-moz-range-thumb {\n border: none;\n height: 16px;\n width: 16px;\n\n border-radius: 50%;\n background: #385380;\n}\n\n/*hide the outline behind the border*/\ninput.vis-configuration.vis-config-range:-moz-focusring{\n outline: 1px solid white;\n outline-offset: -1px;\n}\n\ninput.vis-configuration.vis-config-range::-ms-track {\n width: 300px;\n height: 5px;\n\n /*remove bg colour from the track, we'll use ms-fill-lower and ms-fill-upper instead */\n background: transparent;\n\n /*leave room for the larger thumb to overflow with a transparent border */\n border-color: transparent;\n border-width: 6px 0;\n\n /*remove default tick marks*/\n color: transparent;\n}\ninput.vis-configuration.vis-config-range::-ms-fill-lower {\n background: #777;\n border-radius: 10px;\n}\ninput.vis-configuration.vis-config-range::-ms-fill-upper {\n background: #ddd;\n border-radius: 10px;\n}\ninput.vis-configuration.vis-config-range::-ms-thumb {\n border: none;\n height: 16px;\n width: 16px;\n border-radius: 50%;\n background: #385380;\n}\ninput.vis-configuration.vis-config-range:focus::-ms-fill-lower {\n background: #888;\n}\ninput.vis-configuration.vis-config-range:focus::-ms-fill-upper {\n background: #ccc;\n}\n\n.vis-configuration-popup {\n position: absolute;\n background: rgba(57, 76, 89, 0.85);\n border: 2px solid #f2faff;\n line-height:30px;\n height:30px;\n width:150px;\n text-align:center;\n color: #ffffff;\n font-size:14px;\n border-radius:4px;\n -webkit-transition: opacity 0.3s ease-in-out;\n -moz-transition: opacity 0.3s ease-in-out;\n transition: opacity 0.3s ease-in-out;\n}\n.vis-configuration-popup:after, .vis-configuration-popup:before {\n left: 100%;\n top: 50%;\n border: solid transparent;\n content: \" \";\n height: 0;\n width: 0;\n position: absolute;\n pointer-events: none;\n}\n\n.vis-configuration-popup:after {\n border-color: rgba(136, 183, 213, 0);\n border-left-color: rgba(57, 76, 89, 0.85);\n border-width: 8px;\n margin-top: -8px;\n}\n.vis-configuration-popup:before {\n border-color: rgba(194, 225, 245, 0);\n border-left-color: #f2faff;\n border-width: 12px;\n margin-top: -12px;\n}","div.vis-tooltip {\n position: absolute;\n visibility: hidden;\n padding: 5px;\n white-space: nowrap;\n\n font-family: verdana;\n font-size:14px;\n color:#000000;\n background-color: #f5f4ed;\n\n -moz-border-radius: 3px;\n -webkit-border-radius: 3px;\n border-radius: 3px;\n border: 1px solid #808074;\n\n box-shadow: 3px 3px 10px rgba(0, 0, 0, 0.2);\n pointer-events: none;\n\n z-index: 5;\n}\n",".vis-panel {\n position: absolute;\n\n padding: 0;\n margin: 0;\n\n box-sizing: border-box;\n}\n\n.vis-panel.vis-center,\n.vis-panel.vis-left,\n.vis-panel.vis-right,\n.vis-panel.vis-top,\n.vis-panel.vis-bottom {\n border: 1px #bfbfbf;\n}\n\n.vis-panel.vis-center,\n.vis-panel.vis-left,\n.vis-panel.vis-right {\n border-top-style: solid;\n border-bottom-style: solid;\n overflow: hidden;\n}\n\n.vis-left.vis-panel.vis-vertical-scroll, .vis-right.vis-panel.vis-vertical-scroll {\n height: 100%;\n overflow-x: hidden;\n overflow-y: scroll;\n} \n\n.vis-left.vis-panel.vis-vertical-scroll {\n direction: rtl;\n}\n\n.vis-left.vis-panel.vis-vertical-scroll .vis-content {\n direction: ltr;\n}\n\n.vis-right.vis-panel.vis-vertical-scroll {\n direction: ltr;\n}\n\n.vis-right.vis-panel.vis-vertical-scroll .vis-content {\n direction: rtl;\n}\n\n.vis-panel.vis-center,\n.vis-panel.vis-top,\n.vis-panel.vis-bottom {\n border-left-style: solid;\n border-right-style: solid;\n}\n\n.vis-background {\n overflow: hidden;\n}\n\n.vis-panel > .vis-content {\n position: relative;\n}\n\n.vis-panel .vis-shadow {\n position: absolute;\n width: 100%;\n height: 1px;\n box-shadow: 0 0 10px rgba(0,0,0,0.8);\n /* TODO: find a nice way to ensure vis-shadows are drawn on top of items\n z-index: 1;\n */\n}\n\n.vis-panel .vis-shadow.vis-top {\n top: -1px;\n left: 0;\n}\n\n.vis-panel .vis-shadow.vis-bottom {\n bottom: -1px;\n left: 0;\n}",".vis-current-time {\n background-color: #FF7F6E;\n width: 2px;\n z-index: 1;\n pointer-events: none;\n}\n\n.vis-rolling-mode-btn {\n height: 40px;\n width: 40px;\n position: absolute;\n top: 7px;\n right: 20px;\n border-radius: 50%;\n font-size: 28px;\n cursor: pointer;\n opacity: 0.8;\n color: white;\n font-weight: bold;\n text-align: center;\n background: #3876c2;\n}\n.vis-rolling-mode-btn:before {\n content: \"\\26F6\";\n}\n\n.vis-rolling-mode-btn:hover {\n opacity: 1;\n}",".vis-custom-time {\n background-color: #6E94FF;\n width: 2px;\n cursor: move;\n z-index: 1;\n}\n\n.vis-custom-time > .vis-custom-time-marker {\n background-color: inherit;\n color: white;\n font-size: 12px;\n white-space: nowrap;\n padding: 3px 5px;\n top: 0px;\n cursor: initial;\n z-index: inherit;\n}","\n.vis-timeline {\n position: relative;\n border: 1px solid #bfbfbf;\n overflow: hidden;\n padding: 0;\n margin: 0;\n box-sizing: border-box;\n}\n\n.vis-loading-screen {\n width: 100%;\n height: 100%;\n position: absolute;\n top: 0;\n left: 0;\n}",".vis-graph-group0 {\n fill:#4f81bd;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #4f81bd;\n}\n\n.vis-graph-group1 {\n fill:#f79646;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #f79646;\n}\n\n.vis-graph-group2 {\n fill: #8c51cf;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #8c51cf;\n}\n\n.vis-graph-group3 {\n fill: #75c841;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #75c841;\n}\n\n.vis-graph-group4 {\n fill: #ff0100;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #ff0100;\n}\n\n.vis-graph-group5 {\n fill: #37d8e6;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #37d8e6;\n}\n\n.vis-graph-group6 {\n fill: #042662;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #042662;\n}\n\n.vis-graph-group7 {\n fill:#00ff26;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #00ff26;\n}\n\n.vis-graph-group8 {\n fill:#ff00ff;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #ff00ff;\n}\n\n.vis-graph-group9 {\n fill: #8f3938;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #8f3938;\n}\n\n.vis-timeline .vis-fill {\n fill-opacity:0.1;\n stroke: none;\n}\n\n\n.vis-timeline .vis-bar {\n fill-opacity:0.5;\n stroke-width:1px;\n}\n\n.vis-timeline .vis-point {\n stroke-width:2px;\n fill-opacity:1.0;\n}\n\n\n.vis-timeline .vis-legend-background {\n stroke-width:1px;\n fill-opacity:0.9;\n fill: #ffffff;\n stroke: #c2c2c2;\n}\n\n\n.vis-timeline .vis-outline {\n stroke-width:1px;\n fill-opacity:1;\n fill: #ffffff;\n stroke: #e5e5e5;\n}\n\n.vis-timeline .vis-icon-fill {\n fill-opacity:0.3;\n stroke: none;\n}\n","\n.vis-panel.vis-background.vis-horizontal .vis-grid.vis-horizontal {\n position: absolute;\n width: 100%;\n height: 0;\n border-bottom: 1px solid;\n}\n\n.vis-panel.vis-background.vis-horizontal .vis-grid.vis-minor {\n border-color: #e5e5e5;\n}\n\n.vis-panel.vis-background.vis-horizontal .vis-grid.vis-major {\n border-color: #bfbfbf;\n}\n\n\n.vis-data-axis .vis-y-axis.vis-major {\n width: 100%;\n position: absolute;\n color: #4d4d4d;\n white-space: nowrap;\n}\n\n.vis-data-axis .vis-y-axis.vis-major.vis-measure {\n padding: 0;\n margin: 0;\n border: 0;\n visibility: hidden;\n width: auto;\n}\n\n\n.vis-data-axis .vis-y-axis.vis-minor {\n position: absolute;\n width: 100%;\n color: #bebebe;\n white-space: nowrap;\n}\n\n.vis-data-axis .vis-y-axis.vis-minor.vis-measure {\n padding: 0;\n margin: 0;\n border: 0;\n visibility: hidden;\n width: auto;\n}\n\n.vis-data-axis .vis-y-axis.vis-title {\n position: absolute;\n color: #4d4d4d;\n white-space: nowrap;\n bottom: 20px;\n text-align: center;\n}\n\n.vis-data-axis .vis-y-axis.vis-title.vis-measure {\n padding: 0;\n margin: 0;\n visibility: hidden;\n width: auto;\n}\n\n.vis-data-axis .vis-y-axis.vis-title.vis-left {\n bottom: 0;\n -webkit-transform-origin: left top;\n -moz-transform-origin: left top;\n -ms-transform-origin: left top;\n -o-transform-origin: left top;\n transform-origin: left bottom;\n -webkit-transform: rotate(-90deg);\n -moz-transform: rotate(-90deg);\n -ms-transform: rotate(-90deg);\n -o-transform: rotate(-90deg);\n transform: rotate(-90deg);\n}\n\n.vis-data-axis .vis-y-axis.vis-title.vis-right {\n bottom: 0;\n -webkit-transform-origin: right bottom;\n -moz-transform-origin: right bottom;\n -ms-transform-origin: right bottom;\n -o-transform-origin: right bottom;\n transform-origin: right bottom;\n -webkit-transform: rotate(90deg);\n -moz-transform: rotate(90deg);\n -ms-transform: rotate(90deg);\n -o-transform: rotate(90deg);\n transform: rotate(90deg);\n}\n\n.vis-legend {\n background-color: rgba(247, 252, 255, 0.65);\n padding: 5px;\n border: 1px solid #b3b3b3;\n box-shadow: 2px 2px 10px rgba(154, 154, 154, 0.55);\n}\n\n.vis-legend-text {\n /*font-size: 10px;*/\n white-space: nowrap;\n display: inline-block\n}","\n.vis-itemset {\n position: relative;\n padding: 0;\n margin: 0;\n\n box-sizing: border-box;\n}\n\n.vis-itemset .vis-background,\n.vis-itemset .vis-foreground {\n position: absolute;\n width: 100%;\n height: 100%;\n overflow: visible;\n}\n\n.vis-axis {\n position: absolute;\n width: 100%;\n height: 0;\n left: 0;\n z-index: 1;\n}\n\n.vis-foreground .vis-group {\n position: relative;\n box-sizing: border-box;\n border-bottom: 1px solid #bfbfbf;\n}\n\n.vis-foreground .vis-group:last-child {\n border-bottom: none;\n}\n\n.vis-nesting-group {\n cursor: pointer;\n}\n\n.vis-label.vis-nested-group.vis-group-level-unknown-but-gte1 {\n background: #f5f5f5;\n}\n.vis-label.vis-nested-group.vis-group-level-0 {\n background-color: #ffffff;\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-0 .vis-inner {\n padding-left: 0;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-0 .vis-inner {\n padding-right: 0;\n}\n.vis-label.vis-nested-group.vis-group-level-1 {\n background-color: rgba(0, 0, 0, 0.05);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-1 .vis-inner {\n padding-left: 15px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-1 .vis-inner {\n padding-right: 15px;\n}\n.vis-label.vis-nested-group.vis-group-level-2 {\n background-color: rgba(0, 0, 0, 0.1);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-2 .vis-inner {\n padding-left: 30px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-2 .vis-inner {\n padding-right: 30px;\n}\n.vis-label.vis-nested-group.vis-group-level-3 {\n background-color: rgba(0, 0, 0, 0.15);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-3 .vis-inner {\n padding-left: 45px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-3 .vis-inner {\n padding-right: 45px;\n}\n.vis-label.vis-nested-group.vis-group-level-4 {\n background-color: rgba(0, 0, 0, 0.2);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-4 .vis-inner {\n padding-left: 60px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-4 .vis-inner {\n padding-right: 60px;\n}\n.vis-label.vis-nested-group.vis-group-level-5 {\n background-color: rgba(0, 0, 0, 0.25);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-5 .vis-inner {\n padding-left: 75px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-5 .vis-inner {\n padding-right: 75px;\n}\n.vis-label.vis-nested-group.vis-group-level-6 {\n background-color: rgba(0, 0, 0, 0.3);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-6 .vis-inner {\n padding-left: 90px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-6 .vis-inner {\n padding-right: 90px;\n}\n.vis-label.vis-nested-group.vis-group-level-7 {\n background-color: rgba(0, 0, 0, 0.35);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-7 .vis-inner {\n padding-left: 105px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-7 .vis-inner {\n padding-right: 105px;\n}\n.vis-label.vis-nested-group.vis-group-level-8 {\n background-color: rgba(0, 0, 0, 0.4);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-8 .vis-inner {\n padding-left: 120px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-8 .vis-inner {\n padding-right: 120px;\n}\n.vis-label.vis-nested-group.vis-group-level-9 {\n background-color: rgba(0, 0, 0, 0.45);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-9 .vis-inner {\n padding-left: 135px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-9 .vis-inner {\n padding-right: 135px;\n}\n/* default takes over beginning with level-10 (thats why we add .vis-nested-group\n to the selectors above, to have higher specifity than these rules for the defaults) */\n.vis-label.vis-nested-group {\n background-color: rgba(0, 0, 0, 0.5);\n}\n.vis-ltr .vis-label.vis-nested-group .vis-inner {\n padding-left: 150px;\n}\n.vis-rtl .vis-label.vis-nested-group .vis-inner {\n padding-right: 150px;\n}\n\n.vis-group-level-unknown-but-gte1 {\n border: 1px solid red;\n}\n\n/* expanded/collapsed indicators */\n.vis-label.vis-nesting-group:before,\n.vis-label.vis-nesting-group:before {\n display: inline-block;\n width: 15px;\n}\n.vis-label.vis-nesting-group.expanded:before {\n content: \"\\25BC\";\n}\n.vis-label.vis-nesting-group.collapsed:before {\n content: \"\\25B6\";\n}\n.vis-rtl .vis-label.vis-nesting-group.collapsed:before {\n content: \"\\25C0\";\n}\n/* compensate missing expanded/collapsed indicator, but only at levels > 0 */\n.vis-ltr .vis-label:not(.vis-nesting-group):not(.vis-group-level-0) {\n padding-left: 15px;\n}\n.vis-rtl .vis-label:not(.vis-nesting-group):not(.vis-group-level-0) {\n padding-right: 15px;\n}\n\n.vis-overlay {\n position: absolute;\n top: 0;\n left: 0;\n width: 100%;\n height: 100%;\n z-index: 10;\n}","\n.vis-labelset {\n position: relative;\n\n overflow: hidden;\n\n box-sizing: border-box;\n}\n\n.vis-labelset .vis-label {\n position: relative;\n left: 0;\n top: 0;\n width: 100%;\n color: #4d4d4d;\n\n box-sizing: border-box;\n}\n\n.vis-labelset .vis-label {\n border-bottom: 1px solid #bfbfbf;\n}\n\n.vis-labelset .vis-label.draggable {\n cursor: pointer;\n}\n\n.vis-group-is-dragging {\n background: rgba(0, 0, 0, .1);\n}\n\n.vis-labelset .vis-label:last-child {\n border-bottom: none;\n}\n\n.vis-labelset .vis-label .vis-inner {\n display: inline-block;\n padding: 5px;\n}\n\n.vis-labelset .vis-label .vis-inner.vis-hidden {\n padding: 0;\n}\n",".vis-time-axis {\n position: relative;\n overflow: hidden;\n}\n\n.vis-time-axis.vis-foreground {\n top: 0;\n left: 0;\n width: 100%;\n}\n\n.vis-time-axis.vis-background {\n position: absolute;\n top: 0;\n left: 0;\n width: 100%;\n height: 100%;\n}\n\n.vis-time-axis .vis-text {\n position: absolute;\n color: #4d4d4d;\n padding: 3px;\n overflow: hidden;\n box-sizing: border-box;\n\n white-space: nowrap;\n}\n\n.vis-time-axis .vis-text.vis-measure {\n position: absolute;\n padding-left: 0;\n padding-right: 0;\n margin-left: 0;\n margin-right: 0;\n visibility: hidden;\n}\n\n.vis-time-axis .vis-grid.vis-vertical {\n position: absolute;\n border-left: 1px solid;\n}\n\n.vis-time-axis .vis-grid.vis-vertical-rtl {\n position: absolute;\n border-right: 1px solid;\n}\n\n.vis-time-axis .vis-grid.vis-minor {\n border-color: #e5e5e5;\n}\n\n.vis-time-axis .vis-grid.vis-major {\n border-color: #bfbfbf;\n}\n","\n.vis-item {\n position: absolute;\n color: #1A1A1A;\n border-color: #97B0F8;\n border-width: 1px;\n background-color: #D5DDF6;\n display: inline-block;\n z-index: 1;\n /*overflow: hidden;*/\n}\n\n.vis-item.vis-selected {\n border-color: #FFC200;\n background-color: #FFF785;\n\n /* z-index must be higher than the z-index of custom time bar and current time bar */\n z-index: 2;\n}\n\n.vis-editable.vis-selected {\n cursor: move;\n}\n\n.vis-item.vis-point.vis-selected {\n background-color: #FFF785;\n}\n\n.vis-item.vis-box {\n text-align: center;\n border-style: solid;\n border-radius: 2px;\n}\n\n.vis-item.vis-point {\n background: none;\n}\n\n.vis-item.vis-dot {\n position: absolute;\n padding: 0;\n border-width: 4px;\n border-style: solid;\n border-radius: 4px;\n}\n\n.vis-item.vis-range {\n border-style: solid;\n border-radius: 2px;\n box-sizing: border-box;\n}\n\n.vis-item.vis-background {\n border: none;\n background-color: rgba(213, 221, 246, 0.4);\n box-sizing: border-box;\n padding: 0;\n margin: 0;\n}\n\n.vis-item .vis-item-overflow {\n position: relative;\n width: 100%;\n height: 100%;\n padding: 0;\n margin: 0;\n overflow: hidden;\n}\n\n.vis-item-visible-frame {\n white-space: nowrap;\n}\n\n.vis-item.vis-range .vis-item-content {\n position: relative;\n display: inline-block;\n}\n\n.vis-item.vis-background .vis-item-content {\n position: absolute;\n display: inline-block;\n}\n\n.vis-item.vis-line {\n padding: 0;\n position: absolute;\n width: 0;\n border-left-width: 1px;\n border-left-style: solid;\n}\n\n.vis-item .vis-item-content {\n white-space: nowrap;\n box-sizing: border-box;\n padding: 5px;\n}\n\n.vis-item .vis-onUpdateTime-tooltip {\n position: absolute;\n background: #4f81bd;\n color: white;\n width: 200px;\n text-align: center;\n white-space: nowrap;\n padding: 5px;\n border-radius: 1px;\n transition: 0.4s;\n -o-transition: 0.4s;\n -moz-transition: 0.4s;\n -webkit-transition: 0.4s;\n}\n\n.vis-item .vis-delete, .vis-item .vis-delete-rtl {\n position: absolute;\n top: 0px;\n width: 24px;\n height: 24px;\n box-sizing: border-box;\n padding: 0px 5px;\n cursor: pointer;\n\n -webkit-transition: background 0.2s linear;\n -moz-transition: background 0.2s linear;\n -ms-transition: background 0.2s linear;\n -o-transition: background 0.2s linear;\n transition: background 0.2s linear;\n}\n\n.vis-item .vis-delete {\n right: -24px;\n}\n\n.vis-item .vis-delete-rtl {\n left: -24px;\n}\n\n.vis-item .vis-delete:after, .vis-item .vis-delete-rtl:after {\n content: \"\\00D7\"; /* MULTIPLICATION SIGN */\n color: red;\n font-family: arial, sans-serif;\n font-size: 22px;\n font-weight: bold;\n\n -webkit-transition: color 0.2s linear;\n -moz-transition: color 0.2s linear;\n -ms-transition: color 0.2s linear;\n -o-transition: color 0.2s linear;\n transition: color 0.2s linear;\n}\n\n.vis-item .vis-delete:hover, .vis-item .vis-delete-rtl:hover {\n background: red;\n}\n\n.vis-item .vis-delete:hover:after, .vis-item .vis-delete-rtl:hover:after {\n color: white;\n}\n\n.vis-item .vis-drag-center {\n position: absolute;\n width: 100%;\n height: 100%;\n top: 0;\n left: 0px;\n cursor: move;\n}\n\n.vis-item.vis-range .vis-drag-left {\n position: absolute;\n width: 24px;\n max-width: 20%;\n min-width: 2px;\n height: 100%;\n top: 0;\n left: -4px;\n\n cursor: w-resize;\n}\n\n.vis-item.vis-range .vis-drag-right {\n position: absolute;\n width: 24px;\n max-width: 20%;\n min-width: 2px;\n height: 100%;\n top: 0;\n right: -4px;\n\n cursor: e-resize;\n}\n\n.vis-range.vis-item.vis-readonly .vis-drag-left,\n.vis-range.vis-item.vis-readonly .vis-drag-right {\n cursor: auto;\n}\n\n.vis-item.vis-cluster {\n vertical-align: center;\n text-align: center;\n border-style: solid;\n border-radius: 2px;\n}\n\n.vis-item.vis-cluster-line {\n padding: 0;\n position: absolute;\n width: 0;\n border-left-width: 1px;\n border-left-style: solid;\n}\n\n.vis-item.vis-cluster-dot {\n position: absolute;\n padding: 0;\n border-width: 4px;\n border-style: solid;\n border-radius: 4px;\n}"]} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js index df730409ee491..5cfa0da3d4f38 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js @@ -1,60 +1,30 @@ /** - * vis-timeline and vis-graph2d - * https://visjs.github.io/vis-timeline/ + * vis.js + * https://github.com/almende/vis * - * Create a fully customizable, interactive timeline with items and ranges. + * A dynamic, browser-based visualization library. * - * @version 7.3.4 - * @date 2020-03-18T17:03:58.105Z - * - * @copyright (c) 2011-2017 Almende B.V, http://almende.com - * @copyright (c) 2017-2019 visjs contributors, https://github.com/visjs + * @version 4.21.0 + * @date 2017-10-12 * * @license - * vis.js is dual licensed under both + * Copyright (C) 2011-2017 Almende B.V, http://almende.com + * + * Vis.js is dual licensed under both * - * 1. The Apache 2.0 License - * http://www.apache.org/licenses/LICENSE-2.0 + * * The Apache 2.0 License + * http://www.apache.org/licenses/LICENSE-2.0 * - * and + * and * - * 2. The MIT License - * http://opensource.org/licenses/MIT + * * The MIT License + * http://opensource.org/licenses/MIT * - * vis.js may be distributed under either license. + * Vis.js may be distributed under either license. */ -!function(t,e){"object"==typeof exports&&"undefined"!=typeof module?e(exports):"function"==typeof define&&define.amd?define(["exports"],e):e((t=t||self).vis=t.vis||{})}(this,(function(t){"use strict";var e="undefined"!=typeof globalThis?globalThis:"undefined"!=typeof window?window:"undefined"!=typeof global?global:"undefined"!=typeof self?self:{};function i(){throw new Error("Dynamic requires are not currently supported by @rollup/plugin-commonjs")}function n(t,e){return t(e={exports:{}},e.exports),e.exports}function o(t){return t&&t.default||t}var s=n((function(t,e){t.exports=function(){var e,n;function o(){return e.apply(null,arguments)}function s(t){return t instanceof Array||"[object Array]"===Object.prototype.toString.call(t)}function r(t){return null!=t&&"[object Object]"===Object.prototype.toString.call(t)}function a(t){return void 0===t}function l(t){return"number"==typeof t||"[object Number]"===Object.prototype.toString.call(t)}function h(t){return t instanceof Date||"[object Date]"===Object.prototype.toString.call(t)}function d(t,e){var i,n=[];for(i=0;i>>0,n=0;n0)for(i=0;i=0?i?"+":"":"-")+Math.pow(10,Math.max(0,o)).toString().substr(1)+n}var T=/(\[[^\[]*\])|(\\)?([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,Y=/(\[[^\[]*\])|(\\)?(LTS|LT|LL?L?L?|l{1,4})/g,E={},R={};function U(t,e,i,n){var o=n;"string"==typeof n&&(o=function(){return this[n]()}),t&&(R[t]=o),e&&(R[e[0]]=function(){return X(o.apply(this,arguments),e[1],e[2])}),i&&(R[i]=function(){return this.localeData().ordinal(o.apply(this,arguments),t)})}function z(t,e){return t.isValid()?(e=V(e,t.localeData()),E[e]=E[e]||function(t){var e,i,n,o=t.match(T);for(e=0,i=o.length;e=0&&Y.test(t);)t=t.replace(Y,n),Y.lastIndex=0,i-=1;return t}var j=/\d/,H=/\d\d/,P=/\d{3}/,J=/\d{4}/,K=/[+-]?\d{6}/,q=/\d\d?/,$=/\d\d\d\d?/,tt=/\d\d\d\d\d\d?/,et=/\d{1,3}/,it=/\d{1,4}/,nt=/[+-]?\d{1,6}/,ot=/\d+/,st=/[+-]?\d+/,rt=/Z|[+-]\d\d:?\d\d/gi,at=/Z|[+-]\d\d(?::?\d\d)?/gi,lt=/[0-9]{0,256}['a-z\u00A0-\u05FF\u0700-\uD7FF\uF900-\uFDCF\uFDF0-\uFF07\uFF10-\uFFEF]{1,256}|[\u0600-\u06FF\/]{1,256}(\s*?[\u0600-\u06FF]{1,256}){1,2}/i,ht={};function dt(t,e,i){ht[t]=S(e)?e:function(t,n){return t&&i?i:e}}function ut(t,e){return u(ht,t)?ht[t](e._strict,e._locale):new RegExp(ct(t.replace("\\","").replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,(function(t,e,i,n,o){return e||i||n||o}))))}function ct(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}var pt={};function mt(t,e){var i,n=e;for("string"==typeof t&&(t=[t]),l(e)&&(n=function(t,i){i[e]=k(t)}),i=0;i68?1900:2e3)};var yt,xt=wt("FullYear",!0);function wt(t,e){return function(i){return null!=i?(kt(this,t,i),o.updateOffset(this,e),this):_t(this,t)}}function _t(t,e){return t.isValid()?t._d["get"+(t._isUTC?"UTC":"")+e]():NaN}function kt(t,e,i){t.isValid()&&!isNaN(i)&&("FullYear"===e&&bt(t.year())&&1===t.month()&&29===t.date()?t._d["set"+(t._isUTC?"UTC":"")+e](i,t.month(),Dt(i,t.month())):t._d["set"+(t._isUTC?"UTC":"")+e](i))}function Dt(t,e){if(isNaN(t)||isNaN(e))return NaN;var i,n=(e%(i=12)+i)%i;return t+=(e-n)/12,1===n?bt(t)?29:28:31-n%7%2}yt=Array.prototype.indexOf?Array.prototype.indexOf:function(t){var e;for(e=0;e=0?(a=new Date(t+400,e,i,n,o,s,r),isFinite(a.getFullYear())&&a.setFullYear(t)):a=new Date(t,e,i,n,o,s,r),a}function Bt(t){var e;if(t<100&&t>=0){var i=Array.prototype.slice.call(arguments);i[0]=t+400,e=new Date(Date.UTC.apply(null,i)),isFinite(e.getUTCFullYear())&&e.setUTCFullYear(t)}else e=new Date(Date.UTC.apply(null,arguments));return e}function Ot(t,e,i){var n=7+e-i;return-(7+Bt(t,0,n).getUTCDay()-e)%7+n-1}function Wt(t,e,i,n,o){var s,r,a=1+7*(e-1)+(7+i-n)%7+Ot(t,n,o);return a<=0?r=vt(s=t-1)+a:a>vt(t)?(s=t+1,r=a-vt(t)):(s=t,r=a),{year:s,dayOfYear:r}}function At(t,e,i){var n,o,s=Ot(t.year(),e,i),r=Math.floor((t.dayOfYear()-s-1)/7)+1;return r<1?n=r+Xt(o=t.year()-1,e,i):r>Xt(t.year(),e,i)?(n=r-Xt(t.year(),e,i),o=t.year()+1):(o=t.year(),n=r),{week:n,year:o}}function Xt(t,e,i){var n=Ot(t,e,i),o=Ot(t+1,e,i);return(vt(t)-n+o)/7}function Tt(t,e){return t.slice(e,7).concat(t.slice(0,e))}U("w",["ww",2],"wo","week"),U("W",["WW",2],"Wo","isoWeek"),Z("week","w"),Z("isoWeek","W"),A("week",5),A("isoWeek",5),dt("w",q),dt("ww",q,H),dt("W",q),dt("WW",q,H),ft(["w","ww","W","WW"],(function(t,e,i,n){e[n.substr(0,1)]=k(t)})),U("d",0,"do","day"),U("dd",0,0,(function(t){return this.localeData().weekdaysMin(this,t)})),U("ddd",0,0,(function(t){return this.localeData().weekdaysShort(this,t)})),U("dddd",0,0,(function(t){return this.localeData().weekdays(this,t)})),U("e",0,0,"weekday"),U("E",0,0,"isoWeekday"),Z("day","d"),Z("weekday","e"),Z("isoWeekday","E"),A("day",11),A("weekday",11),A("isoWeekday",11),dt("d",q),dt("e",q),dt("E",q),dt("dd",(function(t,e){return e.weekdaysMinRegex(t)})),dt("ddd",(function(t,e){return e.weekdaysShortRegex(t)})),dt("dddd",(function(t,e){return e.weekdaysRegex(t)})),ft(["dd","ddd","dddd"],(function(t,e,i,n){var o=i._locale.weekdaysParse(t,n,i._strict);null!=o?e.d=o:m(i).invalidWeekday=t})),ft(["d","e","E"],(function(t,e,i,n){e[n]=k(t)}));var Yt="Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),Et="Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),Rt="Su_Mo_Tu_We_Th_Fr_Sa".split("_");function Ut(t,e,i){var n,o,s,r=t.toLocaleLowerCase();if(!this._weekdaysParse)for(this._weekdaysParse=[],this._shortWeekdaysParse=[],this._minWeekdaysParse=[],n=0;n<7;++n)s=p([2e3,1]).day(n),this._minWeekdaysParse[n]=this.weekdaysMin(s,"").toLocaleLowerCase(),this._shortWeekdaysParse[n]=this.weekdaysShort(s,"").toLocaleLowerCase(),this._weekdaysParse[n]=this.weekdays(s,"").toLocaleLowerCase();return i?"dddd"===e?-1!==(o=yt.call(this._weekdaysParse,r))?o:null:"ddd"===e?-1!==(o=yt.call(this._shortWeekdaysParse,r))?o:null:-1!==(o=yt.call(this._minWeekdaysParse,r))?o:null:"dddd"===e?-1!==(o=yt.call(this._weekdaysParse,r))||-1!==(o=yt.call(this._shortWeekdaysParse,r))||-1!==(o=yt.call(this._minWeekdaysParse,r))?o:null:"ddd"===e?-1!==(o=yt.call(this._shortWeekdaysParse,r))||-1!==(o=yt.call(this._weekdaysParse,r))||-1!==(o=yt.call(this._minWeekdaysParse,r))?o:null:-1!==(o=yt.call(this._minWeekdaysParse,r))||-1!==(o=yt.call(this._weekdaysParse,r))||-1!==(o=yt.call(this._shortWeekdaysParse,r))?o:null}var zt=lt,Vt=lt,jt=lt;function Ht(){function t(t,e){return e.length-t.length}var e,i,n,o,s,r=[],a=[],l=[],h=[];for(e=0;e<7;e++)i=p([2e3,1]).day(e),n=this.weekdaysMin(i,""),o=this.weekdaysShort(i,""),s=this.weekdays(i,""),r.push(n),a.push(o),l.push(s),h.push(n),h.push(o),h.push(s);for(r.sort(t),a.sort(t),l.sort(t),h.sort(t),e=0;e<7;e++)a[e]=ct(a[e]),l[e]=ct(l[e]),h[e]=ct(h[e]);this._weekdaysRegex=new RegExp("^("+h.join("|")+")","i"),this._weekdaysShortRegex=this._weekdaysRegex,this._weekdaysMinRegex=this._weekdaysRegex,this._weekdaysStrictRegex=new RegExp("^("+l.join("|")+")","i"),this._weekdaysShortStrictRegex=new RegExp("^("+a.join("|")+")","i"),this._weekdaysMinStrictRegex=new RegExp("^("+r.join("|")+")","i")}function Pt(){return this.hours()%12||12}function Jt(t,e){U(t,0,0,(function(){return this.localeData().meridiem(this.hours(),this.minutes(),e)}))}function Kt(t,e){return e._meridiemParse}U("H",["HH",2],0,"hour"),U("h",["hh",2],0,Pt),U("k",["kk",2],0,(function(){return this.hours()||24})),U("hmm",0,0,(function(){return""+Pt.apply(this)+X(this.minutes(),2)})),U("hmmss",0,0,(function(){return""+Pt.apply(this)+X(this.minutes(),2)+X(this.seconds(),2)})),U("Hmm",0,0,(function(){return""+this.hours()+X(this.minutes(),2)})),U("Hmmss",0,0,(function(){return""+this.hours()+X(this.minutes(),2)+X(this.seconds(),2)})),Jt("a",!0),Jt("A",!1),Z("hour","h"),A("hour",13),dt("a",Kt),dt("A",Kt),dt("H",q),dt("h",q),dt("k",q),dt("HH",q,H),dt("hh",q,H),dt("kk",q,H),dt("hmm",$),dt("hmmss",tt),dt("Hmm",$),dt("Hmmss",tt),mt(["H","HH"],3),mt(["k","kk"],(function(t,e,i){var n=k(t);e[3]=24===n?0:n})),mt(["a","A"],(function(t,e,i){i._isPm=i._locale.isPM(t),i._meridiem=t})),mt(["h","hh"],(function(t,e,i){e[3]=k(t),m(i).bigHour=!0})),mt("hmm",(function(t,e,i){var n=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n)),m(i).bigHour=!0})),mt("hmmss",(function(t,e,i){var n=t.length-4,o=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n,2)),e[5]=k(t.substr(o)),m(i).bigHour=!0})),mt("Hmm",(function(t,e,i){var n=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n))})),mt("Hmmss",(function(t,e,i){var n=t.length-4,o=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n,2)),e[5]=k(t.substr(o))}));var qt,$t=wt("Hours",!0),te={calendar:{sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},longDateFormat:{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"},invalidDate:"Invalid date",ordinal:"%d",dayOfMonthOrdinalParse:/\d{1,2}/,relativeTime:{future:"in %s",past:"%s ago",s:"a few seconds",ss:"%d 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"},months:Ct,monthsShort:Lt,week:{dow:0,doy:6},weekdays:Yt,weekdaysMin:Rt,weekdaysShort:Et,meridiemParse:/[ap]\.?m?\.?/i},ee={},ie={};function ne(t){return t?t.toLowerCase().replace("_","-"):t}function oe(e){var n=null;if(!ee[e]&&t&&t.exports)try{n=qt._abbr,i(),se(n)}catch(t){}return ee[e]}function se(t,e){var i;return t&&((i=a(e)?ae(t):re(t,e))?qt=i:"undefined"!=typeof console&&console.warn&&console.warn("Locale "+t+" not found. Did you forget to load it?")),qt._abbr}function re(t,e){if(null!==e){var i,n=te;if(e.abbr=t,null!=ee[t])M("defineLocaleOverride","use moment.updateLocale(localeName, config) to change an existing locale. moment.defineLocale(localeName, config) should only be used for creating a new locale See http://momentjs.com/guides/#/warnings/define-locale/ for more info."),n=ee[t]._config;else if(null!=e.parentLocale)if(null!=ee[e.parentLocale])n=ee[e.parentLocale]._config;else{if(null==(i=oe(e.parentLocale)))return ie[e.parentLocale]||(ie[e.parentLocale]=[]),ie[e.parentLocale].push({name:t,config:e}),null;n=i._config}return ee[t]=new N(Q(n,e)),ie[t]&&ie[t].forEach((function(t){re(t.name,t.config)})),se(t),ee[t]}return delete ee[t],null}function ae(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return qt;if(!s(t)){if(e=oe(t))return e;t=[t]}return function(t){for(var e,i,n,o,s=0;s0;){if(n=oe(o.slice(0,e).join("-")))return n;if(i&&i.length>=e&&D(o,i,!0)>=e-1)break;e--}s++}return qt}(t)}function le(t){var e,i=t._a;return i&&-2===m(t).overflow&&(e=i[1]<0||i[1]>11?1:i[2]<1||i[2]>Dt(i[0],i[1])?2:i[3]<0||i[3]>24||24===i[3]&&(0!==i[4]||0!==i[5]||0!==i[6])?3:i[4]<0||i[4]>59?4:i[5]<0||i[5]>59?5:i[6]<0||i[6]>999?6:-1,m(t)._overflowDayOfYear&&(e<0||e>2)&&(e=2),m(t)._overflowWeeks&&-1===e&&(e=7),m(t)._overflowWeekday&&-1===e&&(e=8),m(t).overflow=e),t}function he(t,e,i){return null!=t?t:null!=e?e:i}function de(t){var e,i,n,s,r,a=[];if(!t._d){for(n=function(t){var e=new Date(o.now());return t._useUTC?[e.getUTCFullYear(),e.getUTCMonth(),e.getUTCDate()]:[e.getFullYear(),e.getMonth(),e.getDate()]}(t),t._w&&null==t._a[2]&&null==t._a[1]&&function(t){var e,i,n,o,s,r,a,l;if(null!=(e=t._w).GG||null!=e.W||null!=e.E)s=1,r=4,i=he(e.GG,t._a[0],At(Ie(),1,4).year),n=he(e.W,1),((o=he(e.E,1))<1||o>7)&&(l=!0);else{s=t._locale._week.dow,r=t._locale._week.doy;var h=At(Ie(),s,r);i=he(e.gg,t._a[0],h.year),n=he(e.w,h.week),null!=e.d?((o=e.d)<0||o>6)&&(l=!0):null!=e.e?(o=e.e+s,(e.e<0||e.e>6)&&(l=!0)):o=s}n<1||n>Xt(i,s,r)?m(t)._overflowWeeks=!0:null!=l?m(t)._overflowWeekday=!0:(a=Wt(i,n,o,s,r),t._a[0]=a.year,t._dayOfYear=a.dayOfYear)}(t),null!=t._dayOfYear&&(r=he(t._a[0],n[0]),(t._dayOfYear>vt(r)||0===t._dayOfYear)&&(m(t)._overflowDayOfYear=!0),i=Bt(r,0,t._dayOfYear),t._a[1]=i.getUTCMonth(),t._a[2]=i.getUTCDate()),e=0;e<3&&null==t._a[e];++e)t._a[e]=a[e]=n[e];for(;e<7;e++)t._a[e]=a[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[3]&&0===t._a[4]&&0===t._a[5]&&0===t._a[6]&&(t._nextDay=!0,t._a[3]=0),t._d=(t._useUTC?Bt:Zt).apply(null,a),s=t._useUTC?t._d.getUTCDay():t._d.getDay(),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[3]=24),t._w&&void 0!==t._w.d&&t._w.d!==s&&(m(t).weekdayMismatch=!0)}}var ue=/^\s*((?:[+-]\d{6}|\d{4})-(?:\d\d-\d\d|W\d\d-\d|W\d\d|\d\d\d|\d\d))(?:(T| )(\d\d(?::\d\d(?::\d\d(?:[.,]\d+)?)?)?)([\+\-]\d\d(?::?\d\d)?|\s*Z)?)?$/,ce=/^\s*((?:[+-]\d{6}|\d{4})(?:\d\d\d\d|W\d\d\d|W\d\d|\d\d\d|\d\d))(?:(T| )(\d\d(?:\d\d(?:\d\d(?:[.,]\d+)?)?)?)([\+\-]\d\d(?::?\d\d)?|\s*Z)?)?$/,pe=/Z|[+-]\d\d(?::?\d\d)?/,me=[["YYYYYY-MM-DD",/[+-]\d{6}-\d\d-\d\d/],["YYYY-MM-DD",/\d{4}-\d\d-\d\d/],["GGGG-[W]WW-E",/\d{4}-W\d\d-\d/],["GGGG-[W]WW",/\d{4}-W\d\d/,!1],["YYYY-DDD",/\d{4}-\d{3}/],["YYYY-MM",/\d{4}-\d\d/,!1],["YYYYYYMMDD",/[+-]\d{10}/],["YYYYMMDD",/\d{8}/],["GGGG[W]WWE",/\d{4}W\d{3}/],["GGGG[W]WW",/\d{4}W\d{2}/,!1],["YYYYDDD",/\d{7}/]],fe=[["HH:mm:ss.SSSS",/\d\d:\d\d:\d\d\.\d+/],["HH:mm:ss,SSSS",/\d\d:\d\d:\d\d,\d+/],["HH:mm:ss",/\d\d:\d\d:\d\d/],["HH:mm",/\d\d:\d\d/],["HHmmss.SSSS",/\d\d\d\d\d\d\.\d+/],["HHmmss,SSSS",/\d\d\d\d\d\d,\d+/],["HHmmss",/\d\d\d\d\d\d/],["HHmm",/\d\d\d\d/],["HH",/\d\d/]],ge=/^\/?Date\((\-?\d+)/i;function ve(t){var e,i,n,o,s,r,a=t._i,l=ue.exec(a)||ce.exec(a);if(l){for(m(t).iso=!0,e=0,i=me.length;e0&&m(t).unusedInput.push(r),a=a.slice(a.indexOf(i)+i.length),h+=i.length),R[s]?(i?m(t).empty=!1:m(t).unusedTokens.push(s),gt(s,i,t)):t._strict&&!i&&m(t).unusedTokens.push(s);m(t).charsLeftOver=l-h,a.length>0&&m(t).unusedInput.push(a),t._a[3]<=12&&!0===m(t).bigHour&&t._a[3]>0&&(m(t).bigHour=void 0),m(t).parsedDateParts=t._a.slice(0),m(t).meridiem=t._meridiem,t._a[3]=function(t,e,i){var n;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?((n=t.isPM(i))&&e<12&&(e+=12),n||12!==e||(e=0),e):e}(t._locale,t._a[3],t._meridiem),de(t),le(t)}else we(t);else ve(t)}function ke(t){var e=t._i,i=t._f;return t._locale=t._locale||ae(t._l),null===e||void 0===i&&""===e?g({nullInput:!0}):("string"==typeof e&&(t._i=e=t._locale.preparse(e)),w(e)?new x(le(e)):(h(e)?t._d=e:s(i)?function(t){var e,i,n,o,s;if(0===t._f.length)return m(t).invalidFormat=!0,void(t._d=new Date(NaN));for(o=0;othis?this:t:g()}));function Ge(t,e){var i,n;if(1===e.length&&s(e[0])&&(e=e[0]),!e.length)return Ie();for(i=e[0],n=1;n=0?new Date(t+400,e,i)-126227808e5:new Date(t,e,i).valueOf()}function ti(t,e,i){return t<100&&t>=0?Date.UTC(t+400,e,i)-126227808e5:Date.UTC(t,e,i)}function ei(t,e){U(0,[t,t.length],0,e)}function ii(t,e,i,n,o){var s;return null==t?At(this,n,o).year:(e>(s=Xt(t,n,o))&&(e=s),ni.call(this,t,e,i,n,o))}function ni(t,e,i,n,o){var s=Wt(t,e,i,n,o),r=Bt(s.year,0,s.dayOfYear);return this.year(r.getUTCFullYear()),this.month(r.getUTCMonth()),this.date(r.getUTCDate()),this}U(0,["gg",2],0,(function(){return this.weekYear()%100})),U(0,["GG",2],0,(function(){return this.isoWeekYear()%100})),ei("gggg","weekYear"),ei("ggggg","weekYear"),ei("GGGG","isoWeekYear"),ei("GGGGG","isoWeekYear"),Z("weekYear","gg"),Z("isoWeekYear","GG"),A("weekYear",1),A("isoWeekYear",1),dt("G",st),dt("g",st),dt("GG",q,H),dt("gg",q,H),dt("GGGG",it,J),dt("gggg",it,J),dt("GGGGG",nt,K),dt("ggggg",nt,K),ft(["gggg","ggggg","GGGG","GGGGG"],(function(t,e,i,n){e[n.substr(0,2)]=k(t)})),ft(["gg","GG"],(function(t,e,i,n){e[n]=o.parseTwoDigitYear(t)})),U("Q",0,"Qo","quarter"),Z("quarter","Q"),A("quarter",7),dt("Q",j),mt("Q",(function(t,e){e[1]=3*(k(t)-1)})),U("D",["DD",2],"Do","date"),Z("date","D"),A("date",9),dt("D",q),dt("DD",q,H),dt("Do",(function(t,e){return t?e._dayOfMonthOrdinalParse||e._ordinalParse:e._dayOfMonthOrdinalParseLenient})),mt(["D","DD"],2),mt("Do",(function(t,e){e[2]=k(t.match(q)[0])}));var oi=wt("Date",!0);U("DDD",["DDDD",3],"DDDo","dayOfYear"),Z("dayOfYear","DDD"),A("dayOfYear",4),dt("DDD",et),dt("DDDD",P),mt(["DDD","DDDD"],(function(t,e,i){i._dayOfYear=k(t)})),U("m",["mm",2],0,"minute"),Z("minute","m"),A("minute",14),dt("m",q),dt("mm",q,H),mt(["m","mm"],4);var si=wt("Minutes",!1);U("s",["ss",2],0,"second"),Z("second","s"),A("second",15),dt("s",q),dt("ss",q,H),mt(["s","ss"],5);var ri,ai=wt("Seconds",!1);for(U("S",0,0,(function(){return~~(this.millisecond()/100)})),U(0,["SS",2],0,(function(){return~~(this.millisecond()/10)})),U(0,["SSS",3],0,"millisecond"),U(0,["SSSS",4],0,(function(){return 10*this.millisecond()})),U(0,["SSSSS",5],0,(function(){return 100*this.millisecond()})),U(0,["SSSSSS",6],0,(function(){return 1e3*this.millisecond()})),U(0,["SSSSSSS",7],0,(function(){return 1e4*this.millisecond()})),U(0,["SSSSSSSS",8],0,(function(){return 1e5*this.millisecond()})),U(0,["SSSSSSSSS",9],0,(function(){return 1e6*this.millisecond()})),Z("millisecond","ms"),A("millisecond",16),dt("S",et,j),dt("SS",et,H),dt("SSS",et,P),ri="SSSS";ri.length<=9;ri+="S")dt(ri,ot);function li(t,e){e[6]=k(1e3*("0."+t))}for(ri="S";ri.length<=9;ri+="S")mt(ri,li);var hi=wt("Milliseconds",!1);U("z",0,0,"zoneAbbr"),U("zz",0,0,"zoneName");var di=x.prototype;function ui(t){return t}di.add=Ve,di.calendar=function(t,e){var i=t||Ie(),n=Oe(i,this).startOf("day"),s=o.calendarFormat(this,n)||"sameElse",r=e&&(S(e[s])?e[s].call(this,i):e[s]);return this.format(r||this.localeData().calendar(s,this,Ie(i)))},di.clone=function(){return new x(this)},di.diff=function(t,e,i){var n,o,s;if(!this.isValid())return NaN;if(!(n=Oe(t,this)).isValid())return NaN;switch(o=6e4*(n.utcOffset()-this.utcOffset()),e=B(e)){case"year":s=He(this,n)/12;break;case"month":s=He(this,n);break;case"quarter":s=He(this,n)/3;break;case"second":s=(this-n)/1e3;break;case"minute":s=(this-n)/6e4;break;case"hour":s=(this-n)/36e5;break;case"day":s=(this-n-o)/864e5;break;case"week":s=(this-n-o)/6048e5;break;default:s=this-n}return i?s:_(s)},di.endOf=function(t){var e;if(void 0===(t=B(t))||"millisecond"===t||!this.isValid())return this;var i=this._isUTC?ti:$e;switch(t){case"year":e=i(this.year()+1,0,1)-1;break;case"quarter":e=i(this.year(),this.month()-this.month()%3+3,1)-1;break;case"month":e=i(this.year(),this.month()+1,1)-1;break;case"week":e=i(this.year(),this.month(),this.date()-this.weekday()+7)-1;break;case"isoWeek":e=i(this.year(),this.month(),this.date()-(this.isoWeekday()-1)+7)-1;break;case"day":case"date":e=i(this.year(),this.month(),this.date()+1)-1;break;case"hour":e=this._d.valueOf(),e+=36e5-qe(e+(this._isUTC?0:6e4*this.utcOffset()),36e5)-1;break;case"minute":e=this._d.valueOf(),e+=6e4-qe(e,6e4)-1;break;case"second":e=this._d.valueOf(),e+=1e3-qe(e,1e3)-1}return this._d.setTime(e),o.updateOffset(this,!0),this},di.format=function(t){t||(t=this.isUtc()?o.defaultFormatUtc:o.defaultFormat);var e=z(this,t);return this.localeData().postformat(e)},di.from=function(t,e){return this.isValid()&&(w(t)&&t.isValid()||Ie(t).isValid())?Ye({to:this,from:t}).locale(this.locale()).humanize(!e):this.localeData().invalidDate()},di.fromNow=function(t){return this.from(Ie(),t)},di.to=function(t,e){return this.isValid()&&(w(t)&&t.isValid()||Ie(t).isValid())?Ye({from:this,to:t}).locale(this.locale()).humanize(!e):this.localeData().invalidDate()},di.toNow=function(t){return this.to(Ie(),t)},di.get=function(t){return S(this[t=B(t)])?this[t]():this},di.invalidAt=function(){return m(this).overflow},di.isAfter=function(t,e){var i=w(t)?t:Ie(t);return!(!this.isValid()||!i.isValid())&&("millisecond"===(e=B(e)||"millisecond")?this.valueOf()>i.valueOf():i.valueOf()9999?z(i,e?"YYYYYY-MM-DD[T]HH:mm:ss.SSS[Z]":"YYYYYY-MM-DD[T]HH:mm:ss.SSSZ"):S(Date.prototype.toISOString)?e?this.toDate().toISOString():new Date(this.valueOf()+60*this.utcOffset()*1e3).toISOString().replace("Z",z(i,"Z")):z(i,e?"YYYY-MM-DD[T]HH:mm:ss.SSS[Z]":"YYYY-MM-DD[T]HH:mm:ss.SSSZ")},di.inspect=function(){if(!this.isValid())return"moment.invalid(/* "+this._i+" */)";var t="moment",e="";this.isLocal()||(t=0===this.utcOffset()?"moment.utc":"moment.parseZone",e="Z");var i="["+t+'("]',n=0<=this.year()&&this.year()<=9999?"YYYY":"YYYYYY",o=e+'[")]';return this.format(i+n+"-MM-DD[T]HH:mm:ss.SSS"+o)},di.toJSON=function(){return this.isValid()?this.toISOString():null},di.toString=function(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")},di.unix=function(){return Math.floor(this.valueOf()/1e3)},di.valueOf=function(){return this._d.valueOf()-6e4*(this._offset||0)},di.creationData=function(){return{input:this._i,format:this._f,locale:this._locale,isUTC:this._isUTC,strict:this._strict}},di.year=xt,di.isLeapYear=function(){return bt(this.year())},di.weekYear=function(t){return ii.call(this,t,this.week(),this.weekday(),this.localeData()._week.dow,this.localeData()._week.doy)},di.isoWeekYear=function(t){return ii.call(this,t,this.isoWeek(),this.isoWeekday(),1,4)},di.quarter=di.quarters=function(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)},di.month=St,di.daysInMonth=function(){return Dt(this.year(),this.month())},di.week=di.weeks=function(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")},di.isoWeek=di.isoWeeks=function(t){var e=At(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")},di.weeksInYear=function(){var t=this.localeData()._week;return Xt(this.year(),t.dow,t.doy)},di.isoWeeksInYear=function(){return Xt(this.year(),1,4)},di.date=oi,di.day=di.days=function(t){if(!this.isValid())return null!=t?this:NaN;var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=function(t,e){return"string"!=typeof t?t:isNaN(t)?"number"==typeof(t=e.weekdaysParse(t))?t:null:parseInt(t,10)}(t,this.localeData()),this.add(t-e,"d")):e},di.weekday=function(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")},di.isoWeekday=function(t){if(!this.isValid())return null!=t?this:NaN;if(null!=t){var e=function(t,e){return"string"==typeof t?e.weekdaysParse(t)%7||7:isNaN(t)?null:t}(t,this.localeData());return this.day(this.day()%7?e:e-7)}return this.day()||7},di.dayOfYear=function(t){var e=Math.round((this.clone().startOf("day")-this.clone().startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")},di.hour=di.hours=$t,di.minute=di.minutes=si,di.second=di.seconds=ai,di.millisecond=di.milliseconds=hi,di.utcOffset=function(t,e,i){var n,s=this._offset||0;if(!this.isValid())return null!=t?this:NaN;if(null!=t){if("string"==typeof t){if(null===(t=Be(at,t)))return this}else Math.abs(t)<16&&!i&&(t*=60);return!this._isUTC&&e&&(n=We(this)),this._offset=t,this._isUTC=!0,null!=n&&this.add(n,"m"),s!==t&&(!e||this._changeInProgress?ze(this,Ye(t-s,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,o.updateOffset(this,!0),this._changeInProgress=null)),this}return this._isUTC?s:We(this)},di.utc=function(t){return this.utcOffset(0,t)},di.local=function(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(We(this),"m")),this},di.parseZone=function(){if(null!=this._tzm)this.utcOffset(this._tzm,!1,!0);else if("string"==typeof this._i){var t=Be(rt,this._i);null!=t?this.utcOffset(t):this.utcOffset(0,!0)}return this},di.hasAlignedHourOffset=function(t){return!!this.isValid()&&(t=t?Ie(t).utcOffset():0,(this.utcOffset()-t)%60==0)},di.isDST=function(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()},di.isLocal=function(){return!!this.isValid()&&!this._isUTC},di.isUtcOffset=function(){return!!this.isValid()&&this._isUTC},di.isUtc=Ae,di.isUTC=Ae,di.zoneAbbr=function(){return this._isUTC?"UTC":""},di.zoneName=function(){return this._isUTC?"Coordinated Universal Time":""},di.dates=C("dates accessor is deprecated. Use date instead.",oi),di.months=C("months accessor is deprecated. Use month instead",St),di.years=C("years accessor is deprecated. Use year instead",xt),di.zone=C("moment().zone is deprecated, use moment().utcOffset instead. http://momentjs.com/guides/#/warnings/zone/",(function(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()})),di.isDSTShifted=C("isDSTShifted is deprecated. See http://momentjs.com/guides/#/warnings/dst-shifted/ for more information",(function(){if(!a(this._isDSTShifted))return this._isDSTShifted;var t={};if(b(t,this),(t=ke(t))._a){var e=t._isUTC?p(t._a):Ie(t._a);this._isDSTShifted=this.isValid()&&D(t._a,e.toArray())>0}else this._isDSTShifted=!1;return this._isDSTShifted}));var ci=N.prototype;function pi(t,e,i,n){var o=ae(),s=p().set(n,e);return o[i](s,t)}function mi(t,e,i){if(l(t)&&(e=t,t=void 0),t=t||"",null!=e)return pi(t,e,i,"month");var n,o=[];for(n=0;n<12;n++)o[n]=pi(t,n,i,"month");return o}function fi(t,e,i,n){"boolean"==typeof t?(l(e)&&(i=e,e=void 0),e=e||""):(i=e=t,t=!1,l(e)&&(i=e,e=void 0),e=e||"");var o,s=ae(),r=t?s._week.dow:0;if(null!=i)return pi(e,(i+r)%7,n,"day");var a=[];for(o=0;o<7;o++)a[o]=pi(e,(o+r)%7,n,"day");return a}ci.calendar=function(t,e,i){var n=this._calendar[t]||this._calendar.sameElse;return S(n)?n.call(e,i):n},ci.longDateFormat=function(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])},ci.invalidDate=function(){return this._invalidDate},ci.ordinal=function(t){return this._ordinal.replace("%d",t)},ci.preparse=ui,ci.postformat=ui,ci.relativeTime=function(t,e,i,n){var o=this._relativeTime[i];return S(o)?o(t,e,i,n):o.replace(/%d/i,t)},ci.pastFuture=function(t,e){var i=this._relativeTime[t>0?"future":"past"];return S(i)?i(e):i.replace(/%s/i,e)},ci.set=function(t){var e,i;for(i in t)S(e=t[i])?this[i]=e:this["_"+i]=e;this._config=t,this._dayOfMonthOrdinalParseLenient=new RegExp((this._dayOfMonthOrdinalParse.source||this._ordinalParse.source)+"|"+/\d{1,2}/.source)},ci.months=function(t,e){return t?s(this._months)?this._months[t.month()]:this._months[(this._months.isFormat||It).test(e)?"format":"standalone"][t.month()]:s(this._months)?this._months:this._months.standalone},ci.monthsShort=function(t,e){return t?s(this._monthsShort)?this._monthsShort[t.month()]:this._monthsShort[It.test(e)?"format":"standalone"][t.month()]:s(this._monthsShort)?this._monthsShort:this._monthsShort.standalone},ci.monthsParse=function(t,e,i){var n,o,s;if(this._monthsParseExact)return Gt.call(this,t,e,i);for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),n=0;n<12;n++){if(o=p([2e3,n]),i&&!this._longMonthsParse[n]&&(this._longMonthsParse[n]=new RegExp("^"+this.months(o,"").replace(".","")+"$","i"),this._shortMonthsParse[n]=new RegExp("^"+this.monthsShort(o,"").replace(".","")+"$","i")),i||this._monthsParse[n]||(s="^"+this.months(o,"")+"|^"+this.monthsShort(o,""),this._monthsParse[n]=new RegExp(s.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[n].test(t))return n;if(i&&"MMM"===e&&this._shortMonthsParse[n].test(t))return n;if(!i&&this._monthsParse[n].test(t))return n}},ci.monthsRegex=function(t){return this._monthsParseExact?(u(this,"_monthsRegex")||Ft.call(this),t?this._monthsStrictRegex:this._monthsRegex):(u(this,"_monthsRegex")||(this._monthsRegex=Nt),this._monthsStrictRegex&&t?this._monthsStrictRegex:this._monthsRegex)},ci.monthsShortRegex=function(t){return this._monthsParseExact?(u(this,"_monthsRegex")||Ft.call(this),t?this._monthsShortStrictRegex:this._monthsShortRegex):(u(this,"_monthsShortRegex")||(this._monthsShortRegex=Qt),this._monthsShortStrictRegex&&t?this._monthsShortStrictRegex:this._monthsShortRegex)},ci.week=function(t){return At(t,this._week.dow,this._week.doy).week},ci.firstDayOfYear=function(){return this._week.doy},ci.firstDayOfWeek=function(){return this._week.dow},ci.weekdays=function(t,e){var i=s(this._weekdays)?this._weekdays:this._weekdays[t&&!0!==t&&this._weekdays.isFormat.test(e)?"format":"standalone"];return!0===t?Tt(i,this._week.dow):t?i[t.day()]:i},ci.weekdaysMin=function(t){return!0===t?Tt(this._weekdaysMin,this._week.dow):t?this._weekdaysMin[t.day()]:this._weekdaysMin},ci.weekdaysShort=function(t){return!0===t?Tt(this._weekdaysShort,this._week.dow):t?this._weekdaysShort[t.day()]:this._weekdaysShort},ci.weekdaysParse=function(t,e,i){var n,o,s;if(this._weekdaysParseExact)return Ut.call(this,t,e,i);for(this._weekdaysParse||(this._weekdaysParse=[],this._minWeekdaysParse=[],this._shortWeekdaysParse=[],this._fullWeekdaysParse=[]),n=0;n<7;n++){if(o=p([2e3,1]).day(n),i&&!this._fullWeekdaysParse[n]&&(this._fullWeekdaysParse[n]=new RegExp("^"+this.weekdays(o,"").replace(".","\\.?")+"$","i"),this._shortWeekdaysParse[n]=new RegExp("^"+this.weekdaysShort(o,"").replace(".","\\.?")+"$","i"),this._minWeekdaysParse[n]=new RegExp("^"+this.weekdaysMin(o,"").replace(".","\\.?")+"$","i")),this._weekdaysParse[n]||(s="^"+this.weekdays(o,"")+"|^"+this.weekdaysShort(o,"")+"|^"+this.weekdaysMin(o,""),this._weekdaysParse[n]=new RegExp(s.replace(".",""),"i")),i&&"dddd"===e&&this._fullWeekdaysParse[n].test(t))return n;if(i&&"ddd"===e&&this._shortWeekdaysParse[n].test(t))return n;if(i&&"dd"===e&&this._minWeekdaysParse[n].test(t))return n;if(!i&&this._weekdaysParse[n].test(t))return n}},ci.weekdaysRegex=function(t){return this._weekdaysParseExact?(u(this,"_weekdaysRegex")||Ht.call(this),t?this._weekdaysStrictRegex:this._weekdaysRegex):(u(this,"_weekdaysRegex")||(this._weekdaysRegex=zt),this._weekdaysStrictRegex&&t?this._weekdaysStrictRegex:this._weekdaysRegex)},ci.weekdaysShortRegex=function(t){return this._weekdaysParseExact?(u(this,"_weekdaysRegex")||Ht.call(this),t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex):(u(this,"_weekdaysShortRegex")||(this._weekdaysShortRegex=Vt),this._weekdaysShortStrictRegex&&t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex)},ci.weekdaysMinRegex=function(t){return this._weekdaysParseExact?(u(this,"_weekdaysRegex")||Ht.call(this),t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex):(u(this,"_weekdaysMinRegex")||(this._weekdaysMinRegex=jt),this._weekdaysMinStrictRegex&&t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex)},ci.isPM=function(t){return"p"===(t+"").toLowerCase().charAt(0)},ci.meridiem=function(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"},se("en",{dayOfMonthOrdinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10;return t+(1===k(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th")}}),o.lang=C("moment.lang is deprecated. Use moment.locale instead.",se),o.langData=C("moment.langData is deprecated. Use moment.localeData instead.",ae);var gi=Math.abs;function vi(t,e,i,n){var o=Ye(e,i);return t._milliseconds+=n*o._milliseconds,t._days+=n*o._days,t._months+=n*o._months,t._bubble()}function bi(t){return t<0?Math.floor(t):Math.ceil(t)}function yi(t){return 4800*t/146097}function xi(t){return 146097*t/4800}function wi(t){return function(){return this.as(t)}}var _i=wi("ms"),ki=wi("s"),Di=wi("m"),Ii=wi("h"),Ci=wi("d"),Li=wi("w"),Gi=wi("M"),Mi=wi("Q"),Si=wi("y");function Qi(t){return function(){return this.isValid()?this._data[t]:NaN}}var Ni=Qi("milliseconds"),Fi=Qi("seconds"),Zi=Qi("minutes"),Bi=Qi("hours"),Oi=Qi("days"),Wi=Qi("months"),Ai=Qi("years"),Xi=Math.round,Ti={ss:44,s:45,m:45,h:22,d:26,M:11};function Yi(t,e,i,n,o){return o.relativeTime(e||1,!!i,t,n)}function Ei(t,e,i){var n=Ye(t).abs(),o=Xi(n.as("s")),s=Xi(n.as("m")),r=Xi(n.as("h")),a=Xi(n.as("d")),l=Xi(n.as("M")),h=Xi(n.as("y")),d=o<=Ti.ss&&["s",o]||o0,d[4]=i,Yi.apply(null,d)}var Ri=Math.abs;function Ui(t){return(t>0)-(t<0)||+t}function zi(){if(!this.isValid())return this.localeData().invalidDate();var t,e,i=Ri(this._milliseconds)/1e3,n=Ri(this._days),o=Ri(this._months);t=_(i/60),e=_(t/60),i%=60,t%=60;var s=_(o/12),r=o%=12,a=n,l=e,h=t,d=i?i.toFixed(3).replace(/\.?0+$/,""):"",u=this.asSeconds();if(!u)return"P0D";var c=u<0?"-":"",p=Ui(this._months)!==Ui(u)?"-":"",m=Ui(this._days)!==Ui(u)?"-":"",f=Ui(this._milliseconds)!==Ui(u)?"-":"";return c+"P"+(s?p+s+"Y":"")+(r?p+r+"M":"")+(a?m+a+"D":"")+(l||h||d?"T":"")+(l?f+l+"H":"")+(h?f+h+"M":"")+(d?f+d+"S":"")}var Vi=Se.prototype;return Vi.isValid=function(){return this._isValid},Vi.abs=function(){var t=this._data;return this._milliseconds=gi(this._milliseconds),this._days=gi(this._days),this._months=gi(this._months),t.milliseconds=gi(t.milliseconds),t.seconds=gi(t.seconds),t.minutes=gi(t.minutes),t.hours=gi(t.hours),t.months=gi(t.months),t.years=gi(t.years),this},Vi.add=function(t,e){return vi(this,t,e,1)},Vi.subtract=function(t,e){return vi(this,t,e,-1)},Vi.as=function(t){if(!this.isValid())return NaN;var e,i,n=this._milliseconds;if("month"===(t=B(t))||"quarter"===t||"year"===t)switch(e=this._days+n/864e5,i=this._months+yi(e),t){case"month":return i;case"quarter":return i/3;case"year":return i/12}else switch(e=this._days+Math.round(xi(this._months)),t){case"week":return e/7+n/6048e5;case"day":return e+n/864e5;case"hour":return 24*e+n/36e5;case"minute":return 1440*e+n/6e4;case"second":return 86400*e+n/1e3;case"millisecond":return Math.floor(864e5*e)+n;default:throw new Error("Unknown unit "+t)}},Vi.asMilliseconds=_i,Vi.asSeconds=ki,Vi.asMinutes=Di,Vi.asHours=Ii,Vi.asDays=Ci,Vi.asWeeks=Li,Vi.asMonths=Gi,Vi.asQuarters=Mi,Vi.asYears=Si,Vi.valueOf=function(){return this.isValid()?this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*k(this._months/12):NaN},Vi._bubble=function(){var t,e,i,n,o,s=this._milliseconds,r=this._days,a=this._months,l=this._data;return s>=0&&r>=0&&a>=0||s<=0&&r<=0&&a<=0||(s+=864e5*bi(xi(a)+r),r=0,a=0),l.milliseconds=s%1e3,t=_(s/1e3),l.seconds=t%60,e=_(t/60),l.minutes=e%60,i=_(e/60),l.hours=i%24,r+=_(i/24),o=_(yi(r)),a+=o,r-=bi(xi(o)),n=_(a/12),a%=12,l.days=r,l.months=a,l.years=n,this},Vi.clone=function(){return Ye(this)},Vi.get=function(t){return t=B(t),this.isValid()?this[t+"s"]():NaN},Vi.milliseconds=Ni,Vi.seconds=Fi,Vi.minutes=Zi,Vi.hours=Bi,Vi.days=Oi,Vi.weeks=function(){return _(this.days()/7)},Vi.months=Wi,Vi.years=Ai,Vi.humanize=function(t){if(!this.isValid())return this.localeData().invalidDate();var e=this.localeData(),i=Ei(this,!t,e);return t&&(i=e.pastFuture(+this,i)),e.postformat(i)},Vi.toISOString=zi,Vi.toString=zi,Vi.toJSON=zi,Vi.locale=Pe,Vi.localeData=Ke,Vi.toIsoString=C("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",zi),Vi.lang=Je,U("X",0,0,"unix"),U("x",0,0,"valueOf"),dt("x",st),dt("X",/[+-]?\d+(\.\d{1,3})?/),mt("X",(function(t,e,i){i._d=new Date(1e3*parseFloat(t,10))})),mt("x",(function(t,e,i){i._d=new Date(k(t))})),o.version="2.24.0",e=Ie,o.fn=di,o.min=function(){var t=[].slice.call(arguments,0);return Ge("isBefore",t)},o.max=function(){var t=[].slice.call(arguments,0);return Ge("isAfter",t)},o.now=function(){return Date.now?Date.now():+new Date},o.utc=p,o.unix=function(t){return Ie(1e3*t)},o.months=function(t,e){return mi(t,e,"months")},o.isDate=h,o.locale=se,o.invalid=g,o.duration=Ye,o.isMoment=w,o.weekdays=function(t,e,i){return fi(t,e,i,"weekdays")},o.parseZone=function(){return Ie.apply(null,arguments).parseZone()},o.localeData=ae,o.isDuration=Qe,o.monthsShort=function(t,e){return mi(t,e,"monthsShort")},o.weekdaysMin=function(t,e,i){return fi(t,e,i,"weekdaysMin")},o.defineLocale=re,o.updateLocale=function(t,e){if(null!=e){var i,n,o=te;null!=(n=oe(t))&&(o=n._config),(i=new N(e=Q(o,e))).parentLocale=ee[t],ee[t]=i,se(t)}else null!=ee[t]&&(null!=ee[t].parentLocale?ee[t]=ee[t].parentLocale:null!=ee[t]&&delete ee[t]);return ee[t]},o.locales=function(){return L(ee)},o.weekdaysShort=function(t,e,i){return fi(t,e,i,"weekdaysShort")},o.normalizeUnits=B,o.relativeTimeRounding=function(t){return void 0===t?Xi:"function"==typeof t&&(Xi=t,!0)},o.relativeTimeThreshold=function(t,e){return void 0!==Ti[t]&&(void 0===e?Ti[t]:(Ti[t]=e,"s"===t&&(Ti.ss=e-1),!0))},o.calendarFormat=function(t,e){var i=t.diff(e,"days",!0);return i<-6?"sameElse":i<-1?"lastWeek":i<0?"lastDay":i<1?"sameDay":i<2?"nextDay":i<7?"nextWeek":"sameElse"},o.prototype=di,o.HTML5_FMT={DATETIME_LOCAL:"YYYY-MM-DDTHH:mm",DATETIME_LOCAL_SECONDS:"YYYY-MM-DDTHH:mm:ss",DATETIME_LOCAL_MS:"YYYY-MM-DDTHH:mm:ss.SSS",DATE:"YYYY-MM-DD",TIME:"HH:mm",TIME_SECONDS:"HH:mm:ss",TIME_MS:"HH:mm:ss.SSS",WEEK:"GGGG-[W]WW",MONTH:"YYYY-MM"},o}()})),r=(n((function(t,e){(function(t){function e(t,e,i,n){var o={m:["eine Minute","einer Minute"],h:["eine Stunde","einer Stunde"],d:["ein Tag","einem Tag"],dd:[t+" Tage",t+" Tagen"],M:["ein Monat","einem Monat"],MM:[t+" Monate",t+" Monaten"],y:["ein Jahr","einem Jahr"],yy:[t+" Jahre",t+" Jahren"]};return e?o[i][0]:o[i][1]}t.defineLocale("de",{months:"Januar_Februar_März_April_Mai_Juni_Juli_August_September_Oktober_November_Dezember".split("_"),monthsShort:"Jan._Feb._März_Apr._Mai_Juni_Juli_Aug._Sep._Okt._Nov._Dez.".split("_"),monthsParseExact:!0,weekdays:"Sonntag_Montag_Dienstag_Mittwoch_Donnerstag_Freitag_Samstag".split("_"),weekdaysShort:"So._Mo._Di._Mi._Do._Fr._Sa.".split("_"),weekdaysMin:"So_Mo_Di_Mi_Do_Fr_Sa".split("_"),weekdaysParseExact:!0,longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD.MM.YYYY",LL:"D. MMMM YYYY",LLL:"D. MMMM YYYY HH:mm",LLLL:"dddd, D. MMMM YYYY HH:mm"},calendar:{sameDay:"[heute um] LT [Uhr]",sameElse:"L",nextDay:"[morgen um] LT [Uhr]",nextWeek:"dddd [um] LT [Uhr]",lastDay:"[gestern um] LT [Uhr]",lastWeek:"[letzten] dddd [um] LT [Uhr]"},relativeTime:{future:"in %s",past:"vor %s",s:"ein paar Sekunden",ss:"%d Sekunden",m:e,mm:"%d Minuten",h:e,hh:"%d Stunden",d:e,dd:e,M:e,MM:e,y:e,yy:e},dayOfMonthOrdinalParse:/\d{1,2}\./,ordinal:"%d.",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){var e="ene._feb._mar._abr._may._jun._jul._ago._sep._oct._nov._dic.".split("_"),i="ene_feb_mar_abr_may_jun_jul_ago_sep_oct_nov_dic".split("_"),n=[/^ene/i,/^feb/i,/^mar/i,/^abr/i,/^may/i,/^jun/i,/^jul/i,/^ago/i,/^sep/i,/^oct/i,/^nov/i,/^dic/i],o=/^(enero|febrero|marzo|abril|mayo|junio|julio|agosto|septiembre|octubre|noviembre|diciembre|ene\.?|feb\.?|mar\.?|abr\.?|may\.?|jun\.?|jul\.?|ago\.?|sep\.?|oct\.?|nov\.?|dic\.?)/i;t.defineLocale("es",{months:"enero_febrero_marzo_abril_mayo_junio_julio_agosto_septiembre_octubre_noviembre_diciembre".split("_"),monthsShort:function(t,n){return t?/-MMM-/.test(n)?i[t.month()]:e[t.month()]:e},monthsRegex:o,monthsShortRegex:o,monthsStrictRegex:/^(enero|febrero|marzo|abril|mayo|junio|julio|agosto|septiembre|octubre|noviembre|diciembre)/i,monthsShortStrictRegex:/^(ene\.?|feb\.?|mar\.?|abr\.?|may\.?|jun\.?|jul\.?|ago\.?|sep\.?|oct\.?|nov\.?|dic\.?)/i,monthsParse:n,longMonthsParse:n,shortMonthsParse:n,weekdays:"domingo_lunes_martes_miércoles_jueves_viernes_sábado".split("_"),weekdaysShort:"dom._lun._mar._mié._jue._vie._sáb.".split("_"),weekdaysMin:"do_lu_ma_mi_ju_vi_sá".split("_"),weekdaysParseExact:!0,longDateFormat:{LT:"H:mm",LTS:"H:mm:ss",L:"DD/MM/YYYY",LL:"D [de] MMMM [de] YYYY",LLL:"D [de] MMMM [de] YYYY H:mm",LLLL:"dddd, D [de] MMMM [de] YYYY H:mm"},calendar:{sameDay:function(){return"[hoy a la"+(1!==this.hours()?"s":"")+"] LT"},nextDay:function(){return"[mañana a la"+(1!==this.hours()?"s":"")+"] LT"},nextWeek:function(){return"dddd [a la"+(1!==this.hours()?"s":"")+"] LT"},lastDay:function(){return"[ayer a la"+(1!==this.hours()?"s":"")+"] LT"},lastWeek:function(){return"[el] dddd [pasado a la"+(1!==this.hours()?"s":"")+"] LT"},sameElse:"L"},relativeTime:{future:"en %s",past:"hace %s",s:"unos segundos",ss:"%d segundos",m:"un minuto",mm:"%d minutos",h:"una hora",hh:"%d horas",d:"un día",dd:"%d días",M:"un mes",MM:"%d meses",y:"un año",yy:"%d años"},dayOfMonthOrdinalParse:/\d{1,2}º/,ordinal:"%dº",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){t.defineLocale("fr",{months:"janvier_février_mars_avril_mai_juin_juillet_août_septembre_octobre_novembre_décembre".split("_"),monthsShort:"janv._févr._mars_avr._mai_juin_juil._août_sept._oct._nov._déc.".split("_"),monthsParseExact:!0,weekdays:"dimanche_lundi_mardi_mercredi_jeudi_vendredi_samedi".split("_"),weekdaysShort:"dim._lun._mar._mer._jeu._ven._sam.".split("_"),weekdaysMin:"di_lu_ma_me_je_ve_sa".split("_"),weekdaysParseExact:!0,longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD/MM/YYYY",LL:"D MMMM YYYY",LLL:"D MMMM YYYY HH:mm",LLLL:"dddd D MMMM YYYY HH:mm"},calendar:{sameDay:"[Aujourd’hui à] LT",nextDay:"[Demain à] LT",nextWeek:"dddd [à] LT",lastDay:"[Hier à] LT",lastWeek:"dddd [dernier à] LT",sameElse:"L"},relativeTime:{future:"dans %s",past:"il y a %s",s:"quelques secondes",ss:"%d secondes",m:"une minute",mm:"%d minutes",h:"une heure",hh:"%d heures",d:"un jour",dd:"%d jours",M:"un mois",MM:"%d mois",y:"un an",yy:"%d ans"},dayOfMonthOrdinalParse:/\d{1,2}(er|)/,ordinal:function(t,e){switch(e){case"D":return t+(1===t?"er":"");default:case"M":case"Q":case"DDD":case"d":return t+(1===t?"er":"e");case"w":case"W":return t+(1===t?"re":"e")}},week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){t.defineLocale("it",{months:"gennaio_febbraio_marzo_aprile_maggio_giugno_luglio_agosto_settembre_ottobre_novembre_dicembre".split("_"),monthsShort:"gen_feb_mar_apr_mag_giu_lug_ago_set_ott_nov_dic".split("_"),weekdays:"domenica_lunedì_martedì_mercoledì_giovedì_venerdì_sabato".split("_"),weekdaysShort:"dom_lun_mar_mer_gio_ven_sab".split("_"),weekdaysMin:"do_lu_ma_me_gi_ve_sa".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD/MM/YYYY",LL:"D MMMM YYYY",LLL:"D MMMM YYYY HH:mm",LLLL:"dddd D MMMM YYYY HH:mm"},calendar:{sameDay:"[Oggi alle] LT",nextDay:"[Domani alle] LT",nextWeek:"dddd [alle] LT",lastDay:"[Ieri alle] LT",lastWeek:function(){switch(this.day()){case 0:return"[la scorsa] dddd [alle] LT";default:return"[lo scorso] dddd [alle] LT"}},sameElse:"L"},relativeTime:{future:function(t){return(/^[0-9].+$/.test(t)?"tra":"in")+" "+t},past:"%s fa",s:"alcuni secondi",ss:"%d secondi",m:"un minuto",mm:"%d minuti",h:"un'ora",hh:"%d ore",d:"un giorno",dd:"%d giorni",M:"un mese",MM:"%d mesi",y:"un anno",yy:"%d anni"},dayOfMonthOrdinalParse:/\d{1,2}º/,ordinal:"%dº",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){t.defineLocale("ja",{months:"一月_二月_三月_四月_五月_六月_七月_八月_九月_十月_十一月_十二月".split("_"),monthsShort:"1月_2月_3月_4月_5月_6月_7月_8月_9月_10月_11月_12月".split("_"),weekdays:"日曜日_月曜日_火曜日_水曜日_木曜日_金曜日_土曜日".split("_"),weekdaysShort:"日_月_火_水_木_金_土".split("_"),weekdaysMin:"日_月_火_水_木_金_土".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"YYYY/MM/DD",LL:"YYYY年M月D日",LLL:"YYYY年M月D日 HH:mm",LLLL:"YYYY年M月D日 dddd HH:mm",l:"YYYY/MM/DD",ll:"YYYY年M月D日",lll:"YYYY年M月D日 HH:mm",llll:"YYYY年M月D日(ddd) HH:mm"},meridiemParse:/午前|午後/i,isPM:function(t){return"午後"===t},meridiem:function(t,e,i){return t<12?"午前":"午後"},calendar:{sameDay:"[今日] LT",nextDay:"[明日] LT",nextWeek:function(t){return t.week()=20?"ste":"de")},week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){var e="styczeń_luty_marzec_kwiecień_maj_czerwiec_lipiec_sierpień_wrzesień_październik_listopad_grudzień".split("_"),i="stycznia_lutego_marca_kwietnia_maja_czerwca_lipca_sierpnia_września_października_listopada_grudnia".split("_");function n(t){return t%10<5&&t%10>1&&~~(t/10)%10!=1}function o(t,e,i){var o=t+" ";switch(i){case"ss":return o+(n(t)?"sekundy":"sekund");case"m":return e?"minuta":"minutę";case"mm":return o+(n(t)?"minuty":"minut");case"h":return e?"godzina":"godzinę";case"hh":return o+(n(t)?"godziny":"godzin");case"MM":return o+(n(t)?"miesiące":"miesięcy");case"yy":return o+(n(t)?"lata":"lat")}}t.defineLocale("pl",{months:function(t,n){return t?""===n?"("+i[t.month()]+"|"+e[t.month()]+")":/D MMMM/.test(n)?i[t.month()]:e[t.month()]:e},monthsShort:"sty_lut_mar_kwi_maj_cze_lip_sie_wrz_paź_lis_gru".split("_"),weekdays:"niedziela_poniedziałek_wtorek_środa_czwartek_piątek_sobota".split("_"),weekdaysShort:"ndz_pon_wt_śr_czw_pt_sob".split("_"),weekdaysMin:"Nd_Pn_Wt_Śr_Cz_Pt_So".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD.MM.YYYY",LL:"D MMMM YYYY",LLL:"D MMMM YYYY HH:mm",LLLL:"dddd, D MMMM YYYY HH:mm"},calendar:{sameDay:"[Dziś o] LT",nextDay:"[Jutro o] LT",nextWeek:function(){switch(this.day()){case 0:return"[W niedzielę o] LT";case 2:return"[We wtorek o] LT";case 3:return"[W środę o] LT";case 6:return"[W sobotę o] LT";default:return"[W] dddd [o] LT"}},lastDay:"[Wczoraj o] LT",lastWeek:function(){switch(this.day()){case 0:return"[W zeszłą niedzielę o] LT";case 3:return"[W zeszłą środę o] LT";case 6:return"[W zeszłą sobotę o] LT";default:return"[W zeszły] dddd [o] LT"}},sameElse:"L"},relativeTime:{future:"za %s",past:"%s temu",s:"kilka sekund",ss:o,m:o,mm:o,h:o,hh:o,d:"1 dzień",dd:"%d dni",M:"miesiąc",MM:o,y:"rok",yy:o},dayOfMonthOrdinalParse:/\d{1,2}\./,ordinal:"%d.",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){function e(t,e,i){var n,o;return"m"===i?e?"минута":"минуту":t+" "+(n=+t,o={ss:e?"секунда_секунды_секунд":"секунду_секунды_секунд",mm:e?"минута_минуты_минут":"минуту_минуты_минут",hh:"час_часа_часов",dd:"день_дня_дней",MM:"месяц_месяца_месяцев",yy:"год_года_лет"}[i].split("_"),n%10==1&&n%100!=11?o[0]:n%10>=2&&n%10<=4&&(n%100<10||n%100>=20)?o[1]:o[2])}var i=[/^янв/i,/^фев/i,/^мар/i,/^апр/i,/^ма[йя]/i,/^июн/i,/^июл/i,/^авг/i,/^сен/i,/^окт/i,/^ноя/i,/^дек/i];t.defineLocale("ru",{months:{format:"января_февраля_марта_апреля_мая_июня_июля_августа_сентября_октября_ноября_декабря".split("_"),standalone:"январь_февраль_март_апрель_май_июнь_июль_август_сентябрь_октябрь_ноябрь_декабрь".split("_")},monthsShort:{format:"янв._февр._мар._апр._мая_июня_июля_авг._сент._окт._нояб._дек.".split("_"),standalone:"янв._февр._март_апр._май_июнь_июль_авг._сент._окт._нояб._дек.".split("_")},weekdays:{standalone:"воскресенье_понедельник_вторник_среда_четверг_пятница_суббота".split("_"),format:"воскресенье_понедельник_вторник_среду_четверг_пятницу_субботу".split("_"),isFormat:/\[ ?[Вв] ?(?:прошлую|следующую|эту)? ?\] ?dddd/},weekdaysShort:"вс_пн_вт_ср_чт_пт_сб".split("_"),weekdaysMin:"вс_пн_вт_ср_чт_пт_сб".split("_"),monthsParse:i,longMonthsParse:i,shortMonthsParse:i,monthsRegex:/^(январ[ья]|янв\.?|феврал[ья]|февр?\.?|марта?|мар\.?|апрел[ья]|апр\.?|ма[йя]|июн[ья]|июн\.?|июл[ья]|июл\.?|августа?|авг\.?|сентябр[ья]|сент?\.?|октябр[ья]|окт\.?|ноябр[ья]|нояб?\.?|декабр[ья]|дек\.?)/i,monthsShortRegex:/^(январ[ья]|янв\.?|феврал[ья]|февр?\.?|марта?|мар\.?|апрел[ья]|апр\.?|ма[йя]|июн[ья]|июн\.?|июл[ья]|июл\.?|августа?|авг\.?|сентябр[ья]|сент?\.?|октябр[ья]|окт\.?|ноябр[ья]|нояб?\.?|декабр[ья]|дек\.?)/i,monthsStrictRegex:/^(январ[яь]|феврал[яь]|марта?|апрел[яь]|ма[яй]|июн[яь]|июл[яь]|августа?|сентябр[яь]|октябр[яь]|ноябр[яь]|декабр[яь])/i,monthsShortStrictRegex:/^(янв\.|февр?\.|мар[т.]|апр\.|ма[яй]|июн[ья.]|июл[ья.]|авг\.|сент?\.|окт\.|нояб?\.|дек\.)/i,longDateFormat:{LT:"H:mm",LTS:"H:mm:ss",L:"DD.MM.YYYY",LL:"D MMMM YYYY г.",LLL:"D MMMM YYYY г., H:mm",LLLL:"dddd, D MMMM YYYY г., H:mm"},calendar:{sameDay:"[Сегодня, в] LT",nextDay:"[Завтра, в] LT",lastDay:"[Вчера, в] LT",nextWeek:function(t){if(t.week()===this.week())return 2===this.day()?"[Во] dddd, [в] LT":"[В] dddd, [в] LT";switch(this.day()){case 0:return"[В следующее] dddd, [в] LT";case 1:case 2:case 4:return"[В следующий] dddd, [в] LT";case 3:case 5:case 6:return"[В следующую] dddd, [в] LT"}},lastWeek:function(t){if(t.week()===this.week())return 2===this.day()?"[Во] dddd, [в] LT":"[В] dddd, [в] LT";switch(this.day()){case 0:return"[В прошлое] dddd, [в] LT";case 1:case 2:case 4:return"[В прошлый] dddd, [в] LT";case 3:case 5:case 6:return"[В прошлую] dddd, [в] LT"}},sameElse:"L"},relativeTime:{future:"через %s",past:"%s назад",s:"несколько секунд",ss:e,m:e,mm:e,h:"час",hh:e,d:"день",dd:e,M:"месяц",MM:e,y:"год",yy:e},meridiemParse:/ночи|утра|дня|вечера/i,isPM:function(t){return/^(дня|вечера)$/.test(t)},meridiem:function(t,e,i){return t<4?"ночи":t<12?"утра":t<17?"дня":"вечера"},dayOfMonthOrdinalParse:/\d{1,2}-(й|го|я)/,ordinal:function(t,e){switch(e){case"M":case"d":case"DDD":return t+"-й";case"D":return t+"-го";case"w":case"W":return t+"-я";default:return t}},week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){function e(t,e,i){var n,o;return"m"===i?e?"хвилина":"хвилину":"h"===i?e?"година":"годину":t+" "+(n=+t,o={ss:e?"секунда_секунди_секунд":"секунду_секунди_секунд",mm:e?"хвилина_хвилини_хвилин":"хвилину_хвилини_хвилин",hh:e?"година_години_годин":"годину_години_годин",dd:"день_дні_днів",MM:"місяць_місяці_місяців",yy:"рік_роки_років"}[i].split("_"),n%10==1&&n%100!=11?o[0]:n%10>=2&&n%10<=4&&(n%100<10||n%100>=20)?o[1]:o[2])}function i(t){return function(){return t+"о"+(11===this.hours()?"б":"")+"] LT"}}t.defineLocale("uk",{months:{format:"січня_лютого_березня_квітня_травня_червня_липня_серпня_вересня_жовтня_листопада_грудня".split("_"),standalone:"січень_лютий_березень_квітень_травень_червень_липень_серпень_вересень_жовтень_листопад_грудень".split("_")},monthsShort:"січ_лют_бер_квіт_трав_черв_лип_серп_вер_жовт_лист_груд".split("_"),weekdays:function(t,e){var i={nominative:"неділя_понеділок_вівторок_середа_четвер_п’ятниця_субота".split("_"),accusative:"неділю_понеділок_вівторок_середу_четвер_п’ятницю_суботу".split("_"),genitive:"неділі_понеділка_вівторка_середи_четверга_п’ятниці_суботи".split("_")};return!0===t?i.nominative.slice(1,7).concat(i.nominative.slice(0,1)):t?i[/(\[[ВвУу]\]) ?dddd/.test(e)?"accusative":/\[?(?:минулої|наступної)? ?\] ?dddd/.test(e)?"genitive":"nominative"][t.day()]:i.nominative},weekdaysShort:"нд_пн_вт_ср_чт_пт_сб".split("_"),weekdaysMin:"нд_пн_вт_ср_чт_пт_сб".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD.MM.YYYY",LL:"D MMMM YYYY р.",LLL:"D MMMM YYYY р., HH:mm",LLLL:"dddd, D MMMM YYYY р., HH:mm"},calendar:{sameDay:i("[Сьогодні "),nextDay:i("[Завтра "),lastDay:i("[Вчора "),nextWeek:i("[У] dddd ["),lastWeek:function(){switch(this.day()){case 0:case 3:case 5:case 6:return i("[Минулої] dddd [").call(this);case 1:case 2:case 4:return i("[Минулого] dddd [").call(this)}},sameElse:"L"},relativeTime:{future:"за %s",past:"%s тому",s:"декілька секунд",ss:e,m:e,mm:e,h:"годину",hh:e,d:"день",dd:e,M:"місяць",MM:e,y:"рік",yy:e},meridiemParse:/ночі|ранку|дня|вечора/,isPM:function(t){return/^(дня|вечора)$/.test(t)},meridiem:function(t,e,i){return t<4?"ночі":t<12?"ранку":t<17?"дня":"вечора"},dayOfMonthOrdinalParse:/\d{1,2}-(й|го)/,ordinal:function(t,e){switch(e){case"M":case"d":case"DDD":case"w":case"W":return t+"-й";case"D":return t+"-го";default:return t}},week:{dow:1,doy:7}})})(s)})),function(t){return t&&t.Math==Math&&t}),a=r("object"==typeof globalThis&&globalThis)||r("object"==typeof window&&window)||r("object"==typeof self&&self)||r("object"==typeof e&&e)||Function("return this")(),l=function(t){try{return!!t()}catch(t){return!0}},h=!l((function(){return 7!=Object.defineProperty({},1,{get:function(){return 7}})[1]})),d={}.propertyIsEnumerable,u=Object.getOwnPropertyDescriptor,c={f:u&&!d.call({1:2},1)?function(t){var e=u(this,t);return!!e&&e.enumerable}:d},p=function(t,e){return{enumerable:!(1&t),configurable:!(2&t),writable:!(4&t),value:e}},m={}.toString,f=function(t){return m.call(t).slice(8,-1)},g="".split,v=l((function(){return!Object("z").propertyIsEnumerable(0)}))?function(t){return"String"==f(t)?g.call(t,""):Object(t)}:Object,b=function(t){if(null==t)throw TypeError("Can't call method on "+t);return t},y=function(t){return v(b(t))},x=function(t){return"object"==typeof t?null!==t:"function"==typeof t},w=function(t,e){if(!x(t))return t;var i,n;if(e&&"function"==typeof(i=t.toString)&&!x(n=i.call(t)))return n;if("function"==typeof(i=t.valueOf)&&!x(n=i.call(t)))return n;if(!e&&"function"==typeof(i=t.toString)&&!x(n=i.call(t)))return n;throw TypeError("Can't convert object to primitive value")},_={}.hasOwnProperty,k=function(t,e){return _.call(t,e)},D=a.document,I=x(D)&&x(D.createElement),C=function(t){return I?D.createElement(t):{}},L=!h&&!l((function(){return 7!=Object.defineProperty(C("div"),"a",{get:function(){return 7}}).a})),G=Object.getOwnPropertyDescriptor,M={f:h?G:function(t,e){if(t=y(t),e=w(e,!0),L)try{return G(t,e)}catch(t){}if(k(t,e))return p(!c.f.call(t,e),t[e])}},S=/#|\.prototype\./,Q=function(t,e){var i=F[N(t)];return i==B||i!=Z&&("function"==typeof e?l(e):!!e)},N=Q.normalize=function(t){return String(t).replace(S,".").toLowerCase()},F=Q.data={},Z=Q.NATIVE="N",B=Q.POLYFILL="P",O=Q,W={},A=function(t){if("function"!=typeof t)throw TypeError(String(t)+" is not a function");return t},X=function(t,e,i){if(A(t),void 0===e)return t;switch(i){case 0:return function(){return t.call(e)};case 1:return function(i){return t.call(e,i)};case 2:return function(i,n){return t.call(e,i,n)};case 3:return function(i,n,o){return t.call(e,i,n,o)}}return function(){return t.apply(e,arguments)}},T=function(t){if(!x(t))throw TypeError(String(t)+" is not an object");return t},Y=Object.defineProperty,E={f:h?Y:function(t,e,i){if(T(t),e=w(e,!0),T(i),L)try{return Y(t,e,i)}catch(t){}if("get"in i||"set"in i)throw TypeError("Accessors not supported");return"value"in i&&(t[e]=i.value),t}},R=h?function(t,e,i){return E.f(t,e,p(1,i))}:function(t,e,i){return t[e]=i,t},U=M.f,z=function(t){var e=function(e,i,n){if(this instanceof t){switch(arguments.length){case 0:return new t;case 1:return new t(e);case 2:return new t(e,i)}return new t(e,i,n)}return t.apply(this,arguments)};return e.prototype=t.prototype,e},V=function(t,e){var i,n,o,s,r,l,h,d,u=t.target,c=t.global,p=t.stat,m=t.proto,f=c?a:p?a[u]:(a[u]||{}).prototype,g=c?W:W[u]||(W[u]={}),v=g.prototype;for(o in e)i=!O(c?o:u+(p?".":"#")+o,t.forced)&&f&&k(f,o),r=g[o],i&&(l=t.noTargetGet?(d=U(f,o))&&d.value:f[o]),s=i&&l?l:e[o],i&&typeof r==typeof s||(h=t.bind&&i?X(s,a):t.wrap&&i?z(s):m&&"function"==typeof s?X(Function.call,s):s,(t.sham||s&&s.sham||r&&r.sham)&&R(h,"sham",!0),g[o]=h,m&&(k(W,n=u+"Prototype")||R(W,n,{}),W[n][o]=s,t.real&&v&&!v[o]&&R(v,o,s)))};V({target:"Object",stat:!0,forced:!h,sham:!h},{defineProperty:E.f});var j=n((function(t){var e=W.Object,i=t.exports=function(t,i,n){return e.defineProperty(t,i,n)};e.defineProperty.sham&&(i.sham=!0)})),H=j,P=Math.ceil,J=Math.floor,K=function(t){return isNaN(t=+t)?0:(t>0?J:P)(t)},q=Math.min,$=function(t){return t>0?q(K(t),9007199254740991):0},tt=Math.max,et=Math.min,it=function(t,e){var i=K(t);return i<0?tt(i+e,0):et(i,e)},nt=function(t){return function(e,i,n){var o,s=y(e),r=$(s.length),a=it(n,r);if(t&&i!=i){for(;r>a;)if((o=s[a++])!=o)return!0}else for(;r>a;a++)if((t||a in s)&&s[a]===i)return t||a||0;return!t&&-1}},ot={includes:nt(!0),indexOf:nt(!1)},st={},rt=ot.indexOf,at=function(t,e){var i,n=y(t),o=0,s=[];for(i in n)!k(st,i)&&k(n,i)&&s.push(i);for(;e.length>o;)k(n,i=e[o++])&&(~rt(s,i)||s.push(i));return s},lt=["constructor","hasOwnProperty","isPrototypeOf","propertyIsEnumerable","toLocaleString","toString","valueOf"],ht=Object.keys||function(t){return at(t,lt)},dt=h?Object.defineProperties:function(t,e){T(t);for(var i,n=ht(e),o=n.length,s=0;o>s;)E.f(t,i=n[s++],e[i]);return t};V({target:"Object",stat:!0,forced:!h,sham:!h},{defineProperties:dt});var ut=n((function(t){var e=W.Object,i=t.exports=function(t,i){return e.defineProperties(t,i)};e.defineProperties.sham&&(i.sham=!0)})),ct=function(t){return"function"==typeof t?t:void 0},pt=function(t,e){return arguments.length<2?ct(W[t])||ct(a[t]):W[t]&&W[t][e]||a[t]&&a[t][e]},mt=lt.concat("length","prototype"),ft={f:Object.getOwnPropertyNames||function(t){return at(t,mt)}},gt={f:Object.getOwnPropertySymbols},vt=pt("Reflect","ownKeys")||function(t){var e=ft.f(T(t)),i=gt.f;return i?e.concat(i(t)):e},bt=function(t,e,i){var n=w(e);n in t?E.f(t,n,p(0,i)):t[n]=i};V({target:"Object",stat:!0,sham:!h},{getOwnPropertyDescriptors:function(t){for(var e,i,n=y(t),o=M.f,s=vt(n),r={},a=0;s.length>a;)void 0!==(i=o(n,e=s[a++]))&&bt(r,e,i);return r}});var yt=W.Object.getOwnPropertyDescriptors,xt=M.f,wt=l((function(){xt(1)}));V({target:"Object",stat:!0,forced:!h||wt,sham:!h},{getOwnPropertyDescriptor:function(t,e){return xt(y(t),e)}});var _t,kt=n((function(t){var e=W.Object,i=t.exports=function(t,i){return e.getOwnPropertyDescriptor(t,i)};e.getOwnPropertyDescriptor.sham&&(i.sham=!0)})),Dt=!!Object.getOwnPropertySymbols&&!l((function(){return!String(Symbol())})),It=Dt&&!Symbol.sham&&"symbol"==typeof Symbol.iterator,Ct=Array.isArray||function(t){return"Array"==f(t)},Lt=function(t){return Object(b(t))},Gt=pt("document","documentElement"),Mt=a["__core-js_shared__"]||function(t,e){try{R(a,t,e)}catch(i){a[t]=e}return e}("__core-js_shared__",{}),St=n((function(t){(t.exports=function(t,e){return Mt[t]||(Mt[t]=void 0!==e?e:{})})("versions",[]).push({version:"3.6.4",mode:"pure",copyright:"© 2020 Denis Pushkarev (zloirock.ru)"})})),Qt=0,Nt=Math.random(),Ft=function(t){return"Symbol("+String(void 0===t?"":t)+")_"+(++Qt+Nt).toString(36)},Zt=St("keys"),Bt=function(t){return Zt[t]||(Zt[t]=Ft(t))},Ot=Bt("IE_PROTO"),Wt=function(){},At=function(t){return"