From 221b418b1c9db7b04c600b6300d18b034a4f444e Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Oct 2016 14:54:55 -0700 Subject: [PATCH 001/177] [SPARK-17778][TESTS] Mock SparkContext to reduce memory usage of BlockManagerSuite ## What changes were proposed in this pull request? Mock SparkContext to reduce memory usage of BlockManagerSuite ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15350 from zsxwing/SPARK-17778. --- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 1652fcdb964da..705c355234425 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -107,7 +107,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) - sc = new SparkContext("local", "test", conf) + // Mock SparkContext to reduce the memory usage of tests. It's fine since the only reason we + // need to create a SparkContext is to initialize LiveListenerBus. + sc = mock(classOf[SparkContext]) + when(sc.conf).thenReturn(conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus(sc))), conf, true) From 5fd54b994e2078dbf0794932b4e0ffa9a9eda0c3 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 5 Oct 2016 16:05:30 -0700 Subject: [PATCH 002/177] [SPARK-17758][SQL] Last returns wrong result in case of empty partition ## What changes were proposed in this pull request? The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order: ``` - Partition 1 [Row1, Row2] - Partition 2 [Row3] - Partition 3 [] ``` In this case the `Last` function will currently return a null, instead of the value of `Row3`. This PR fixes this by adding a `valueSet` flag to the `Last` function. ## How was this patch tested? We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function. Author: Herman van Hovell Closes #15348 from hvanhovell/SPARK-17758. --- .../catalyst/expressions/aggregate/Last.scala | 27 ++--- .../DeclarativeAggregateEvaluator.scala | 61 ++++++++++ .../expressions/aggregate/LastTestSuite.scala | 109 ++++++++++++++++++ 3 files changed, 184 insertions(+), 13 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index af8840305805f..8579f7292d3ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -55,34 +55,35 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) extends Declarat private lazy val last = AttributeReference("last", child.dataType)() - override lazy val aggBufferAttributes: Seq[AttributeReference] = last :: Nil + private lazy val valueSet = AttributeReference("valueSet", BooleanType)() + + override lazy val aggBufferAttributes: Seq[AttributeReference] = last :: valueSet :: Nil override lazy val initialValues: Seq[Literal] = Seq( - /* last = */ Literal.create(null, child.dataType) + /* last = */ Literal.create(null, child.dataType), + /* valueSet = */ Literal.create(false, BooleanType) ) override lazy val updateExpressions: Seq[Expression] = { if (ignoreNulls) { Seq( - /* last = */ If(IsNull(child), last, child) + /* last = */ If(IsNull(child), last, child), + /* valueSet = */ Or(valueSet, IsNotNull(child)) ) } else { Seq( - /* last = */ child + /* last = */ child, + /* valueSet = */ Literal.create(true, BooleanType) ) } } override lazy val mergeExpressions: Seq[Expression] = { - if (ignoreNulls) { - Seq( - /* last = */ If(IsNull(last.right), last.left, last.right) - ) - } else { - Seq( - /* last = */ last.right - ) - } + // Prefer the right hand expression if it has been set. + Seq( + /* last = */ If(valueSet.right, last.right, last.left), + /* valueSet = */ Or(valueSet.right, valueSet.left) + ) } override lazy val evaluateExpression: AttributeReference = last diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala new file mode 100644 index 0000000000000..614f24db0aafb --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection + +/** + * Evaluator for a [[DeclarativeAggregate]]. + */ +case class DeclarativeAggregateEvaluator(function: DeclarativeAggregate, input: Seq[Attribute]) { + + lazy val initializer = GenerateSafeProjection.generate(function.initialValues) + + lazy val updater = GenerateSafeProjection.generate( + function.updateExpressions, + function.aggBufferAttributes ++ input) + + lazy val merger = GenerateSafeProjection.generate( + function.mergeExpressions, + function.aggBufferAttributes ++ function.inputAggBufferAttributes) + + lazy val evaluator = GenerateSafeProjection.generate( + function.evaluateExpression :: Nil, + function.aggBufferAttributes) + + def initialize(): InternalRow = initializer.apply(InternalRow.empty).copy() + + def update(values: InternalRow*): InternalRow = { + val joiner = new JoinedRow + val buffer = values.foldLeft(initialize()) { (buffer, input) => + updater(joiner(buffer, input)) + } + buffer.copy() + } + + def merge(buffers: InternalRow*): InternalRow = { + val joiner = new JoinedRow + val buffer = buffers.foldLeft(initialize()) { (left, right) => + merger(joiner(left, right)) + } + buffer.copy() + } + + def eval(buffer: InternalRow): InternalRow = evaluator(buffer).copy() +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala new file mode 100644 index 0000000000000..ba36bc074e154 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Literal} +import org.apache.spark.sql.types.IntegerType + +class LastTestSuite extends SparkFunSuite { + val input = AttributeReference("input", IntegerType, nullable = true)() + val evaluator = DeclarativeAggregateEvaluator(Last(input, Literal(false)), Seq(input)) + val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(Last(input, Literal(true)), Seq(input)) + + test("empty buffer") { + assert(evaluator.initialize() === InternalRow(null, false)) + } + + test("update") { + val result = evaluator.update( + InternalRow(1), + InternalRow(9), + InternalRow(-1)) + assert(result === InternalRow(-1, true)) + } + + test("update - ignore nulls") { + val result1 = evaluatorIgnoreNulls.update( + InternalRow(null), + InternalRow(9), + InternalRow(null)) + assert(result1 === InternalRow(9, true)) + + val result2 = evaluatorIgnoreNulls.update( + InternalRow(null), + InternalRow(null)) + assert(result2 === InternalRow(null, false)) + } + + test("merge") { + // Empty merge + val p0 = evaluator.initialize() + assert(evaluator.merge(p0) === InternalRow(null, false)) + + // Single merge + val p1 = evaluator.update(InternalRow(1), InternalRow(-99)) + assert(evaluator.merge(p1) === p1) + + // Multiple merges. + val p2 = evaluator.update(InternalRow(2), InternalRow(10)) + assert(evaluator.merge(p1, p2) === p2) + + // Empty partitions (p0 is empty) + assert(evaluator.merge(p1, p0, p2) === p2) + assert(evaluator.merge(p2, p1, p0) === p1) + } + + test("merge - ignore nulls") { + // Multi merges + val p1 = evaluatorIgnoreNulls.update(InternalRow(1), InternalRow(null)) + val p2 = evaluatorIgnoreNulls.update(InternalRow(null), InternalRow(null)) + assert(evaluatorIgnoreNulls.merge(p1, p2) === p1) + } + + test("eval") { + // Null Eval + assert(evaluator.eval(InternalRow(null, true)) === InternalRow(null)) + assert(evaluator.eval(InternalRow(null, false)) === InternalRow(null)) + + // Empty Eval + val p0 = evaluator.initialize() + assert(evaluator.eval(p0) === InternalRow(null)) + + // Update - Eval + val p1 = evaluator.update(InternalRow(1), InternalRow(-99)) + assert(evaluator.eval(p1) === InternalRow(-99)) + + // Update - Merge - Eval + val p2 = evaluator.update(InternalRow(2), InternalRow(10)) + val m1 = evaluator.merge(p1, p0, p2) + assert(evaluator.eval(m1) === InternalRow(10)) + + // Update - Merge - Eval (empty partition at the end) + val m2 = evaluator.merge(p2, p1, p0) + assert(evaluator.eval(m2) === InternalRow(-99)) + } + + test("eval - ignore nulls") { + // Update - Merge - Eval + val p1 = evaluatorIgnoreNulls.update(InternalRow(1), InternalRow(null)) + val p2 = evaluatorIgnoreNulls.update(InternalRow(null), InternalRow(null)) + val m1 = evaluatorIgnoreNulls.merge(p1, p2) + assert(evaluatorIgnoreNulls.eval(m1) === InternalRow(1)) + } +} From 9293734d35eb3d6e4fd4ebb86f54dd5d3a35e6db Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Oct 2016 16:45:45 -0700 Subject: [PATCH 003/177] [SPARK-17346][SQL] Add Kafka source for Structured Streaming ## What changes were proposed in this pull request? This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source. It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing tdas did most of work and part of them was inspired by koeninger's work. ### Introduction The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows: Column | Type ---- | ---- key | binary value | binary topic | string partition | int offset | long timestamp | long timestampType | int The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic. ### Configuration The user can use `DataStreamReader.option` to set the following configurations. Kafka Source's options | value | default | meaning ------ | ------- | ------ | ----- startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off. failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected. subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets. fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")` ### Usage * Subscribe to 1 topic ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1") .load() ``` * Subscribe to multiple topics ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1,topic2") .load() ``` * Subscribe to a pattern ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribePattern", "topic.*") .load() ``` ## How was this patch tested? The new unit tests. Author: Shixiong Zhu Author: Tathagata Das Author: Shixiong Zhu Author: cody koeninger Closes #15102 from zsxwing/kafka-source. --- .../spark/util/UninterruptibleThread.scala | 7 - dev/run-tests.py | 2 +- dev/sparktestsupport/modules.py | 12 + .../structured-streaming-kafka-integration.md | 239 ++++++++++ .../structured-streaming-programming-guide.md | 7 +- external/kafka-0-10-sql/pom.xml | 82 ++++ ...pache.spark.sql.sources.DataSourceRegister | 1 + .../sql/kafka010/CachedKafkaConsumer.scala | 152 +++++++ .../spark/sql/kafka010/KafkaSource.scala | 399 ++++++++++++++++ .../sql/kafka010/KafkaSourceOffset.scala | 54 +++ .../sql/kafka010/KafkaSourceProvider.scala | 282 ++++++++++++ .../spark/sql/kafka010/KafkaSourceRDD.scala | 148 ++++++ .../spark/sql/kafka010/package-info.java | 21 + .../src/test/resources/log4j.properties | 28 ++ .../sql/kafka010/KafkaSourceOffsetSuite.scala | 39 ++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 424 ++++++++++++++++++ .../spark/sql/kafka010/KafkaTestUtils.scala | 339 ++++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 6 +- .../execution/streaming/StreamExecution.scala | 8 +- .../spark/sql/streaming/StreamTest.scala | 40 +- 21 files changed, 2268 insertions(+), 23 deletions(-) create mode 100644 docs/structured-streaming-kafka-integration.md create mode 100644 external/kafka-0-10-sql/pom.xml create mode 100644 external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java create mode 100644 external/kafka-0-10-sql/src/test/resources/log4j.properties create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala index 4dcf95177aa78..f0b68f0cb7e29 100644 --- a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala @@ -89,13 +89,6 @@ private[spark] class UninterruptibleThread(name: String) extends Thread(name) { } } - /** - * Tests whether `interrupt()` has been called. - */ - override def isInterrupted: Boolean = { - super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread } - } - /** * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be * interrupted until it enters into the interruptible status. diff --git a/dev/run-tests.py b/dev/run-tests.py index ae4b5306fc5cf..5d661f5f1a1c5 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -110,7 +110,7 @@ def determine_modules_to_test(changed_modules): ['graphx', 'examples'] >>> x = [x.name for x in determine_modules_to_test([modules.sql])] >>> x # doctest: +NORMALIZE_WHITESPACE - ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver', + ['sql', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver', 'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] """ modules_to_test = set() diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 050cdf043757f..5f14683d9a52f 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -158,6 +158,18 @@ def __hash__(self): ) +sql_kafka = Module( + name="sql-kafka-0-10", + dependencies=[sql], + source_file_regexes=[ + "external/kafka-0-10-sql", + ], + sbt_test_goals=[ + "sql-kafka-0-10/test", + ] +) + + sketch = Module( name="sketch", dependencies=[tags], diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md new file mode 100644 index 0000000000000..668489addf82c --- /dev/null +++ b/docs/structured-streaming-kafka-integration.md @@ -0,0 +1,239 @@ +--- +layout: global +title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) +--- + +Structured Streaming integration for Kafka 0.10 to poll data from Kafka. + +### Linking +For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact: + + groupId = org.apache.spark + artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +For Python applications, you need to add this above library and its dependencies when deploying your +application. See the [Deploying](#deploying) subsection below. + +### Creating a Kafka Source Stream + +
+
+ + // Subscribe to 1 topic + val ds1 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() + ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + // Subscribe to multiple topics + val ds2 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() + ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + // Subscribe to a pattern + val ds3 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() + ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +
+
+ + // Subscribe to 1 topic + Dataset ds1 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() + ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + // Subscribe to multiple topics + Dataset ds2 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() + ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + // Subscribe to a pattern + Dataset ds3 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() + ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +
+
+ + # Subscribe to 1 topic + ds1 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() + ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + # Subscribe to multiple topics + ds2 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() + ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + # Subscribe to a pattern + ds3 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() + ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +
+
+ +Each row in the source has the following schema: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ColumnType
keybinary
valuebinary
topicstring
partitionint
offsetlong
timestamplong
timestampTypeint
+ +The following options must be set for the Kafka source. + + + + + + + + + + + + + + + + + + +
Optionvaluemeaning
subscribeA comma-separated list of topicsThe topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be + specified for Kafka source.
subscribePatternJava regex stringThe pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" + options can be specified for Kafka source.
kafka.bootstrap.serversA comma-separated list of host:portThe Kafka "bootstrap.servers" configuration.
+ +The following configurations are optional: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Optionvaluedefaultmeaning
startingOffset["earliest", "latest"]"latest"The start point when a query is started, either "earliest" which is from the earliest offset, + or "latest" which is just from the latest offset. Note: This only applies when a new Streaming q + uery is started, and that resuming will always pick up from where the query left off.
failOnDataLoss[true, false]trueWhether to fail the query when it's possible that data is lost (e.g., topics are deleted, or + offsets are out of range). This may be a false alarm. You can disable it when it doesn't work + as you expected.
kafkaConsumer.pollTimeoutMslong512The timeout in milliseconds to poll data from Kafka in executors.
fetchOffset.numRetriesint3Number of times to retry before giving up fatch Kafka latest offsets.
fetchOffset.retryIntervalMslong10milliseconds to wait before retrying to fetch Kafka offsets
+ +Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, +`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafkaParams, see +[Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). + +Note that the following Kafka params cannot be set and the Kafka source will throw an exception: +- **group.id**: Kafka source will create a unique group id for each query automatically. +- **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify + where to start instead. Structured Streaming manages which offsets are consumed internally, rather + than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new + topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new + Streaming query is started, and that resuming will always pick up from where the query left off. +- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use + DataFrame operations to explicitly deserialize the keys. +- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. + Use DataFrame operations to explicitly deserialize the values. +- **enable.auto.commit**: Kafka source doesn't commit any offset. +- **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to + use ConsumerInterceptor as it may break the query. + +### Deploying + +As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` +and its dependencies can be directly added to `spark-submit` using `--packages`, such as, + + ./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + +See [Application Submission Guide](submitting-applications.html) for more details about submitting +applications with external dependencies. diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 2e6df94823d38..173fd6e8c73b9 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -418,10 +418,15 @@ Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as Streaming DataFrames can be created through the `DataStreamReader` interface ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/ [Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. + +#### Data Sources +In Spark 2.0, there are a few built-in sources. - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. + - **Kafka source** - Poll data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. + - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. Here are some examples. diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml new file mode 100644 index 0000000000000..b96445a11f858 --- /dev/null +++ b/external/kafka-0-10-sql/pom.xml @@ -0,0 +1,82 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-sql-kafka-0-10_2.11 + + sql-kafka-0-10 + + jar + Kafka 0.10 Source for Structured Streaming + http://spark.apache.org/ + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka-clients + 0.10.0.1 + + + org.apache.kafka + kafka_${scala.binary.version} + 0.10.0.1 + test + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister new file mode 100644 index 0000000000000..2f9e9fc0396d5 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -0,0 +1 @@ +org.apache.spark.sql.kafka010.KafkaSourceProvider diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala new file mode 100644 index 0000000000000..3b5a96534f9b6 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} + +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{SparkEnv, SparkException, TaskContext} +import org.apache.spark.internal.Logging + + +/** + * Consumer of single topicpartition, intended for cached reuse. + * Underlying consumer is not threadsafe, so neither is this, + * but processing the same topicpartition and group id in multiple threads is usually bad anyway. + */ +private[kafka010] case class CachedKafkaConsumer private( + topicPartition: TopicPartition, + kafkaParams: ju.Map[String, Object]) extends Logging { + + private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + + private val consumer = { + val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + val tps = new ju.ArrayList[TopicPartition]() + tps.add(topicPartition) + c.assign(tps) + c + } + + /** Iterator to the already fetch data */ + private var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] + private var nextOffsetInFetchedData = -2L + + /** + * Get the record for the given offset, waiting up to timeout ms if IO is necessary. + * Sequential forward access will use buffers, but random access will be horribly inefficient. + */ + def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[Array[Byte], Array[Byte]] = { + logDebug(s"Get $groupId $topicPartition nextOffset $nextOffsetInFetchedData requested $offset") + if (offset != nextOffsetInFetchedData) { + logInfo(s"Initial fetch for $topicPartition $offset") + seek(offset) + poll(pollTimeoutMs) + } + + if (!fetchedData.hasNext()) { poll(pollTimeoutMs) } + assert(fetchedData.hasNext(), + s"Failed to get records for $groupId $topicPartition $offset " + + s"after polling for $pollTimeoutMs") + var record = fetchedData.next() + + if (record.offset != offset) { + logInfo(s"Buffer miss for $groupId $topicPartition $offset") + seek(offset) + poll(pollTimeoutMs) + assert(fetchedData.hasNext(), + s"Failed to get records for $groupId $topicPartition $offset " + + s"after polling for $pollTimeoutMs") + record = fetchedData.next() + assert(record.offset == offset, + s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset") + } + + nextOffsetInFetchedData = offset + 1 + record + } + + private def close(): Unit = consumer.close() + + private def seek(offset: Long): Unit = { + logDebug(s"Seeking to $groupId $topicPartition $offset") + consumer.seek(topicPartition, offset) + } + + private def poll(pollTimeoutMs: Long): Unit = { + val p = consumer.poll(pollTimeoutMs) + val r = p.records(topicPartition) + logDebug(s"Polled $groupId ${p.partitions()} ${r.size}") + fetchedData = r.iterator + } +} + +private[kafka010] object CachedKafkaConsumer extends Logging { + + private case class CacheKey(groupId: String, topicPartition: TopicPartition) + + private lazy val cache = { + val conf = SparkEnv.get.conf + val capacity = conf.getInt("spark.sql.kafkaConsumerCache.capacity", 64) + new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer](capacity, 0.75f, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = { + if (this.size > capacity) { + logWarning(s"KafkaConsumer cache hitting max capacity of $capacity, " + + s"removing consumer for ${entry.getKey}") + try { + entry.getValue.close() + } catch { + case e: SparkException => + logError(s"Error closing earliest Kafka consumer for ${entry.getKey}", e) + } + true + } else { + false + } + } + } + } + + /** + * Get a cached consumer for groupId, assigned to topic and partition. + * If matching consumer doesn't already exist, will be created using kafkaParams. + */ + def getOrCreate( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = synchronized { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + // If this is reattempt at running the task, then invalidate cache and start with + // a new consumer + if (TaskContext.get != null && TaskContext.get.attemptNumber > 1) { + cache.remove(key) + new CachedKafkaConsumer(topicPartition, kafkaParams) + } else { + if (!cache.containsKey(key)) { + cache.put(key, new CachedKafkaConsumer(topicPartition, kafkaParams)) + } + cache.get(key) + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala new file mode 100644 index 0000000000000..1be70db87497e --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -0,0 +1,399 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer} +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.kafka010.KafkaSource._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.UninterruptibleThread + +/** + * A [[Source]] that uses Kafka's own [[KafkaConsumer]] API to reads data from Kafka. The design + * for this source is as follows. + * + * - The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains + * a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For + * example if the last record in a Kafka topic "t", partition 2 is offset 5, then + * KafkaSourceOffset will contain TopicPartition("t", 2) -> 6. This is done keep it consistent + * with the semantics of `KafkaConsumer.position()`. + * + * - The [[ConsumerStrategy]] class defines which Kafka topics and partitions should be read + * by this source. These strategies directly correspond to the different consumption options + * in . This class is designed to return a configured [[KafkaConsumer]] that is used by the + * [[KafkaSource]] to query for the offsets. See the docs on + * [[org.apache.spark.sql.kafka010.KafkaSource.ConsumerStrategy]] for more details. + * + * - The [[KafkaSource]] written to do the following. + * + * - As soon as the source is created, the pre-configured KafkaConsumer returned by the + * [[ConsumerStrategy]] is used to query the initial offsets that this source should + * start reading from. This used to create the first batch. + * + * - `getOffset()` uses the KafkaConsumer to query the latest available offsets, which are + * returned as a [[KafkaSourceOffset]]. + * + * - `getBatch()` returns a DF that reads from the 'start offset' until the 'end offset' in + * for each partition. The end offset is excluded to be consistent with the semantics of + * [[KafkaSourceOffset]] and `KafkaConsumer.position()`. + * + * - The DF returned is based on [[KafkaSourceRDD]] which is constructed such that the + * data from Kafka topic + partition is consistently read by the same executors across + * batches, and cached KafkaConsumers in the executors can be reused efficiently. See the + * docs on [[KafkaSourceRDD]] for more details. + * + * Zero data lost is not guaranteed when topics are deleted. If zero data lost is critical, the user + * must make sure all messages in a topic have been processed when deleting a topic. + * + * There is a known issue caused by KAFKA-1894: the query using KafkaSource maybe cannot be stopped. + * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers + * and not use wrong broker addresses. + */ +private[kafka010] case class KafkaSource( + sqlContext: SQLContext, + consumerStrategy: ConsumerStrategy, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + metadataPath: String, + failOnDataLoss: Boolean) + extends Source with Logging { + + private val sc = sqlContext.sparkContext + + private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong + + private val maxOffsetFetchAttempts = + sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt + + private val offsetFetchAttemptIntervalMs = + sourceOptions.getOrElse("fetchOffset.retryIntervalMs", "10").toLong + + /** + * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the + * offsets and never commits them. + */ + private val consumer = consumerStrategy.createConsumer() + + /** + * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only + * called in StreamExecutionThread. Otherwise, interrupting a thread while running + * `KafkaConsumer.poll` may hang forever (KAFKA-1894). + */ + private lazy val initialPartitionOffsets = { + val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) + metadataLog.get(0).getOrElse { + val offsets = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = false)) + metadataLog.add(0, offsets) + logInfo(s"Initial offsets: $offsets") + offsets + }.partitionToOffsets + } + + override def schema: StructType = KafkaSource.kafkaSchema + + /** Returns the maximum available offset for this source. */ + override def getOffset: Option[Offset] = { + // Make sure initialPartitionOffsets is initialized + initialPartitionOffsets + + val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = true)) + logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") + Some(offset) + } + + /** + * Returns the data that is between the offsets + * [`start.get.partitionToOffsets`, `end.partitionToOffsets`), i.e. end.partitionToOffsets is + * exclusive. + */ + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + // Make sure initialPartitionOffsets is initialized + initialPartitionOffsets + + logInfo(s"GetBatch called with start = $start, end = $end") + val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) + val fromPartitionOffsets = start match { + case Some(prevBatchEndOffset) => + KafkaSourceOffset.getPartitionOffsets(prevBatchEndOffset) + case None => + initialPartitionOffsets + } + + // Find the new partitions, and get their earliest offsets + val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) + val newPartitionOffsets = if (newPartitions.nonEmpty) { + fetchNewPartitionEarliestOffsets(newPartitions.toSeq) + } else { + Map.empty[TopicPartition, Long] + } + if (newPartitionOffsets.keySet != newPartitions) { + // We cannot get from offsets for some partitions. It means they got deleted. + val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) + reportDataLoss( + s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") + } + logInfo(s"Partitions added: $newPartitionOffsets") + newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => + reportDataLoss( + s"Added partition $p starts from $o instead of 0. Some data may have been missed") + } + + val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) + if (deletedPartitions.nonEmpty) { + reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + } + + // Use the until partitions to calculate offset ranges to ignore partitions that have + // been deleted + val topicPartitions = untilPartitionOffsets.keySet.filter { tp => + // Ignore partitions that we don't know the from offsets. + newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp) + }.toSeq + logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) + + val sortedExecutors = getSortedExecutorList(sc) + val numExecutors = sortedExecutors.length + logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) + + // Calculate offset ranges + val offsetRanges = topicPartitions.map { tp => + val fromOffset = fromPartitionOffsets.get(tp).getOrElse { + newPartitionOffsets.getOrElse(tp, { + // This should not happen since newPartitionOffsets contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset") + }) + } + val untilOffset = untilPartitionOffsets(tp) + val preferredLoc = if (numExecutors > 0) { + // This allows cached KafkaConsumers in the executors to be re-used to read the same + // partition in every batch. + Some(sortedExecutors(floorMod(tp.hashCode, numExecutors))) + } else None + KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) + }.filter { range => + if (range.untilOffset < range.fromOffset) { + reportDataLoss(s"Partition ${range.topicPartition}'s offset was changed from " + + s"${range.fromOffset} to ${range.untilOffset}, some data may have been missed") + false + } else { + true + } + }.toArray + + // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. + val rdd = new KafkaSourceRDD( + sc, executorKafkaParams, offsetRanges, pollTimeoutMs).map { cr => + Row(cr.key, cr.value, cr.topic, cr.partition, cr.offset, cr.timestamp, cr.timestampType.id) + } + + logInfo("GetBatch generating RDD of offset range: " + + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + sqlContext.createDataFrame(rdd, schema) + } + + /** Stop this source and free any resources it has allocated. */ + override def stop(): Unit = synchronized { + consumer.close() + } + + override def toString(): String = s"KafkaSource[$consumerStrategy]" + + /** + * Fetch the offset of a partition, either seek to the latest offsets or use the current offsets + * in the consumer. + */ + private def fetchPartitionOffsets( + seekToEnd: Boolean): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitioned assigned to consumer: $partitions") + + // Get the current or latest offset of each partition + if (seekToEnd) { + consumer.seekToEnd(partitions) + logDebug("Seeked to the end") + } + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got offsets for partition : $partitionOffsets") + partitionOffsets + } + + /** + * Fetch the earliest offsets for newly discovered partitions. The return result may not contain + * some partitions if they are deleted. + */ + private def fetchNewPartitionEarliestOffsets( + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + logDebug(s"\tPartitioned assigned to consumer: $partitions") + + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionToOffsets = newPartitions.filter { p => + // When deleting topics happen at the same time, some partitions may not be in `partitions`. + // So we need to ignore them + partitions.contains(p) + }.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got offsets for new partitions: $partitionToOffsets") + partitionToOffsets + } + + /** + * Helper function that does multiple retries on the a body of code that returns offsets. + * Retries are needed to handle transient failures. For e.g. race conditions between getting + * assignment and getting position while topics/partitions are deleted can cause NPEs. + * + * This method also makes sure `body` won't be interrupted to workaround a potential issue in + * `KafkaConsumer.poll`. (KAFKA-1894) + */ + private def withRetriesWithoutInterrupt( + body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + synchronized { + var result: Option[Map[TopicPartition, Long]] = None + var attempt = 1 + var lastException: Throwable = null + while (result.isEmpty && attempt <= maxOffsetFetchAttempts + && !Thread.currentThread().isInterrupted) { + Thread.currentThread match { + case ut: UninterruptibleThread => + // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query + // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it. + // + // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may + // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the + // issue. + ut.runUninterruptibly { + try { + result = Some(body) + } catch { + case NonFatal(e) => + lastException = e + logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) + attempt += 1 + Thread.sleep(offsetFetchAttemptIntervalMs) + } + } + case _ => + throw new IllegalStateException( + "Kafka APIs must be executed on a o.a.spark.util.UninterruptibleThread") + } + } + if (Thread.interrupted()) { + throw new InterruptedException() + } + if (result.isEmpty) { + assert(attempt > maxOffsetFetchAttempts) + assert(lastException != null) + throw lastException + } + result.get + } + } + + /** + * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. + * Otherwise, just log a warning. + */ + private def reportDataLoss(message: String): Unit = { + if (failOnDataLoss) { + throw new IllegalStateException(message + + ". Set the source option 'failOnDataLoss' to 'false' if you want to ignore these checks.") + } else { + logWarning(message) + } + } +} + + +/** Companion object for the [[KafkaSource]]. */ +private[kafka010] object KafkaSource { + + def kafkaSchema: StructType = StructType(Seq( + StructField("key", BinaryType), + StructField("value", BinaryType), + StructField("topic", StringType), + StructField("partition", IntegerType), + StructField("offset", LongType), + StructField("timestamp", LongType), + StructField("timestampType", IntegerType) + )) + + sealed trait ConsumerStrategy { + def createConsumer(): Consumer[Array[Byte], Array[Byte]] + } + + case class SubscribeStrategy(topics: Seq[String], kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe(topics.asJava) + consumer + } + + override def toString: String = s"Subscribe[${topics.mkString(", ")}]" + } + + case class SubscribePatternStrategy( + topicPattern: String, kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe( + ju.regex.Pattern.compile(topicPattern), + new NoOpConsumerRebalanceListener()) + consumer + } + + override def toString: String = s"SubscribePattern[$topicPattern]" + } + + private def getSortedExecutorList(sc: SparkContext): Array[String] = { + val bm = sc.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compare) + .map(_.toString) + } + + private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } + } + + private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala new file mode 100644 index 0000000000000..b5ade982515f0 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.sql.execution.streaming.Offset + +/** + * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and + * their offsets. + */ +private[kafka010] +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { + override def toString(): String = { + partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") + } +} + +/** Companion object of the [[KafkaSourceOffset]] */ +private[kafka010] object KafkaSourceOffset { + + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + offset match { + case o: KafkaSourceOffset => o.partitionToOffsets + case _ => + throw new IllegalArgumentException( + s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") + } + } + + /** + * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) + * tuples. + */ + def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { + KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala new file mode 100644 index 0000000000000..1b0a2fe955d03 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -0,0 +1,282 @@ +/* + * 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.kafka010 + +import java.{util => ju} +import java.util.UUID + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.serialization.ByteArrayDeserializer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.execution.streaming.Source +import org.apache.spark.sql.kafka010.KafkaSource._ +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} +import org.apache.spark.sql.types.StructType + +/** + * The provider class for the [[KafkaSource]]. This provider is designed such that it throws + * IllegalArgumentException when the Kafka Dataset is created, so that it can catch + * missing options even before the query is started. + */ +private[kafka010] class KafkaSourceProvider extends StreamSourceProvider + with DataSourceRegister with Logging { + + import KafkaSourceProvider._ + + /** + * Returns the name and schema of the source. In addition, it also verifies whether the options + * are correct and sufficient to create the [[KafkaSource]] when the query is started. + */ + override def sourceSchema( + sqlContext: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") + validateOptions(parameters) + ("kafka", KafkaSource.kafkaSchema) + } + + override def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + validateOptions(parameters) + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase.startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + + val deserClassName = classOf[ByteArrayDeserializer].getName + // Each running query should use its own group id. Otherwise, the query may be only assigned + // partial data since Kafka will assign partitions to multiple consumers having the same group + // id. Hence, we should generate a unique id for each query. + val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" + + val autoOffsetResetValue = caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { + case Some(value) => value.trim() // same values as those supported by auto.offset.reset + case None => "latest" + } + + val kafkaParamsForStrategy = + ConfigUpdater("source", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // So that consumers in Kafka source do not mess with any existing group id + .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver") + + // So that consumers can start from earliest or latest + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetResetValue) + + // So that consumers in the driver does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // So that the driver does not pull too much data + .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1)) + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val kafkaParamsForExecutors = + ConfigUpdater("executor", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // Make sure executors do only what the driver tells them. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // So that consumers in executors do not mess with any existing group id + .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") + + // So that consumers in executors does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("subscribe", value) => + SubscribeStrategy( + value.split(",").map(_.trim()).filter(_.nonEmpty), + kafkaParamsForStrategy) + case ("subscribepattern", value) => + SubscribePatternStrategy( + value.trim(), + kafkaParamsForStrategy) + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + val failOnDataLoss = + caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean + + new KafkaSource( + sqlContext, + strategy, + kafkaParamsForExecutors, + parameters, + metadataPath, + failOnDataLoss) + } + + private def validateOptions(parameters: Map[String, String]): Unit = { + + // Validate source options + + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val specifiedStrategies = + caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq + if (specifiedStrategies.isEmpty) { + throw new IllegalArgumentException( + "One of the following options must be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } else if (specifiedStrategies.size > 1) { + throw new IllegalArgumentException( + "Only one of the following options can be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } + + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("subscribe", value) => + val topics = value.split(",").map(_.trim).filter(_.nonEmpty) + if (topics.isEmpty) { + throw new IllegalArgumentException( + "No topics to subscribe to as specified value for option " + + s"'subscribe' is '$value'") + } + case ("subscribepattern", value) => + val pattern = caseInsensitiveParams("subscribepattern").trim() + if (pattern.isEmpty) { + throw new IllegalArgumentException( + "Pattern to subscribe is empty as specified value for option " + + s"'subscribePattern' is '$value'") + } + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { + case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) => + throw new IllegalArgumentException( + s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " + + s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}") + case _ => + } + + // Validate user-specified Kafka options + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + + s"user-specified consumer groups is not used to track offsets.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { + throw new IllegalArgumentException( + s""" + |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. + |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to + |specify where to start. Structured Streaming manages which offsets are consumed + |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no + |data is missed when when new topics/partitions are dynamically subscribed. Note that + |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and + |that resuming will always pick up from where the query left off. See the docs for more + |details. + """.stripMargin) + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " + + "to explicitly deserialize the keys.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " + + "operations to explicitly deserialize the values.") + } + + val otherUnsupportedConfigs = Seq( + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source + ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe + + otherUnsupportedConfigs.foreach { c => + if (caseInsensitiveParams.contains(s"kafka.$c")) { + throw new IllegalArgumentException(s"Kafka option '$c' is not supported") + } + } + + if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { + throw new IllegalArgumentException( + s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " + + s"configuring Kafka consumer") + } + } + + override def shortName(): String = "kafka" + + /** Class to conveniently update Kafka config params, while logging the changes */ + private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + + def set(key: String, value: Object): this.type = { + map.put(key, value) + logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") + this + } + + def setIfUnset(key: String, value: Object): ConfigUpdater = { + if (!map.containsKey(key)) { + map.put(key, value) + logInfo(s"$module: Set $key to $value") + } + this + } + + def build(): ju.Map[String, Object] = map + } +} + +private[kafka010] object KafkaSourceProvider { + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") + private val STARTING_OFFSET_OPTION_KEY = "startingoffset" + private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") + private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala new file mode 100644 index 0000000000000..496af7e39abab --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + + +/** Offset range that one partition of the KafkaSourceRDD has to read */ +private[kafka010] case class KafkaSourceRDDOffsetRange( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long, + preferredLoc: Option[String]) { + def topic: String = topicPartition.topic + def partition: Int = topicPartition.partition + def size: Long = untilOffset - fromOffset +} + + +/** Partition of the KafkaSourceRDD */ +private[kafka010] case class KafkaSourceRDDPartition( + index: Int, offsetRange: KafkaSourceRDDOffsetRange) extends Partition + + +/** + * An RDD that reads data from Kafka based on offset ranges across multiple partitions. + * Additionally, it allows preferred locations to be set for each topic + partition, so that + * the [[KafkaSource]] can ensure the same executor always reads the same topic + partition + * and cached KafkaConsuemrs (see [[CachedKafkaConsumer]] can be used read data efficiently. + * + * @param sc the [[SparkContext]] + * @param executorKafkaParams Kafka configuration for creating KafkaConsumer on the executors + * @param offsetRanges Offset ranges that define the Kafka data belonging to this RDD + */ +private[kafka010] class KafkaSourceRDD( + sc: SparkContext, + executorKafkaParams: ju.Map[String, Object], + offsetRanges: Seq[KafkaSourceRDDOffsetRange], + pollTimeoutMs: Long) + extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { + + override def persist(newLevel: StorageLevel): this.type = { + logError("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => new KafkaSourceRDDPartition(i, o) }.toArray + } + + override def count(): Long = offsetRanges.map(_.size).sum + + override def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[ConsumerRecord[Array[Byte], Array[Byte]]] = { + val nonEmptyPartitions = + this.partitions.map(_.asInstanceOf[KafkaSourceRDDPartition]).filter(_.offsetRange.size > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[ConsumerRecord[Array[Byte], Array[Byte]]](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.offsetRange.size) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]) => + it.take(parts(tc.partitionId)).toArray, parts.keys.toArray + ) + res.foreach(buf ++= _) + buf.toArray + } + + override def compute( + thePart: Partition, + context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { + val range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange + assert( + range.fromOffset <= range.untilOffset, + s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " + + s"for topic ${range.topic} partition ${range.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged") + if (range.fromOffset == range.untilOffset) { + logInfo(s"Beginning offset ${range.fromOffset} is the same as ending offset " + + s"skipping ${range.topic} ${range.partition}") + Iterator.empty + + } else { + + val consumer = CachedKafkaConsumer.getOrCreate( + range.topic, range.partition, executorKafkaParams) + var requestOffset = range.fromOffset + + logDebug(s"Creating iterator for $range") + + new Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { + override def hasNext(): Boolean = requestOffset < range.untilOffset + override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { + assert(hasNext(), "Can't call next() once untilOffset has been reached") + val r = consumer.get(requestOffset, pollTimeoutMs) + requestOffset += 1 + r + } + } + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java new file mode 100644 index 0000000000000..596f775c56dbc --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ + +/** + * Structured Streaming Data Source for Kafka 0.10 + */ +package org.apache.spark.sql.kafka010; diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..75e3b53a093f6 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark-project.jetty=WARN + diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala new file mode 100644 index 0000000000000..7056a41b1751e --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.spark.sql.streaming.OffsetSuite + +class KafkaSourceOffsetSuite extends OffsetSuite { + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("t", 1, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("T", 0, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("T", 0, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala new file mode 100644 index 0000000000000..64bf503058027 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -0,0 +1,424 @@ +/* + * 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.kafka010 + +import java.util.concurrent.atomic.AtomicInteger + +import scala.util.Random + +import org.apache.kafka.clients.producer.RecordMetadata +import org.scalatest.BeforeAndAfter +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.test.SharedSQLContext + + +abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { + + protected var testUtils: KafkaTestUtils = _ + + override val streamingTimeout = 30.seconds + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + protected def makeSureGetOffsetCalled = AssertOnQuery { q => + // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure + // its "getOffset" is called before pushing any data. Otherwise, because of the race contion, + // we don't know which data should be fetched when `startingOffset` is latest. + q.processAllAvailable() + true + } + + /** + * Add data to Kafka. + * + * `topicAction` can be used to run actions for each topic before inserting data. + */ + case class AddKafkaData(topics: Set[String], data: Int*) + (implicit ensureDataInMultiplePartition: Boolean = false, + concurrent: Boolean = false, + message: String = "", + topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends AddData { + + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + if (query.get.isActive) { + // Make sure no Spark job is running when deleting a topic + query.get.processAllAvailable() + } + + val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap + val newTopics = topics.diff(existingTopics.keySet) + for (newTopic <- newTopics) { + topicAction(newTopic, None) + } + for (existingTopicPartitions <- existingTopics) { + topicAction(existingTopicPartitions._1, Some(existingTopicPartitions._2)) + } + + // Read all topics again in case some topics are delete. + val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys + require( + query.nonEmpty, + "Cannot add data when there is no query for finding the active kafka source") + + val sources = query.get.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[KafkaSource] => + source.asInstanceOf[KafkaSource] + } + if (sources.isEmpty) { + throw new Exception( + "Could not find Kafka source in the StreamExecution logical plan to add data to") + } else if (sources.size > 1) { + throw new Exception( + "Could not select the Kafka source in the StreamExecution logical plan as there" + + "are multiple Kafka sources:\n\t" + sources.mkString("\n\t")) + } + val kafkaSource = sources.head + val topic = topics.toSeq(Random.nextInt(topics.size)) + val sentMetadata = testUtils.sendMessages(topic, data.map { _.toString }.toArray) + + def metadataToStr(m: (String, RecordMetadata)): String = { + s"Sent ${m._1} to partition ${m._2.partition()}, offset ${m._2.offset()}" + } + // Verify that the test data gets inserted into multiple partitions + if (ensureDataInMultiplePartition) { + require( + sentMetadata.groupBy(_._2.partition).size > 1, + s"Added data does not test multiple partitions: ${sentMetadata.map(metadataToStr)}") + } + + val offset = KafkaSourceOffset(testUtils.getLatestOffsets(topics)) + logInfo(s"Added data, expected offset $offset") + (kafkaSource, offset) + } + + override def toString: String = + s"AddKafkaData(topics = $topics, data = $data, message = $message)" + } +} + + +class KafkaSourceSuite extends KafkaSourceTest { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + + test("cannot stop Kafka stream") { + val topic = newTopic() + testUtils.createTopic(newTopic(), partitions = 5) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"topic-.*") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + StopStream + ) + } + + test("subscribing topic by name from latest offsets") { + val topic = newTopic() + testFromLatestOffsets(topic, "subscribe" -> topic) + } + + test("subscribing topic by name from earliest offsets") { + val topic = newTopic() + testFromEarliestOffsets(topic, "subscribe" -> topic) + } + + test("subscribing topic by pattern from latest offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromLatestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("subscribing topic by pattern from earliest offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromEarliestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("subscribing topic by pattern with topic deletions") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-seems" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"$topicPrefix-.*") + .option("failOnDataLoss", "false") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + Assert { + testUtils.deleteTopic(topic) + testUtils.createTopic(topic2, partitions = 5) + true + }, + AddKafkaData(Set(topic2), 4, 5, 6), + CheckAnswer(2, 3, 4, 5, 6, 7) + ) + } + + test("bad source options") { + def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + options.foreach { case (k, v) => reader.option(k, v) } + reader.load() + } + expectedMsgs.foreach { m => + assert(ex.getMessage.toLowerCase.contains(m.toLowerCase)) + } + } + + // No strategy specified + testBadOptions()("options must be specified", "subscribe", "subscribePattern") + + // Multiple strategies specified + testBadOptions("subscribe" -> "t", "subscribePattern" -> "t.*")( + "only one", "options can be specified") + + testBadOptions("subscribe" -> "")("no topics to subscribe") + testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") + } + + test("unsupported kafka configs") { + def testUnsupportedConfig(key: String, value: String = "someValue"): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + .option("subscribe", "topic") + .option("kafka.bootstrap.servers", "somehost") + .option(s"$key", value) + reader.load() + } + assert(ex.getMessage.toLowerCase.contains("not supported")) + } + + testUnsupportedConfig("kafka.group.id") + testUnsupportedConfig("kafka.auto.offset.reset") + testUnsupportedConfig("kafka.enable.auto.commit") + testUnsupportedConfig("kafka.interceptor.classes") + testUnsupportedConfig("kafka.key.deserializer") + testUnsupportedConfig("kafka.value.deserializer") + + testUnsupportedConfig("kafka.auto.offset.reset", "none") + testUnsupportedConfig("kafka.auto.offset.reset", "someValue") + testUnsupportedConfig("kafka.auto.offset.reset", "earliest") + testUnsupportedConfig("kafka.auto.offset.reset", "latest") + } + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffset", s"latest") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4), // Should get the data back on recovery + StopStream, + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), // Should get the added data + AddKafkaData(Set(topic), 7, 8), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + testUtils.addPartitions(topic, 10) + true + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) + ) + } + + private def testFromEarliestOffsets(topic: String, options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, (1 to 3).map { _.toString }.toArray) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark.readStream + reader + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("startingOffset", s"earliest") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + AddKafkaData(Set(topic), 7, 8), + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + testUtils.addPartitions(topic, 10) + true + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) + ) + } +} + + +class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { + + import testImplicits._ + + val topicId = new AtomicInteger(1) + + @volatile var topics: Seq[String] = (1 to 5).map(_ => newStressTopic) + + def newStressTopic: String = s"stress${topicId.getAndIncrement()}" + + private def nextInt(start: Int, end: Int): Int = { + start + Random.nextInt(start + end - 1) + } + + after { + for (topic <- testUtils.getAllTopicsAndPartitionSize().toMap.keys) { + testUtils.deleteTopic(topic) + } + } + + test("stress test with multiple topics and partitions") { + topics.foreach { topic => + testUtils.createTopic(topic, partitions = nextInt(1, 6)) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + } + + // Create Kafka source that reads from latest offset + val kafka = + spark.readStream + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "stress.*") + .option("failOnDataLoss", "false") + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + + runStressTest( + mapped, + Seq(makeSureGetOffsetCalled), + (d, running) => { + Random.nextInt(5) match { + case 0 => // Add a new topic + topics = topics ++ Seq(newStressTopic) + AddKafkaData(topics.toSet, d: _*)(message = s"Add topic $newStressTopic", + topicAction = (topic, partition) => { + if (partition.isEmpty) { + testUtils.createTopic(topic, partitions = nextInt(1, 6)) + } + }) + case 1 if running => + // Only delete a topic when the query is running. Otherwise, we may lost data and + // cannot check the correctness. + val deletedTopic = topics(Random.nextInt(topics.size)) + if (deletedTopic != topics.head) { + topics = topics.filterNot(_ == deletedTopic) + } + AddKafkaData(topics.toSet, d: _*)(message = s"Delete topic $deletedTopic", + topicAction = (topic, partition) => { + // Never remove the first topic to make sure we have at least one topic + if (topic == deletedTopic && deletedTopic != topics.head) { + testUtils.deleteTopic(deletedTopic) + } + }) + case 2 => // Add new partitions + AddKafkaData(topics.toSet, d: _*)(message = "Add partitiosn", + topicAction = (topic, partition) => { + testUtils.addPartitions(topic, partition.get + nextInt(1, 6)) + }) + case _ => // Just add new data + AddKafkaData(topics.toSet, d: _*) + } + }, + iterations = 50) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala new file mode 100644 index 0000000000000..3eb8a737ba4c8 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -0,0 +1,339 @@ +/* + * 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.kafka010 + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.language.postfixOps +import scala.util.Random + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.common.TopicAndPartition +import kafka.server.{KafkaConfig, KafkaServer, OffsetCheckpoint} +import kafka.utils.ZkUtils +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils +import org.apache.spark.SparkConf + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkUtils: ZkUtils = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 0 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkUtils = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkUtils).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration, doLog = false) + server = new KafkaServer(brokerConf) + server.startup() + brokerPort = server.boundPort() + (server, brokerPort) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server = null + } + + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + if (zkUtils != null) { + zkUtils.close() + zkUtils = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkUtils, topic, partitions, 1) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + def getAllTopicsAndPartitionSize(): Seq[(String, Int)] = { + zkUtils.getPartitionsForTopics(zkUtils.getAllTopics()).mapValues(_.size).toSeq + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + createTopic(topic, 1) + } + + /** Delete a Kafka topic and wait until it is propagated to the whole cluster */ + def deleteTopic(topic: String): Unit = { + val partitions = zkUtils.getPartitionsForTopics(Seq(topic))(topic).size + AdminUtils.deleteTopic(zkUtils, topic) + verifyTopicDeletion(zkUtils, topic, partitions, List(this.server)) + } + + /** Add new paritions to a Kafka topic */ + def addPartitions(topic: String, partitions: Int): Unit = { + AdminUtils.addPartitions(zkUtils, topic, partitions) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Seq[(String, RecordMetadata)] = { + producer = new KafkaProducer[String, String](producerConfiguration) + val offsets = try { + messages.map { m => + val metadata = + producer.send(new ProducerRecord[String, String](topic, m)).get(10, TimeUnit.SECONDS) + logInfo(s"\tSent $m to partition ${metadata.partition}, offset ${metadata.offset}") + (m, metadata) + } + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + offsets + } + + def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { + val kc = new KafkaConsumer[String, String](consumerConfiguration) + logInfo("Created consumer to get latest offsets") + kc.subscribe(topics.asJavaCollection) + kc.poll(0) + val partitions = kc.assignment() + kc.pause(partitions) + kc.seekToEnd(partitions) + val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap + kc.close() + logInfo("Closed consumer to get latest offsets") + offsets + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("advertised.host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props.put("delete.topic.enable", "true") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("value.serializer", classOf[StringSerializer].getName) + props.put("key.serializer", classOf[StringSerializer].getName) + // wait for all in-sync replicas to ack sends + props.put("acks", "all") + props + } + + private def consumerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("group.id", "group-KafkaTestUtils-" + Random.nextInt) + props.put("value.deserializer", classOf[StringDeserializer].getName) + props.put("key.deserializer", classOf[StringDeserializer].getName) + props.put("enable.auto.commit", "false") + props + } + + private def verifyTopicDeletion( + zkUtils: ZkUtils, + topic: String, + numPartitions: Int, + servers: Seq[KafkaServer]) { + import ZkUtils._ + val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _)) + def isDeleted(): Boolean = { + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + val deletePath = !zkUtils.pathExists(getDeleteTopicPath(topic)) + val topicPath = !zkUtils.pathExists(getTopicPath(topic)) + // ensure that the topic-partition has been deleted from all brokers' replica managers + val replicaManager = servers.forall(server => topicAndPartitions.forall(tp => + server.replicaManager.getPartition(tp.topic, tp.partition) == None)) + // ensure that logs from all replicas are deleted if delete topic is marked successful + val logManager = servers.forall(server => topicAndPartitions.forall(tp => + server.getLogManager().getLog(tp).isEmpty)) + // ensure that topic is removed from all cleaner offsets + val cleaner = servers.forall(server => topicAndPartitions.forall { tp => + val checkpoints = server.getLogManager().logDirs.map { logDir => + new OffsetCheckpoint(new File(logDir, "cleaner-offset-checkpoint")).read() + } + checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) + }) + deletePath && topicPath && replicaManager && logManager && cleaner + } + eventually(timeout(10.seconds)) { + assert(isDeleted, s"$topic not deleted after timeout") + } + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + zkUtils.getLeaderForPartition(topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.size >= 1 + + case _ => + false + } + eventually(timeout(10.seconds)) { + assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } +} + diff --git a/pom.xml b/pom.xml index 8408f4b1fa5ed..37976b0359ad4 100644 --- a/pom.xml +++ b/pom.xml @@ -111,6 +111,7 @@ external/kafka-0-8-assembly external/kafka-0-10 external/kafka-0-10-assembly + external/kafka-0-10-sql diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8e47e7f13d367..88d5dc9b02dd9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -39,8 +39,8 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer) = Seq( - "catalyst", "sql", "hive", "hive-thriftserver" + val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, sqlKafka010) = Seq( + "catalyst", "sql", "hive", "hive-thriftserver", "sql-kafka-0-10" ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( @@ -353,7 +353,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags + unsafe, tags, sqlKafka010 ).contains(x) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 9825f19b86a55..b3a0d6ad0bd4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -116,7 +116,7 @@ class StreamExecution( * [[HDFSMetadataLog]]. See SPARK-14131 for more details. */ val microBatchThread = - new UninterruptibleThread(s"stream execution thread for $name") { + new StreamExecutionThread(s"stream execution thread for $name") { override def run(): Unit = { // To fix call site like "run at :0", we bridge the call site from the caller // thread to this micro batch thread @@ -530,3 +530,9 @@ object StreamExecution { def nextId: Long = _nextId.getAndIncrement() } + +/** + * A special thread to run the stream query. Some codes require to run in the StreamExecutionThread + * and will use `classOf[StreamExecutionThread]` to check. + */ +abstract class StreamExecutionThread(name: String) extends UninterruptibleThread(name) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index aa6515bc7a909..09140a1d6e76b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -50,11 +50,11 @@ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} * * {{{ * val inputData = MemoryStream[Int] - val mapped = inputData.toDS().map(_ + 1) - - testStream(mapped)( - AddData(inputData, 1, 2, 3), - CheckAnswer(2, 3, 4)) + * val mapped = inputData.toDS().map(_ + 1) + * + * testStream(mapped)( + * AddData(inputData, 1, 2, 3), + * CheckAnswer(2, 3, 4)) * }}} * * Note that while we do sleep to allow the other thread to progress without spinning, @@ -477,21 +477,41 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } + + /** + * Creates a stress test that randomly starts/stops/adds data/checks the result. + * + * @param ds a dataframe that executes + 1 on a stream of integers, returning the result + * @param addData an add data action that adds the given numbers to the stream, encoding them + * as needed + * @param iterations the iteration number + */ + def runStressTest( + ds: Dataset[Int], + addData: Seq[Int] => StreamAction, + iterations: Int = 100): Unit = { + runStressTest(ds, Seq.empty, (data, running) => addData(data), iterations) + } + /** * Creates a stress test that randomly starts/stops/adds data/checks the result. * - * @param ds a dataframe that executes + 1 on a stream of integers, returning the result. - * @param addData and add data action that adds the given numbers to the stream, encoding them + * @param ds a dataframe that executes + 1 on a stream of integers, returning the result + * @param prepareActions actions need to run before starting the stress test. + * @param addData an add data action that adds the given numbers to the stream, encoding them * as needed + * @param iterations the iteration number */ def runStressTest( ds: Dataset[Int], - addData: Seq[Int] => StreamAction, - iterations: Int = 100): Unit = { + prepareActions: Seq[StreamAction], + addData: (Seq[Int], Boolean) => StreamAction, + iterations: Int): Unit = { implicit val intEncoder = ExpressionEncoder[Int]() var dataPos = 0 var running = true val actions = new ArrayBuffer[StreamAction]() + actions ++= prepareActions def addCheck() = { actions += CheckAnswer(1 to dataPos: _*) } @@ -499,7 +519,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val numItems = Random.nextInt(10) val data = dataPos until (dataPos + numItems) dataPos += numItems - actions += addData(data) + actions += addData(data, running) } (1 to iterations).foreach { i => From b678e465afa417780b54db0fbbaa311621311f15 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Oct 2016 18:11:31 -0700 Subject: [PATCH 004/177] [SPARK-17346][SQL][TEST-MAVEN] Generate the sql test jar to fix the maven build ## What changes were proposed in this pull request? Generate the sql test jar to fix the maven build ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15368 from zsxwing/sql-test-jar. --- external/kafka-0-10-sql/pom.xml | 14 ++++++++++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 1 + sql/core/pom.xml | 27 +++++++++++++++++++ 3 files changed, 42 insertions(+) diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index b96445a11f858..ebff5fd07a9b9 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -41,6 +41,20 @@ ${project.version} provided + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + org.apache.spark spark-sql_${scala.binary.version} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 64bf503058027..6c03070398fca 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -151,6 +151,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val mapped = kafka.map(kv => kv._2.toInt + 1) testStream(mapped)( + makeSureGetOffsetCalled, StopStream ) } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 84de1d4a6e2d1..7da77158ff07e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -132,6 +132,33 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + test-jar-on-test-compile + test-compile + + test-jar + + + + org.codehaus.mojo build-helper-maven-plugin From 7aeb20be7e999523784aca7be1a7c9c99dec125e Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 5 Oct 2016 23:03:09 -0700 Subject: [PATCH 005/177] [MINOR][ML] Avoid 2D array flatten in NB training. ## What changes were proposed in this pull request? Avoid 2D array flatten in ```NaiveBayes``` training, since flatten method might be expensive (It will create another array and copy data there). ## How was this patch tested? Existing tests. Author: Yanbo Liang Closes #15359 from yanboliang/nb-theta. --- .../org/apache/spark/ml/classification/NaiveBayes.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 6775745167b08..e565a6fd3ece2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -176,8 +176,8 @@ class NaiveBayes @Since("1.5.0") ( val numLabels = aggregated.length val numDocuments = aggregated.map(_._2._1).sum - val piArray = Array.fill[Double](numLabels)(0.0) - val thetaArrays = Array.fill[Double](numLabels, numFeatures)(0.0) + val piArray = new Array[Double](numLabels) + val thetaArray = new Array[Double](numLabels * numFeatures) val lambda = $(smoothing) val piLogDenom = math.log(numDocuments + numLabels * lambda) @@ -193,14 +193,14 @@ class NaiveBayes @Since("1.5.0") ( } var j = 0 while (j < numFeatures) { - thetaArrays(i)(j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + thetaArray(i * numFeatures + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom j += 1 } i += 1 } val pi = Vectors.dense(piArray) - val theta = new DenseMatrix(numLabels, thetaArrays(0).length, thetaArrays.flatten, true) + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) new NaiveBayesModel(uid, pi, theta) } From 5e9f32dd87e58e909a579eaa310e67d31c3b6573 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 6 Oct 2016 09:58:58 +0100 Subject: [PATCH 006/177] [BUILD] Closing some stale PRs ## What changes were proposed in this pull request? This PR proposes to close some stale PRs and ones suggested to be closed by committer(s) or obviously inappropriate PRs (e.g. branch to branch). Closes #13458 Closes #15278 Closes #15294 Closes #15339 Closes #15283 ## How was this patch tested? N/A Author: hyukjinkwon Closes #15356 from HyukjinKwon/closing-prs. From 92b7e5728025b1bb6ed3aab5f1753c946a73568c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 6 Oct 2016 09:42:30 -0700 Subject: [PATCH 007/177] [SPARK-17750][SQL] Fix CREATE VIEW with INTERVAL arithmetic. ## What changes were proposed in this pull request? Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions. ```scala scala> sql("CREATE TABLE dates (ts TIMESTAMP)") scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates") java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ... ``` ## How was this patch tested? Pass Jenkins with a new testcase. Author: Dongjoon Hyun Closes #15318 from dongjoon-hyun/SPARK-17750. --- .../expressions/datetimeExpressions.scala | 2 ++ .../resources/sqlgen/interval_arithmetic.sql | 8 ++++++++ .../catalyst/ExpressionSQLBuilderSuite.scala | 18 +++++++++++++++++- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 16 ++++++++++++++++ 4 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 04c17bdaf2989..7ab68a13e09cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -682,6 +682,7 @@ case class TimeAdd(start: Expression, interval: Expression) override def right: Expression = interval override def toString: String = s"$left + $right" + override def sql: String = s"${left.sql} + ${right.sql}" override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, CalendarIntervalType) override def dataType: DataType = TimestampType @@ -762,6 +763,7 @@ case class TimeSub(start: Expression, interval: Expression) override def right: Expression = interval override def toString: String = s"$left - $right" + override def sql: String = s"${left.sql} - ${right.sql}" override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, CalendarIntervalType) override def dataType: DataType = TimestampType diff --git a/sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql b/sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql new file mode 100644 index 0000000000000..31d00348769f5 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select ts + interval 1 day, ts + interval 2 days, + ts - interval 1 day, ts - interval 2 days, + ts + interval '1' day, ts + interval '2' days, + ts - interval '1' day, ts - interval '2' days +from dates +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `CAST(ts + interval 1 days AS TIMESTAMP)`, `gen_attr_2` AS `CAST(ts + interval 2 days AS TIMESTAMP)`, `gen_attr_3` AS `CAST(ts - interval 1 days AS TIMESTAMP)`, `gen_attr_4` AS `CAST(ts - interval 2 days AS TIMESTAMP)`, `gen_attr_5` AS `CAST(ts + interval 1 days AS TIMESTAMP)`, `gen_attr_6` AS `CAST(ts + interval 2 days AS TIMESTAMP)`, `gen_attr_7` AS `CAST(ts - interval 1 days AS TIMESTAMP)`, `gen_attr_8` AS `CAST(ts - interval 2 days AS TIMESTAMP)` FROM (SELECT CAST(`gen_attr_1` + interval 1 days AS TIMESTAMP) AS `gen_attr_0`, CAST(`gen_attr_1` + interval 2 days AS TIMESTAMP) AS `gen_attr_2`, CAST(`gen_attr_1` - interval 1 days AS TIMESTAMP) AS `gen_attr_3`, CAST(`gen_attr_1` - interval 2 days AS TIMESTAMP) AS `gen_attr_4`, CAST(`gen_attr_1` + interval 1 days AS TIMESTAMP) AS `gen_attr_5`, CAST(`gen_attr_1` + interval 2 days AS TIMESTAMP) AS `gen_attr_6`, CAST(`gen_attr_1` - interval 1 days AS TIMESTAMP) AS `gen_attr_7`, CAST(`gen_attr_1` - interval 2 days AS TIMESTAMP) AS `gen_attr_8` FROM (SELECT `ts` AS `gen_attr_1` FROM `default`.`dates`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index ce5efe853ca4f..149ce1e195111 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.catalyst import java.sql.Timestamp import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{If, Literal, SpecifiedWindowFrame, WindowSpecDefinition} +import org.apache.spark.sql.catalyst.expressions.{If, Literal, SpecifiedWindowFrame, TimeAdd, + TimeSub, WindowSpecDefinition} +import org.apache.spark.unsafe.types.CalendarInterval class ExpressionSQLBuilderSuite extends SQLBuilderTest { test("literal") { @@ -119,4 +121,18 @@ class ExpressionSQLBuilderSuite extends SQLBuilderTest { s"(PARTITION BY `a`, `b` ORDER BY `c` ASC NULLS FIRST, `d` DESC NULLS LAST $frame)" ) } + + test("interval arithmetic") { + val interval = Literal(new CalendarInterval(0, CalendarInterval.MICROS_PER_DAY)) + + checkSQL( + TimeAdd('a, interval), + "`a` + interval 1 days" + ) + + checkSQL( + TimeSub('a, interval), + "`a` - interval 1 days" + ) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 7fa5c29dc5b8f..9ac1e86fc82cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -1145,4 +1145,20 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { """.stripMargin, "inline_tables") } + + test("SPARK-17750 - interval arithmetic") { + withTable("dates") { + sql("create table dates (ts timestamp)") + checkSQL( + """ + |select ts + interval 1 day, ts + interval 2 days, + | ts - interval 1 day, ts - interval 2 days, + | ts + interval '1' day, ts + interval '2' days, + | ts - interval '1' day, ts - interval '2' days + |from dates + """.stripMargin, + "interval_arithmetic" + ) + } + } } From 79accf45ace5549caa0cbab02f94fc87bedb5587 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 6 Oct 2016 10:33:45 -0700 Subject: [PATCH 008/177] [SPARK-17798][SQL] Remove redundant Experimental annotations in sql.streaming ## What changes were proposed in this pull request? I was looking through API annotations to catch mislabeled APIs, and realized DataStreamReader and DataStreamWriter classes are already annotated as Experimental, and as a result there is no need to annotate each method within them. ## How was this patch tested? N/A Author: Reynold Xin Closes #15373 from rxin/SPARK-17798. --- .../sql/streaming/DataStreamReader.scala | 28 ------------------ .../sql/streaming/DataStreamWriter.scala | 29 ------------------- .../streaming/StreamingQueryListener.scala | 4 +-- 3 files changed, 1 insertion(+), 60 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index d437c16a25b01..864a9cd3eb89d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -35,89 +35,73 @@ import org.apache.spark.sql.types.StructType @Experimental final class DataStreamReader private[sql](sparkSession: SparkSession) extends Logging { /** - * :: Experimental :: * Specifies the input data source format. * * @since 2.0.0 */ - @Experimental def format(source: String): DataStreamReader = { this.source = source this } /** - * :: Experimental :: * Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema * automatically from data. By specifying the schema here, the underlying data source can * skip the schema inference step, and thus speed up data loading. * * @since 2.0.0 */ - @Experimental def schema(schema: StructType): DataStreamReader = { this.userSpecifiedSchema = Option(schema) this } /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: String): DataStreamReader = { this.extraOptions += (key -> value) this } /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Boolean): DataStreamReader = option(key, value.toString) /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Long): DataStreamReader = option(key, value.toString) /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Double): DataStreamReader = option(key, value.toString) /** - * :: Experimental :: * (Scala-specific) Adds input options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: scala.collection.Map[String, String]): DataStreamReader = { this.extraOptions ++= options this } /** - * :: Experimental :: * Adds input options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: java.util.Map[String, String]): DataStreamReader = { this.options(options.asScala) this @@ -125,13 +109,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo /** - * :: Experimental :: * Loads input data stream in as a [[DataFrame]], for data streams that don't require a path * (e.g. external key-value stores). * * @since 2.0.0 */ - @Experimental def load(): DataFrame = { val dataSource = DataSource( @@ -143,18 +125,15 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo } /** - * :: Experimental :: * Loads input in as a [[DataFrame]], for data streams that read from some path. * * @since 2.0.0 */ - @Experimental def load(path: String): DataFrame = { option("path", path).load() } /** - * :: Experimental :: * Loads a JSON file stream (one object per line) and returns the result as a [[DataFrame]]. * * This function goes through the input once to determine the input schema. If you know the @@ -198,11 +177,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def json(path: String): DataFrame = format("json").load(path) /** - * :: Experimental :: * Loads a CSV file stream and returns the result as a [[DataFrame]]. * * This function will go through the input once to determine the input schema if `inferSchema` @@ -262,11 +239,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def csv(path: String): DataFrame = format("csv").load(path) /** - * :: Experimental :: * Loads a Parquet file stream, returning the result as a [[DataFrame]]. * * You can set the following Parquet-specific option(s) for reading Parquet files: @@ -281,13 +256,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def parquet(path: String): DataFrame = { format("parquet").load(path) } /** - * :: Experimental :: * Loads text files and returns a [[DataFrame]] whose schema starts with a string column named * "value", and followed by partitioned columns if there are any. * @@ -308,7 +281,6 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def text(path: String): DataFrame = format("text").load(path) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index f70c7d08a691c..b959444b49298 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -37,7 +37,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { private val df = ds.toDF() /** - * :: Experimental :: * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be * written to the sink @@ -46,15 +45,12 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def outputMode(outputMode: OutputMode): DataStreamWriter[T] = { this.outputMode = outputMode this } - /** - * :: Experimental :: * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to * the sink @@ -63,7 +59,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def outputMode(outputMode: String): DataStreamWriter[T] = { this.outputMode = outputMode.toLowerCase match { case "append" => @@ -78,7 +73,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } /** - * :: Experimental :: * Set the trigger for the stream query. The default value is `ProcessingTime(0)` and it will run * the query as fast as possible. * @@ -100,7 +94,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def trigger(trigger: Trigger): DataStreamWriter[T] = { this.trigger = trigger this @@ -108,25 +101,21 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** - * :: Experimental :: * Specifies the name of the [[StreamingQuery]] that can be started with `start()`. * This name must be unique among all the currently active queries in the associated SQLContext. * * @since 2.0.0 */ - @Experimental def queryName(queryName: String): DataStreamWriter[T] = { this.extraOptions += ("queryName" -> queryName) this } /** - * :: Experimental :: * Specifies the underlying output data source. Built-in options include "parquet" for now. * * @since 2.0.0 */ - @Experimental def format(source: String): DataStreamWriter[T] = { this.source = source this @@ -156,90 +145,74 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: String): DataStreamWriter[T] = { this.extraOptions += (key -> value) this } /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Boolean): DataStreamWriter[T] = option(key, value.toString) /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Long): DataStreamWriter[T] = option(key, value.toString) /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Double): DataStreamWriter[T] = option(key, value.toString) /** - * :: Experimental :: * (Scala-specific) Adds output options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: scala.collection.Map[String, String]): DataStreamWriter[T] = { this.extraOptions ++= options this } /** - * :: Experimental :: * Adds output options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: java.util.Map[String, String]): DataStreamWriter[T] = { this.options(options.asScala) this } /** - * :: Experimental :: * Starts the execution of the streaming query, which will continually output results to the given * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with * the stream. * * @since 2.0.0 */ - @Experimental def start(path: String): StreamingQuery = { option("path", path).start() } /** - * :: Experimental :: * Starts the execution of the streaming query, which will continually output results to the given * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with * the stream. * * @since 2.0.0 */ - @Experimental def start(): StreamingQuery = { if (source == "memory") { assertNotPartitioned("memory") @@ -297,7 +270,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } /** - * :: Experimental :: * Starts the execution of the streaming query, which will continually send results to the given * [[ForeachWriter]] as as new data arrives. The [[ForeachWriter]] can be used to send the data * generated by the [[DataFrame]]/[[Dataset]] to an external system. @@ -343,7 +315,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { this.source = "foreach" this.foreachWriter = if (writer != null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index db606abb8ce43..8a8855d85a4c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -35,7 +35,7 @@ abstract class StreamingQueryListener { /** * Called when a query is started. * @note This is called synchronously with - * [[org.apache.spark.sql.DataStreamWriter `DataStreamWriter.start()`]], + * [[org.apache.spark.sql.streaming.DataStreamWriter `DataStreamWriter.start()`]], * that is, `onQueryStart` will be called on all listeners before * `DataStreamWriter.start()` returns the corresponding [[StreamingQuery]]. Please * don't block this method as it will block your query. @@ -101,8 +101,6 @@ object StreamingQueryListener { * @param queryInfo Information about the status of the query. * @param exception The exception message of the [[StreamingQuery]] if the query was terminated * with an exception. Otherwise, it will be `None`. - * @param stackTrace The stack trace of the exception if the query was terminated with an - * exception. It will be empty if there was no error. * @since 2.0.0 */ @Experimental From 9a48e60e6319d85f2c3be3a3c608dab135e18a73 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 6 Oct 2016 12:51:12 -0700 Subject: [PATCH 009/177] [SPARK-17780][SQL] Report Throwable to user in StreamExecution ## What changes were proposed in this pull request? When using an incompatible source for structured streaming, it may throw NoClassDefFoundError. It's better to just catch Throwable and report it to the user since the streaming thread is dying. ## How was this patch tested? `test("NoClassDefFoundError from an incompatible source")` Author: Shixiong Zhu Closes #15352 from zsxwing/SPARK-17780. --- .../execution/streaming/StreamExecution.scala | 7 ++++- .../spark/sql/streaming/StreamSuite.scala | 31 ++++++++++++++++++- .../spark/sql/streaming/StreamTest.scala | 3 +- 3 files changed, 37 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index b3a0d6ad0bd4c..333239f875bd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -207,13 +207,18 @@ class StreamExecution( }) } catch { case _: InterruptedException if state == TERMINATED => // interrupted by stop() - case NonFatal(e) => + case e: Throwable => streamDeathCause = new StreamingQueryException( this, s"Query $name terminated with exception: ${e.getMessage}", e, Some(committedOffsets.toCompositeOffset(sources))) logError(s"Query $name terminated with error", e) + // Rethrow the fatal errors to allow the user using `Thread.UncaughtExceptionHandler` to + // handle them + if (!NonFatal(e)) { + throw e + } } finally { state = TERMINATED sparkSession.streams.notifyQueryTermination(StreamExecution.this) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 1caafb9d74440..cdbad901dba8e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.streaming +import scala.reflect.ClassTag +import scala.util.control.ControlThrowable + import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.ManualClock @@ -236,6 +238,33 @@ class StreamSuite extends StreamTest { } } + testQuietly("fatal errors from a source should be sent to the user") { + for (e <- Seq( + new VirtualMachineError {}, + new ThreadDeath, + new LinkageError, + new ControlThrowable {} + )) { + val source = new Source { + override def getOffset: Option[Offset] = { + throw e + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + throw e + } + + override def schema: StructType = StructType(Array(StructField("value", IntegerType))) + + override def stop(): Unit = {} + } + val df = Dataset[Int](sqlContext.sparkSession, StreamingExecutionRelation(source)) + testStream(df)( + ExpectFailure()(ClassTag(e.getClass)) + ) + } + } + test("output mode API in Scala") { val o1 = OutputMode.Append assert(o1 === InternalOutputModes.Append) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 09140a1d6e76b..fa13d385cce75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -167,7 +167,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { /** Signals that a failure is expected and should not kill the test. */ case class ExpectFailure[T <: Throwable : ClassTag]() extends StreamAction { val causeClass: Class[T] = implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] - override def toString(): String = s"ExpectFailure[${causeClass.getCanonicalName}]" + override def toString(): String = s"ExpectFailure[${causeClass.getName}]" } /** Assert that a body is true */ @@ -322,7 +322,6 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { new UncaughtExceptionHandler { override def uncaughtException(t: Thread, e: Throwable): Unit = { streamDeathCause = e - testThread.interrupt() } }) From 49d11d49983fbe270f4df4fb1e34b5fbe854c5ec Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Thu, 6 Oct 2016 14:28:49 -0700 Subject: [PATCH 010/177] [SPARK-17803][TESTS] Upgrade docker-client dependency [SPARK-17803: Docker integration tests don't run with "Docker for Mac"](https://issues.apache.org/jira/browse/SPARK-17803) ## What changes were proposed in this pull request? This PR upgrades the [docker-client](https://mvnrepository.com/artifact/com.spotify/docker-client) dependency from [3.6.6](https://mvnrepository.com/artifact/com.spotify/docker-client/3.6.6) to [5.0.2](https://mvnrepository.com/artifact/com.spotify/docker-client/5.0.2) to enable _Docker for Mac_ users to run the `docker-integration-tests` out of the box. The very latest docker-client version is [6.0.0](https://mvnrepository.com/artifact/com.spotify/docker-client/6.0.0) but that has one additional dependency and no usage yet. ## How was this patch tested? The code change was tested on Mac OS X Yosemite with both _Docker Toolbox_ as well as _Docker for Mac_ and on Linux Ubuntu 14.04. ``` $ build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -Phive -Phive-thriftserver -DskipTests clean package $ build/mvn -Pdocker-integration-tests -Pscala-2.11 -pl :spark-docker-integration-tests_2.11 clean compile test ``` Author: Christian Kadner Closes #15378 from ckadner/SPARK-17803_Docker_for_Mac. --- .../org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala | 1 + pom.xml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index c36f4d5f95482..609696bc8a2c7 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import scala.util.control.NonFatal import com.spotify.docker.client._ +import com.spotify.docker.client.exceptions.ImageNotFoundException import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding} import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually diff --git a/pom.xml b/pom.xml index 37976b0359ad4..7d13c51b2a596 100644 --- a/pom.xml +++ b/pom.xml @@ -744,7 +744,7 @@ com.spotify docker-client - 3.6.6 + 5.0.2 test From 3713bb199142c5e06e2e527c99650f02f41f47b1 Mon Sep 17 00:00:00 2001 From: sethah Date: Thu, 6 Oct 2016 21:10:17 -0700 Subject: [PATCH 011/177] [SPARK-17792][ML] L-BFGS solver for linear regression does not accept general numeric label column types ## What changes were proposed in this pull request? Before, we computed `instances` in LinearRegression in two spots, even though they did the same thing. One of them did not cast the label column to `DoubleType`. This patch consolidates the computation and always casts the label column to `DoubleType`. ## How was this patch tested? Added a unit test to check all solvers. This test failed before this patch. Author: sethah Closes #15364 from sethah/linreg_numeric_type. --- .../spark/ml/regression/LinearRegression.scala | 17 ++++++----------- .../ml/regression/LinearRegressionSuite.scala | 8 +++++--- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 536c58f998080..025ed20c75a04 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -188,17 +188,18 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + val instances: RDD[Instance] = dataset.select( + col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } + if (($(solver) == "auto" && $(elasticNetParam) == 0.0 && numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == "normal") { require($(elasticNetParam) == 0.0, "Only L2 regularization can be used when normal " + "solver is used.'") // For low dimensional data, WeightedLeastSquares is more efficiently since the // training algorithm only requires one pass through the data. (SPARK-10668) - val instances: RDD[Instance] = dataset.select( - col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), $(standardization), true) @@ -221,12 +222,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String return lrModel.setSummary(trainingSummary) } - val instances: RDD[Instance] = - dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index 5ae371b489aa5..1c94ec67d79d1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -1015,12 +1015,14 @@ class LinearRegressionSuite } test("should support all NumericType labels and not support other types") { - val lr = new LinearRegression().setMaxIter(1) - MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( - lr, spark, isClassification = false) { (expected, actual) => + for (solver <- Seq("auto", "l-bfgs", "normal")) { + val lr = new LinearRegression().setMaxIter(1).setSolver(solver) + MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( + lr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } + } } } From bcaa799cb01289f73e9f48526e94653a07628983 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Fri, 7 Oct 2016 00:27:55 -0700 Subject: [PATCH 012/177] [SPARK-17805][PYSPARK] Fix in sqlContext.read.text when pass in list of paths ## What changes were proposed in this pull request? If given a list of paths, `pyspark.sql.readwriter.text` will attempt to use an undefined variable `paths`. This change checks if the param `paths` is a basestring and then converts it to a list, so that the same variable `paths` can be used for both cases ## How was this patch tested? Added unit test for reading list of files Author: Bryan Cutler Closes #15379 from BryanCutler/sql-readtext-paths-SPARK-17805. --- python/pyspark/sql/readwriter.py | 4 ++-- python/pyspark/sql/tests.py | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 3ad6f80de9fdf..91c2b17049fa1 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -289,8 +289,8 @@ def text(self, paths): [Row(value=u'hello'), Row(value=u'this')] """ if isinstance(paths, basestring): - path = [paths] - return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(path))) + paths = [paths] + return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths))) @since(2.0) def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index c2171c277cac3..a9e455565a6cd 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1702,6 +1702,12 @@ def test_cache(self): "does_not_exist", lambda: spark.catalog.uncacheTable("does_not_exist")) + def test_read_text_file_list(self): + df = self.spark.read.text(['python/test_support/sql/text-test.txt', + 'python/test_support/sql/text-test.txt']) + count = df.count() + self.assertEquals(count, 4) + class HiveSparkSubmitTests(SparkSubmitTests): From 18bf9d2b2d7eae0574102d4f15ac27dc71dea18a Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Fri, 7 Oct 2016 11:46:39 +0100 Subject: [PATCH 013/177] [SPARK-17782][STREAMING][BUILD] Add Kafka 0.10 project to build modules ## What changes were proposed in this pull request? This PR adds the Kafka 0.10 subproject to the build infrastructure. This makes sure Kafka 0.10 tests are only triggers when it or of its dependencies change. Author: Herman van Hovell Closes #15355 from hvanhovell/SPARK-17782. --- dev/sparktestsupport/modules.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 5f14683d9a52f..b34ab51f3b996 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -241,6 +241,17 @@ def __hash__(self): ] ) +streaming_kafka_0_10 = Module( + name="streaming-kafka-0-10", + dependencies=[streaming], + source_file_regexes=[ + "external/kafka-0-10", + "external/kafka-0-10-assembly", + ], + sbt_test_goals=[ + "streaming-kafka-0-10/test", + ] +) streaming_flume_sink = Module( name="streaming-flume-sink", From 24097d84743d3e792e395410139e8d486b75a3ef Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Fri, 7 Oct 2016 11:47:37 +0100 Subject: [PATCH 014/177] =?UTF-8?q?[SPARK-17795][WEB=20UI]=20Sorting=20on?= =?UTF-8?q?=20stage=20or=20job=20tables=20doesn=E2=80=99t=20reload=20page?= =?UTF-8?q?=20on=20that=20table?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Added anchor on table header id to sorting links on job and stage tables. This make the page reload after a sort load the page at the sorted table. This only changes page load behavior so no UI changes ## How was this patch tested? manually tested and dev/run-tests Author: Alex Bozarth Closes #15369 from ajbozarth/spark17795. --- .../apache/spark/ui/jobs/AllJobsPage.scala | 20 ++++++++++++------- .../apache/spark/ui/jobs/AllStagesPage.scala | 12 +++++------ .../org/apache/spark/ui/jobs/JobPage.scala | 17 +++++++++++----- .../org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../org/apache/spark/ui/jobs/StageTable.scala | 14 +++++++++---- 5 files changed, 42 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index c04964ec66479..19bb41a1417c7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -216,6 +216,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { private def jobsTable( request: HttpServletRequest, + tableHeaderId: String, jobTag: String, jobs: Seq[JobUIData]): Seq[Node] = { val allParameters = request.getParameterMap.asScala.toMap @@ -256,6 +257,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { try { new JobPagedTable( jobs, + tableHeaderId, jobTag, UIUtils.prependBaseUri(parent.basePath), "jobs", // subPath @@ -288,9 +290,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val completedJobs = listener.completedJobs.reverse.toSeq val failedJobs = listener.failedJobs.reverse.toSeq - val activeJobsTable = jobsTable(request, "activeJob", activeJobs) - val completedJobsTable = jobsTable(request, "completedJob", completedJobs) - val failedJobsTable = jobsTable(request, "failedJob", failedJobs) + val activeJobsTable = jobsTable(request, "active", "activeJob", activeJobs) + val completedJobsTable = jobsTable(request, "completed", "completedJob", completedJobs) + val failedJobsTable = jobsTable(request, "failed", "failedJob", failedJobs) val shouldShowActiveJobs = activeJobs.nonEmpty val shouldShowCompletedJobs = completedJobs.nonEmpty @@ -486,6 +488,7 @@ private[ui] class JobDataSource( } private[ui] class JobPagedTable( data: Seq[JobUIData], + tableHeaderId: String, jobTag: String, basePath: String, subPath: String, @@ -528,12 +531,13 @@ private[ui] class JobPagedTable( s"&$pageNumberFormField=$page" + s"&$jobTag.sort=$encodedSortColumn" + s"&$jobTag.desc=$desc" + - s"&$pageSizeFormField=$pageSize" + s"&$pageSizeFormField=$pageSize" + + s"#$tableHeaderId" } override def goButtonFormPath: String = { val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - s"$parameterPath&$jobTag.sort=$encodedSortColumn&$jobTag.desc=$desc" + s"$parameterPath&$jobTag.sort=$encodedSortColumn&$jobTag.desc=$desc#$tableHeaderId" } override def headers: Seq[Node] = { @@ -557,7 +561,8 @@ private[ui] class JobPagedTable( parameterPath + s"&$jobTag.sort=${URLEncoder.encode(header, "UTF-8")}" + s"&$jobTag.desc=${!desc}" + - s"&$jobTag.pageSize=$pageSize") + s"&$jobTag.pageSize=$pageSize" + + s"#$tableHeaderId") val arrow = if (desc) "▾" else "▴" // UP or DOWN @@ -572,7 +577,8 @@ private[ui] class JobPagedTable( val headerLink = Unparsed( parameterPath + s"&$jobTag.sort=${URLEncoder.encode(header, "UTF-8")}" + - s"&$jobTag.pageSize=$pageSize") + s"&$jobTag.pageSize=$pageSize" + + s"#$tableHeaderId") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index cba8f82dd77a6..fe6ca1099e6b0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -41,19 +41,19 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val subPath = "stages" val activeStagesTable = - new StageTableBase(request, activeStages, "activeStage", parent.basePath, subPath, + new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, subPath, parent.progressListener, parent.isFairScheduler, killEnabled = parent.killEnabled, isFailedStage = false) val pendingStagesTable = - new StageTableBase(request, pendingStages, "pendingStage", parent.basePath, subPath, - parent.progressListener, parent.isFairScheduler, + new StageTableBase(request, pendingStages, "pending", "pendingStage", parent.basePath, + subPath, parent.progressListener, parent.isFairScheduler, killEnabled = false, isFailedStage = false) val completedStagesTable = - new StageTableBase(request, completedStages, "completedStage", parent.basePath, subPath, - parent.progressListener, parent.isFairScheduler, + new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, + subPath, parent.progressListener, parent.isFairScheduler, killEnabled = false, isFailedStage = false) val failedStagesTable = - new StageTableBase(request, failedStages, "failedStage", parent.basePath, subPath, + new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, subPath, parent.progressListener, parent.isFairScheduler, killEnabled = false, isFailedStage = true) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 2f7f8976a8899..0ff9e5e9411ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -230,20 +230,27 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val basePath = "jobs/job" + val pendingOrSkippedTableId = + if (isComplete) { + "pending" + } else { + "skipped" + } + val activeStagesTable = - new StageTableBase(request, activeStages, "activeStage", parent.basePath, + new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, basePath, parent.jobProgresslistener, parent.isFairScheduler, killEnabled = parent.killEnabled, isFailedStage = false) val pendingOrSkippedStagesTable = - new StageTableBase(request, pendingOrSkippedStages, "pendingStage", parent.basePath, - basePath, parent.jobProgresslistener, parent.isFairScheduler, + new StageTableBase(request, pendingOrSkippedStages, pendingOrSkippedTableId, "pendingStage", + parent.basePath, basePath, parent.jobProgresslistener, parent.isFairScheduler, killEnabled = false, isFailedStage = false) val completedStagesTable = - new StageTableBase(request, completedStages, "completedStage", parent.basePath, + new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, basePath, parent.jobProgresslistener, parent.isFairScheduler, killEnabled = false, isFailedStage = false) val failedStagesTable = - new StageTableBase(request, failedStages, "failedStage", parent.basePath, + new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, basePath, parent.jobProgresslistener, parent.isFairScheduler, killEnabled = false, isFailedStage = true) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index f9cb717918592..8ee70d27cc09f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -44,7 +44,7 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { } val shouldShowActiveStages = activeStages.nonEmpty val activeStagesTable = - new StageTableBase(request, activeStages, "activeStage", parent.basePath, "stages/pool", + new StageTableBase(request, activeStages, "", "activeStage", parent.basePath, "stages/pool", parent.progressListener, parent.isFairScheduler, parent.killEnabled, isFailedStage = false) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2a04e8fc7d007..40a6762c281ce 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -34,6 +34,7 @@ import org.apache.spark.util.Utils private[ui] class StageTableBase( request: HttpServletRequest, stages: Seq[StageInfo], + tableHeaderID: String, stageTag: String, basePath: String, subPath: String, @@ -77,6 +78,7 @@ private[ui] class StageTableBase( val toNodeSeq = try { new StagePagedTable( stages, + tableHeaderID, stageTag, basePath, subPath, @@ -131,6 +133,7 @@ private[ui] class MissingStageTableRowData( /** Page showing list of all ongoing and recently finished stages */ private[ui] class StagePagedTable( stages: Seq[StageInfo], + tableHeaderId: String, stageTag: String, basePath: String, subPath: String, @@ -173,12 +176,13 @@ private[ui] class StagePagedTable( s"&$pageNumberFormField=$page" + s"&$stageTag.sort=$encodedSortColumn" + s"&$stageTag.desc=$desc" + - s"&$pageSizeFormField=$pageSize" + s"&$pageSizeFormField=$pageSize" + + s"#$tableHeaderId" } override def goButtonFormPath: String = { val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - s"$parameterPath&$stageTag.sort=$encodedSortColumn&$stageTag.desc=$desc" + s"$parameterPath&$stageTag.sort=$encodedSortColumn&$stageTag.desc=$desc#$tableHeaderId" } override def headers: Seq[Node] = { @@ -226,7 +230,8 @@ private[ui] class StagePagedTable( parameterPath + s"&$stageTag.sort=${URLEncoder.encode(header, "UTF-8")}" + s"&$stageTag.desc=${!desc}" + - s"&$stageTag.pageSize=$pageSize") + s"&$stageTag.pageSize=$pageSize") + + s"#$tableHeaderId" val arrow = if (desc) "▾" else "▴" // UP or DOWN @@ -241,7 +246,8 @@ private[ui] class StagePagedTable( val headerLink = Unparsed( parameterPath + s"&$stageTag.sort=${URLEncoder.encode(header, "UTF-8")}" + - s"&$stageTag.pageSize=$pageSize") + s"&$stageTag.pageSize=$pageSize") + + s"#$tableHeaderId" From 2b01d3c701c58f07fa42afd570523dd161384882 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 7 Oct 2016 11:49:34 +0100 Subject: [PATCH 015/177] [SPARK-16960][SQL] Deprecate approxCountDistinct, toDegrees and toRadians according to FunctionRegistry ## What changes were proposed in this pull request? It seems `approxCountDistinct`, `toDegrees` and `toRadians` are also missed while matching the names to the ones in `FunctionRegistry`. (please see [approx_count_distinct](https://github.com/apache/spark/blob/5c2ae79bfcf448d8dc9217efafa1409997c739de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L244), [degrees](https://github.com/apache/spark/blob/5c2ae79bfcf448d8dc9217efafa1409997c739de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L203) and [radians](https://github.com/apache/spark/blob/5c2ae79bfcf448d8dc9217efafa1409997c739de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L222) in `FunctionRegistry`). I took a scan between `functions.scala` and `FunctionRegistry` and it seems these are all left. For `countDistinct` and `sumDistinct`, they are not registered in `FunctionRegistry`. This PR deprecates `approxCountDistinct`, `toDegrees` and `toRadians` and introduces `approx_count_distinct`, `degrees` and `radians`. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon Author: Hyukjin Kwon Closes #14538 from HyukjinKwon/SPARK-16588-followup. --- python/pyspark/sql/functions.py | 33 +++++-- .../org/apache/spark/sql/functions.scala | 91 +++++++++++++++---- .../spark/sql/DataFrameWindowSuite.scala | 2 +- .../spark/sql/MathExpressionsSuite.scala | 12 +-- 4 files changed, 105 insertions(+), 33 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 45d6bf944b702..7fa3fd2de7ddf 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -112,11 +112,8 @@ def _(): 'sinh': 'Computes the hyperbolic sine of the given value.', 'tan': 'Computes the tangent of the given value.', 'tanh': 'Computes the hyperbolic tangent of the given value.', - 'toDegrees': 'Converts an angle measured in radians to an approximately equivalent angle ' + - 'measured in degrees.', - 'toRadians': 'Converts an angle measured in degrees to an approximately equivalent angle ' + - 'measured in radians.', - + 'toDegrees': '.. note:: Deprecated in 2.1, use degrees instead.', + 'toRadians': '.. note:: Deprecated in 2.1, use radians instead.', 'bitwiseNOT': 'Computes bitwise not.', } @@ -135,7 +132,15 @@ def _(): 'kurtosis': 'Aggregate function: returns the kurtosis of the values in a group.', 'collect_list': 'Aggregate function: returns a list of objects with duplicates.', 'collect_set': 'Aggregate function: returns a set of objects with duplicate elements' + - ' eliminated.' + ' eliminated.', +} + +_functions_2_1 = { + # unary math functions + 'degrees': 'Converts an angle measured in radians to an approximately equivalent angle ' + + 'measured in degrees.', + 'radians': 'Converts an angle measured in degrees to an approximately equivalent angle ' + + 'measured in radians.', } # math functions that take two arguments as input @@ -182,21 +187,31 @@ def _(): globals()[_name] = since(1.6)(_create_window_function(_name, _doc)) for _name, _doc in _functions_1_6.items(): globals()[_name] = since(1.6)(_create_function(_name, _doc)) +for _name, _doc in _functions_2_1.items(): + globals()[_name] = since(2.1)(_create_function(_name, _doc)) del _name, _doc @since(1.3) def approxCountDistinct(col, rsd=None): + """ + .. note:: Deprecated in 2.1, use approx_count_distinct instead. + """ + return approx_count_distinct(col, rsd) + + +@since(2.1) +def approx_count_distinct(col, rsd=None): """Returns a new :class:`Column` for approximate distinct count of ``col``. - >>> df.agg(approxCountDistinct(df.age).alias('c')).collect() + >>> df.agg(approx_count_distinct(df.age).alias('c')).collect() [Row(c=2)] """ sc = SparkContext._active_spark_context if rsd is None: - jc = sc._jvm.functions.approxCountDistinct(_to_java_column(col)) + jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col)) else: - jc = sc._jvm.functions.approxCountDistinct(_to_java_column(col), rsd) + jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col), rsd) return Column(jc) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 3bc1c5b90031d..40f82d895d43b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -182,13 +182,43 @@ object functions { // Aggregate functions ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(e: Column): Column = approx_count_distinct(e) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(columnName: String): Column = approx_count_distinct(columnName) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(e: Column, rsd: Double): Column = approx_count_distinct(e, rsd) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(columnName: String, rsd: Double): Column = { + approx_count_distinct(Column(columnName), rsd) + } + /** * Aggregate function: returns the approximate number of distinct items in a group. * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(e: Column): Column = withAggregateFunction { + def approx_count_distinct(e: Column): Column = withAggregateFunction { HyperLogLogPlusPlus(e.expr) } @@ -196,9 +226,9 @@ object functions { * Aggregate function: returns the approximate number of distinct items in a group. * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(columnName: String): Column = approxCountDistinct(column(columnName)) + def approx_count_distinct(columnName: String): Column = approx_count_distinct(column(columnName)) /** * Aggregate function: returns the approximate number of distinct items in a group. @@ -206,9 +236,9 @@ object functions { * @param rsd maximum estimation error allowed (default = 0.05) * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(e: Column, rsd: Double): Column = withAggregateFunction { + def approx_count_distinct(e: Column, rsd: Double): Column = withAggregateFunction { HyperLogLogPlusPlus(e.expr, rsd, 0, 0) } @@ -218,10 +248,10 @@ object functions { * @param rsd maximum estimation error allowed (default = 0.05) * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(columnName: String, rsd: Double): Column = { - approxCountDistinct(Column(columnName), rsd) + def approx_count_distinct(columnName: String, rsd: Double): Column = { + approx_count_distinct(Column(columnName), rsd) } /** @@ -1949,37 +1979,65 @@ object functions { */ def tanh(columnName: String): Column = tanh(Column(columnName)) + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use degrees", "2.1.0") + def toDegrees(e: Column): Column = degrees(e) + + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use degrees", "2.1.0") + def toDegrees(columnName: String): Column = degrees(Column(columnName)) + /** * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. * * @group math_funcs - * @since 1.4.0 + * @since 2.1.0 */ - def toDegrees(e: Column): Column = withExpr { ToDegrees(e.expr) } + def degrees(e: Column): Column = withExpr { ToDegrees(e.expr) } /** * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. * + * @group math_funcs + * @since 2.1.0 + */ + def degrees(columnName: String): Column = degrees(Column(columnName)) + + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use radians", "2.1.0") + def toRadians(e: Column): Column = radians(e) + + /** * @group math_funcs * @since 1.4.0 */ - def toDegrees(columnName: String): Column = toDegrees(Column(columnName)) + @deprecated("Use radians", "2.1.0") + def toRadians(columnName: String): Column = radians(Column(columnName)) /** * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. * * @group math_funcs - * @since 1.4.0 + * @since 2.1.0 */ - def toRadians(e: Column): Column = withExpr { ToRadians(e.expr) } + def radians(e: Column): Column = withExpr { ToRadians(e.expr) } /** * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. * * @group math_funcs - * @since 1.4.0 + * @since 2.1.0 */ - def toRadians(columnName: String): Column = toRadians(Column(columnName)) + def radians(columnName: String): Column = radians(Column(columnName)) ////////////////////////////////////////////////////////////////////////////////////////////// // Misc functions @@ -3096,5 +3154,4 @@ object functions { def callUDF(udfName: String, cols: Column*): Column = withExpr { UnresolvedFunction(udfName, cols.map(_.expr), isDistinct = false) } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala index c6f8c3ad3fc93..c2b47cae8f4c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala @@ -228,7 +228,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { $"key", var_pop($"value").over(window), var_samp($"value").over(window), - approxCountDistinct($"value").over(window)), + approx_count_distinct($"value").over(window)), Seq.fill(4)(Row("a", 1.0d / 4.0d, 1.0d / 3.0d, 2)) ++ Seq.fill(3)(Row("b", 2.0d / 3.0d, 1.0d, 3))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala index 0de7f2321f398..6944c6f848179 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala @@ -148,19 +148,19 @@ class MathExpressionsSuite extends QueryTest with SharedSQLContext { testOneToOneMathFunction(tanh, math.tanh) } - test("toDegrees") { - testOneToOneMathFunction(toDegrees, math.toDegrees) + test("degrees") { + testOneToOneMathFunction(degrees, math.toDegrees) checkAnswer( sql("SELECT degrees(0), degrees(1), degrees(1.5)"), - Seq((1, 2)).toDF().select(toDegrees(lit(0)), toDegrees(lit(1)), toDegrees(lit(1.5))) + Seq((1, 2)).toDF().select(degrees(lit(0)), degrees(lit(1)), degrees(lit(1.5))) ) } - test("toRadians") { - testOneToOneMathFunction(toRadians, math.toRadians) + test("radians") { + testOneToOneMathFunction(radians, math.toRadians) checkAnswer( sql("SELECT radians(0), radians(1), radians(1.5)"), - Seq((1, 2)).toDF().select(toRadians(lit(0)), toRadians(lit(1)), toRadians(lit(1.5))) + Seq((1, 2)).toDF().select(radians(lit(0)), radians(lit(1)), radians(lit(1.5))) ) } From e56614cba99bfdf5fa8a6c617fdd56eca2b34694 Mon Sep 17 00:00:00 2001 From: Brian Cho Date: Fri, 7 Oct 2016 11:37:18 -0400 Subject: [PATCH 016/177] [SPARK-16827] Stop reporting spill metrics as shuffle metrics ## What changes were proposed in this pull request? Fix a bug where spill metrics were being reported as shuffle metrics. Eventually these spill metrics should be reported (SPARK-3577), but separate from shuffle metrics. The fix itself basically reverts the line to what it was in 1.6. ## How was this patch tested? Tested on a job that was reporting shuffle writes even for the final stage, when no shuffle writes should take place. After the change the job no longer shows these writes. Before: ![screen shot 2016-10-03 at 6 39 59 pm](https://cloud.githubusercontent.com/assets/1514239/19085897/dbf59a92-8a20-11e6-9f68-a978860c0d74.png) After: screen shot 2016-10-03 at 11 44 44 pm Author: Brian Cho Closes #15347 from dafrista/shuffle-metrics. --- .../util/collection/unsafe/sort/UnsafeExternalSorter.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 428ff72e71a43..7835017910232 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -145,7 +145,9 @@ private UnsafeExternalSorter( // Use getSizeAsKb (not bytes) to maintain backwards compatibility for units // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; this.fileBufferSizeBytes = 32 * 1024; - this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); + // The spill metrics are stored in a new ShuffleWriteMetrics, and then discarded (this fixes SPARK-16827). + // TODO: Instead, separate spill metrics should be stored and reported (tracked in SPARK-3577). + this.writeMetrics = new ShuffleWriteMetrics(); if (existingInMemorySorter == null) { this.inMemSorter = new UnsafeInMemorySorter( From dd16b52cf785ae06026bd00e8e6bedfffa791f5d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 7 Oct 2016 10:24:42 -0700 Subject: [PATCH 017/177] [SPARK-17800] Introduce InterfaceStability annotation ## What changes were proposed in this pull request? This patch introduces three new annotations under InterfaceStability: - Stable - Evolving - Unstable This is inspired by Hadoop's InterfaceStability, and the first step towards switching over to a new API stability annotation framework. ## How was this patch tested? N/A Author: Reynold Xin Closes #15374 from rxin/SPARK-17800. --- .../spark/annotation/InterfaceStability.java | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) create mode 100644 common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java diff --git a/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java b/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java new file mode 100644 index 0000000000000..323098f69c6e1 --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java @@ -0,0 +1,49 @@ +/* + * 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.annotation; + +import java.lang.annotation.Documented; + +/** + * Annotation to inform users of how much to rely on a particular package, + * class or method not changing over time. + */ +public class InterfaceStability { + + /** + * Stable APIs that retain source and binary compatibility within a major release. + * These interfaces can change from one major release to another major release + * (e.g. from 1.0 to 2.0). + */ + @Documented + public @interface Stable {}; + + /** + * APIs that are meant to evolve towards becoming stable APIs, but are not stable APIs yet. + * Evolving interfaces can change from one feature release to another release (i.e. 2.1 to 2.2). + */ + @Documented + public @interface Evolving {}; + + /** + * Unstable APIs, with no guarantee on stability. + * Classes that are unannotated are considered Unstable. + */ + @Documented + public @interface Unstable {}; +} From cff560755244dd4ccb998e0c56e81d2620cd4cff Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 7 Oct 2016 10:31:41 -0700 Subject: [PATCH 018/177] [SPARK-17707][WEBUI] Web UI prevents spark-submit application to be finished ## What changes were proposed in this pull request? This expands calls to Jetty's simple `ServerConnector` constructor to explicitly specify a `ScheduledExecutorScheduler` that makes daemon threads. It should otherwise result in exactly the same configuration, because the other args are copied from the constructor that is currently called. (I'm not sure we should change the Hive Thriftserver impl, but I did anyway.) This also adds `sc.stop()` to the quick start guide example. ## How was this patch tested? Existing tests; _pending_ at least manual verification of the fix. Author: Sean Owen Closes #15381 from srowen/SPARK-17707. --- .../deploy/rest/RestSubmissionServer.scala | 14 +++++++++--- .../org/apache/spark/ui/JettyUtils.scala | 14 +++++++++--- docs/quick-start.md | 7 +++++- .../cli/thrift/ThriftHttpCLIService.java | 22 +++++++++++++++++-- 4 files changed, 48 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index fa55d470842b3..b30c980e95a9a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -22,9 +22,9 @@ import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.{Server, ServerConnector} +import org.eclipse.jetty.server.{HttpConnectionFactory, Server, ServerConnector} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} -import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.json4s._ import org.json4s.jackson.JsonMethods._ @@ -83,7 +83,15 @@ private[spark] abstract class RestSubmissionServer( threadPool.setDaemon(true) val server = new Server(threadPool) - val connector = new ServerConnector(server) + val connector = new ServerConnector( + server, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("RestSubmissionServer-JettyScheduler", true), + null, + -1, + -1, + new HttpConnectionFactory()) connector.setHost(host) connector.setPort(startPort) server.addConnector(connector) 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 24f3f757157f3..35c3c8d00f99b 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -27,12 +27,12 @@ import scala.xml.Node import org.eclipse.jetty.client.api.Response import org.eclipse.jetty.proxy.ProxyServlet -import org.eclipse.jetty.server.{Request, Server, ServerConnector} +import org.eclipse.jetty.server.{HttpConnectionFactory, Request, Server, ServerConnector} import org.eclipse.jetty.server.handler._ import org.eclipse.jetty.servlet._ import org.eclipse.jetty.servlets.gzip.GzipHandler import org.eclipse.jetty.util.component.LifeCycle -import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} @@ -294,7 +294,15 @@ private[spark] object JettyUtils extends Logging { val server = new Server(pool) val connectors = new ArrayBuffer[ServerConnector] // Create a connector on port currentPort to listen for HTTP requests - val httpConnector = new ServerConnector(server) + val httpConnector = new ServerConnector( + server, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler(s"$serverName-JettyScheduler", true), + null, + -1, + -1, + new HttpConnectionFactory()) httpConnector.setPort(currentPort) connectors += httpConnector diff --git a/docs/quick-start.md b/docs/quick-start.md index 2eab8d19aa4c6..cb9a378199562 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -240,7 +240,8 @@ object SimpleApp { val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() - println("Lines with a: %s, Lines with b: %s".format(numAs, numBs)) + println(s"Lines with a: $numAs, Lines with b: $numBs") + sc.stop() } } {% endhighlight %} @@ -328,6 +329,8 @@ public class SimpleApp { }).count(); System.out.println("Lines with a: " + numAs + ", lines with b: " + numBs); + + sc.stop() } } {% endhighlight %} @@ -407,6 +410,8 @@ numAs = logData.filter(lambda s: 'a' in s).count() numBs = logData.filter(lambda s: 'b' in s).count() print("Lines with a: %i, lines with b: %i" % (numAs, numBs)) + +sc.stop() {% endhighlight %} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 37e4845cceb9e..341a7fdbb59b8 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -37,11 +37,15 @@ import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.TServlet; +import org.eclipse.jetty.server.AbstractConnectionFactory; +import org.eclipse.jetty.server.ConnectionFactory; +import org.eclipse.jetty.server.HttpConnectionFactory; import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.eclipse.jetty.util.thread.ExecutorThreadPool; +import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler; public class ThriftHttpCLIService extends ThriftCLIService { @@ -70,7 +74,8 @@ public void run() { httpServer = new org.eclipse.jetty.server.Server(threadPool); // Connector configs - ServerConnector connector = new ServerConnector(httpServer); + + ConnectionFactory[] connectionFactories; boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); String schemeName = useSsl ? "https" : "http"; // Change connector if SSL is used @@ -90,8 +95,21 @@ public void run() { Arrays.toString(sslContextFactory.getExcludeProtocols())); sslContextFactory.setKeyStorePath(keyStorePath); sslContextFactory.setKeyStorePassword(keyStorePassword); - connector = new ServerConnector(httpServer, sslContextFactory); + connectionFactories = AbstractConnectionFactory.getFactories( + sslContextFactory, new HttpConnectionFactory()); + } else { + connectionFactories = new ConnectionFactory[] { new HttpConnectionFactory() }; } + ServerConnector connector = new ServerConnector( + httpServer, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("HiveServer2-HttpHandler-JettyScheduler", true), + null, + -1, + -1, + connectionFactories); + connector.setPort(portNum); // Linux:yes, Windows:no connector.setReuseAddress(!Shell.WINDOWS); From aa3a6841ebaf45efb5d3930a93869948bdd0d2b6 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 7 Oct 2016 10:52:32 -0700 Subject: [PATCH 019/177] [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvider ## What changes were proposed in this pull request? This PR proposes cleaning up the confusing part in `createRelation` as discussed in https://github.com/apache/spark/pull/12601/files#r80627940 Also, this PR proposes the changes below: - Add documentation for `batchsize` and `isolationLevel`. - Move property names into `JDBCOptions` so that they can be managed in a single place. which were, `fetchsize`, `batchsize`, `isolationLevel` and `driver`. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon Closes #15263 from HyukjinKwon/SPARK-14525. --- .../jdbc/JdbcRelationProvider.scala | 82 ++++++++----------- .../datasources/jdbc/JdbcUtils.scala | 29 ++++++- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 13 +++ 4 files changed, 74 insertions(+), 52 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index ae04af2479c8d..3a8a197ef5241 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -22,6 +22,7 @@ import java.util.Properties import scala.collection.JavaConverters.mapAsJavaMapConverter import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister, RelationProvider} class JdbcRelationProvider extends CreatableRelationProvider @@ -50,67 +51,52 @@ class JdbcRelationProvider extends CreatableRelationProvider JDBCRelation(jdbcOptions.url, jdbcOptions.table, parts, properties)(sqlContext.sparkSession) } - /* - * The following structure applies to this code: - * | tableExists | !tableExists - *------------------------------------------------------------------------------------ - * Ignore | BaseRelation | CreateTable, saveTable, BaseRelation - * ErrorIfExists | ERROR | CreateTable, saveTable, BaseRelation - * Overwrite* | (DropTable, CreateTable,) | CreateTable, saveTable, BaseRelation - * | saveTable, BaseRelation | - * Append | saveTable, BaseRelation | CreateTable, saveTable, BaseRelation - * - * *Overwrite & tableExists with truncate, will not drop & create, but instead truncate - */ override def createRelation( sqlContext: SQLContext, mode: SaveMode, parameters: Map[String, String], - data: DataFrame): BaseRelation = { - val jdbcOptions = new JDBCOptions(parameters) - val url = jdbcOptions.url - val table = jdbcOptions.table - + df: DataFrame): BaseRelation = { + val options = new JDBCOptions(parameters) + val url = options.url + val table = options.table + val createTableOptions = options.createTableOptions + val isTruncate = options.isTruncate val props = new Properties() props.putAll(parameters.asJava) - val conn = JdbcUtils.createConnectionFactory(url, props)() + val conn = JdbcUtils.createConnectionFactory(url, props)() try { val tableExists = JdbcUtils.tableExists(conn, url, table) + if (tableExists) { + mode match { + case SaveMode.Overwrite => + if (isTruncate && isCascadingTruncateTable(url).contains(false)) { + // In this case, we should truncate table and then load. + truncateTable(conn, table) + saveTable(df, url, table, props) + } else { + // Otherwise, do not truncate the table, instead drop and recreate it + dropTable(conn, table) + createTable(df.schema, url, table, createTableOptions, conn) + saveTable(df, url, table, props) + } - val (doCreate, doSave) = (mode, tableExists) match { - case (SaveMode.Ignore, true) => (false, false) - case (SaveMode.ErrorIfExists, true) => throw new AnalysisException( - s"Table or view '$table' already exists, and SaveMode is set to ErrorIfExists.") - case (SaveMode.Overwrite, true) => - if (jdbcOptions.isTruncate && JdbcUtils.isCascadingTruncateTable(url) == Some(false)) { - JdbcUtils.truncateTable(conn, table) - (false, true) - } else { - JdbcUtils.dropTable(conn, table) - (true, true) - } - case (SaveMode.Append, true) => (false, true) - case (_, true) => throw new IllegalArgumentException(s"Unexpected SaveMode, '$mode'," + - " for handling existing tables.") - case (_, false) => (true, true) - } + case SaveMode.Append => + saveTable(df, url, table, props) + + case SaveMode.ErrorIfExists => + throw new AnalysisException( + s"Table or view '$table' already exists. SaveMode: ErrorIfExists.") - if (doCreate) { - val schema = JdbcUtils.schemaString(data, url) - // To allow certain options to append when create a new table, which can be - // table_options or partition_options. - // E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" - val createtblOptions = jdbcOptions.createTableOptions - val sql = s"CREATE TABLE $table ($schema) $createtblOptions" - val statement = conn.createStatement - try { - statement.executeUpdate(sql) - } finally { - statement.close() + case SaveMode.Ignore => + // With `SaveMode.Ignore` mode, if table already exists, the save operation is expected + // to not save the contents of the DataFrame and to not change the existing data. + // Therefore, it is okay to do nothing here and then just return the relation below. } + } else { + createTable(df.schema, url, table, createTableOptions, conn) + saveTable(df, url, table, props) } - if (doSave) JdbcUtils.saveTable(data, url, table, props) } finally { conn.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 3db1d1f109fb7..66f2bada2e3d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -552,7 +552,7 @@ object JdbcUtils extends Logging { isolationLevel: Int): Iterator[Byte] = { require(batchSize >= 1, s"Invalid value `${batchSize.toString}` for parameter " + - s"`${JdbcUtils.JDBC_BATCH_INSERT_SIZE}`. The minimum value is 1.") + s"`$JDBC_BATCH_INSERT_SIZE`. The minimum value is 1.") val conn = getConnection() var committed = false @@ -657,10 +657,10 @@ object JdbcUtils extends Logging { /** * Compute the schema string for this RDD. */ - def schemaString(df: DataFrame, url: String): String = { + def schemaString(schema: StructType, url: String): String = { val sb = new StringBuilder() val dialect = JdbcDialects.get(url) - df.schema.fields foreach { field => + schema.fields foreach { field => val name = dialect.quoteIdentifier(field.name) val typ: String = getJdbcType(field.dataType, dialect).databaseTypeDefinition val nullable = if (field.nullable) "" else "NOT NULL" @@ -697,4 +697,27 @@ object JdbcUtils extends Logging { getConnection, table, iterator, rddSchema, nullTypes, batchSize, dialect, isolationLevel) ) } + + /** + * Creates a table with a given schema. + */ + def createTable( + schema: StructType, + url: String, + table: String, + createTableOptions: String, + conn: Connection): Unit = { + val strSchema = schemaString(schema, url) + // Create the table if the table does not exist. + // To allow certain options to append when create a new table, which can be + // table_options or partition_options. + // E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" + val sql = s"CREATE TABLE $table ($strSchema) $createTableOptions" + val statement = conn.createStatement + try { + statement.executeUpdate(sql) + } finally { + statement.close() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 10f15ca280689..7cc3989b791ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -788,7 +788,7 @@ class JDBCSuite extends SparkFunSuite test("SPARK-16387: Reserved SQL words are not escaped by JDBC writer") { val df = spark.createDataset(Seq("a", "b", "c")).toDF("order") - val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") + val schema = JdbcUtils.schemaString(df.schema, "jdbc:mysql://localhost:3306/temp") assert(schema.contains("`order` TEXT")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 506971362f867..62b29db4d5524 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -132,6 +132,19 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { } } + test("CREATE with ignore") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + + df.write.mode(SaveMode.Ignore).jdbc(url1, "TEST.DROPTEST", properties) + assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) + assert(3 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) + + df2.write.mode(SaveMode.Ignore).jdbc(url1, "TEST.DROPTEST", properties) + assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) + assert(3 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) + } + test("CREATE with overwrite") { val df = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) From bb1aaf28eca6d9ae9af664ac3ad35cafdfc01a3b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 7 Oct 2016 11:16:24 -0700 Subject: [PATCH 020/177] [SPARK-16411][SQL][STREAMING] Add textFile to Structured Streaming. ## What changes were proposed in this pull request? Adds the textFile API which exists in DataFrameReader and serves same purpose. ## How was this patch tested? Added corresponding testcase. Author: Prashant Sharma Closes #14087 from ScrapCodes/textFile. --- .../sql/streaming/DataStreamReader.scala | 33 ++++++++++++++++++- .../sql/streaming/FileStreamSourceSuite.scala | 18 ++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 864a9cd3eb89d..87b73062180e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.StreamingRelation import org.apache.spark.sql.types.StructType @@ -283,6 +283,37 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo */ def text(path: String): DataFrame = format("text").load(path) + /** + * Loads text file(s) and returns a [[Dataset]] of String. The underlying schema of the Dataset + * contains a single string column named "value". + * + * If the directory structure of the text files contains partitioning information, those are + * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. + * + * Each line in the text file is a new element in the resulting Dataset. For example: + * {{{ + * // Scala: + * spark.readStream.textFile("/path/to/spark/README.md") + * + * // Java: + * spark.readStream().textFile("/path/to/spark/README.md") + * }}} + * + * You can set the following text-specific options to deal with text files: + *
    + *
  • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
  • + *
+ * + * @param path input path + * @since 2.1.0 + */ + def textFile(path: String): Dataset[String] = { + if (userSpecifiedSchema.nonEmpty) { + throw new AnalysisException("User specified schema not supported with `textFile`") + } + text(path).select("value").as[String](sparkSession.implicits.newStringEncoder) + } /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options 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 3157afe5a56c0..7f9c981a4e9c9 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 @@ -342,6 +342,24 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + test("read from textfile") { + withTempDirs { case (src, tmp) => + val textStream = spark.readStream.textFile(src.getCanonicalPath) + val filtered = textStream.filter(_.contains("keep")) + + testStream(filtered)( + AddTextFileData("drop1\nkeep2\nkeep3", src, tmp), + CheckAnswer("keep2", "keep3"), + StopStream, + AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), + StartStream(), + CheckAnswer("keep2", "keep3", "keep5", "keep6"), + AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") + ) + } + } + test("SPARK-17165 should not track the list of seen files indefinitely") { // This test works by: // 1. Create a file From 9d8ae853ecc5600f5c2f69565b96d5c46a8c0048 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 7 Oct 2016 11:34:49 -0700 Subject: [PATCH 021/177] [SPARK-17665][SPARKR] Support options/mode all for read/write APIs and options in other types ## What changes were proposed in this pull request? This PR includes the changes below: - Support `mode`/`options` in `read.parquet`, `write.parquet`, `read.orc`, `write.orc`, `read.text`, `write.text`, `read.json` and `write.json` APIs - Support other types (logical, numeric and string) as options for `write.df`, `read.df`, `read.parquet`, `write.parquet`, `read.orc`, `write.orc`, `read.text`, `write.text`, `read.json` and `write.json` ## How was this patch tested? Unit tests in `test_sparkSQL.R`/ `utils.R`. Author: hyukjinkwon Closes #15239 from HyukjinKwon/SPARK-17665. --- R/pkg/R/DataFrame.R | 43 +++++++++---- R/pkg/R/SQLContext.R | 23 +++++-- R/pkg/R/generics.R | 10 +-- R/pkg/R/utils.R | 22 +++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 75 +++++++++++++++++++++++ R/pkg/inst/tests/testthat/test_utils.R | 9 +++ 6 files changed, 160 insertions(+), 22 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 75861d5de7092..801d2ed4e7500 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -55,6 +55,19 @@ setMethod("initialize", "SparkDataFrame", function(.Object, sdf, isCached) { .Object }) +#' Set options/mode and then return the write object +#' @noRd +setWriteOptions <- function(write, path = NULL, mode = "error", ...) { + options <- varargsToStrEnv(...) + if (!is.null(path)) { + options[["path"]] <- path + } + jmode <- convertToJSaveMode(mode) + write <- callJMethod(write, "mode", jmode) + write <- callJMethod(write, "options", options) + write +} + #' @export #' @param sdf A Java object reference to the backing Scala DataFrame #' @param isCached TRUE if the SparkDataFrame is cached @@ -727,6 +740,8 @@ setMethod("toJSON", #' #' @param x A SparkDataFrame #' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. #' #' @family SparkDataFrame functions #' @rdname write.json @@ -743,8 +758,9 @@ setMethod("toJSON", #' @note write.json since 1.6.0 setMethod("write.json", signature(x = "SparkDataFrame", path = "character"), - function(x, path) { + function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) invisible(callJMethod(write, "json", path)) }) @@ -755,6 +771,8 @@ setMethod("write.json", #' #' @param x A SparkDataFrame #' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. #' #' @family SparkDataFrame functions #' @aliases write.orc,SparkDataFrame,character-method @@ -771,8 +789,9 @@ setMethod("write.json", #' @note write.orc since 2.0.0 setMethod("write.orc", signature(x = "SparkDataFrame", path = "character"), - function(x, path) { + function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) invisible(callJMethod(write, "orc", path)) }) @@ -783,6 +802,8 @@ setMethod("write.orc", #' #' @param x A SparkDataFrame #' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. #' #' @family SparkDataFrame functions #' @rdname write.parquet @@ -800,8 +821,9 @@ setMethod("write.orc", #' @note write.parquet since 1.6.0 setMethod("write.parquet", signature(x = "SparkDataFrame", path = "character"), - function(x, path) { + function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) invisible(callJMethod(write, "parquet", path)) }) @@ -825,6 +847,8 @@ setMethod("saveAsParquetFile", #' #' @param x A SparkDataFrame #' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. #' #' @family SparkDataFrame functions #' @aliases write.text,SparkDataFrame,character-method @@ -841,8 +865,9 @@ setMethod("saveAsParquetFile", #' @note write.text since 2.0.0 setMethod("write.text", signature(x = "SparkDataFrame", path = "character"), - function(x, path) { + function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) invisible(callJMethod(write, "text", path)) }) @@ -2637,15 +2662,9 @@ setMethod("write.df", if (is.null(source)) { source <- getDefaultSqlSource() } - jmode <- convertToJSaveMode(mode) - options <- varargsToEnv(...) - if (!is.null(path)) { - options[["path"]] <- path - } write <- callJMethod(df@sdf, "write") write <- callJMethod(write, "format", source) - write <- callJMethod(write, "mode", jmode) - write <- callJMethod(write, "options", options) + write <- setWriteOptions(write, path = path, mode = mode, ...) write <- handledCallJMethod(write, "save") }) @@ -2701,7 +2720,7 @@ setMethod("saveAsTable", source <- getDefaultSqlSource() } jmode <- convertToJSaveMode(mode) - options <- varargsToEnv(...) + options <- varargsToStrEnv(...) write <- callJMethod(df@sdf, "write") write <- callJMethod(write, "format", source) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index baa87824beb91..0d6a229e63455 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -328,6 +328,7 @@ setMethod("toDF", signature(x = "RDD"), #' It goes through the entire dataset once to determine the schema. #' #' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param ... additional external data source specific named properties. #' @return SparkDataFrame #' @rdname read.json #' @export @@ -341,11 +342,13 @@ setMethod("toDF", signature(x = "RDD"), #' @name read.json #' @method read.json default #' @note read.json since 1.6.0 -read.json.default <- function(path) { +read.json.default <- function(path, ...) { sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) sdf <- callJMethod(read, "json", paths) dataFrame(sdf) } @@ -405,16 +408,19 @@ jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { #' Loads an ORC file, returning the result as a SparkDataFrame. #' #' @param path Path of file to read. +#' @param ... additional external data source specific named properties. #' @return SparkDataFrame #' @rdname read.orc #' @export #' @name read.orc #' @note read.orc since 2.0.0 -read.orc <- function(path) { +read.orc <- function(path, ...) { sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) # Allow the user to have a more flexible definiton of the ORC file path path <- suppressWarnings(normalizePath(path)) read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) sdf <- callJMethod(read, "orc", path) dataFrame(sdf) } @@ -430,11 +436,13 @@ read.orc <- function(path) { #' @name read.parquet #' @method read.parquet default #' @note read.parquet since 1.6.0 -read.parquet.default <- function(path) { +read.parquet.default <- function(path, ...) { sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) # Allow the user to have a more flexible definiton of the Parquet file path paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) sdf <- callJMethod(read, "parquet", paths) dataFrame(sdf) } @@ -467,6 +475,7 @@ parquetFile <- function(x, ...) { #' Each line in the text file is a new row in the resulting SparkDataFrame. #' #' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param ... additional external data source specific named properties. #' @return SparkDataFrame #' @rdname read.text #' @export @@ -479,11 +488,13 @@ parquetFile <- function(x, ...) { #' @name read.text #' @method read.text default #' @note read.text since 1.6.1 -read.text.default <- function(path) { +read.text.default <- function(path, ...) { sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) sdf <- callJMethod(read, "text", paths) dataFrame(sdf) } @@ -779,7 +790,7 @@ read.df.default <- function(path = NULL, source = NULL, schema = NULL, na.string "in 'spark.sql.sources.default' configuration by default.") } sparkSession <- getSparkSession() - options <- varargsToEnv(...) + options <- varargsToStrEnv(...) if (!is.null(path)) { options[["path"]] <- path } @@ -842,7 +853,7 @@ loadDF <- function(x = NULL, ...) { #' @note createExternalTable since 1.4.0 createExternalTable.default <- function(tableName, path = NULL, source = NULL, ...) { sparkSession <- getSparkSession() - options <- varargsToEnv(...) + options <- varargsToStrEnv(...) if (!is.null(path)) { options[["path"]] <- path } diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 90a02e2778310..810aea9017743 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -651,15 +651,17 @@ setGeneric("write.jdbc", function(x, url, tableName, mode = "error", ...) { #' @rdname write.json #' @export -setGeneric("write.json", function(x, path) { standardGeneric("write.json") }) +setGeneric("write.json", function(x, path, ...) { standardGeneric("write.json") }) #' @rdname write.orc #' @export -setGeneric("write.orc", function(x, path) { standardGeneric("write.orc") }) +setGeneric("write.orc", function(x, path, ...) { standardGeneric("write.orc") }) #' @rdname write.parquet #' @export -setGeneric("write.parquet", function(x, path) { standardGeneric("write.parquet") }) +setGeneric("write.parquet", function(x, path, ...) { + standardGeneric("write.parquet") +}) #' @rdname write.parquet #' @export @@ -667,7 +669,7 @@ setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParqu #' @rdname write.text #' @export -setGeneric("write.text", function(x, path) { standardGeneric("write.text") }) +setGeneric("write.text", function(x, path, ...) { standardGeneric("write.text") }) #' @rdname schema #' @export diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index e69666453480c..fa8bb0f79ce80 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -334,6 +334,28 @@ varargsToEnv <- function(...) { env } +# Utility function to capture the varargs into environment object but all values are converted +# into string. +varargsToStrEnv <- function(...) { + pairs <- list(...) + env <- new.env() + for (name in names(pairs)) { + value <- pairs[[name]] + if (!(is.logical(value) || is.numeric(value) || is.character(value) || is.null(value))) { + stop(paste0("Unsupported type for ", name, " : ", class(value), + ". Supported types are logical, numeric, character and NULL.")) + } + if (is.logical(value)) { + env[[name]] <- tolower(as.character(value)) + } else if (is.null(value)) { + env[[name]] <- value + } else { + env[[name]] <- as.character(value) + } + } + env +} + getStorageLevel <- function(newLevel = c("DISK_ONLY", "DISK_ONLY_2", "MEMORY_AND_DISK", diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index f5ab601f274fe..6d8cfad5c1f93 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -256,6 +256,23 @@ test_that("read/write csv as DataFrame", { unlink(csvPath2) }) +test_that("Support other types for options", { + csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") + mockLinesCsv <- c("year,make,model,comment,blank", + "\"2012\",\"Tesla\",\"S\",\"No comment\",", + "1997,Ford,E350,\"Go get one now they are going fast\",", + "2015,Chevy,Volt", + "NA,Dummy,Placeholder") + writeLines(mockLinesCsv, csvPath) + + csvDf <- read.df(csvPath, "csv", header = "true", inferSchema = "true") + expected <- read.df(csvPath, "csv", header = TRUE, inferSchema = TRUE) + expect_equal(collect(csvDf), collect(expected)) + + expect_error(read.df(csvPath, "csv", header = TRUE, maxColumns = 3)) + unlink(csvPath) +}) + test_that("convert NAs to null type in DataFrames", { rdd <- parallelize(sc, list(list(1L, 2L), list(NA, 4L))) df <- createDataFrame(rdd, list("a", "b")) @@ -497,6 +514,19 @@ test_that("read/write json files", { unlink(jsonPath3) }) +test_that("read/write json files - compression option", { + df <- read.df(jsonPath, "json") + + jsonPath <- tempfile(pattern = "jsonPath", fileext = ".json") + write.json(df, jsonPath, compression = "gzip") + jsonDF <- read.json(jsonPath) + expect_is(jsonDF, "SparkDataFrame") + expect_equal(count(jsonDF), count(df)) + expect_true(length(list.files(jsonPath, pattern = ".gz")) > 0) + + unlink(jsonPath) +}) + test_that("jsonRDD() on a RDD with json string", { sqlContext <- suppressWarnings(sparkRSQL.init(sc)) rdd <- parallelize(sc, mockLines) @@ -1786,6 +1816,21 @@ test_that("read/write ORC files", { unsetHiveContext() }) +test_that("read/write ORC files - compression option", { + setHiveContext(sc) + df <- read.df(jsonPath, "json") + + orcPath2 <- tempfile(pattern = "orcPath2", fileext = ".orc") + write.orc(df, orcPath2, compression = "ZLIB") + orcDF <- read.orc(orcPath2) + expect_is(orcDF, "SparkDataFrame") + expect_equal(count(orcDF), count(df)) + expect_true(length(list.files(orcPath2, pattern = ".zlib.orc")) > 0) + + unlink(orcPath2) + unsetHiveContext() +}) + test_that("read/write Parquet files", { df <- read.df(jsonPath, "json") # Test write.df and read.df @@ -1817,6 +1862,23 @@ test_that("read/write Parquet files", { unlink(parquetPath4) }) +test_that("read/write Parquet files - compression option/mode", { + df <- read.df(jsonPath, "json") + tempPath <- tempfile(pattern = "tempPath", fileext = ".parquet") + + # Test write.df and read.df + write.parquet(df, tempPath, compression = "GZIP") + df2 <- read.parquet(tempPath) + expect_is(df2, "SparkDataFrame") + expect_equal(count(df2), 3) + expect_true(length(list.files(tempPath, pattern = ".gz.parquet")) > 0) + + write.parquet(df, tempPath, mode = "overwrite") + df3 <- read.parquet(tempPath) + expect_is(df3, "SparkDataFrame") + expect_equal(count(df3), 3) +}) + test_that("read/write text files", { # Test write.df and read.df df <- read.df(jsonPath, "text") @@ -1838,6 +1900,19 @@ test_that("read/write text files", { unlink(textPath2) }) +test_that("read/write text files - compression option", { + df <- read.df(jsonPath, "text") + + textPath <- tempfile(pattern = "textPath", fileext = ".txt") + write.text(df, textPath, compression = "GZIP") + textDF <- read.text(textPath) + expect_is(textDF, "SparkDataFrame") + expect_equal(count(textDF), count(df)) + expect_true(length(list.files(textPath, pattern = ".gz")) > 0) + + unlink(textPath) +}) + test_that("describe() and summarize() on a DataFrame", { df <- read.json(jsonPath) stats <- describe(df, "age") diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 69ed5549168b1..a20254e9b3fa9 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -217,4 +217,13 @@ test_that("rbindRaws", { }) +test_that("varargsToStrEnv", { + strenv <- varargsToStrEnv(a = 1, b = 1.1, c = TRUE, d = "abcd") + env <- varargsToEnv(a = "1", b = "1.1", c = "true", d = "abcd") + expect_equal(strenv, env) + expect_error(varargsToStrEnv(a = list(1, "a")), + paste0("Unsupported type for a : list. Supported types are logical, ", + "numeric, character and NULL.")) +}) + sparkR.session.stop() From 2badb58cdd7833465202197c4c52db5aa3d4c6e7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 7 Oct 2016 13:45:00 -0700 Subject: [PATCH 022/177] [SPARK-15621][SQL] Support spilling for Python UDF ## What changes were proposed in this pull request? When execute a Python UDF, we buffer the input row into as queue, then pull them out to join with the result from Python UDF. In the case that Python UDF is slow or the input row is too wide, we could ran out of memory because of the queue. Since we can't flush all the buffers (sockets) between JVM and Python process from JVM side, we can't limit the rows in the queue, otherwise it could deadlock. This PR will manage the memory used by the queue, spill that into disk when there is no enough memory (also release the memory and disk space as soon as possible). ## How was this patch tested? Added unit tests. Also manually ran a workload with large input row and slow python UDF (with large broadcast) like this: ``` b = range(1<<24) add = udf(lambda x: x + len(b), IntegerType()) df = sqlContext.range(1, 1<<26, 1, 4) print df.select(df.id, lit("adf"*10000).alias("s"), add(df.id).alias("add")).groupBy(length("s")).sum().collect() ``` It ran out of memory (hang because of full GC) before the patch, ran smoothly after the patch. Author: Davies Liu Closes #15089 from davies/spill_udf. --- .../python/BatchEvalPythonExec.scala | 36 ++- .../spark/sql/execution/python/RowQueue.scala | 280 ++++++++++++++++++ .../sql/execution/python/RowQueueSuite.scala | 127 ++++++++ 3 files changed, 436 insertions(+), 7 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index d9bf4d3ccf698..f9d20ad090056 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -17,18 +17,21 @@ package org.apache.spark.sql.execution.python +import java.io.File + import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import net.razorvine.pickle.{Pickler, Unpickler} -import org.apache.spark.TaskContext +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.api.python.{ChainedPythonFunctions, PythonRunner} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.util.Utils /** @@ -37,9 +40,25 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} * Python evaluation works by sending the necessary (projected) input data via a socket to an * external Python process, and combine the result from the Python process with the original row. * - * For each row we send to Python, we also put it in a queue. For each output row from Python, + * For each row we send to Python, we also put it in a queue first. For each output row from Python, * we drain the queue to find the original input row. Note that if the Python process is way too - * slow, this could lead to the queue growing unbounded and eventually run out of memory. + * slow, this could lead to the queue growing unbounded and spill into disk when run out of memory. + * + * Here is a diagram to show how this works: + * + * Downstream (for parent) + * / \ + * / socket (output of UDF) + * / \ + * RowQueue Python + * \ / + * \ socket (input of UDF) + * \ / + * upstream (from child) + * + * The rows sent to and received from Python are packed into batches (100 rows) and serialized, + * there should be always some rows buffered in the socket or Python process, so the pulling from + * RowQueue ALWAYS happened after pushing into it. */ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], child: SparkPlan) extends SparkPlan { @@ -70,7 +89,11 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi // The queue used to buffer input rows so we can drain it to // combine input with output from Python. - val queue = new java.util.concurrent.ConcurrentLinkedQueue[InternalRow]() + val queue = HybridRowQueue(TaskContext.get().taskMemoryManager(), + new File(Utils.getLocalDir(SparkEnv.get.conf)), child.output.length) + TaskContext.get().addTaskCompletionListener({ ctx => + queue.close() + }) val (pyFuncs, inputs) = udfs.map(collectFunctions).unzip @@ -98,7 +121,7 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi // For each row, add it to the queue. val inputIterator = iter.grouped(100).map { inputRows => val toBePickled = inputRows.map { inputRow => - queue.add(inputRow) + queue.add(inputRow.asInstanceOf[UnsafeRow]) val row = projection(inputRow) if (needConversion) { EvaluatePython.toJava(row, schema) @@ -132,7 +155,6 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi StructType(udfs.map(u => StructField("", u.dataType, u.nullable))) } val resultProj = UnsafeProjection.create(output, output) - outputIterator.flatMap { pickedResult => val unpickledBatch = unpickle.loads(pickedResult) unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala @@ -144,7 +166,7 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi } else { EvaluatePython.fromJava(result, resultType).asInstanceOf[InternalRow] } - resultProj(joined(queue.poll(), row)) + resultProj(joined(queue.remove(), row)) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala new file mode 100644 index 0000000000000..422a3f862d96f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -0,0 +1,280 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.execution.python + +import java.io._ + +import com.google.common.io.Closeables + +import org.apache.spark.SparkException +import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.memory.MemoryBlock + +/** + * A RowQueue is an FIFO queue for UnsafeRow. + * + * This RowQueue is ONLY designed and used for Python UDF, which has only one writer and only one + * reader, the reader ALWAYS ran behind the writer. See the doc of class [[BatchEvalPythonExec]] + * on how it works. + */ +private[python] trait RowQueue { + + /** + * Add a row to the end of it, returns true iff the row has been added to the queue. + */ + def add(row: UnsafeRow): Boolean + + /** + * Retrieve and remove the first row, returns null if it's empty. + * + * It can only be called after add is called, otherwise it will fail (NPE). + */ + def remove(): UnsafeRow + + /** + * Cleanup all the resources. + */ + def close(): Unit +} + +/** + * A RowQueue that is based on in-memory page. UnsafeRows are appended into it until it's full. + * Another thread could read from it at the same time (behind the writer). + * + * The format of UnsafeRow in page: + * [4 bytes to hold length of record (N)] [N bytes to hold record] [...] + * + * -1 length means end of page. + */ +private[python] abstract class InMemoryRowQueue(val page: MemoryBlock, numFields: Int) + extends RowQueue { + private val base: AnyRef = page.getBaseObject + private val endOfPage: Long = page.getBaseOffset + page.size + // the first location where a new row would be written + private var writeOffset = page.getBaseOffset + // points to the start of the next row to read + private var readOffset = page.getBaseOffset + private val resultRow = new UnsafeRow(numFields) + + def add(row: UnsafeRow): Boolean = synchronized { + val size = row.getSizeInBytes + if (writeOffset + 4 + size > endOfPage) { + // if there is not enough space in this page to hold the new record + if (writeOffset + 4 <= endOfPage) { + // if there's extra space at the end of the page, store a special "end-of-page" length (-1) + Platform.putInt(base, writeOffset, -1) + } + false + } else { + Platform.putInt(base, writeOffset, size) + Platform.copyMemory(row.getBaseObject, row.getBaseOffset, base, writeOffset + 4, size) + writeOffset += 4 + size + true + } + } + + def remove(): UnsafeRow = synchronized { + assert(readOffset <= writeOffset, "reader should not go beyond writer") + if (readOffset + 4 > endOfPage || Platform.getInt(base, readOffset) < 0) { + null + } else { + val size = Platform.getInt(base, readOffset) + resultRow.pointTo(base, readOffset + 4, size) + readOffset += 4 + size + resultRow + } + } +} + +/** + * A RowQueue that is backed by a file on disk. This queue will stop accepting new rows once any + * reader has begun reading from the queue. + */ +private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueue { + private var out = new DataOutputStream( + new BufferedOutputStream(new FileOutputStream(file.toString))) + private var unreadBytes = 0L + + private var in: DataInputStream = _ + private val resultRow = new UnsafeRow(fields) + + def add(row: UnsafeRow): Boolean = synchronized { + if (out == null) { + // Another thread is reading, stop writing this one + return false + } + out.writeInt(row.getSizeInBytes) + out.write(row.getBytes) + unreadBytes += 4 + row.getSizeInBytes + true + } + + def remove(): UnsafeRow = synchronized { + if (out != null) { + out.close() + out = null + in = new DataInputStream(new BufferedInputStream(new FileInputStream(file.toString))) + } + + if (unreadBytes > 0) { + val size = in.readInt() + val bytes = new Array[Byte](size) + in.readFully(bytes) + unreadBytes -= 4 + size + resultRow.pointTo(bytes, size) + resultRow + } else { + null + } + } + + def close(): Unit = synchronized { + Closeables.close(out, true) + out = null + Closeables.close(in, true) + in = null + if (file.exists()) { + file.delete() + } + } +} + +/** + * A RowQueue that has a list of RowQueues, which could be in memory or disk. + * + * HybridRowQueue could be safely appended in one thread, and pulled in another thread in the same + * time. + */ +private[python] case class HybridRowQueue( + memManager: TaskMemoryManager, + tempDir: File, + numFields: Int) + extends MemoryConsumer(memManager) with RowQueue { + + // Each buffer should have at least one row + private var queues = new java.util.LinkedList[RowQueue]() + + private var writing: RowQueue = _ + private var reading: RowQueue = _ + + // exposed for testing + private[python] def numQueues(): Int = queues.size() + + def spill(size: Long, trigger: MemoryConsumer): Long = { + if (trigger == this) { + // When it's triggered by itself, it should write upcoming rows into disk instead of copying + // the rows already in the queue. + return 0L + } + var released = 0L + synchronized { + // poll out all the buffers and add them back in the same order to make sure that the rows + // are in correct order. + val newQueues = new java.util.LinkedList[RowQueue]() + while (!queues.isEmpty) { + val queue = queues.remove() + val newQueue = if (!queues.isEmpty && queue.isInstanceOf[InMemoryRowQueue]) { + val diskQueue = createDiskQueue() + var row = queue.remove() + while (row != null) { + diskQueue.add(row) + row = queue.remove() + } + released += queue.asInstanceOf[InMemoryRowQueue].page.size() + queue.close() + diskQueue + } else { + queue + } + newQueues.add(newQueue) + } + queues = newQueues + } + released + } + + private def createDiskQueue(): RowQueue = { + DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields) + } + + private def createNewQueue(required: Long): RowQueue = { + val page = try { + allocatePage(required) + } catch { + case _: OutOfMemoryError => + null + } + val buffer = if (page != null) { + new InMemoryRowQueue(page, numFields) { + override def close(): Unit = { + freePage(page) + } + } + } else { + createDiskQueue() + } + + synchronized { + queues.add(buffer) + } + buffer + } + + def add(row: UnsafeRow): Boolean = { + if (writing == null || !writing.add(row)) { + writing = createNewQueue(4 + row.getSizeInBytes) + if (!writing.add(row)) { + throw new SparkException(s"failed to push a row into $writing") + } + } + true + } + + def remove(): UnsafeRow = { + var row: UnsafeRow = null + if (reading != null) { + row = reading.remove() + } + if (row == null) { + if (reading != null) { + reading.close() + } + synchronized { + reading = queues.remove() + } + assert(reading != null, s"queue should not be empty") + row = reading.remove() + assert(row != null, s"$reading should have at least one row") + } + row + } + + def close(): Unit = { + if (reading != null) { + reading.close() + reading = null + } + synchronized { + while (!queues.isEmpty) { + queues.remove().close() + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala new file mode 100644 index 0000000000000..ffda33cf906c5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import java.io.File + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.util.Utils + +class RowQueueSuite extends SparkFunSuite { + + test("in-memory queue") { + val page = MemoryBlock.fromLongArray(new Array[Long](1<<10)) + val queue = new InMemoryRowQueue(page, 1) { + override def close() {} + } + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](16), 16) + val n = page.size() / (4 + row.getSizeInBytes) + var i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + assert(!queue.add(row), "should not add more") + i = 0 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + assert(queue.remove() == null, "should be empty") + queue.close() + } + + test("disk queue") { + val dir = Utils.createTempDir().getCanonicalFile + dir.mkdirs() + val queue = DiskRowQueue(new File(dir, "buffer"), 1) + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](16), 16) + val n = 1000 + var i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + val first = queue.remove() + assert(first != null, "first should not be null") + assert(first.getLong(0) == 0, "first should be 0") + assert(!queue.add(row), "should not add more") + i = 1 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + assert(queue.remove() == null, "should be empty") + queue.close() + } + + test("hybrid queue") { + val mem = new TestMemoryManager(new SparkConf()) + mem.limit(4<<10) + val taskM = new TaskMemoryManager(mem, 0) + val queue = HybridRowQueue(taskM, Utils.createTempDir().getCanonicalFile, 1) + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](16), 16) + val n = (4<<10) / 16 * 3 + var i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + assert(queue.numQueues() > 1, "should have more than one queue") + queue.spill(1<<20, null) + i = 0 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + + // fill again and spill + i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + assert(queue.numQueues() > 1, "should have more than one queue") + queue.spill(1<<20, null) + assert(queue.numQueues() > 1, "should have more than one queue") + i = 0 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + queue.close() + } +} From 97594c29b723f372a5c4c061760015bd78d01f50 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Fri, 7 Oct 2016 14:03:45 -0700 Subject: [PATCH 023/177] [SPARK-17761][SQL] Remove MutableRow ## What changes were proposed in this pull request? In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`. The code below illustrates the immutability issue with InternalRow: ```scala import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericMutableRow val struct = new GenericMutableRow(1) val row = InternalRow(struct, 1) println(row) scala> [[null], 1] struct.setInt(0, 42) println(row) scala> [[42], 1] ``` This might be somewhat controversial, so feedback is appreciated. ## How was this patch tested? Existing tests. Author: Herman van Hovell Closes #15333 from hvanhovell/SPARK-17761. --- .../apache/spark/ml/linalg/MatrixUDT.scala | 4 +- .../apache/spark/ml/linalg/VectorUDT.scala | 6 +- .../apache/spark/mllib/linalg/Matrices.scala | 4 +- .../apache/spark/mllib/linalg/Vectors.scala | 6 +- .../sql/catalyst/expressions/UnsafeRow.java | 2 +- .../spark/sql/catalyst/InternalRow.scala | 23 +++++- .../catalyst/encoders/ExpressionEncoder.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/JoinedRow.scala | 16 +++++ .../sql/catalyst/expressions/Projection.scala | 4 +- ...bleRow.scala => SpecificInternalRow.scala} | 5 +- .../aggregate/HyperLogLogPlusPlus.scala | 6 +- .../expressions/aggregate/PivotFirst.scala | 10 +-- .../expressions/aggregate/collect.scala | 6 +- .../expressions/aggregate/interfaces.scala | 14 ++-- .../expressions/codegen/CodeGenerator.scala | 3 +- .../codegen/GenerateMutableProjection.scala | 8 +-- .../codegen/GenerateSafeProjection.scala | 8 +-- .../sql/catalyst/expressions/package.scala | 2 +- .../spark/sql/catalyst/expressions/rows.scala | 44 +----------- .../sql/catalyst/json/JacksonParser.scala | 4 +- .../sql/catalyst/ScalaReflectionSuite.scala | 4 +- .../expressions/CodeGenerationSuite.scala | 16 ++--- .../catalyst/expressions/MapDataSuite.scala | 2 +- .../expressions/UnsafeRowConverterSuite.scala | 26 +++---- .../ApproximatePercentileSuite.scala | 9 +-- .../aggregate/HyperLogLogPlusPlusSuite.scala | 13 ++-- .../execution/vectorized/ColumnarBatch.java | 7 +- .../spark/sql/execution/ExistingRDD.scala | 4 +- .../aggregate/AggregationIterator.scala | 26 +++---- .../SortBasedAggregationIterator.scala | 6 +- .../TungstenAggregationIterator.scala | 8 +-- .../spark/sql/execution/aggregate/udaf.scala | 38 +++++----- .../execution/columnar/ColumnAccessor.scala | 13 ++-- .../sql/execution/columnar/ColumnType.scala | 72 +++++++++---------- .../columnar/GenerateColumnAccessor.scala | 6 +- .../columnar/NullableColumnAccessor.scala | 4 +- .../CompressibleColumnAccessor.scala | 4 +- .../compression/CompressionScheme.scala | 3 +- .../compression/compressionSchemes.scala | 20 +++--- .../datasources/DataSourceStrategy.scala | 2 +- .../datasources/csv/CSVRelation.scala | 4 +- .../datasources/jdbc/JdbcUtils.scala | 34 ++++----- .../parquet/ParquetRowConverter.scala | 6 +- .../joins/BroadcastNestedLoopJoinExec.scala | 10 +-- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../apache/spark/sql/execution/objects.scala | 4 +- .../python/BatchEvalPythonExec.scala | 2 +- .../sql/execution/stat/StatFunctions.scala | 4 +- .../execution/window/AggregateProcessor.scala | 4 +- .../sql/execution/window/WindowExec.scala | 12 ++-- .../window/WindowFunctionFrame.scala | 10 +-- .../scala/org/apache/spark/sql/RowSuite.scala | 6 +- .../sql/TypedImperativeAggregateSuite.scala | 6 +- .../execution/columnar/ColumnTypeSuite.scala | 4 +- .../columnar/ColumnarTestUtils.scala | 12 ++-- .../NullableColumnAccessorSuite.scala | 4 +- .../columnar/NullableColumnBuilderSuite.scala | 4 +- .../compression/BooleanBitSetSuite.scala | 4 +- .../CompressionSchemeBenchmark.scala | 4 +- .../compression/DictionaryEncodingSuite.scala | 4 +- .../compression/IntegralDeltaSuite.scala | 6 +- .../compression/RunLengthEncodingSuite.scala | 4 +- .../datasources/parquet/ParquetIOSuite.scala | 4 +- .../parquet/ParquetQuerySuite.scala | 4 +- .../spark/sql/hive/HiveInspectors.scala | 18 ++--- .../apache/spark/sql/hive/TableReader.scala | 38 +++++----- .../hive/execution/ScriptTransformation.scala | 2 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 6 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 2 +- 71 files changed, 343 insertions(+), 347 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{SpecificMutableRow.scala => SpecificInternalRow.scala} (98%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala index a1e53662f02a8..f4a8556c71f6e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.linalg import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ /** @@ -46,7 +46,7 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { } override def serialize(obj: Matrix): InternalRow = { - val row = new GenericMutableRow(7) + val row = new GenericInternalRow(7) obj match { case sm: SparseMatrix => row.setByte(0, 0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala index 0b9b2ff5c5e26..917861309c573 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.linalg import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ /** @@ -42,14 +42,14 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { override def serialize(obj: Vector): InternalRow = { obj match { case SparseVector(size, indices, values) => - val row = new GenericMutableRow(4) + val row = new GenericInternalRow(4) row.setByte(0, 0) row.setInt(1, size) row.update(2, UnsafeArrayData.fromPrimitiveArray(indices)) row.update(3, UnsafeArrayData.fromPrimitiveArray(values)) row case DenseVector(values) => - val row = new GenericMutableRow(4) + val row = new GenericInternalRow(4) row.setByte(0, 1) row.setNullAt(1) row.setNullAt(2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 6642999a2121f..542a69b3ef8cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -28,7 +28,7 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.annotation.Since import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ /** @@ -189,7 +189,7 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { } override def serialize(obj: Matrix): InternalRow = { - val row = new GenericMutableRow(7) + val row = new GenericInternalRow(7) obj match { case sm: SparseMatrix => row.setByte(0, 0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 91f065831c804..fbd217af74ecb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -34,7 +34,7 @@ import org.apache.spark.annotation.{AlphaComponent, Since} import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.NumericParser import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ /** @@ -214,14 +214,14 @@ class VectorUDT extends UserDefinedType[Vector] { override def serialize(obj: Vector): InternalRow = { obj match { case SparseVector(size, indices, values) => - val row = new GenericMutableRow(4) + val row = new GenericInternalRow(4) row.setByte(0, 0) row.setInt(1, size) row.update(2, UnsafeArrayData.fromPrimitiveArray(indices)) row.update(3, UnsafeArrayData.fromPrimitiveArray(values)) row case DenseVector(values) => - val row = new GenericMutableRow(4) + val row = new GenericInternalRow(4) row.setByte(0, 1) row.setNullAt(1) row.setNullAt(2) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 9027652d57f14..c3f0abac244cf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -59,7 +59,7 @@ * * Instances of `UnsafeRow` act as pointers to row data stored in this format. */ -public final class UnsafeRow extends MutableRow implements Externalizable, KryoSerializable { +public final class UnsafeRow extends InternalRow implements Externalizable, KryoSerializable { ////////////////////////////////////////////////////////////////////////////// // Static methods diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index eba95c5c8b908..f498e071b50a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.{DataType, Decimal, StructType} /** * An abstract class for row used internal in Spark SQL, which only contain the columns as @@ -31,6 +31,27 @@ abstract class InternalRow extends SpecializedGetters with Serializable { // This is only use for test and will throw a null pointer exception if the position is null. def getString(ordinal: Int): String = getUTF8String(ordinal).toString + def setNullAt(i: Int): Unit + + def update(i: Int, value: Any): Unit + + // default implementation (slow) + def setBoolean(i: Int, value: Boolean): Unit = update(i, value) + def setByte(i: Int, value: Byte): Unit = update(i, value) + def setShort(i: Int, value: Short): Unit = update(i, value) + def setInt(i: Int, value: Int): Unit = update(i, value) + def setLong(i: Int, value: Long): Unit = update(i, value) + def setFloat(i: Int, value: Float): Unit = update(i, value) + def setDouble(i: Int, value: Double): Unit = update(i, value) + + /** + * Update the decimal column at `i`. + * + * Note: In order to support update decimal with precision > 18 in UnsafeRow, + * CAN NOT call setNullAt() for decimal column on UnsafeRow, call setDecimal(i, null, precision). + */ + def setDecimal(i: Int, value: Decimal, precision: Int) { update(i, value) } + /** * Make a copy of the current [[InternalRow]] object. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index b96b744b4fa98..82e1a8a7cad96 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -256,7 +256,7 @@ case class ExpressionEncoder[T]( private lazy val extractProjection = GenerateUnsafeProjection.generate(serializer) @transient - private lazy val inputRow = new GenericMutableRow(1) + private lazy val inputRow = new GenericInternalRow(1) @transient private lazy val constructProjection = GenerateSafeProjection.generate(deserializer :: Nil) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 70fff51956255..1314c416510dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -403,7 +403,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case (fromField, toField) => cast(fromField.dataType, toField.dataType) } // TODO: Could be faster? - val newRow = new GenericMutableRow(from.fields.length) + val newRow = new GenericInternalRow(from.fields.length) buildCast[InternalRow](_, row => { var i = 0 while (i < row.numFields) { @@ -892,7 +892,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w val fieldsCasts = from.fields.zip(to.fields).map { case (fromField, toField) => nullSafeCastFunction(fromField.dataType, toField.dataType, ctx) } - val rowClass = classOf[GenericMutableRow].getName + val rowClass = classOf[GenericInternalRow].getName val result = ctx.freshName("result") val tmpRow = ctx.freshName("tmpRow") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala index ed894f6d6e10e..7770684a5b399 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala @@ -123,6 +123,22 @@ class JoinedRow extends InternalRow { override def anyNull: Boolean = row1.anyNull || row2.anyNull + override def setNullAt(i: Int): Unit = { + if (i < row1.numFields) { + row1.setNullAt(i) + } else { + row2.setNullAt(i - row1.numFields) + } + } + + override def update(i: Int, value: Any): Unit = { + if (i < row1.numFields) { + row1.update(i, value) + } else { + row2.update(i - row1.numFields, value) + } + } + override def copy(): InternalRow = { val copy1 = row1.copy() val copy2 = row2.copy() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index c8d18667f7c4a..a81fa1ce3adcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -69,10 +69,10 @@ case class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mu }) private[this] val exprArray = expressions.toArray - private[this] var mutableRow: MutableRow = new GenericMutableRow(exprArray.length) + private[this] var mutableRow: InternalRow = new GenericInternalRow(exprArray.length) def currentValue: InternalRow = mutableRow - override def target(row: MutableRow): MutableProjection = { + override def target(row: InternalRow): MutableProjection = { mutableRow = row this } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala similarity index 98% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala index 61ca7272dfa61..74e0b4691d4cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.types._ /** * A parent class for mutable container objects that are reused when the values are changed, - * resulting in less garbage. These values are held by a [[SpecificMutableRow]]. + * resulting in less garbage. These values are held by a [[SpecificInternalRow]]. * * The following code was roughly used to generate these objects: * {{{ @@ -191,8 +191,7 @@ final class MutableAny extends MutableValue { * based on the dataTypes of each column. The intent is to decrease garbage when modifying the * values of primitive columns. */ -final class SpecificMutableRow(val values: Array[MutableValue]) - extends MutableRow with BaseGenericInternalRow { +final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGenericInternalRow { def this(dataTypes: Seq[DataType]) = this( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala index 1d218da6db806..83c8d400c5d6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala @@ -155,7 +155,7 @@ case class HyperLogLogPlusPlus( aggBufferAttributes.map(_.newInstance()) /** Fill all words with zeros. */ - override def initialize(buffer: MutableRow): Unit = { + override def initialize(buffer: InternalRow): Unit = { var word = 0 while (word < numWords) { buffer.setLong(mutableAggBufferOffset + word, 0) @@ -168,7 +168,7 @@ case class HyperLogLogPlusPlus( * * Variable names in the HLL++ paper match variable names in the code. */ - override def update(buffer: MutableRow, input: InternalRow): Unit = { + override def update(buffer: InternalRow, input: InternalRow): Unit = { val v = child.eval(input) if (v != null) { // Create the hashed value 'x'. @@ -200,7 +200,7 @@ case class HyperLogLogPlusPlus( * Merge the HLL buffers by iterating through the registers in both buffers and select the * maximum number of leading zeros for each register. */ - override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = { + override def merge(buffer1: InternalRow, buffer2: InternalRow): Unit = { var idx = 0 var wordOffset = 0 while (wordOffset < numWords) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala index 16c03c500ad08..087606077295f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala @@ -30,7 +30,7 @@ object PivotFirst { // Currently UnsafeRow does not support the generic update method (throws // UnsupportedOperationException), so we need to explicitly support each DataType. - private val updateFunction: PartialFunction[DataType, (MutableRow, Int, Any) => Unit] = { + private val updateFunction: PartialFunction[DataType, (InternalRow, Int, Any) => Unit] = { case DoubleType => (row, offset, value) => row.setDouble(offset, value.asInstanceOf[Double]) case IntegerType => @@ -89,9 +89,9 @@ case class PivotFirst( val indexSize = pivotIndex.size - private val updateRow: (MutableRow, Int, Any) => Unit = PivotFirst.updateFunction(valueDataType) + private val updateRow: (InternalRow, Int, Any) => Unit = PivotFirst.updateFunction(valueDataType) - override def update(mutableAggBuffer: MutableRow, inputRow: InternalRow): Unit = { + override def update(mutableAggBuffer: InternalRow, inputRow: InternalRow): Unit = { val pivotColValue = pivotColumn.eval(inputRow) if (pivotColValue != null) { // We ignore rows whose pivot column value is not in the list of pivot column values. @@ -105,7 +105,7 @@ case class PivotFirst( } } - override def merge(mutableAggBuffer: MutableRow, inputAggBuffer: InternalRow): Unit = { + override def merge(mutableAggBuffer: InternalRow, inputAggBuffer: InternalRow): Unit = { for (i <- 0 until indexSize) { if (!inputAggBuffer.isNullAt(inputAggBufferOffset + i)) { val value = inputAggBuffer.get(inputAggBufferOffset + i, valueDataType) @@ -114,7 +114,7 @@ case class PivotFirst( } } - override def initialize(mutableAggBuffer: MutableRow): Unit = valueDataType match { + override def initialize(mutableAggBuffer: InternalRow): Unit = valueDataType match { case d: DecimalType => // Per doc of setDecimal we need to do this instead of setNullAt for DecimalType. for (i <- 0 until indexSize) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 78a388d20630b..89eb864e94702 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -60,11 +60,11 @@ abstract class Collect extends ImperativeAggregate { protected[this] val buffer: Growable[Any] with Iterable[Any] - override def initialize(b: MutableRow): Unit = { + override def initialize(b: InternalRow): Unit = { buffer.clear() } - override def update(b: MutableRow, input: InternalRow): Unit = { + override def update(b: InternalRow, input: InternalRow): Unit = { // Do not allow null values. We follow the semantics of Hive's collect_list/collect_set here. // See: org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMkCollectionEvaluator val value = child.eval(input) @@ -73,7 +73,7 @@ abstract class Collect extends ImperativeAggregate { } } - override def merge(buffer: MutableRow, input: InternalRow): Unit = { + override def merge(buffer: InternalRow, input: InternalRow): Unit = { sys.error("Collect cannot be used in partial aggregations.") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index b5c0844fbf310..f3fd58bc98ef6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -307,14 +307,14 @@ abstract class ImperativeAggregate extends AggregateFunction with CodegenFallbac * * Use `fieldNumber + mutableAggBufferOffset` to access fields of `mutableAggBuffer`. */ - def initialize(mutableAggBuffer: MutableRow): Unit + def initialize(mutableAggBuffer: InternalRow): Unit /** * Updates its aggregation buffer, located in `mutableAggBuffer`, based on the given `inputRow`. * * Use `fieldNumber + mutableAggBufferOffset` to access fields of `mutableAggBuffer`. */ - def update(mutableAggBuffer: MutableRow, inputRow: InternalRow): Unit + def update(mutableAggBuffer: InternalRow, inputRow: InternalRow): Unit /** * Combines new intermediate results from the `inputAggBuffer` with the existing intermediate @@ -323,7 +323,7 @@ abstract class ImperativeAggregate extends AggregateFunction with CodegenFallbac * Use `fieldNumber + mutableAggBufferOffset` to access fields of `mutableAggBuffer`. * Use `fieldNumber + inputAggBufferOffset` to access fields of `inputAggBuffer`. */ - def merge(mutableAggBuffer: MutableRow, inputAggBuffer: InternalRow): Unit + def merge(mutableAggBuffer: InternalRow, inputAggBuffer: InternalRow): Unit } /** @@ -504,16 +504,16 @@ abstract class TypedImperativeAggregate[T] extends ImperativeAggregate { /** De-serializes the serialized format Array[Byte], and produces aggregation buffer object T */ def deserialize(storageFormat: Array[Byte]): T - final override def initialize(buffer: MutableRow): Unit = { + final override def initialize(buffer: InternalRow): Unit = { val bufferObject = createAggregationBuffer() buffer.update(mutableAggBufferOffset, bufferObject) } - final override def update(buffer: MutableRow, input: InternalRow): Unit = { + final override def update(buffer: InternalRow, input: InternalRow): Unit = { update(getBufferObject(buffer), input) } - final override def merge(buffer: MutableRow, inputBuffer: InternalRow): Unit = { + final override def merge(buffer: InternalRow, inputBuffer: InternalRow): Unit = { val bufferObject = getBufferObject(buffer) // The inputBuffer stores serialized aggregation buffer object produced by partial aggregate val inputObject = deserialize(inputBuffer.getBinary(inputAggBufferOffset)) @@ -547,7 +547,7 @@ abstract class TypedImperativeAggregate[T] extends ImperativeAggregate { * This is only called when doing Partial or PartialMerge mode aggregation, before the framework * shuffle out aggregate buffers. */ - final def serializeAggregateBufferInPlace(buffer: MutableRow): Unit = { + final def serializeAggregateBufferInPlace(buffer: InternalRow): Unit = { buffer(mutableAggBufferOffset) = serialize(getBufferObject(buffer)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 574943d3d21f0..6cab50ae1bf8d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -819,7 +819,7 @@ class CodeAndComment(val body: String, val comment: collection.Map[String, Strin */ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Logging { - protected val genericMutableRowType: String = classOf[GenericMutableRow].getName + protected val genericMutableRowType: String = classOf[GenericInternalRow].getName /** * Generates a class for a given input expression. Called when there is not cached code @@ -889,7 +889,6 @@ object CodeGenerator extends Logging { classOf[UnsafeArrayData].getName, classOf[MapData].getName, classOf[UnsafeMapData].getName, - classOf[MutableRow].getName, classOf[Expression].getName )) evaluator.setExtendedClass(classOf[GeneratedClass]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 13d61af1c9b40..5c4b56b0b224c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp abstract class BaseMutableProjection extends MutableProjection /** - * Generates byte code that produces a [[MutableRow]] object that can update itself based on a new + * Generates byte code that produces a [[InternalRow]] object that can update itself based on a new * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. * It exposes a `target` method, which is used to set the row that will be updated. - * The internal [[MutableRow]] object created internally is used only when `target` is not used. + * The internal [[InternalRow]] object created internally is used only when `target` is not used. */ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableProjection] { @@ -102,7 +102,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP class SpecificMutableProjection extends ${classOf[BaseMutableProjection].getName} { private Object[] references; - private MutableRow mutableRow; + private InternalRow mutableRow; ${ctx.declareMutableStates()} public SpecificMutableProjection(Object[] references) { @@ -113,7 +113,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP ${ctx.declareAddedFunctions()} - public ${classOf[BaseMutableProjection].getName} target(MutableRow row) { + public ${classOf[BaseMutableProjection].getName} target(InternalRow row) { mutableRow = row; return this; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index 1c98c9ed10705..2773e1a666212 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.types._ abstract class BaseProjection extends Projection {} /** - * Generates byte code that produces a [[MutableRow]] object (not an [[UnsafeRow]]) that can update + * Generates byte code that produces a [[InternalRow]] object (not an [[UnsafeRow]]) that can update * itself based on a new input [[InternalRow]] for a fixed set of [[Expression Expressions]]. */ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] { @@ -164,12 +164,12 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] class SpecificSafeProjection extends ${classOf[BaseProjection].getName} { private Object[] references; - private MutableRow mutableRow; + private InternalRow mutableRow; ${ctx.declareMutableStates()} public SpecificSafeProjection(Object[] references) { this.references = references; - mutableRow = (MutableRow) references[references.length - 1]; + mutableRow = (InternalRow) references[references.length - 1]; ${ctx.initMutableStates()} } @@ -188,7 +188,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) - val resultRow = new SpecificMutableRow(expressions.map(_.dataType)) + val resultRow = new SpecificInternalRow(expressions.map(_.dataType)) c.generate(ctx.references.toArray :+ resultRow).asInstanceOf[Projection] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index a6125c61e508a..1510a4796683c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -81,7 +81,7 @@ package object expressions { def currentValue: InternalRow /** Uses the given row to store the output of the projection. */ - def target(row: MutableRow): MutableProjection + def target(row: InternalRow): MutableProjection } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 73dceb35ac50e..751b821e1b009 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -157,33 +157,6 @@ trait BaseGenericInternalRow extends InternalRow { } } -/** - * An extended interface to [[InternalRow]] that allows the values for each column to be updated. - * Setting a value through a primitive function implicitly marks that column as not null. - */ -abstract class MutableRow extends InternalRow { - def setNullAt(i: Int): Unit - - def update(i: Int, value: Any): Unit - - // default implementation (slow) - def setBoolean(i: Int, value: Boolean): Unit = { update(i, value) } - def setByte(i: Int, value: Byte): Unit = { update(i, value) } - def setShort(i: Int, value: Short): Unit = { update(i, value) } - def setInt(i: Int, value: Int): Unit = { update(i, value) } - def setLong(i: Int, value: Long): Unit = { update(i, value) } - def setFloat(i: Int, value: Float): Unit = { update(i, value) } - def setDouble(i: Int, value: Double): Unit = { update(i, value) } - - /** - * Update the decimal column at `i`. - * - * Note: In order to support update decimal with precision > 18 in UnsafeRow, - * CAN NOT call setNullAt() for decimal column on UnsafeRow, call setDecimal(i, null, precision). - */ - def setDecimal(i: Int, value: Decimal, precision: Int) { update(i, value) } -} - /** * A row implementation that uses an array of objects as the underlying storage. Note that, while * the array is not copied, and thus could technically be mutated after creation, this is not @@ -230,24 +203,9 @@ class GenericInternalRow(val values: Array[Any]) extends BaseGenericInternalRow override def numFields: Int = values.length - override def copy(): GenericInternalRow = this -} - -class GenericMutableRow(values: Array[Any]) extends MutableRow with BaseGenericInternalRow { - /** No-arg constructor for serialization. */ - protected def this() = this(null) - - def this(size: Int) = this(new Array[Any](size)) - - override protected def genericGet(ordinal: Int) = values(ordinal) - - override def toSeq(fieldTypes: Seq[DataType]): Seq[Any] = values - - override def numFields: Int = values.length - override def setNullAt(i: Int): Unit = { values(i) = null} override def update(i: Int, value: Any): Unit = { values(i) = value } - override def copy(): InternalRow = new GenericInternalRow(values.clone()) + override def copy(): GenericInternalRow = this } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index f80e6373d2f89..e476cb11a3517 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -105,7 +105,7 @@ class JacksonParser( } emptyRow } else { - val row = new GenericMutableRow(schema.length) + val row = new GenericInternalRow(schema.length) for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) { require(schema(corruptIndex).dataType == StringType) row.update(corruptIndex, UTF8String.fromString(record)) @@ -363,7 +363,7 @@ class JacksonParser( parser: JsonParser, schema: StructType, fieldConverters: Seq[ValueConverter]): InternalRow = { - val row = new GenericMutableRow(schema.length) + val row = new GenericInternalRow(schema.length) while (nextUntil(parser, JsonToken.END_OBJECT)) { schema.getFieldIndex(parser.getCurrentName) match { case Some(index) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 85563ddedc165..43b6afd9ad896 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -23,7 +23,7 @@ import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.typeOf import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, SpecificInternalRow} import org.apache.spark.sql.catalyst.expressions.objects.NewInstance import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -94,7 +94,7 @@ object TestingUDT { .add("c", DoubleType, nullable = false) override def serialize(n: NestedStruct): Any = { - val row = new SpecificMutableRow(sqlType.asInstanceOf[StructType].map(_.dataType)) + val row = new SpecificInternalRow(sqlType.asInstanceOf[StructType].map(_.dataType)) row.setInt(0, n.a) row.setLong(1, n.b) row.setDouble(2, n.c) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 5588b4429164c..0cb201e4dae3e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -68,7 +68,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val length = 5000 val expressions = List.fill(length)(EqualTo(Literal(1), Literal(1))) val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq.fill(length)(true) if (!checkResult(actual, expected)) { @@ -91,7 +91,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val expression = CaseWhen((1 to cases).map(generateCase(_))) val plan = GenerateMutableProjection.generate(Seq(expression)) - val input = new GenericMutableRow(Array[Any](UTF8String.fromString(s"${clauses}:${cases}"))) + val input = new GenericInternalRow(Array[Any](UTF8String.fromString(s"${clauses}:${cases}"))) val actual = plan(input).toSeq(Seq(expression.dataType)) assert(actual(0) == cases) @@ -101,7 +101,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val length = 5000 val expressions = Seq(CreateArray(List.fill(length)(EqualTo(Literal(1), Literal(1))))) val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq(new GenericArrayData(Seq.fill(length)(true))) if (!checkResult(actual, expected)) { @@ -116,7 +116,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { case (expr, i) => Seq(Literal(i), expr) })) val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)).map { + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)).map { case m: ArrayBasedMapData => ArrayBasedMapData.toScalaMap(m) } val expected = (0 until length).map((_, true)).toMap :: Nil @@ -130,7 +130,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val length = 5000 val expressions = Seq(CreateStruct(List.fill(length)(EqualTo(Literal(1), Literal(1))))) val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq(InternalRow(Seq.fill(length)(true): _*)) if (!checkResult(actual, expected)) { @@ -145,7 +145,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { expr => Seq(Literal(expr.toString), expr) })) val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq(InternalRow(Seq.fill(length)(true): _*)) if (!checkResult(actual, expected)) { @@ -158,7 +158,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val schema = StructType(Seq.fill(length)(StructField("int", IntegerType))) val expressions = Seq(CreateExternalRow(Seq.fill(length)(Literal(1)), schema)) val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq(Row.fromSeq(Seq.fill(length)(1))) if (!checkResult(actual, expected)) { @@ -174,7 +174,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { Literal.create("PST", StringType)) } val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq.fill(length)( DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-07-24 07:00:00"))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala index 0f1264c7c3269..25a675a90276d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala @@ -45,7 +45,7 @@ class MapDataSuite extends SparkFunSuite { // UnsafeMapData val unsafeConverter = UnsafeProjection.create(Array[DataType](MapType(StringType, IntegerType))) - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) def toUnsafeMap(map: ArrayBasedMapData): UnsafeMapData = { row.update(0, map) val unsafeRow = unsafeConverter.apply(row) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 90790dda753f8..cf3cbe270753e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -37,7 +37,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val fieldTypes: Array[DataType] = Array(LongType, LongType, IntegerType) val converter = UnsafeProjection.create(fieldTypes) - val row = new SpecificMutableRow(fieldTypes) + val row = new SpecificInternalRow(fieldTypes) row.setLong(0, 0) row.setLong(1, 1) row.setInt(2, 2) @@ -75,7 +75,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val fieldTypes: Array[DataType] = Array(LongType, StringType, BinaryType) val converter = UnsafeProjection.create(fieldTypes) - val row = new SpecificMutableRow(fieldTypes) + val row = new SpecificInternalRow(fieldTypes) row.setLong(0, 0) row.update(1, UTF8String.fromString("Hello")) row.update(2, "World".getBytes(StandardCharsets.UTF_8)) @@ -94,7 +94,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val fieldTypes: Array[DataType] = Array(LongType, StringType, DateType, TimestampType) val converter = UnsafeProjection.create(fieldTypes) - val row = new SpecificMutableRow(fieldTypes) + val row = new SpecificInternalRow(fieldTypes) row.setLong(0, 0) row.update(1, UTF8String.fromString("Hello")) row.update(2, DateTimeUtils.fromJavaDate(Date.valueOf("1970-01-01"))) @@ -138,7 +138,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val converter = UnsafeProjection.create(fieldTypes) val rowWithAllNullColumns: InternalRow = { - val r = new SpecificMutableRow(fieldTypes) + val r = new SpecificInternalRow(fieldTypes) for (i <- fieldTypes.indices) { r.setNullAt(i) } @@ -167,7 +167,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { // columns, then the serialized row representation should be identical to what we would get by // creating an entirely null row via the converter val rowWithNoNullColumns: InternalRow = { - val r = new SpecificMutableRow(fieldTypes) + val r = new SpecificInternalRow(fieldTypes) r.setNullAt(0) r.setBoolean(1, false) r.setByte(2, 20) @@ -243,11 +243,11 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { test("NaN canonicalization") { val fieldTypes: Array[DataType] = Array(FloatType, DoubleType) - val row1 = new SpecificMutableRow(fieldTypes) + val row1 = new SpecificInternalRow(fieldTypes) row1.setFloat(0, java.lang.Float.intBitsToFloat(0x7f800001)) row1.setDouble(1, java.lang.Double.longBitsToDouble(0x7ff0000000000001L)) - val row2 = new SpecificMutableRow(fieldTypes) + val row2 = new SpecificInternalRow(fieldTypes) row2.setFloat(0, java.lang.Float.intBitsToFloat(0x7fffffff)) row2.setDouble(1, java.lang.Double.longBitsToDouble(0x7fffffffffffffffL)) @@ -263,7 +263,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val converter = UnsafeProjection.create(fieldTypes) - val row = new GenericMutableRow(fieldTypes.length) + val row = new GenericInternalRow(fieldTypes.length) row.update(0, InternalRow(1)) row.update(1, InternalRow(InternalRow(2L))) @@ -324,7 +324,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { ) val converter = UnsafeProjection.create(fieldTypes) - val row = new GenericMutableRow(fieldTypes.length) + val row = new GenericInternalRow(fieldTypes.length) row.update(0, createArray(1, 2)) row.update(1, createArray(createArray(3, 4))) @@ -359,7 +359,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerMap = createMap(5, 6)(7, 8) val map2 = createMap(9)(innerMap) - val row = new GenericMutableRow(fieldTypes.length) + val row = new GenericInternalRow(fieldTypes.length) row.update(0, map1) row.update(1, map2) @@ -400,7 +400,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { ) val converter = UnsafeProjection.create(fieldTypes) - val row = new GenericMutableRow(fieldTypes.length) + val row = new GenericInternalRow(fieldTypes.length) row.update(0, InternalRow(createArray(1))) row.update(1, createArray(InternalRow(2L))) @@ -439,7 +439,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { ) val converter = UnsafeProjection.create(fieldTypes) - val row = new GenericMutableRow(fieldTypes.length) + val row = new GenericInternalRow(fieldTypes.length) row.update(0, InternalRow(createMap(1)(2))) row.update(1, createMap(3)(InternalRow(4L))) @@ -485,7 +485,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { ) val converter = UnsafeProjection.create(fieldTypes) - val row = new GenericMutableRow(fieldTypes.length) + val row = new GenericInternalRow(fieldTypes.length) row.update(0, createArray(createMap(1)(2))) row.update(1, createMap(3)(createArray(4))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala index 61298a1b72d77..8456e244609bc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.{SimpleAnalyzer, UnresolvedAttribu import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, BoundReference, Cast, CreateArray, DecimalLiteral, GenericMutableRow, Literal} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, BoundReference, Cast, CreateArray, DecimalLiteral, GenericInternalRow, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.{PercentileDigest, PercentileDigestSerializer} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.util.ArrayData @@ -144,7 +144,8 @@ class ApproximatePercentileSuite extends SparkFunSuite { .withNewInputAggBufferOffset(inputAggregationBufferOffset) .withNewMutableAggBufferOffset(mutableAggregationBufferOffset) - val mutableAggBuffer = new GenericMutableRow(new Array[Any](mutableAggregationBufferOffset + 1)) + val mutableAggBuffer = new GenericInternalRow( + new Array[Any](mutableAggregationBufferOffset + 1)) agg.initialize(mutableAggBuffer) val dataCount = 10 (1 to dataCount).foreach { data => @@ -154,7 +155,7 @@ class ApproximatePercentileSuite extends SparkFunSuite { // Serialize the aggregation buffer val serialized = mutableAggBuffer.getBinary(mutableAggregationBufferOffset) - val inputAggBuffer = new GenericMutableRow(Array[Any](null, serialized)) + val inputAggBuffer = new GenericInternalRow(Array[Any](null, serialized)) // Phase 2: final mode aggregation // Re-initialize the aggregation buffer @@ -311,7 +312,7 @@ class ApproximatePercentileSuite extends SparkFunSuite { test("class ApproximatePercentile, null handling") { val childExpression = Cast(BoundReference(0, IntegerType, nullable = true), DoubleType) val agg = new ApproximatePercentile(childExpression, Literal(0.5D)) - val buffer = new GenericMutableRow(new Array[Any](1)) + val buffer = new GenericInternalRow(new Array[Any](1)) agg.initialize(buffer) // Empty aggregation buffer assert(agg.eval(buffer) == null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala index f5374229ca5cd..17f6b71bb270b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala @@ -22,28 +22,29 @@ import java.util.Random import scala.collection.mutable import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{BoundReference, MutableRow, SpecificMutableRow} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{BoundReference, SpecificInternalRow} import org.apache.spark.sql.types.{DataType, IntegerType} class HyperLogLogPlusPlusSuite extends SparkFunSuite { /** Create a HLL++ instance and an input and output buffer. */ def createEstimator(rsd: Double, dt: DataType = IntegerType): - (HyperLogLogPlusPlus, MutableRow, MutableRow) = { - val input = new SpecificMutableRow(Seq(dt)) + (HyperLogLogPlusPlus, InternalRow, InternalRow) = { + val input = new SpecificInternalRow(Seq(dt)) val hll = new HyperLogLogPlusPlus(new BoundReference(0, dt, true), rsd) val buffer = createBuffer(hll) (hll, input, buffer) } - def createBuffer(hll: HyperLogLogPlusPlus): MutableRow = { - val buffer = new SpecificMutableRow(hll.aggBufferAttributes.map(_.dataType)) + def createBuffer(hll: HyperLogLogPlusPlus): InternalRow = { + val buffer = new SpecificInternalRow(hll.aggBufferAttributes.map(_.dataType)) hll.initialize(buffer) buffer } /** Evaluate the estimate. It should be within 3*SD's of the given true rsd. */ - def evaluateEstimate(hll: HyperLogLogPlusPlus, buffer: MutableRow, cardinality: Int): Unit = { + def evaluateEstimate(hll: HyperLogLogPlusPlus, buffer: InternalRow, cardinality: Int): Unit = { val estimate = hll.eval(buffer).asInstanceOf[Long].toDouble val error = math.abs((estimate / cardinality.toDouble) - 1.0d) assert(error < hll.trueRsd * 3.0d, "Error should be within 3 std. errors.") diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index 62abc2a821a3a..a6ce4c2edc232 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -21,8 +21,7 @@ import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow; -import org.apache.spark.sql.catalyst.expressions.MutableRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; @@ -91,7 +90,7 @@ public void close() { * Adapter class to interop with existing components that expect internal row. A lot of * performance is lost with this translation. */ - public static final class Row extends MutableRow { + public static final class Row extends InternalRow { protected int rowId; private final ColumnarBatch parent; private final int fixedLenRowSize; @@ -129,7 +128,7 @@ public void markFiltered() { * Revisit this. This is expensive. This is currently only used in test paths. */ public InternalRow copy() { - GenericMutableRow row = new GenericMutableRow(columns.length); + GenericInternalRow row = new GenericInternalRow(columns.length); for (int i = 0; i < numFields(); i++) { if (isNullAt(i)) { row.setNullAt(i); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 6c4248c60e893..d3a22228623e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -32,7 +32,7 @@ object RDDConversions { def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { data.mapPartitions { iterator => val numColumns = outputTypes.length - val mutableRow = new GenericMutableRow(numColumns) + val mutableRow = new GenericInternalRow(numColumns) val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) iterator.map { r => var i = 0 @@ -52,7 +52,7 @@ object RDDConversions { def rowToRowRdd(data: RDD[Row], outputTypes: Seq[DataType]): RDD[InternalRow] = { data.mapPartitions { iterator => val numColumns = outputTypes.length - val mutableRow = new GenericMutableRow(numColumns) + val mutableRow = new GenericInternalRow(numColumns) val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) iterator.map { r => var i = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index f335912ba2c32..7c11fdb9792e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -153,7 +153,7 @@ abstract class AggregationIterator( protected def generateProcessRow( expressions: Seq[AggregateExpression], functions: Seq[AggregateFunction], - inputAttributes: Seq[Attribute]): (MutableRow, InternalRow) => Unit = { + inputAttributes: Seq[Attribute]): (InternalRow, InternalRow) => Unit = { val joinedRow = new JoinedRow if (expressions.nonEmpty) { val mergeExpressions = functions.zipWithIndex.flatMap { @@ -168,9 +168,9 @@ abstract class AggregationIterator( case (ae: ImperativeAggregate, i) => expressions(i).mode match { case Partial | Complete => - (buffer: MutableRow, row: InternalRow) => ae.update(buffer, row) + (buffer: InternalRow, row: InternalRow) => ae.update(buffer, row) case PartialMerge | Final => - (buffer: MutableRow, row: InternalRow) => ae.merge(buffer, row) + (buffer: InternalRow, row: InternalRow) => ae.merge(buffer, row) } }.toArray // This projection is used to merge buffer values for all expression-based aggregates. @@ -178,7 +178,7 @@ abstract class AggregationIterator( val updateProjection = newMutableProjection(mergeExpressions, aggregationBufferSchema ++ inputAttributes) - (currentBuffer: MutableRow, row: InternalRow) => { + (currentBuffer: InternalRow, row: InternalRow) => { // Process all expression-based aggregate functions. updateProjection.target(currentBuffer)(joinedRow(currentBuffer, row)) // Process all imperative aggregate functions. @@ -190,11 +190,11 @@ abstract class AggregationIterator( } } else { // Grouping only. - (currentBuffer: MutableRow, row: InternalRow) => {} + (currentBuffer: InternalRow, row: InternalRow) => {} } } - protected val processRow: (MutableRow, InternalRow) => Unit = + protected val processRow: (InternalRow, InternalRow) => Unit = generateProcessRow(aggregateExpressions, aggregateFunctions, inputAttributes) protected val groupingProjection: UnsafeProjection = @@ -202,7 +202,7 @@ abstract class AggregationIterator( protected val groupingAttributes = groupingExpressions.map(_.toAttribute) // Initializing the function used to generate the output row. - protected def generateResultProjection(): (UnsafeRow, MutableRow) => UnsafeRow = { + protected def generateResultProjection(): (UnsafeRow, InternalRow) => UnsafeRow = { val joinedRow = new JoinedRow val modes = aggregateExpressions.map(_.mode).distinct val bufferAttributes = aggregateFunctions.flatMap(_.aggBufferAttributes) @@ -211,14 +211,14 @@ abstract class AggregationIterator( case ae: DeclarativeAggregate => ae.evaluateExpression case agg: AggregateFunction => NoOp } - val aggregateResult = new SpecificMutableRow(aggregateAttributes.map(_.dataType)) + val aggregateResult = new SpecificInternalRow(aggregateAttributes.map(_.dataType)) val expressionAggEvalProjection = newMutableProjection(evalExpressions, bufferAttributes) expressionAggEvalProjection.target(aggregateResult) val resultProjection = UnsafeProjection.create(resultExpressions, groupingAttributes ++ aggregateAttributes) - (currentGroupingKey: UnsafeRow, currentBuffer: MutableRow) => { + (currentGroupingKey: UnsafeRow, currentBuffer: InternalRow) => { // Generate results for all expression-based aggregate functions. expressionAggEvalProjection(currentBuffer) // Generate results for all imperative aggregate functions. @@ -244,7 +244,7 @@ abstract class AggregationIterator( } } - (currentGroupingKey: UnsafeRow, currentBuffer: MutableRow) => { + (currentGroupingKey: UnsafeRow, currentBuffer: InternalRow) => { // Serializes the generic object stored in aggregation buffer var i = 0 while (i < typedImperativeAggregates.length) { @@ -256,17 +256,17 @@ abstract class AggregationIterator( } else { // Grouping-only: we only output values based on grouping expressions. val resultProjection = UnsafeProjection.create(resultExpressions, groupingAttributes) - (currentGroupingKey: UnsafeRow, currentBuffer: MutableRow) => { + (currentGroupingKey: UnsafeRow, currentBuffer: InternalRow) => { resultProjection(currentGroupingKey) } } } - protected val generateOutput: (UnsafeRow, MutableRow) => UnsafeRow = + protected val generateOutput: (UnsafeRow, InternalRow) => UnsafeRow = generateResultProjection() /** Initializes buffer values for all aggregate functions. */ - protected def initializeBuffer(buffer: MutableRow): Unit = { + protected def initializeBuffer(buffer: InternalRow): Unit = { expressionAggInitialProjection.target(buffer)(EmptyRow) var i = 0 while (i < allImperativeAggregateFunctions.length) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala index c2b1ef0fe3c2c..bea2dce1a7657 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -49,11 +49,11 @@ class SortBasedAggregationIterator( * Creates a new aggregation buffer and initializes buffer values * for all aggregate functions. */ - private def newBuffer: MutableRow = { + private def newBuffer: InternalRow = { val bufferSchema = aggregateFunctions.flatMap(_.aggBufferAttributes) val bufferRowSize: Int = bufferSchema.length - val genericMutableBuffer = new GenericMutableRow(bufferRowSize) + val genericMutableBuffer = new GenericInternalRow(bufferRowSize) val useUnsafeBuffer = bufferSchema.map(_.dataType).forall(UnsafeRow.isMutable) val buffer = if (useUnsafeBuffer) { @@ -84,7 +84,7 @@ class SortBasedAggregationIterator( private[this] var sortedInputHasNewGroup: Boolean = false // The aggregation buffer used by the sort-based aggregation. - private[this] val sortBasedAggregationBuffer: MutableRow = newBuffer + private[this] val sortBasedAggregationBuffer: InternalRow = newBuffer // This safe projection is used to turn the input row into safe row. This is necessary // because the input row may be produced by unsafe projection in child operator and all the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 4e072a92cc772..2988161ee5e7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -118,7 +118,7 @@ class TungstenAggregationIterator( private def createNewAggregationBuffer(): UnsafeRow = { val bufferSchema = aggregateFunctions.flatMap(_.aggBufferAttributes) val buffer: UnsafeRow = UnsafeProjection.create(bufferSchema.map(_.dataType)) - .apply(new GenericMutableRow(bufferSchema.length)) + .apply(new GenericInternalRow(bufferSchema.length)) // Initialize declarative aggregates' buffer values expressionAggInitialProjection.target(buffer)(EmptyRow) // Initialize imperative aggregates' buffer values @@ -127,7 +127,7 @@ class TungstenAggregationIterator( } // Creates a function used to generate output rows. - override protected def generateResultProjection(): (UnsafeRow, MutableRow) => UnsafeRow = { + override protected def generateResultProjection(): (UnsafeRow, InternalRow) => UnsafeRow = { val modes = aggregateExpressions.map(_.mode).distinct if (modes.nonEmpty && !modes.contains(Final) && !modes.contains(Complete)) { // Fast path for partial aggregation, UnsafeRowJoiner is usually faster than projection @@ -137,7 +137,7 @@ class TungstenAggregationIterator( val bufferSchema = StructType.fromAttributes(bufferAttributes) val unsafeRowJoiner = GenerateUnsafeRowJoiner.create(groupingKeySchema, bufferSchema) - (currentGroupingKey: UnsafeRow, currentBuffer: MutableRow) => { + (currentGroupingKey: UnsafeRow, currentBuffer: InternalRow) => { unsafeRowJoiner.join(currentGroupingKey, currentBuffer.asInstanceOf[UnsafeRow]) } } else { @@ -300,7 +300,7 @@ class TungstenAggregationIterator( private[this] val sortBasedAggregationBuffer: UnsafeRow = createNewAggregationBuffer() // The function used to process rows in a group - private[this] var sortBasedProcessRow: (MutableRow, InternalRow) => Unit = null + private[this] var sortBasedProcessRow: (InternalRow, InternalRow) => Unit = null // Processes rows in the current group. It will stop when it find a new group. private def processCurrentSortedGroup(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 586e1456ac69e..67760f334e406 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, MutableRow, _} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, _} import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} @@ -96,18 +96,18 @@ sealed trait BufferSetterGetterUtils { getters } - def createSetters(schema: StructType): Array[((MutableRow, Int, Any) => Unit)] = { + def createSetters(schema: StructType): Array[((InternalRow, Int, Any) => Unit)] = { val dataTypes = schema.fields.map(_.dataType) - val setters = new Array[(MutableRow, Int, Any) => Unit](dataTypes.length) + val setters = new Array[(InternalRow, Int, Any) => Unit](dataTypes.length) var i = 0 while (i < setters.length) { setters(i) = dataTypes(i) match { case NullType => - (row: MutableRow, ordinal: Int, value: Any) => row.setNullAt(ordinal) + (row: InternalRow, ordinal: Int, value: Any) => row.setNullAt(ordinal) case b: BooleanType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setBoolean(ordinal, value.asInstanceOf[Boolean]) } else { @@ -115,7 +115,7 @@ sealed trait BufferSetterGetterUtils { } case ByteType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setByte(ordinal, value.asInstanceOf[Byte]) } else { @@ -123,7 +123,7 @@ sealed trait BufferSetterGetterUtils { } case ShortType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setShort(ordinal, value.asInstanceOf[Short]) } else { @@ -131,7 +131,7 @@ sealed trait BufferSetterGetterUtils { } case IntegerType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setInt(ordinal, value.asInstanceOf[Int]) } else { @@ -139,7 +139,7 @@ sealed trait BufferSetterGetterUtils { } case LongType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setLong(ordinal, value.asInstanceOf[Long]) } else { @@ -147,7 +147,7 @@ sealed trait BufferSetterGetterUtils { } case FloatType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setFloat(ordinal, value.asInstanceOf[Float]) } else { @@ -155,7 +155,7 @@ sealed trait BufferSetterGetterUtils { } case DoubleType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setDouble(ordinal, value.asInstanceOf[Double]) } else { @@ -164,13 +164,13 @@ sealed trait BufferSetterGetterUtils { case dt: DecimalType => val precision = dt.precision - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => // To make it work with UnsafeRow, we cannot use setNullAt. // Please see the comment of UnsafeRow's setDecimal. row.setDecimal(ordinal, value.asInstanceOf[Decimal], precision) case DateType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setInt(ordinal, value.asInstanceOf[Int]) } else { @@ -178,7 +178,7 @@ sealed trait BufferSetterGetterUtils { } case TimestampType => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.setLong(ordinal, value.asInstanceOf[Long]) } else { @@ -186,7 +186,7 @@ sealed trait BufferSetterGetterUtils { } case other => - (row: MutableRow, ordinal: Int, value: Any) => + (row: InternalRow, ordinal: Int, value: Any) => if (value != null) { row.update(ordinal, value) } else { @@ -209,7 +209,7 @@ private[aggregate] class MutableAggregationBufferImpl( toCatalystConverters: Array[Any => Any], toScalaConverters: Array[Any => Any], bufferOffset: Int, - var underlyingBuffer: MutableRow) + var underlyingBuffer: InternalRow) extends MutableAggregationBuffer with BufferSetterGetterUtils { private[this] val offsets: Array[Int] = { @@ -413,13 +413,13 @@ case class ScalaUDAF( null) } - override def initialize(buffer: MutableRow): Unit = { + override def initialize(buffer: InternalRow): Unit = { mutableAggregateBuffer.underlyingBuffer = buffer udaf.initialize(mutableAggregateBuffer) } - override def update(buffer: MutableRow, input: InternalRow): Unit = { + override def update(buffer: InternalRow, input: InternalRow): Unit = { mutableAggregateBuffer.underlyingBuffer = buffer udaf.update( @@ -427,7 +427,7 @@ case class ScalaUDAF( inputToScalaConverters(inputProjection(input)).asInstanceOf[Row]) } - override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = { + override def merge(buffer1: InternalRow, buffer2: InternalRow): Unit = { mutableAggregateBuffer.underlyingBuffer = buffer1 inputAggregateBuffer.underlyingInputBuffer = buffer2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala index 7cde04b62619e..6241b79d9affc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala @@ -21,15 +21,16 @@ import java.nio.{ByteBuffer, ByteOrder} import scala.annotation.tailrec -import org.apache.spark.sql.catalyst.expressions.{MutableRow, UnsafeArrayData, UnsafeMapData, UnsafeRow} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeMapData, UnsafeRow} import org.apache.spark.sql.execution.columnar.compression.CompressibleColumnAccessor import org.apache.spark.sql.types._ /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is * extracted from the buffer, instead of directly returning it, the value is set into some field of - * a [[MutableRow]]. In this way, boxing cost can be avoided by leveraging the setter methods - * for primitive values provided by [[MutableRow]]. + * a [[InternalRow]]. In this way, boxing cost can be avoided by leveraging the setter methods + * for primitive values provided by [[InternalRow]]. */ private[columnar] trait ColumnAccessor { initialize() @@ -38,7 +39,7 @@ private[columnar] trait ColumnAccessor { def hasNext: Boolean - def extractTo(row: MutableRow, ordinal: Int): Unit + def extractTo(row: InternalRow, ordinal: Int): Unit protected def underlyingBuffer: ByteBuffer } @@ -52,11 +53,11 @@ private[columnar] abstract class BasicColumnAccessor[JvmType]( override def hasNext: Boolean = buffer.hasRemaining - override def extractTo(row: MutableRow, ordinal: Int): Unit = { + override def extractTo(row: InternalRow, ordinal: Int): Unit = { extractSingle(row, ordinal) } - def extractSingle(row: MutableRow, ordinal: Int): Unit = { + def extractSingle(row: InternalRow, ordinal: Int): Unit = { columnType.extract(buffer, row, ordinal) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index d27d8c362dd9a..703bde25316df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -92,7 +92,7 @@ private[columnar] sealed abstract class ColumnType[JvmType] { * `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs whenever * possible. */ - def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { setField(row, ordinal, extract(buffer)) } @@ -125,13 +125,13 @@ private[columnar] sealed abstract class ColumnType[JvmType] { * Sets `row(ordinal)` to `field`. Subclasses should override this method to avoid boxing/unboxing * costs whenever possible. */ - def setField(row: MutableRow, ordinal: Int, value: JvmType): Unit + def setField(row: InternalRow, ordinal: Int, value: JvmType): Unit /** * Copies `from(fromOrdinal)` to `to(toOrdinal)`. Subclasses should override this method to avoid * boxing/unboxing costs whenever possible. */ - def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int): Unit = { setField(to, toOrdinal, getField(from, fromOrdinal)) } @@ -149,7 +149,7 @@ private[columnar] object NULL extends ColumnType[Any] { override def defaultSize: Int = 0 override def append(v: Any, buffer: ByteBuffer): Unit = {} override def extract(buffer: ByteBuffer): Any = null - override def setField(row: MutableRow, ordinal: Int, value: Any): Unit = row.setNullAt(ordinal) + override def setField(row: InternalRow, ordinal: Int, value: Any): Unit = row.setNullAt(ordinal) override def getField(row: InternalRow, ordinal: Int): Any = null } @@ -177,18 +177,18 @@ private[columnar] object INT extends NativeColumnType(IntegerType, 4) { ByteBufferHelper.getInt(buffer) } - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { row.setInt(ordinal, ByteBufferHelper.getInt(buffer)) } - override def setField(row: MutableRow, ordinal: Int, value: Int): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Int): Unit = { row.setInt(ordinal, value) } override def getField(row: InternalRow, ordinal: Int): Int = row.getInt(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { to.setInt(toOrdinal, from.getInt(fromOrdinal)) } } @@ -206,17 +206,17 @@ private[columnar] object LONG extends NativeColumnType(LongType, 8) { ByteBufferHelper.getLong(buffer) } - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { row.setLong(ordinal, ByteBufferHelper.getLong(buffer)) } - override def setField(row: MutableRow, ordinal: Int, value: Long): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Long): Unit = { row.setLong(ordinal, value) } override def getField(row: InternalRow, ordinal: Int): Long = row.getLong(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { to.setLong(toOrdinal, from.getLong(fromOrdinal)) } } @@ -234,17 +234,17 @@ private[columnar] object FLOAT extends NativeColumnType(FloatType, 4) { ByteBufferHelper.getFloat(buffer) } - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { row.setFloat(ordinal, ByteBufferHelper.getFloat(buffer)) } - override def setField(row: MutableRow, ordinal: Int, value: Float): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Float): Unit = { row.setFloat(ordinal, value) } override def getField(row: InternalRow, ordinal: Int): Float = row.getFloat(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { to.setFloat(toOrdinal, from.getFloat(fromOrdinal)) } } @@ -262,17 +262,17 @@ private[columnar] object DOUBLE extends NativeColumnType(DoubleType, 8) { ByteBufferHelper.getDouble(buffer) } - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { row.setDouble(ordinal, ByteBufferHelper.getDouble(buffer)) } - override def setField(row: MutableRow, ordinal: Int, value: Double): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Double): Unit = { row.setDouble(ordinal, value) } override def getField(row: InternalRow, ordinal: Int): Double = row.getDouble(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { to.setDouble(toOrdinal, from.getDouble(fromOrdinal)) } } @@ -288,17 +288,17 @@ private[columnar] object BOOLEAN extends NativeColumnType(BooleanType, 1) { override def extract(buffer: ByteBuffer): Boolean = buffer.get() == 1 - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { row.setBoolean(ordinal, buffer.get() == 1) } - override def setField(row: MutableRow, ordinal: Int, value: Boolean): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Boolean): Unit = { row.setBoolean(ordinal, value) } override def getField(row: InternalRow, ordinal: Int): Boolean = row.getBoolean(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { to.setBoolean(toOrdinal, from.getBoolean(fromOrdinal)) } } @@ -316,17 +316,17 @@ private[columnar] object BYTE extends NativeColumnType(ByteType, 1) { buffer.get() } - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { row.setByte(ordinal, buffer.get()) } - override def setField(row: MutableRow, ordinal: Int, value: Byte): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Byte): Unit = { row.setByte(ordinal, value) } override def getField(row: InternalRow, ordinal: Int): Byte = row.getByte(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { to.setByte(toOrdinal, from.getByte(fromOrdinal)) } } @@ -344,17 +344,17 @@ private[columnar] object SHORT extends NativeColumnType(ShortType, 2) { buffer.getShort() } - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { row.setShort(ordinal, buffer.getShort()) } - override def setField(row: MutableRow, ordinal: Int, value: Short): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Short): Unit = { row.setShort(ordinal, value) } override def getField(row: InternalRow, ordinal: Int): Short = row.getShort(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { to.setShort(toOrdinal, from.getShort(fromOrdinal)) } } @@ -366,7 +366,7 @@ private[columnar] object SHORT extends NativeColumnType(ShortType, 2) { private[columnar] trait DirectCopyColumnType[JvmType] extends ColumnType[JvmType] { // copy the bytes from ByteBuffer to UnsafeRow - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { if (row.isInstanceOf[MutableUnsafeRow]) { val numBytes = buffer.getInt val cursor = buffer.position() @@ -407,7 +407,7 @@ private[columnar] object STRING UTF8String.fromBytes(buffer.array(), buffer.arrayOffset() + cursor, length) } - override def setField(row: MutableRow, ordinal: Int, value: UTF8String): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: UTF8String): Unit = { if (row.isInstanceOf[MutableUnsafeRow]) { row.asInstanceOf[MutableUnsafeRow].writer.write(ordinal, value) } else { @@ -419,7 +419,7 @@ private[columnar] object STRING row.getUTF8String(ordinal) } - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { setField(to, toOrdinal, getField(from, fromOrdinal)) } @@ -433,7 +433,7 @@ private[columnar] case class COMPACT_DECIMAL(precision: Int, scale: Int) Decimal(ByteBufferHelper.getLong(buffer), precision, scale) } - override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { if (row.isInstanceOf[MutableUnsafeRow]) { // copy it as Long row.setLong(ordinal, ByteBufferHelper.getLong(buffer)) @@ -459,11 +459,11 @@ private[columnar] case class COMPACT_DECIMAL(precision: Int, scale: Int) row.getDecimal(ordinal, precision, scale) } - override def setField(row: MutableRow, ordinal: Int, value: Decimal): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Decimal): Unit = { row.setDecimal(ordinal, value, precision) } - override def copyField(from: InternalRow, fromOrdinal: Int, to: MutableRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { setField(to, toOrdinal, getField(from, fromOrdinal)) } } @@ -497,7 +497,7 @@ private[columnar] object BINARY extends ByteArrayColumnType[Array[Byte]](16) { def dataType: DataType = BinaryType - override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Array[Byte]): Unit = { row.update(ordinal, value) } @@ -522,7 +522,7 @@ private[columnar] case class LARGE_DECIMAL(precision: Int, scale: Int) row.getDecimal(ordinal, precision, scale) } - override def setField(row: MutableRow, ordinal: Int, value: Decimal): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: Decimal): Unit = { row.setDecimal(ordinal, value, precision) } @@ -553,7 +553,7 @@ private[columnar] case class STRUCT(dataType: StructType) override def defaultSize: Int = 20 - override def setField(row: MutableRow, ordinal: Int, value: UnsafeRow): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: UnsafeRow): Unit = { row.update(ordinal, value) } @@ -591,7 +591,7 @@ private[columnar] case class ARRAY(dataType: ArrayType) override def defaultSize: Int = 28 - override def setField(row: MutableRow, ordinal: Int, value: UnsafeArrayData): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: UnsafeArrayData): Unit = { row.update(ordinal, value) } @@ -630,7 +630,7 @@ private[columnar] case class MAP(dataType: MapType) override def defaultSize: Int = 68 - override def setField(row: MutableRow, ordinal: Int, value: UnsafeMapData): Unit = { + override def setField(row: InternalRow, ordinal: Int, value: UnsafeMapData): Unit = { row.update(ordinal, value) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 96bd338f092e5..14024d6c10558 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -36,8 +36,7 @@ abstract class ColumnarIterator extends Iterator[InternalRow] { * * WARNING: These setter MUST be called in increasing order of ordinals. */ -class MutableUnsafeRow(val writer: UnsafeRowWriter) extends GenericMutableRow(null) { - +class MutableUnsafeRow(val writer: UnsafeRowWriter) extends BaseGenericInternalRow { override def isNullAt(i: Int): Boolean = writer.isNullAt(i) override def setNullAt(i: Int): Unit = writer.setNullAt(i) @@ -55,6 +54,9 @@ class MutableUnsafeRow(val writer: UnsafeRowWriter) extends GenericMutableRow(nu override def update(i: Int, v: Any): Unit = throw new UnsupportedOperationException // all other methods inherited from GenericMutableRow are not need + override protected def genericGet(ordinal: Int): Any = throw new UnsupportedOperationException + override def numFields: Int = throw new UnsupportedOperationException + override def copy(): InternalRow = throw new UnsupportedOperationException } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessor.scala index 2465633162c4e..2f09757aa341c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessor.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.sql.catalyst.expressions.MutableRow +import org.apache.spark.sql.catalyst.InternalRow private[columnar] trait NullableColumnAccessor extends ColumnAccessor { private var nullsBuffer: ByteBuffer = _ @@ -39,7 +39,7 @@ private[columnar] trait NullableColumnAccessor extends ColumnAccessor { super.initialize() } - abstract override def extractTo(row: MutableRow, ordinal: Int): Unit = { + abstract override def extractTo(row: InternalRow, ordinal: Int): Unit = { if (pos == nextNullIndex) { seenNulls += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala index 6579b5068e65a..e1d13ad0e94e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.columnar.compression -import org.apache.spark.sql.catalyst.expressions.MutableRow +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnAccessor, NativeColumnAccessor} import org.apache.spark.sql.types.AtomicType @@ -33,7 +33,7 @@ private[columnar] trait CompressibleColumnAccessor[T <: AtomicType] extends Colu abstract override def hasNext: Boolean = super.hasNext || decoder.hasNext - override def extractSingle(row: MutableRow, ordinal: Int): Unit = { + override def extractSingle(row: InternalRow, ordinal: Int): Unit = { decoder.next(row, ordinal) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala index b90d00b15b180..6e4f1c5b80684 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.columnar.compression import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.execution.columnar.{ColumnType, NativeColumnType} import org.apache.spark.sql.types.AtomicType @@ -39,7 +38,7 @@ private[columnar] trait Encoder[T <: AtomicType] { } private[columnar] trait Decoder[T <: AtomicType] { - def next(row: MutableRow, ordinal: Int): Unit + def next(row: InternalRow, ordinal: Int): Unit def hasNext: Boolean } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 941f03b745a07..ee99c90a751d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.types._ @@ -56,7 +56,7 @@ private[columnar] case object PassThrough extends CompressionScheme { class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { - override def next(row: MutableRow, ordinal: Int): Unit = { + override def next(row: InternalRow, ordinal: Int): Unit = { columnType.extract(buffer, row, ordinal) } @@ -86,7 +86,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { private var _compressedSize = 0 // Using `MutableRow` to store the last value to avoid boxing/unboxing cost. - private val lastValue = new SpecificMutableRow(Seq(columnType.dataType)) + private val lastValue = new SpecificInternalRow(Seq(columnType.dataType)) private var lastRun = 0 override def uncompressedSize: Int = _uncompressedSize @@ -117,9 +117,9 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { to.putInt(RunLengthEncoding.typeId) if (from.hasRemaining) { - val currentValue = new SpecificMutableRow(Seq(columnType.dataType)) + val currentValue = new SpecificInternalRow(Seq(columnType.dataType)) var currentRun = 1 - val value = new SpecificMutableRow(Seq(columnType.dataType)) + val value = new SpecificInternalRow(Seq(columnType.dataType)) columnType.extract(from, currentValue, 0) @@ -156,7 +156,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { private var valueCount = 0 private var currentValue: T#InternalType = _ - override def next(row: MutableRow, ordinal: Int): Unit = { + override def next(row: InternalRow, ordinal: Int): Unit = { if (valueCount == run) { currentValue = columnType.extract(buffer) run = ByteBufferHelper.getInt(buffer) @@ -273,7 +273,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { Array.fill[Any](elementNum)(columnType.extract(buffer).asInstanceOf[Any]) } - override def next(row: MutableRow, ordinal: Int): Unit = { + override def next(row: InternalRow, ordinal: Int): Unit = { columnType.setField(row, ordinal, dictionary(buffer.getShort()).asInstanceOf[T#InternalType]) } @@ -356,7 +356,7 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { private var visited: Int = 0 - override def next(row: MutableRow, ordinal: Int): Unit = { + override def next(row: InternalRow, ordinal: Int): Unit = { val bit = visited % BITS_PER_LONG visited += 1 @@ -443,7 +443,7 @@ private[columnar] case object IntDelta extends CompressionScheme { override def hasNext: Boolean = buffer.hasRemaining - override def next(row: MutableRow, ordinal: Int): Unit = { + override def next(row: InternalRow, ordinal: Int): Unit = { val delta = buffer.get() prev = if (delta > Byte.MinValue) prev + delta else ByteBufferHelper.getInt(buffer) row.setInt(ordinal, prev) @@ -523,7 +523,7 @@ private[columnar] case object LongDelta extends CompressionScheme { override def hasNext: Boolean = buffer.hasRemaining - override def next(row: MutableRow, ordinal: Int): Unit = { + override def next(row: InternalRow, ordinal: Int): Unit = { val delta = buffer.get() prev = if (delta > Byte.MinValue) prev + delta else ByteBufferHelper.getLong(buffer) row.setLong(ordinal, prev) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 693b4c4d0e5e9..6f9ed50a02b09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -273,7 +273,7 @@ object DataSourceStrategy extends Strategy with Logging { // Get the bucket ID based on the bucketing values. // Restriction: Bucket pruning works iff the bucketing column has one and only one column. def getBucketId(bucketColumn: Attribute, numBuckets: Int, value: Any): Int = { - val mutableRow = new SpecificMutableRow(Seq(bucketColumn.dataType)) + val mutableRow = new SpecificInternalRow(Seq(bucketColumn.dataType)) mutableRow(0) = Cast(Literal(value), bucketColumn.dataType).eval(null) val bucketIdGeneration = UnsafeProjection.create( HashPartitioning(bucketColumn :: Nil, numBuckets).partitionIdExpression :: Nil, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 33b170bc31f62..55cb26d6513af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -29,7 +29,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile, WriterContainer} import org.apache.spark.sql.types._ @@ -88,7 +88,7 @@ object CSVRelation extends Logging { case (field, index) => safeRequiredIndices(safeRequiredFields.indexOf(field)) = index } val requiredSize = requiredFields.length - val row = new GenericMutableRow(requiredSize) + val row = new GenericInternalRow(requiredSize) (tokens: Array[String], numMalformedRows) => { if (params.dropMalformed && schemaFields.length != tokens.length) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 66f2bada2e3d8..47549637b5813 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.util.{DateTimeUtils, GenericArrayData} import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.types._ @@ -283,7 +283,7 @@ object JdbcUtils extends Logging { new NextIterator[InternalRow] { private[this] val rs = resultSet private[this] val getters: Array[JDBCValueGetter] = makeGetters(schema) - private[this] val mutableRow = new SpecificMutableRow(schema.fields.map(x => x.dataType)) + private[this] val mutableRow = new SpecificInternalRow(schema.fields.map(x => x.dataType)) override protected def close(): Unit = { try { @@ -314,22 +314,22 @@ object JdbcUtils extends Logging { // A `JDBCValueGetter` is responsible for getting a value from `ResultSet` into a field // for `MutableRow`. The last argument `Int` means the index for the value to be set in // the row and also used for the value in `ResultSet`. - private type JDBCValueGetter = (ResultSet, MutableRow, Int) => Unit + private type JDBCValueGetter = (ResultSet, InternalRow, Int) => Unit /** * Creates `JDBCValueGetter`s according to [[StructType]], which can set - * each value from `ResultSet` to each field of [[MutableRow]] correctly. + * each value from `ResultSet` to each field of [[InternalRow]] correctly. */ private def makeGetters(schema: StructType): Array[JDBCValueGetter] = schema.fields.map(sf => makeGetter(sf.dataType, sf.metadata)) private def makeGetter(dt: DataType, metadata: Metadata): JDBCValueGetter = dt match { case BooleanType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => row.setBoolean(pos, rs.getBoolean(pos + 1)) case DateType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => // DateTimeUtils.fromJavaDate does not handle null value, so we need to check it. val dateVal = rs.getDate(pos + 1) if (dateVal != null) { @@ -347,25 +347,25 @@ object JdbcUtils extends Logging { // retrieve it, you will get wrong result 199.99. // So it is needed to set precision and scale for Decimal based on JDBC metadata. case DecimalType.Fixed(p, s) => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => val decimal = nullSafeConvert[java.math.BigDecimal](rs.getBigDecimal(pos + 1), d => Decimal(d, p, s)) row.update(pos, decimal) case DoubleType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => row.setDouble(pos, rs.getDouble(pos + 1)) case FloatType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => row.setFloat(pos, rs.getFloat(pos + 1)) case IntegerType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => row.setInt(pos, rs.getInt(pos + 1)) case LongType if metadata.contains("binarylong") => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => val bytes = rs.getBytes(pos + 1) var ans = 0L var j = 0 @@ -376,20 +376,20 @@ object JdbcUtils extends Logging { row.setLong(pos, ans) case LongType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => row.setLong(pos, rs.getLong(pos + 1)) case ShortType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => row.setShort(pos, rs.getShort(pos + 1)) case StringType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => // TODO(davies): use getBytes for better performance, if the encoding is UTF-8 row.update(pos, UTF8String.fromString(rs.getString(pos + 1))) case TimestampType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => val t = rs.getTimestamp(pos + 1) if (t != null) { row.setLong(pos, DateTimeUtils.fromJavaTimestamp(t)) @@ -398,7 +398,7 @@ object JdbcUtils extends Logging { } case BinaryType => - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => row.update(pos, rs.getBytes(pos + 1)) case ArrayType(et, _) => @@ -437,7 +437,7 @@ object JdbcUtils extends Logging { case _ => (array: Object) => array.asInstanceOf[Array[Any]] } - (rs: ResultSet, row: MutableRow, pos: Int) => + (rs: ResultSet, row: InternalRow, pos: Int) => val array = nullSafeConvert[Object]( rs.getArray(pos + 1).getArray, array => new GenericArrayData(elementConversion.apply(array))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 9ffc2b5dd8a56..33dcf2f3fd167 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -40,7 +40,7 @@ import org.apache.spark.unsafe.types.UTF8String /** * A [[ParentContainerUpdater]] is used by a Parquet converter to set converted values to some * corresponding parent container. For example, a converter for a `StructType` field may set - * converted values to a [[MutableRow]]; or a converter for array elements may append converted + * converted values to a [[InternalRow]]; or a converter for array elements may append converted * values to an [[ArrayBuffer]]. */ private[parquet] trait ParentContainerUpdater { @@ -155,7 +155,7 @@ private[parquet] class ParquetRowConverter( * Updater used together with field converters within a [[ParquetRowConverter]]. It propagates * converted filed values to the `ordinal`-th cell in `currentRow`. */ - private final class RowUpdater(row: MutableRow, ordinal: Int) extends ParentContainerUpdater { + private final class RowUpdater(row: InternalRow, ordinal: Int) extends ParentContainerUpdater { override def set(value: Any): Unit = row(ordinal) = value override def setBoolean(value: Boolean): Unit = row.setBoolean(ordinal, value) override def setByte(value: Byte): Unit = row.setByte(ordinal, value) @@ -166,7 +166,7 @@ private[parquet] class ParquetRowConverter( override def setFloat(value: Float): Unit = row.setFloat(ordinal, value) } - private val currentRow = new SpecificMutableRow(catalystType.map(_.dataType)) + private val currentRow = new SpecificInternalRow(catalystType.map(_.dataType)) private val unsafeProjection = UnsafeProjection.create(catalystType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index 43cdce7de8c7f..bfe7e3dea45df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -119,7 +119,7 @@ case class BroadcastNestedLoopJoinExec( streamed.execute().mapPartitionsInternal { streamedIter => val buildRows = relation.value val joinedRow = new JoinedRow - val nulls = new GenericMutableRow(broadcast.output.size) + val nulls = new GenericInternalRow(broadcast.output.size) // Returns an iterator to avoid copy the rows. new Iterator[InternalRow] { @@ -205,14 +205,14 @@ case class BroadcastNestedLoopJoinExec( val joinedRow = new JoinedRow if (condition.isDefined) { - val resultRow = new GenericMutableRow(Array[Any](null)) + val resultRow = new GenericInternalRow(Array[Any](null)) streamedIter.map { row => val result = buildRows.exists(r => boundCondition(joinedRow(row, r))) resultRow.setBoolean(0, result) joinedRow(row, resultRow) } } else { - val resultRow = new GenericMutableRow(Array[Any](buildRows.nonEmpty)) + val resultRow = new GenericInternalRow(Array[Any](buildRows.nonEmpty)) streamedIter.map { row => joinedRow(row, resultRow) } @@ -293,7 +293,7 @@ case class BroadcastNestedLoopJoinExec( } val notMatchedBroadcastRows: Seq[InternalRow] = { - val nulls = new GenericMutableRow(streamed.output.size) + val nulls = new GenericInternalRow(streamed.output.size) val buf: CompactBuffer[InternalRow] = new CompactBuffer() val joinedRow = new JoinedRow joinedRow.withLeft(nulls) @@ -311,7 +311,7 @@ case class BroadcastNestedLoopJoinExec( val matchedStreamRows = streamRdd.mapPartitionsInternal { streamedIter => val buildRows = relation.value val joinedRow = new JoinedRow - val nulls = new GenericMutableRow(broadcast.output.size) + val nulls = new GenericInternalRow(broadcast.output.size) streamedIter.flatMap { streamedRow => var i = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index fb6bfa7b2735c..8ddac19bf1b57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -192,7 +192,7 @@ trait HashJoin { streamIter: Iterator[InternalRow], hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinKeys = streamSideKeyGenerator() - val result = new GenericMutableRow(Array[Any](null)) + val result = new GenericInternalRow(Array[Any](null)) val joinedRow = new JoinedRow streamIter.map { current => val key = joinKeys(current) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 81b3e1d224ab6..ecf7cf289f034 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -275,7 +275,7 @@ case class SortMergeJoinExec( case j: ExistenceJoin => new RowIterator { private[this] var currentLeftRow: InternalRow = _ - private[this] val result: MutableRow = new GenericMutableRow(Array[Any](null)) + private[this] val result: InternalRow = new GenericInternalRow(Array[Any](null)) private[this] val smjScanner = new SortMergeJoinScanner( createLeftKeyGenerator(), createRightKeyGenerator(), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index c7e267152b5cd..2acc5110e8950 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -141,7 +141,7 @@ object ObjectOperator { def serializeObjectToRow(serializer: Seq[Expression]): Any => UnsafeRow = { val proj = GenerateUnsafeProjection.generate(serializer) val objType = serializer.head.collect { case b: BoundReference => b.dataType }.head - val objRow = new SpecificMutableRow(objType :: Nil) + val objRow = new SpecificInternalRow(objType :: Nil) (o: Any) => { objRow(0) = o proj(objRow) @@ -149,7 +149,7 @@ object ObjectOperator { } def wrapObjectToRow(objType: DataType): Any => InternalRow = { - val outputRow = new SpecificMutableRow(objType :: Nil) + val outputRow = new SpecificInternalRow(objType :: Nil) (o: Any) => { outputRow(0) = o outputRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index f9d20ad090056..dcaf2c76d479d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -147,7 +147,7 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi .compute(inputIterator, context.partitionId(), context) val unpickle = new Unpickler - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) val joined = new JoinedRow val resultType = if (udfs.length == 1) { udfs.head.dataType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index 822f49ecab47b..c02b15498748f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.stat import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} -import org.apache.spark.sql.catalyst.expressions.{Cast, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.{Cast, GenericInternalRow} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.util.QuantileSummaries import org.apache.spark.sql.functions._ @@ -186,7 +186,7 @@ object StatFunctions extends Logging { require(columnSize < 1e4, s"The number of distinct values for $col2, can't " + s"exceed 1e4. Currently $columnSize") val table = counts.groupBy(_.get(0)).map { case (col1Item, rows) => - val countsRow = new GenericMutableRow(columnSize + 1) + val countsRow = new GenericInternalRow(columnSize + 1) rows.foreach { (row: Row) => // row.get(0) is column 1 // row.get(1) is column 2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala index d3a46d020dbbf..c9f5d3b3d92d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala @@ -123,7 +123,7 @@ private[window] final class AggregateProcessor( private[this] val join = new JoinedRow private[this] val numImperatives = imperatives.length - private[this] val buffer = new SpecificMutableRow(bufferSchema.toSeq.map(_.dataType)) + private[this] val buffer = new SpecificInternalRow(bufferSchema.toSeq.map(_.dataType)) initialProjection.target(buffer) updateProjection.target(buffer) @@ -154,6 +154,6 @@ private[window] final class AggregateProcessor( } /** Evaluate buffer. */ - def evaluate(target: MutableRow): Unit = + def evaluate(target: InternalRow): Unit = evaluateProjection.target(target)(buffer) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala index 7a6a30f120386..1dd281ebf1034 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -204,7 +204,7 @@ case class WindowExec( val factory = key match { // Offset Frame case ("OFFSET", RowFrame, Some(offset), Some(h)) if offset == h => - target: MutableRow => + target: InternalRow => new OffsetWindowFunctionFrame( target, ordinal, @@ -217,7 +217,7 @@ case class WindowExec( // Growing Frame. case ("AGGREGATE", frameType, None, Some(high)) => - target: MutableRow => { + target: InternalRow => { new UnboundedPrecedingWindowFunctionFrame( target, processor, @@ -226,7 +226,7 @@ case class WindowExec( // Shrinking Frame. case ("AGGREGATE", frameType, Some(low), None) => - target: MutableRow => { + target: InternalRow => { new UnboundedFollowingWindowFunctionFrame( target, processor, @@ -235,7 +235,7 @@ case class WindowExec( // Moving Frame. case ("AGGREGATE", frameType, Some(low), Some(high)) => - target: MutableRow => { + target: InternalRow => { new SlidingWindowFunctionFrame( target, processor, @@ -245,7 +245,7 @@ case class WindowExec( // Entire Partition Frame. case ("AGGREGATE", frameType, None, None) => - target: MutableRow => { + target: InternalRow => { new UnboundedWindowFunctionFrame(target, processor) } } @@ -312,7 +312,7 @@ case class WindowExec( val inputFields = child.output.length var sorter: UnsafeExternalSorter = null var rowBuffer: RowBuffer = null - val windowFunctionResult = new SpecificMutableRow(expressions.map(_.dataType)) + val windowFunctionResult = new SpecificInternalRow(expressions.map(_.dataType)) val frames = factories.map(_(windowFunctionResult)) val numFrames = frames.length private[this] def fetchNextPartition() { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala index 2ab9faab7a59b..70efc0f78ddb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -56,7 +56,7 @@ private[window] abstract class WindowFunctionFrame { * @param offset by which rows get moved within a partition. */ private[window] final class OffsetWindowFunctionFrame( - target: MutableRow, + target: InternalRow, ordinal: Int, expressions: Array[OffsetWindowFunction], inputSchema: Seq[Attribute], @@ -136,7 +136,7 @@ private[window] final class OffsetWindowFunctionFrame( * @param ubound comparator used to identify the upper bound of an output row. */ private[window] final class SlidingWindowFunctionFrame( - target: MutableRow, + target: InternalRow, processor: AggregateProcessor, lbound: BoundOrdering, ubound: BoundOrdering) @@ -217,7 +217,7 @@ private[window] final class SlidingWindowFunctionFrame( * @param processor to calculate the row values with. */ private[window] final class UnboundedWindowFunctionFrame( - target: MutableRow, + target: InternalRow, processor: AggregateProcessor) extends WindowFunctionFrame { @@ -255,7 +255,7 @@ private[window] final class UnboundedWindowFunctionFrame( * @param ubound comparator used to identify the upper bound of an output row. */ private[window] final class UnboundedPrecedingWindowFunctionFrame( - target: MutableRow, + target: InternalRow, processor: AggregateProcessor, ubound: BoundOrdering) extends WindowFunctionFrame { @@ -317,7 +317,7 @@ private[window] final class UnboundedPrecedingWindowFunctionFrame( * @param lbound comparator used to identify the lower bound of an output row. */ private[window] final class UnboundedFollowingWindowFunctionFrame( - target: MutableRow, + target: InternalRow, processor: AggregateProcessor, lbound: BoundOrdering) extends WindowFunctionFrame { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 34936b38fb5d4..7516be315dd2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, SpecificInternalRow} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -27,7 +27,7 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ test("create row") { - val expected = new GenericMutableRow(4) + val expected = new GenericInternalRow(4) expected.setInt(0, 2147483647) expected.update(1, UTF8String.fromString("this is a string")) expected.setBoolean(2, false) @@ -49,7 +49,7 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { } test("SpecificMutableRow.update with null") { - val row = new SpecificMutableRow(Seq(IntegerType)) + val row = new SpecificInternalRow(Seq(IntegerType)) row(0) = null assert(row.isNullAt(0)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala index b5eb16b6f650b..ffa26f1f8250f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala @@ -21,7 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, Da import org.apache.spark.sql.TypedImperativeAggregateSuite.TypedMax import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, GenericMutableRow, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, GenericInternalRow, SpecificInternalRow} import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate import org.apache.spark.sql.execution.aggregate.SortAggregateExec import org.apache.spark.sql.expressions.Window @@ -64,7 +64,7 @@ class TypedImperativeAggregateSuite extends QueryTest with SharedSQLContext { assert(agg.eval(mergeBuffer) == data.map(_._1).max) // Tests low level eval(row: InternalRow) API. - val row = new GenericMutableRow(Array(mergeBuffer): Array[Any]) + val row = new GenericInternalRow(Array(mergeBuffer): Array[Any]) // Evaluates directly on row consist of aggregation buffer object. assert(agg.eval(row) == data.map(_._1).max) @@ -73,7 +73,7 @@ class TypedImperativeAggregateSuite extends QueryTest with SharedSQLContext { test("supports SpecificMutableRow as mutable row") { val aggregationBufferSchema = Seq(IntegerType, LongType, BinaryType, IntegerType) val aggBufferOffset = 2 - val buffer = new SpecificMutableRow(aggregationBufferSchema) + val buffer = new SpecificInternalRow(aggregationBufferSchema) val agg = new TypedMax(BoundReference(ordinal = 1, dataType = IntegerType, nullable = false)) .withNewMutableAggBufferOffset(aggBufferOffset) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 805b5667287ea..8bf9f521e2f06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types._ @@ -54,7 +54,7 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { expected: Int): Unit = { assertResult(expected, s"Wrong actualSize for $columnType") { - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) row.update(0, CatalystTypeConverters.convertToCatalyst(value)) val proj = UnsafeProjection.create(Array[DataType](columnType.dataType)) columnType.actualSize(proj(row), 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala index 1529313dfbd51..686c8fa6f5fa9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala @@ -21,14 +21,14 @@ import scala.collection.immutable.HashSet import scala.util.Random import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types.{AtomicType, Decimal} import org.apache.spark.unsafe.types.UTF8String object ColumnarTestUtils { - def makeNullRow(length: Int): GenericMutableRow = { - val row = new GenericMutableRow(length) + def makeNullRow(length: Int): GenericInternalRow = { + val row = new GenericInternalRow(length) (0 until length).foreach(row.setNullAt) row } @@ -86,7 +86,7 @@ object ColumnarTestUtils { tail: ColumnType[_]*): InternalRow = makeRandomRow(Seq(head) ++ tail) def makeRandomRow(columnTypes: Seq[ColumnType[_]]): InternalRow = { - val row = new GenericMutableRow(columnTypes.length) + val row = new GenericInternalRow(columnTypes.length) makeRandomValues(columnTypes).zipWithIndex.foreach { case (value, index) => row(index) = value } @@ -95,11 +95,11 @@ object ColumnarTestUtils { def makeUniqueValuesAndSingleValueRows[T <: AtomicType]( columnType: NativeColumnType[T], - count: Int): (Seq[T#InternalType], Seq[GenericMutableRow]) = { + count: Int): (Seq[T#InternalType], Seq[GenericInternalRow]) = { val values = makeUniqueRandomValues(columnType, count) val rows = values.map { value => - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) row(0) = value row } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala index dc22d3e8e4d3a..8f4ca3cea77a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.types._ class TestNullableColumnAccessor[JvmType]( @@ -72,7 +72,7 @@ class NullableColumnAccessorSuite extends SparkFunSuite { } val accessor = TestNullableColumnAccessor(builder.build(), columnType) - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) val converter = CatalystTypeConverters.createToScalaConverter(columnType.dataType) (0 until 4).foreach { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala index cdd4551d64b50..b2b6e92e9a056 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.types._ class TestNullableColumnBuilder[JvmType](columnType: ColumnType[JvmType]) @@ -94,7 +94,7 @@ class NullableColumnBuilderSuite extends SparkFunSuite { (1 to 7 by 2).foreach(assertResult(_, "Wrong null position")(buffer.getInt())) // For non-null values - val actual = new GenericMutableRow(new Array[Any](1)) + val actual = new GenericInternalRow(new Array[Any](1)) (0 until 4).foreach { _ => columnType.extract(buffer, actual, 0) assert(converter(actual.get(0, dataType)) === converter(randomRow.get(0, dataType)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala index f67e9c7dae278..d01bf911e3a77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar.{BOOLEAN, NoopColumnStats} import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ @@ -72,7 +72,7 @@ class BooleanBitSetSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (values.nonEmpty) { values.foreach { assert(decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala index babf944e6aa8e..9005ec93e786e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets import org.apache.commons.lang3.RandomStringUtils import org.apache.commons.math3.distribution.LogNormalDistribution -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar.{BOOLEAN, INT, LONG, NativeColumnType, SHORT, STRING} import org.apache.spark.sql.types.AtomicType import org.apache.spark.util.Benchmark @@ -111,7 +111,7 @@ object CompressionSchemeBenchmark extends AllCompressionSchemes { input.rewind() benchmark.addCase(label)({ i: Int => - val rowBuf = new GenericMutableRow(1) + val rowBuf = new GenericInternalRow(1) for (n <- 0L until iters) { compressedBuf.rewind.position(4) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala index 830ca0294e1b8..67139b13d7882 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.columnar.compression import java.nio.ByteBuffer import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.AtomicType @@ -97,7 +97,7 @@ class DictionaryEncodingSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = DictionaryEncoding.decoder(buffer, columnType) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index a530e270746c5..411d31fa0e29b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.IntegralType @@ -48,7 +48,7 @@ class IntegralDeltaSuite extends SparkFunSuite { } input.foreach { value => - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) columnType.setField(row, 0, value) builder.appendFrom(row, 0) } @@ -95,7 +95,7 @@ class IntegralDeltaSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = scheme.decoder(buffer, columnType) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (input.nonEmpty) { input.foreach{ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala index 95642e93ae9f0..dffa9b364ebfe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.AtomicType @@ -80,7 +80,7 @@ class RunLengthEncodingSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = RunLengthEncoding.decoder(buffer, columnType) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 3161a630af0f1..580eade4b1412 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -38,7 +38,7 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -716,7 +716,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { dataTypes.zip(constantValues).foreach { case (dt, v) => val schema = StructType(StructField("pcol", dt) :: Nil) val vectorizedReader = new VectorizedParquetRecordReader - val partitionValues = new GenericMutableRow(Array(v)) + val partitionValues = new GenericInternalRow(Array(v)) val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) try { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 9dd8d9f80496c..4c4a7d86f2bd3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -24,7 +24,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow +import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStruct, NestedStructUDT, SingleElement} import org.apache.spark.sql.internal.SQLConf @@ -719,7 +719,7 @@ object TestingUDT { .add("c", DoubleType, nullable = false) override def serialize(n: NestedStruct): Any = { - val row = new SpecificMutableRow(sqlType.asInstanceOf[StructType].map(_.dataType)) + val row = new SpecificInternalRow(sqlType.asInstanceOf[StructType].map(_.dataType)) row.setInt(0, n.a) row.setLong(1, n.b) row.setDouble(2, n.c) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index fe34caa0a3e48..1625116803505 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -688,25 +688,25 @@ private[hive] trait HiveInspectors { * @return A function that performs in-place updating of a MutableRow. * Use the overloaded ObjectInspector version for assignments. */ - def unwrapperFor(field: HiveStructField): (Any, MutableRow, Int) => Unit = + def unwrapperFor(field: HiveStructField): (Any, InternalRow, Int) => Unit = field.getFieldObjectInspector match { case oi: BooleanObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) case oi: ByteObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) case oi: ShortObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) case oi: IntObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) case oi: LongObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) case oi: FloatObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) case oi: DoubleObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) case oi => val unwrapper = unwrapperFor(oi) - (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrapper(value) + (value: Any, row: InternalRow, ordinal: Int) => row(ordinal) = unwrapper(value) } def wrap(a: Any, oi: ObjectInspector, dataType: DataType): AnyRef = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index ec7e53efc87f9..2a54163a04e9b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -120,7 +120,7 @@ class HadoopTableReader( val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) val attrsWithIndex = attributes.zipWithIndex - val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHadoopConf.value.value @@ -215,7 +215,7 @@ class HadoopTableReader( val tableDesc = relation.tableDesc val broadcastedHiveConf = _broadcastedHadoopConf val localDeserializer = partDeserializer - val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) // Splits all attributes into two groups, partition key attributes and those that are not. // Attached indices indicate the position of each attribute in the output schema. @@ -224,7 +224,7 @@ class HadoopTableReader( relation.partitionKeys.contains(attr) } - def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow): Unit = { + def fillPartitionKeys(rawPartValues: Array[String], row: InternalRow): Unit = { partitionKeyAttrs.foreach { case (attr, ordinal) => val partOrdinal = relation.partitionKeys.indexOf(attr) row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null) @@ -360,7 +360,7 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { iterator: Iterator[Writable], rawDeser: Deserializer, nonPartitionKeyAttrs: Seq[(Attribute, Int)], - mutableRow: MutableRow, + mutableRow: InternalRow, tableDeser: Deserializer): Iterator[InternalRow] = { val soi = if (rawDeser.getObjectInspector.equals(tableDeser.getObjectInspector)) { @@ -381,43 +381,43 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { * Builds specific unwrappers ahead of time according to object inspector * types to avoid pattern matching and branching costs per row. */ - val unwrappers: Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map { + val unwrappers: Seq[(Any, InternalRow, Int) => Unit] = fieldRefs.map { _.getFieldObjectInspector match { case oi: BooleanObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) case oi: ByteObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) case oi: ShortObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) case oi: IntObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) case oi: LongObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) case oi: FloatObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) case oi: DoubleObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) case oi: HiveVarcharObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, UTF8String.fromString(oi.getPrimitiveJavaObject(value).getValue)) case oi: HiveCharObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, UTF8String.fromString(oi.getPrimitiveJavaObject(value).getValue)) case oi: HiveDecimalObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, HiveShim.toCatalystDecimal(oi, value)) case oi: TimestampObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(oi.getPrimitiveJavaObject(value))) case oi: DateObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.setInt(ordinal, DateTimeUtils.fromJavaDate(oi.getPrimitiveJavaObject(value))) case oi: BinaryObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, oi.getPrimitiveJavaObject(value)) case oi => val unwrapper = unwrapperFor(oi) - (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrapper(value) + (value: Any, row: InternalRow, ordinal: Int) => row(ordinal) = unwrapper(value) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index c553c03a9b708..1025b8f70d9ff 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -124,7 +124,7 @@ case class ScriptTransformation( } else { null } - val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + val mutableRow = new SpecificInternalRow(output.map(_.dataType)) @transient lazy val unwrappers = outputSoi.getAllStructFieldRefs.asScala.map(unwrapperFor) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index d54913518bb33..42033080dc34b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -329,17 +329,17 @@ private[hive] case class HiveUDAFFunction( // buffer for it. override def aggBufferSchema: StructType = StructType(Nil) - override def update(_buffer: MutableRow, input: InternalRow): Unit = { + override def update(_buffer: InternalRow, input: InternalRow): Unit = { val inputs = inputProjection(input) function.iterate(buffer, wrap(inputs, wrappers, cached, inputDataTypes)) } - override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = { + override def merge(buffer1: InternalRow, buffer2: InternalRow): Unit = { throw new UnsupportedOperationException( "Hive UDAF doesn't support partial aggregate") } - override def initialize(_buffer: MutableRow): Unit = { + override def initialize(_buffer: InternalRow): Unit = { buffer = function.getNewAggregationBuffer } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 15b72d8d2179f..e94f49ea81177 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -281,7 +281,7 @@ private[orc] object OrcRelation extends HiveInspectors { maybeStructOI: Option[StructObjectInspector], iterator: Iterator[Writable]): Iterator[InternalRow] = { val deserializer = new OrcSerde - val mutableRow = new SpecificMutableRow(dataSchema.map(_.dataType)) + val mutableRow = new SpecificInternalRow(dataSchema.map(_.dataType)) val unsafeProjection = UnsafeProjection.create(dataSchema) def unwrap(oi: StructObjectInspector): Iterator[InternalRow] = { From 94b24b84a666517e31e9c9d693f92d9bbfd7f9ad Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 7 Oct 2016 15:03:47 -0700 Subject: [PATCH 024/177] [SPARK-17806] [SQL] fix bug in join key rewritten in HashJoin ## What changes were proposed in this pull request? In HashJoin, we try to rewrite the join key as Long to improve the performance of finding a match. The rewriting part is not well tested, has a bug that could cause wrong result when there are at least three integral columns in the joining key also the total length of the key exceed 8 bytes. ## How was this patch tested? Added unit test to covering the rewriting with different number of columns and different data types. Manually test the reported case and confirmed that this PR fix the bug. Author: Davies Liu Closes #15390 from davies/rewrite_key. --- .../spark/sql/execution/joins/HashJoin.scala | 65 +++++++++---------- .../execution/joins/BroadcastJoinSuite.scala | 47 ++++++++++++++ 2 files changed, 79 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 8ddac19bf1b57..05c5e2f4cd77b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -63,45 +63,16 @@ trait HashJoin { protected lazy val (buildKeys, streamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") - val lkeys = rewriteKeyExpr(leftKeys).map(BindReferences.bindReference(_, left.output)) - val rkeys = rewriteKeyExpr(rightKeys).map(BindReferences.bindReference(_, right.output)) + val lkeys = HashJoin.rewriteKeyExpr(leftKeys).map(BindReferences.bindReference(_, left.output)) + val rkeys = HashJoin.rewriteKeyExpr(rightKeys) + .map(BindReferences.bindReference(_, right.output)) buildSide match { case BuildLeft => (lkeys, rkeys) case BuildRight => (rkeys, lkeys) } } - /** - * Try to rewrite the key as LongType so we can use getLong(), if they key can fit with a long. - * - * If not, returns the original expressions. - */ - private def rewriteKeyExpr(keys: Seq[Expression]): Seq[Expression] = { - var keyExpr: Expression = null - var width = 0 - keys.foreach { e => - e.dataType match { - case dt: IntegralType if dt.defaultSize <= 8 - width => - if (width == 0) { - if (e.dataType != LongType) { - keyExpr = Cast(e, LongType) - } else { - keyExpr = e - } - width = dt.defaultSize - } else { - val bits = dt.defaultSize * 8 - keyExpr = BitwiseOr(ShiftLeft(keyExpr, Literal(bits)), - BitwiseAnd(Cast(e, LongType), Literal((1L << bits) - 1))) - width -= bits - } - // TODO: support BooleanType, DateType and TimestampType - case other => - return keys - } - } - keyExpr :: Nil - } + protected def buildSideKeyGenerator(): Projection = UnsafeProjection.create(buildKeys) @@ -247,3 +218,31 @@ trait HashJoin { } } } + +object HashJoin { + /** + * Try to rewrite the key as LongType so we can use getLong(), if they key can fit with a long. + * + * If not, returns the original expressions. + */ + private[joins] def rewriteKeyExpr(keys: Seq[Expression]): Seq[Expression] = { + assert(keys.nonEmpty) + // TODO: support BooleanType, DateType and TimestampType + if (keys.exists(!_.dataType.isInstanceOf[IntegralType]) + || keys.map(_.dataType.defaultSize).sum > 8) { + return keys + } + + var keyExpr: Expression = if (keys.head.dataType != LongType) { + Cast(keys.head, LongType) + } else { + keys.head + } + keys.tail.foreach { e => + val bits = e.dataType.defaultSize * 8 + keyExpr = BitwiseOr(ShiftLeft(keyExpr, Literal(bits)), + BitwiseAnd(Cast(e, LongType), Literal((1L << bits) - 1))) + } + keyExpr :: Nil + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 97adffa8ce101..83db81ea3f1c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -21,11 +21,13 @@ import scala.reflect.ClassTag import org.apache.spark.AccumulatorSuite import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.{LongType, ShortType} /** * Test various broadcast join operators. @@ -153,4 +155,49 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { cases.foreach(assertBroadcastJoin) } } + + test("join key rewritten") { + val l = Literal(1L) + val i = Literal(2) + val s = Literal.create(3, ShortType) + val ss = Literal("hello") + + assert(HashJoin.rewriteKeyExpr(l :: Nil) === l :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: l :: Nil) === l :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: i :: Nil) === l :: i :: Nil) + + assert(HashJoin.rewriteKeyExpr(i :: Nil) === Cast(i, LongType) :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: l :: Nil) === i :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: i :: Nil) === + BitwiseOr(ShiftLeft(Cast(i, LongType), Literal(32)), + BitwiseAnd(Cast(i, LongType), Literal((1L << 32) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: i :: i :: Nil) === i :: i :: i :: Nil) + + assert(HashJoin.rewriteKeyExpr(s :: Nil) === Cast(s, LongType) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: l :: Nil) === s :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: Nil) === + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: Nil) === + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: s :: Nil) === + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: s :: s :: Nil) === + s :: s :: s :: s :: s :: Nil) + + assert(HashJoin.rewriteKeyExpr(ss :: Nil) === ss :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: ss :: Nil) === l :: ss :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: ss :: Nil) === i :: ss :: Nil) + } } From 24850c9415bfe18dc1edf66e5a7b4c554fff4f23 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 7 Oct 2016 17:59:24 -0700 Subject: [PATCH 025/177] [HOTFIX][BUILD] Do not use contains in Option in JdbcRelationProvider ## What changes were proposed in this pull request? This PR proposes the fix the use of `contains` API which only exists from Scala 2.11. ## How was this patch tested? Manually checked: ```scala scala> val o: Option[Boolean] = None o: Option[Boolean] = None scala> o == Some(false) res17: Boolean = false scala> val o: Option[Boolean] = Some(true) o: Option[Boolean] = Some(true) scala> o == Some(false) res18: Boolean = false scala> val o: Option[Boolean] = Some(false) o: Option[Boolean] = Some(false) scala> o == Some(false) res19: Boolean = true ``` Author: hyukjinkwon Closes #15393 from HyukjinKwon/hotfix. --- .../sql/execution/datasources/jdbc/JdbcRelationProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index 3a8a197ef5241..b1a061b6f7422 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -70,7 +70,7 @@ class JdbcRelationProvider extends CreatableRelationProvider if (tableExists) { mode match { case SaveMode.Overwrite => - if (isTruncate && isCascadingTruncateTable(url).contains(false)) { + if (isTruncate && isCascadingTruncateTable(url) == Some(false)) { // In this case, we should truncate table and then load. truncateTable(conn, table) saveTable(df, url, table, props) From 471690f90f3bf29735faecd83d4671842c57b164 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 7 Oct 2016 18:00:26 -0700 Subject: [PATCH 026/177] [MINOR][ML] remove redundant comment in LogisticRegression ## What changes were proposed in this pull request? While adding R wrapper for LogisticRegression, I found one extra comment. It is minor and I just remove it. ## How was this patch tested? Unit tests Author: wm624@hotmail.com Closes #15391 from wangmiao1981/mlordoc. --- .../org/apache/spark/ml/classification/LogisticRegression.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 329961a25d984..862a468745fbd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -78,7 +78,6 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas /** * Param for the name of family which is a description of the label distribution * to be used in the model. - * Supported options: "auto", "multinomial", "binomial". * Supported options: * - "auto": Automatically select the family based on the number of classes: * If numClasses == 1 || numClasses == 2, set to "binomial". From 362ba4b6f8e8fc2355368742c5adced7573fec00 Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Sat, 8 Oct 2016 11:24:00 +0100 Subject: [PATCH 027/177] =?UTF-8?q?[SPARK-17793][WEB=20UI]=20Sorting=20on?= =?UTF-8?q?=20the=20description=20on=20the=20Job=20or=20Stage=20page=20doe?= =?UTF-8?q?sn=E2=80=99t=20always=20work?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Added secondary sorting on stage name for the description column. This provide a clearer behavior in the common case where the Description column only comprises of Stage names instead of the option description value. ## How was this patch tested? manual testing and dev/run-tests Screenshots of sorting on both description and stage name as well as an example of both: ![screen shot 2016-10-04 at 1 09 39 pm](https://cloud.githubusercontent.com/assets/13952758/19135523/067b042e-8b1a-11e6-912e-e6371d006d21.png) ![screen shot 2016-10-04 at 1 09 51 pm](https://cloud.githubusercontent.com/assets/13952758/19135526/06960936-8b1a-11e6-85e9-8aaf694c5f7b.png) ![screen shot 2016-10-05 at 1 14 45 pm](https://cloud.githubusercontent.com/assets/13952758/19135525/069547da-8b1a-11e6-8692-6524c75c4c07.png) ![screen shot 2016-10-05 at 1 14 51 pm](https://cloud.githubusercontent.com/assets/13952758/19135524/0694b4d2-8b1a-11e6-92dc-c8aa514e4f62.png) ![screen shot 2016-10-05 at 4 42 52 pm](https://cloud.githubusercontent.com/assets/13952758/19135618/e232eafe-8b1a-11e6-88b3-ff0bbb26b7f8.png) Author: Alex Bozarth Closes #15366 from ajbozarth/spark17793. --- .../apache/spark/ui/jobs/AllJobsPage.scala | 25 +--- .../org/apache/spark/ui/jobs/StagePage.scala | 134 ++++-------------- .../org/apache/spark/ui/jobs/StageTable.scala | 51 ++----- .../org/apache/spark/ui/storage/RDDPage.scala | 27 +--- 4 files changed, 49 insertions(+), 188 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 19bb41a1417c7..f6713097b9349 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -457,23 +457,11 @@ private[ui] class JobDataSource( * Return Ordering according to sortColumn and desc */ private def ordering(sortColumn: String, desc: Boolean): Ordering[JobTableRowData] = { - val ordering = sortColumn match { - case "Job Id" | "Job Id (Job Group)" => new Ordering[JobTableRowData] { - override def compare(x: JobTableRowData, y: JobTableRowData): Int = - Ordering.Int.compare(x.jobData.jobId, y.jobData.jobId) - } - case "Description" => new Ordering[JobTableRowData] { - override def compare(x: JobTableRowData, y: JobTableRowData): Int = - Ordering.String.compare(x.lastStageDescription, y.lastStageDescription) - } - case "Submitted" => new Ordering[JobTableRowData] { - override def compare(x: JobTableRowData, y: JobTableRowData): Int = - Ordering.Long.compare(x.submissionTime, y.submissionTime) - } - case "Duration" => new Ordering[JobTableRowData] { - override def compare(x: JobTableRowData, y: JobTableRowData): Int = - Ordering.Long.compare(x.duration, y.duration) - } + val ordering: Ordering[JobTableRowData] = sortColumn match { + case "Job Id" | "Job Id (Job Group)" => Ordering.by(_.jobData.jobId) + case "Description" => Ordering.by(x => (x.lastStageDescription, x.lastStageName)) + case "Submitted" => Ordering.by(_.submissionTime) + case "Duration" => Ordering.by(_.duration) case "Stages: Succeeded/Total" | "Tasks (for all stages): Succeeded/Total" => throw new IllegalArgumentException(s"Unsortable column: $sortColumn") case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") @@ -501,8 +489,7 @@ private[ui] class JobPagedTable( sortColumn: String, desc: Boolean ) extends PagedTable[JobTableRowData] { - val parameterPath = UIUtils.prependBaseUri(basePath) + s"/$subPath/?" + - parameterOtherTable.mkString("&") + val parameterPath = basePath + s"/$subPath/?" + parameterOtherTable.mkString("&") override def tableId: String = jobTag + "-table" diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index c322ae0972ad7..8c7cefe200739 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -1050,89 +1050,38 @@ private[ui] class TaskDataSource( * Return Ordering according to sortColumn and desc */ private def ordering(sortColumn: String, desc: Boolean): Ordering[TaskTableRowData] = { - val ordering = sortColumn match { - case "Index" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Int.compare(x.index, y.index) - } - case "ID" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.taskId, y.taskId) - } - case "Attempt" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Int.compare(x.attempt, y.attempt) - } - case "Status" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.status, y.status) - } - case "Locality Level" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.taskLocality, y.taskLocality) - } - case "Executor ID / Host" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.executorIdAndHost, y.executorIdAndHost) - } - case "Launch Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.launchTime, y.launchTime) - } - case "Duration" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.duration, y.duration) - } - case "Scheduler Delay" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.schedulerDelay, y.schedulerDelay) - } - case "Task Deserialization Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.taskDeserializationTime, y.taskDeserializationTime) - } - case "GC Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.gcTime, y.gcTime) - } - case "Result Serialization Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.serializationTime, y.serializationTime) - } - case "Getting Result Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.gettingResultTime, y.gettingResultTime) - } - case "Peak Execution Memory" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.peakExecutionMemoryUsed, y.peakExecutionMemoryUsed) - } + val ordering: Ordering[TaskTableRowData] = sortColumn match { + case "Index" => Ordering.by(_.index) + case "ID" => Ordering.by(_.taskId) + case "Attempt" => Ordering.by(_.attempt) + case "Status" => Ordering.by(_.status) + case "Locality Level" => Ordering.by(_.taskLocality) + case "Executor ID / Host" => Ordering.by(_.executorIdAndHost) + case "Launch Time" => Ordering.by(_.launchTime) + case "Duration" => Ordering.by(_.duration) + case "Scheduler Delay" => Ordering.by(_.schedulerDelay) + case "Task Deserialization Time" => Ordering.by(_.taskDeserializationTime) + case "GC Time" => Ordering.by(_.gcTime) + case "Result Serialization Time" => Ordering.by(_.serializationTime) + case "Getting Result Time" => Ordering.by(_.gettingResultTime) + case "Peak Execution Memory" => Ordering.by(_.peakExecutionMemoryUsed) case "Accumulators" => if (hasAccumulators) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.accumulators.get, y.accumulators.get) - } + Ordering.by(_.accumulators.get) } else { throw new IllegalArgumentException( "Cannot sort by Accumulators because of no accumulators") } case "Input Size / Records" => if (hasInput) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.input.get.inputSortable, y.input.get.inputSortable) - } + Ordering.by(_.input.get.inputSortable) } else { throw new IllegalArgumentException( "Cannot sort by Input Size / Records because of no inputs") } case "Output Size / Records" => if (hasOutput) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.output.get.outputSortable, y.output.get.outputSortable) - } + Ordering.by(_.output.get.outputSortable) } else { throw new IllegalArgumentException( "Cannot sort by Output Size / Records because of no outputs") @@ -1140,33 +1089,21 @@ private[ui] class TaskDataSource( // ShuffleRead case "Shuffle Read Blocked Time" => if (hasShuffleRead) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleRead.get.shuffleReadBlockedTimeSortable, - y.shuffleRead.get.shuffleReadBlockedTimeSortable) - } + Ordering.by(_.shuffleRead.get.shuffleReadBlockedTimeSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Read Blocked Time because of no shuffle reads") } case "Shuffle Read Size / Records" => if (hasShuffleRead) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleRead.get.shuffleReadSortable, - y.shuffleRead.get.shuffleReadSortable) - } + Ordering.by(_.shuffleRead.get.shuffleReadSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Read Size / Records because of no shuffle reads") } case "Shuffle Remote Reads" => if (hasShuffleRead) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleRead.get.shuffleReadRemoteSortable, - y.shuffleRead.get.shuffleReadRemoteSortable) - } + Ordering.by(_.shuffleRead.get.shuffleReadRemoteSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Remote Reads because of no shuffle reads") @@ -1174,22 +1111,14 @@ private[ui] class TaskDataSource( // ShuffleWrite case "Write Time" => if (hasShuffleWrite) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleWrite.get.writeTimeSortable, - y.shuffleWrite.get.writeTimeSortable) - } + Ordering.by(_.shuffleWrite.get.writeTimeSortable) } else { throw new IllegalArgumentException( "Cannot sort by Write Time because of no shuffle writes") } case "Shuffle Write Size / Records" => if (hasShuffleWrite) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleWrite.get.shuffleWriteSortable, - y.shuffleWrite.get.shuffleWriteSortable) - } + Ordering.by(_.shuffleWrite.get.shuffleWriteSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Write Size / Records because of no shuffle writes") @@ -1197,30 +1126,19 @@ private[ui] class TaskDataSource( // BytesSpilled case "Shuffle Spill (Memory)" => if (hasBytesSpilled) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.bytesSpilled.get.memoryBytesSpilledSortable, - y.bytesSpilled.get.memoryBytesSpilledSortable) - } + Ordering.by(_.bytesSpilled.get.memoryBytesSpilledSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Spill (Memory) because of no spills") } case "Shuffle Spill (Disk)" => if (hasBytesSpilled) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.bytesSpilled.get.diskBytesSpilledSortable, - y.bytesSpilled.get.diskBytesSpilledSortable) - } + Ordering.by(_.bytesSpilled.get.diskBytesSpilledSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Spill (Disk) because of no spills") } - case "Errors" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.error, y.error) - } + case "Errors" => Ordering.by(_.error) case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") } if (desc) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 40a6762c281ce..9b9b4681ba5db 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -109,7 +109,6 @@ private[ui] class StageTableRowData( val stageId: Int, val attemptId: Int, val schedulingPool: String, - val description: String, val descriptionOption: Option[String], val submissionTime: Long, val formattedSubmissionTime: String, @@ -128,7 +127,7 @@ private[ui] class MissingStageTableRowData( stageInfo: StageInfo, stageId: Int, attemptId: Int) extends StageTableRowData( - stageInfo, None, stageId, attemptId, "", "", None, 0, "", -1, "", 0, "", 0, "", 0, "", 0, "") + stageInfo, None, stageId, attemptId, "", None, 0, "", -1, "", 0, "", 0, "", 0, "", 0, "") /** Page showing list of all ongoing and recently finished stages */ private[ui] class StagePagedTable( @@ -470,7 +469,6 @@ private[ui] class StageDataSource( s.stageId, s.attemptId, stageData.schedulingPool, - description.getOrElse(""), description, s.submissionTime.getOrElse(0), formattedSubmissionTime, @@ -491,43 +489,16 @@ private[ui] class StageDataSource( * Return Ordering according to sortColumn and desc */ private def ordering(sortColumn: String, desc: Boolean): Ordering[StageTableRowData] = { - val ordering = sortColumn match { - case "Stage Id" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.Int.compare(x.stageId, y.stageId) - } - case "Pool Name" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.String.compare(x.schedulingPool, y.schedulingPool) - } - case "Description" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.String.compare(x.description, y.description) - } - case "Submitted" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.Long.compare(x.submissionTime, y.submissionTime) - } - case "Duration" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.Long.compare(x.duration, y.duration) - } - case "Input" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.Long.compare(x.inputRead, y.inputRead) - } - case "Output" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.Long.compare(x.outputWrite, y.outputWrite) - } - case "Shuffle Read" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.Long.compare(x.shuffleRead, y.shuffleRead) - } - case "Shuffle Write" => new Ordering[StageTableRowData] { - override def compare(x: StageTableRowData, y: StageTableRowData): Int = - Ordering.Long.compare(x.shuffleWrite, y.shuffleWrite) - } + val ordering: Ordering[StageTableRowData] = sortColumn match { + case "Stage Id" => Ordering.by(_.stageId) + case "Pool Name" => Ordering.by(_.schedulingPool) + case "Description" => Ordering.by(x => (x.descriptionOption, x.stageInfo.name)) + case "Submitted" => Ordering.by(_.submissionTime) + case "Duration" => Ordering.by(_.duration) + case "Input" => Ordering.by(_.inputRead) + case "Output" => Ordering.by(_.outputWrite) + case "Shuffle Read" => Ordering.by(_.shuffleRead) + case "Shuffle Write" => Ordering.by(_.shuffleWrite) case "Tasks: Succeeded/Total" => throw new IllegalArgumentException(s"Unsortable column: $sortColumn") case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 606d15d599e81..227e940c9c50c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -197,27 +197,12 @@ private[ui] class BlockDataSource( * Return Ordering according to sortColumn and desc */ private def ordering(sortColumn: String, desc: Boolean): Ordering[BlockTableRowData] = { - val ordering = sortColumn match { - case "Block Name" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.String.compare(x.blockName, y.blockName) - } - case "Storage Level" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.String.compare(x.storageLevel, y.storageLevel) - } - case "Size in Memory" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.Long.compare(x.memoryUsed, y.memoryUsed) - } - case "Size on Disk" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.Long.compare(x.diskUsed, y.diskUsed) - } - case "Executors" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.String.compare(x.executors, y.executors) - } + val ordering: Ordering[BlockTableRowData] = sortColumn match { + case "Block Name" => Ordering.by(_.blockName) + case "Storage Level" => Ordering.by(_.storageLevel) + case "Size in Memory" => Ordering.by(_.memoryUsed) + case "Size on Disk" => Ordering.by(_.diskUsed) + case "Executors" => Ordering.by(_.executors) case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") } if (desc) { From 4201ddcc07ca2e9af78bf4a74fdb3900c1783347 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 8 Oct 2016 11:31:12 +0100 Subject: [PATCH 028/177] [SPARK-17768][CORE] Small (Sum,Count,Mean)Evaluator problems and suboptimalities MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Fix: - GroupedMeanEvaluator and GroupedSumEvaluator are unused, as is the StudentTCacher support class - CountEvaluator can return a lower bound < 0, when counts can't be negative - MeanEvaluator will actually fail on exactly 1 datum (yields t-test with 0 DOF) - CountEvaluator uses a normal distribution, which may be an inappropriate approximation (leading to above) - Test for SumEvaluator asserts incorrect expected sums – e.g. after observing 10% of data has sum of 2, expectation should be 20, not 38 - CountEvaluator, MeanEvaluator have no unit tests to catch these - Duplication of distribution code across CountEvaluator, GroupedCountEvaluator - The stats in each could use a bit of documentation as I had to guess at them - (Code could use a few cleanups and optimizations too) ## How was this patch tested? Existing and new tests Author: Sean Owen Closes #15341 from srowen/SPARK-17768. --- .../apache/spark/partial/CountEvaluator.scala | 53 +++++++---- .../spark/partial/GroupedCountEvaluator.scala | 30 ++----- .../spark/partial/GroupedMeanEvaluator.scala | 80 ----------------- .../spark/partial/GroupedSumEvaluator.scala | 88 ------------------- .../apache/spark/partial/MeanEvaluator.scala | 23 +++-- .../apache/spark/partial/StudentTCacher.scala | 46 ---------- .../apache/spark/partial/SumEvaluator.scala | 33 ++++--- .../spark/partial/CountEvaluatorSuite.scala | 43 +++++++++ .../spark/partial/MeanEvaluatorSuite.scala | 57 ++++++++++++ .../spark/partial/SumEvaluatorSuite.scala | 82 ++++++----------- 10 files changed, 203 insertions(+), 332 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala delete mode 100644 core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala delete mode 100644 core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala create mode 100644 core/src/test/scala/org/apache/spark/partial/CountEvaluatorSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/partial/MeanEvaluatorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala index 637492a97551b..5a5bd7fbbe2f8 100644 --- a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala @@ -17,21 +17,18 @@ package org.apache.spark.partial -import org.apache.commons.math3.distribution.NormalDistribution +import org.apache.commons.math3.distribution.{PascalDistribution, PoissonDistribution} /** * An ApproximateEvaluator for counts. - * - * TODO: There's currently a lot of shared code between this and GroupedCountEvaluator. It might - * be best to make this a special case of GroupedCountEvaluator with one group. */ private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[Long, BoundedDouble] { - var outputsMerged = 0 - var sum: Long = 0 + private var outputsMerged = 0 + private var sum: Long = 0 - override def merge(outputId: Int, taskResult: Long) { + override def merge(outputId: Int, taskResult: Long): Unit = { outputsMerged += 1 sum += taskResult } @@ -39,18 +36,40 @@ private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double) override def currentResult(): BoundedDouble = { if (outputsMerged == totalOutputs) { new BoundedDouble(sum, 1.0, sum, sum) - } else if (outputsMerged == 0) { - new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) + } else if (outputsMerged == 0 || sum == 0) { + new BoundedDouble(0, 0.0, 0.0, Double.PositiveInfinity) } else { val p = outputsMerged.toDouble / totalOutputs - val mean = (sum + 1 - p) / p - val variance = (sum + 1) * (1 - p) / (p * p) - val stdev = math.sqrt(variance) - val confFactor = new NormalDistribution(). - inverseCumulativeProbability(1 - (1 - confidence) / 2) - val low = mean - confFactor * stdev - val high = mean + confFactor * stdev - new BoundedDouble(mean, confidence, low, high) + CountEvaluator.bound(confidence, sum, p) } } } + +private[partial] object CountEvaluator { + + def bound(confidence: Double, sum: Long, p: Double): BoundedDouble = { + // Let the total count be N. A fraction p has been counted already, with sum 'sum', + // as if each element from the total data set had been seen with probability p. + val dist = + if (sum <= 10000) { + // The remaining count, k=N-sum, may be modeled as negative binomial (aka Pascal), + // where there have been 'sum' successes of probability p already. (There are several + // conventions, but this is the one followed by Commons Math3.) + new PascalDistribution(sum.toInt, p) + } else { + // For large 'sum' (certainly, > Int.MaxValue!), use a Poisson approximation, which has + // a different interpretation. "sum" elements have been observed having scanned a fraction + // p of the data. This suggests data is counted at a rate of sum / p across the whole data + // set. The total expected count from the rest is distributed as + // (1-p) Poisson(sum / p) = Poisson(sum*(1-p)/p) + new PoissonDistribution(sum * (1 - p) / p) + } + // Not quite symmetric; calculate interval straight from discrete distribution + val low = dist.inverseCumulativeProbability((1 - confidence) / 2) + val high = dist.inverseCumulativeProbability((1 + confidence) / 2) + // Add 'sum' to each because distribution is just of remaining count, not observed + new BoundedDouble(sum + dist.getNumericalMean, confidence, sum + low, sum + high) + } + + +} diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 5afce75680f94..d2b4187df5d50 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -17,15 +17,10 @@ package org.apache.spark.partial -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import org.apache.commons.math3.distribution.NormalDistribution - import org.apache.spark.util.collection.OpenHashMap /** @@ -34,10 +29,10 @@ import org.apache.spark.util.collection.OpenHashMap private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[OpenHashMap[T, Long], Map[T, BoundedDouble]] { - var outputsMerged = 0 - var sums = new OpenHashMap[T, Long]() // Sum of counts for each key + private var outputsMerged = 0 + private val sums = new OpenHashMap[T, Long]() // Sum of counts for each key - override def merge(outputId: Int, taskResult: OpenHashMap[T, Long]) { + override def merge(outputId: Int, taskResult: OpenHashMap[T, Long]): Unit = { outputsMerged += 1 taskResult.foreach { case (key, value) => sums.changeValue(key, value, _ + value) @@ -46,27 +41,12 @@ private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, conf override def currentResult(): Map[T, BoundedDouble] = { if (outputsMerged == totalOutputs) { - val result = new JHashMap[T, BoundedDouble](sums.size) - sums.foreach { case (key, sum) => - result.put(key, new BoundedDouble(sum, 1.0, sum, sum)) - } - result.asScala + sums.map { case (key, sum) => (key, new BoundedDouble(sum, 1.0, sum, sum)) }.toMap } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { val p = outputsMerged.toDouble / totalOutputs - val confFactor = new NormalDistribution(). - inverseCumulativeProbability(1 - (1 - confidence) / 2) - val result = new JHashMap[T, BoundedDouble](sums.size) - sums.foreach { case (key, sum) => - val mean = (sum + 1 - p) / p - val variance = (sum + 1) * (1 - p) / (p * p) - val stdev = math.sqrt(variance) - val low = mean - confFactor * stdev - val high = mean + confFactor * stdev - result.put(key, new BoundedDouble(mean, confidence, low, high)) - } - result.asScala + sums.map { case (key, sum) => (key, CountEvaluator.bound(confidence, sum, p)) }.toMap } } } diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala deleted file mode 100644 index a164040684803..0000000000000 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.partial - -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.Map -import scala.collection.mutable.HashMap - -import org.apache.spark.util.StatCounter - -/** - * An ApproximateEvaluator for means by key. Returns a map of key to confidence interval. - */ -private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] { - - var outputsMerged = 0 - var sums = new JHashMap[T, StatCounter] // Sum of counts for each key - - override def merge(outputId: Int, taskResult: JHashMap[T, StatCounter]) { - outputsMerged += 1 - val iter = taskResult.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val old = sums.get(entry.getKey) - if (old != null) { - old.merge(entry.getValue) - } else { - sums.put(entry.getKey, entry.getValue) - } - } - } - - override def currentResult(): Map[T, BoundedDouble] = { - if (outputsMerged == totalOutputs) { - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val mean = entry.getValue.mean - result.put(entry.getKey, new BoundedDouble(mean, 1.0, mean, mean)) - } - result.asScala - } else if (outputsMerged == 0) { - new HashMap[T, BoundedDouble] - } else { - val studentTCacher = new StudentTCacher(confidence) - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val counter = entry.getValue - val mean = counter.mean - val stdev = math.sqrt(counter.sampleVariance / counter.count) - val confFactor = studentTCacher.get(counter.count) - val low = mean - confFactor * stdev - val high = mean + confFactor * stdev - result.put(entry.getKey, new BoundedDouble(mean, confidence, low, high)) - } - result.asScala - } - } -} diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala deleted file mode 100644 index 54a1beab3514b..0000000000000 --- a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.partial - -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.Map -import scala.collection.mutable.HashMap - -import org.apache.spark.util.StatCounter - -/** - * An ApproximateEvaluator for sums by key. Returns a map of key to confidence interval. - */ -private[spark] class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] { - - var outputsMerged = 0 - var sums = new JHashMap[T, StatCounter] // Sum of counts for each key - - override def merge(outputId: Int, taskResult: JHashMap[T, StatCounter]) { - outputsMerged += 1 - val iter = taskResult.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val old = sums.get(entry.getKey) - if (old != null) { - old.merge(entry.getValue) - } else { - sums.put(entry.getKey, entry.getValue) - } - } - } - - override def currentResult(): Map[T, BoundedDouble] = { - if (outputsMerged == totalOutputs) { - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getValue.sum - result.put(entry.getKey, new BoundedDouble(sum, 1.0, sum, sum)) - } - result.asScala - } else if (outputsMerged == 0) { - new HashMap[T, BoundedDouble] - } else { - val p = outputsMerged.toDouble / totalOutputs - val studentTCacher = new StudentTCacher(confidence) - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val counter = entry.getValue - val meanEstimate = counter.mean - val meanVar = counter.sampleVariance / counter.count - val countEstimate = (counter.count + 1 - p) / p - val countVar = (counter.count + 1) * (1 - p) / (p * p) - val sumEstimate = meanEstimate * countEstimate - val sumVar = (meanEstimate * meanEstimate * countVar) + - (countEstimate * countEstimate * meanVar) + - (meanVar * countVar) - val sumStdev = math.sqrt(sumVar) - val confFactor = studentTCacher.get(counter.count) - val low = sumEstimate - confFactor * sumStdev - val high = sumEstimate + confFactor * sumStdev - result.put(entry.getKey, new BoundedDouble(sumEstimate, confidence, low, high)) - } - result.asScala - } - } -} diff --git a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala index 787a21a61fdcf..3fb2d30a800b6 100644 --- a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala @@ -27,10 +27,10 @@ import org.apache.spark.util.StatCounter private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[StatCounter, BoundedDouble] { - var outputsMerged = 0 - var counter = new StatCounter + private var outputsMerged = 0 + private val counter = new StatCounter() - override def merge(outputId: Int, taskResult: StatCounter) { + override def merge(outputId: Int, taskResult: StatCounter): Unit = { outputsMerged += 1 counter.merge(taskResult) } @@ -38,19 +38,24 @@ private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double) override def currentResult(): BoundedDouble = { if (outputsMerged == totalOutputs) { new BoundedDouble(counter.mean, 1.0, counter.mean, counter.mean) - } else if (outputsMerged == 0) { + } else if (outputsMerged == 0 || counter.count == 0) { new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) + } else if (counter.count == 1) { + new BoundedDouble(counter.mean, confidence, Double.NegativeInfinity, Double.PositiveInfinity) } else { val mean = counter.mean val stdev = math.sqrt(counter.sampleVariance / counter.count) - val confFactor = { - if (counter.count > 100) { - new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) + val confFactor = if (counter.count > 100) { + // For large n, the normal distribution is a good approximation to t-distribution + new NormalDistribution().inverseCumulativeProbability((1 + confidence) / 2) } else { + // t-distribution describes distribution of actual population mean + // note that if this goes to 0, TDistribution will throw an exception. + // Hence special casing 1 above. val degreesOfFreedom = (counter.count - 1).toInt - new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) + new TDistribution(degreesOfFreedom).inverseCumulativeProbability((1 + confidence) / 2) } - } + // Symmetric, so confidence interval is symmetric about mean of distribution val low = mean - confFactor * stdev val high = mean + confFactor * stdev new BoundedDouble(mean, confidence, low, high) diff --git a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala deleted file mode 100644 index 55acb9ca64d3f..0000000000000 --- a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.partial - -import org.apache.commons.math3.distribution.{NormalDistribution, TDistribution} - -/** - * A utility class for caching Student's T distribution values for a given confidence level - * and various sample sizes. This is used by the MeanEvaluator to efficiently calculate - * confidence intervals for many keys. - */ -private[spark] class StudentTCacher(confidence: Double) { - - val NORMAL_APPROX_SAMPLE_SIZE = 100 // For samples bigger than this, use Gaussian approximation - - val normalApprox = new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) - val cache = Array.fill[Double](NORMAL_APPROX_SAMPLE_SIZE)(-1.0) - - def get(sampleSize: Long): Double = { - if (sampleSize >= NORMAL_APPROX_SAMPLE_SIZE) { - normalApprox - } else { - val size = sampleSize.toInt - if (cache(size) < 0) { - val tDist = new TDistribution(size - 1) - cache(size) = tDist.inverseCumulativeProbability(1 - (1 - confidence) / 2) - } - cache(size) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index 5fe33583166c3..1988052b733e6 100644 --- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -30,10 +30,10 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[StatCounter, BoundedDouble] { // modified in merge - var outputsMerged = 0 - val counter = new StatCounter + private var outputsMerged = 0 + private val counter = new StatCounter() - override def merge(outputId: Int, taskResult: StatCounter) { + override def merge(outputId: Int, taskResult: StatCounter): Unit = { outputsMerged += 1 counter.merge(taskResult) } @@ -45,34 +45,45 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) } else { val p = outputsMerged.toDouble / totalOutputs + // Expected value of unobserved is presumed equal to that of the observed data val meanEstimate = counter.mean - val countEstimate = (counter.count + 1 - p) / p + // Expected size of rest of the data is proportional + val countEstimate = counter.count * (1 - p) / p + // Expected sum is simply their product val sumEstimate = meanEstimate * countEstimate + // Variance of unobserved data is presumed equal to that of the observed data val meanVar = counter.sampleVariance / counter.count - // branch at this point because counter.count == 1 implies counter.sampleVariance == Nan + // branch at this point because count == 1 implies counter.sampleVariance == Nan // and we don't want to ever return a bound of NaN if (meanVar.isNaN || counter.count == 1) { - new BoundedDouble(sumEstimate, confidence, Double.NegativeInfinity, Double.PositiveInfinity) + // add sum because estimate is of unobserved data sum + new BoundedDouble( + counter.sum + sumEstimate, confidence, Double.NegativeInfinity, Double.PositiveInfinity) } else { - val countVar = (counter.count + 1) * (1 - p) / (p * p) + // See CountEvaluator. Variance of population count here follows from negative binomial + val countVar = counter.count * (1 - p) / (p * p) + // Var(Sum) = Var(Mean*Count) = + // [E(Mean)]^2 * Var(Count) + [E(Count)]^2 * Var(Mean) + Var(Mean) * Var(Count) val sumVar = (meanEstimate * meanEstimate * countVar) + (countEstimate * countEstimate * meanVar) + (meanVar * countVar) val sumStdev = math.sqrt(sumVar) val confFactor = if (counter.count > 100) { - new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) + new NormalDistribution().inverseCumulativeProbability((1 + confidence) / 2) } else { // note that if this goes to 0, TDistribution will throw an exception. // Hence special casing 1 above. val degreesOfFreedom = (counter.count - 1).toInt - new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) + new TDistribution(degreesOfFreedom).inverseCumulativeProbability((1 + confidence) / 2) } - + // Symmetric, so confidence interval is symmetric about mean of distribution val low = sumEstimate - confFactor * sumStdev val high = sumEstimate + confFactor * sumStdev - new BoundedDouble(sumEstimate, confidence, low, high) + // add sum because estimate is of unobserved data sum + new BoundedDouble( + counter.sum + sumEstimate, confidence, counter.sum + low, counter.sum + high) } } } diff --git a/core/src/test/scala/org/apache/spark/partial/CountEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/CountEvaluatorSuite.scala new file mode 100644 index 0000000000000..da3256bd882e8 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/partial/CountEvaluatorSuite.scala @@ -0,0 +1,43 @@ +/* + * 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.partial + +import org.apache.spark.SparkFunSuite + +class CountEvaluatorSuite extends SparkFunSuite { + + test("test count 0") { + val evaluator = new CountEvaluator(10, 0.95) + assert(new BoundedDouble(0.0, 0.0, 0.0, Double.PositiveInfinity) == evaluator.currentResult()) + evaluator.merge(1, 0) + assert(new BoundedDouble(0.0, 0.0, 0.0, Double.PositiveInfinity) == evaluator.currentResult()) + } + + test("test count >= 1") { + val evaluator = new CountEvaluator(10, 0.95) + evaluator.merge(1, 1) + assert(new BoundedDouble(10.0, 0.95, 1.0, 36.0) == evaluator.currentResult()) + evaluator.merge(1, 3) + assert(new BoundedDouble(20.0, 0.95, 7.0, 41.0) == evaluator.currentResult()) + evaluator.merge(1, 8) + assert(new BoundedDouble(40.0, 0.95, 24.0, 61.0) == evaluator.currentResult()) + (4 to 10).foreach(_ => evaluator.merge(1, 10)) + assert(new BoundedDouble(82.0, 1.0, 82.0, 82.0) == evaluator.currentResult()) + } + +} diff --git a/core/src/test/scala/org/apache/spark/partial/MeanEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/MeanEvaluatorSuite.scala new file mode 100644 index 0000000000000..eaa1262b4199f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/partial/MeanEvaluatorSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.partial + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.StatCounter + +class MeanEvaluatorSuite extends SparkFunSuite { + + test("test count 0") { + val evaluator = new MeanEvaluator(10, 0.95) + assert(new BoundedDouble(0.0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter()) + assert(new BoundedDouble(0.0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter(Seq(0.0))) + assert(new BoundedDouble(0.0, 0.95, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + } + + test("test count 1") { + val evaluator = new MeanEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter(Seq(1.0))) + assert(new BoundedDouble(1.0, 0.95, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + } + + test("test count > 1") { + val evaluator = new MeanEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter(Seq(1.0))) + evaluator.merge(1, new StatCounter(Seq(3.0))) + assert(new BoundedDouble(2.0, 0.95, -10.706204736174746, 14.706204736174746) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter(Seq(8.0))) + assert(new BoundedDouble(4.0, 0.95, -4.9566858949231225, 12.956685894923123) == + evaluator.currentResult()) + (4 to 10).foreach(_ => evaluator.merge(1, new StatCounter(Seq(9.0)))) + assert(new BoundedDouble(7.5, 1.0, 7.5, 7.5) == evaluator.currentResult()) + } + +} diff --git a/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala index a79f5b4d74467..e212db73627e7 100644 --- a/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala @@ -17,61 +17,34 @@ package org.apache.spark.partial -import org.apache.spark._ +import org.apache.spark.SparkFunSuite import org.apache.spark.util.StatCounter -class SumEvaluatorSuite extends SparkFunSuite with SharedSparkContext { +class SumEvaluatorSuite extends SparkFunSuite { test("correct handling of count 1") { + // sanity check: + assert(new BoundedDouble(2.0, 0.95, 1.1, 1.2) == new BoundedDouble(2.0, 0.95, 1.1, 1.2)) - // setup - val counter = new StatCounter(List(2.0)) // count of 10 because it's larger than 1, // and 0.95 because that's the default val evaluator = new SumEvaluator(10, 0.95) // arbitrarily assign id 1 - evaluator.merge(1, counter) - - // execute - val res = evaluator.currentResult() - // 38.0 - 7.1E-15 because that's how the maths shakes out - val targetMean = 38.0 - 7.1E-15 - - // Sanity check that equality works on BoundedDouble - assert(new BoundedDouble(2.0, 0.95, 1.1, 1.2) == new BoundedDouble(2.0, 0.95, 1.1, 1.2)) - - // actual test - assert(res == - new BoundedDouble(targetMean, 0.950, Double.NegativeInfinity, Double.PositiveInfinity)) + evaluator.merge(1, new StatCounter(Seq(2.0))) + assert(new BoundedDouble(20.0, 0.95, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) } test("correct handling of count 0") { - - // setup - val counter = new StatCounter(List()) - // count of 10 because it's larger than 0, - // and 0.95 because that's the default val evaluator = new SumEvaluator(10, 0.95) - // arbitrarily assign id 1 - evaluator.merge(1, counter) - - // execute - val res = evaluator.currentResult() - // assert - assert(res == new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity)) + evaluator.merge(1, new StatCounter()) + assert(new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) } test("correct handling of NaN") { - - // setup - val counter = new StatCounter(List(1, Double.NaN, 2)) - // count of 10 because it's larger than 0, - // and 0.95 because that's the default val evaluator = new SumEvaluator(10, 0.95) - // arbitrarily assign id 1 - evaluator.merge(1, counter) - - // execute + evaluator.merge(1, new StatCounter(Seq(1, Double.NaN, 2))) val res = evaluator.currentResult() // assert - note semantics of == in face of NaN assert(res.mean.isNaN) @@ -81,27 +54,24 @@ class SumEvaluatorSuite extends SparkFunSuite with SharedSparkContext { } test("correct handling of > 1 values") { - - // setup - val counter = new StatCounter(List(1, 3, 2)) - // count of 10 because it's larger than 0, - // and 0.95 because that's the default val evaluator = new SumEvaluator(10, 0.95) - // arbitrarily assign id 1 - evaluator.merge(1, counter) - - // execute + evaluator.merge(1, new StatCounter(Seq(1.0, 3.0, 2.0))) val res = evaluator.currentResult() + assert(new BoundedDouble(60.0, 0.95, -101.7362525347778, 221.7362525347778) == + evaluator.currentResult()) + } - // These vals because that's how the maths shakes out - val targetMean = 78.0 - val targetLow = -117.617 + 2.732357258139473E-5 - val targetHigh = 273.617 - 2.7323572624027292E-5 - val target = new BoundedDouble(targetMean, 0.95, targetLow, targetHigh) - - - // check that values are within expected tolerance of expectation - assert(res == target) + test("test count > 1") { + val evaluator = new SumEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter().merge(1.0)) + evaluator.merge(1, new StatCounter().merge(3.0)) + assert(new BoundedDouble(20.0, 0.95, -186.4513905077019, 226.4513905077019) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter().merge(8.0)) + assert(new BoundedDouble(40.0, 0.95, -72.75723361226733, 152.75723361226733) == + evaluator.currentResult()) + (4 to 10).foreach(_ => evaluator.merge(1, new StatCounter().merge(9.0))) + assert(new BoundedDouble(75.0, 1.0, 75.0, 75.0) == evaluator.currentResult()) } } From 8a6bbe095b6a9aa33989c0deaa5ed0128d70320f Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Sat, 8 Oct 2016 12:12:35 +0100 Subject: [PATCH 029/177] [MINOR][SQL] Use resource path for test_script.sh ## What changes were proposed in this pull request? This PR modified the test case `test("script")` to use resource path for `test_script.sh`. Make the test case portable (even in IntelliJ). ## How was this patch tested? Passed the test case. Before: Run `test("script")` in IntelliJ: ``` Caused by: org.apache.spark.SparkException: Subprocess exited with status 127. Error: bash: src/test/resources/test_script.sh: No such file or directory ``` After: Test passed. Author: Weiqing Yang Closes #15246 from weiqingy/hivetest. --- .../scala/org/apache/spark/SparkFunSuite.scala | 11 +++++++++++ .../spark/deploy/history/HistoryServerSuite.scala | 6 +++--- .../test/scala/org/apache/spark/ui/UISuite.scala | 3 ++- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 2 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 3 ++- .../spark/sql/hive/execution/SQLQuerySuite.scala | 15 +++++++++------ 6 files changed, 28 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index cd876807f890e..18077c08c9dcc 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark // scalastyle:off +import java.io.File + import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} import org.apache.spark.internal.Logging @@ -41,6 +43,15 @@ abstract class SparkFunSuite } } + // helper function + protected final def getTestResourceFile(file: String): File = { + new File(getClass.getClassLoader.getResource(file).getFile) + } + + protected final def getTestResourcePath(file: String): String = { + getTestResourceFile(file).getCanonicalPath + } + /** * Log the suite name and the test name before and after each test. * diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 5b316b2f6b4b7..a595bc174a310 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -59,8 +59,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers with JsonTestUtils with Eventually with WebBrowser with LocalSparkContext with ResetSystemProperties { - private val logDir = new File("src/test/resources/spark-events") - private val expRoot = new File("src/test/resources/HistoryServerExpectations/") + private val logDir = getTestResourcePath("spark-events") + private val expRoot = getTestResourceFile("HistoryServerExpectations") private var provider: FsHistoryProvider = null private var server: HistoryServer = null @@ -68,7 +68,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers def init(): Unit = { val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) + .set("spark.history.fs.logDirectory", logDir) .set("spark.history.fs.update.interval", "0") .set("spark.testing", "true") provider = new FsHistoryProvider(conf) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index dbb8dca4c8dab..4abcfb7e51914 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -53,9 +53,10 @@ class UISuite extends SparkFunSuite { } private def sslEnabledConf(): (SparkConf, SSLOptions) = { + val keyStoreFilePath = getTestResourcePath("spark.keystore") val conf = new SparkConf() .set("spark.ssl.ui.enabled", "true") - .set("spark.ssl.ui.keyStore", "./src/test/resources/spark.keystore") + .set("spark.ssl.ui.keyStore", keyStoreFilePath) .set("spark.ssl.ui.keyStorePassword", "123456") .set("spark.ssl.ui.keyPassword", "123456") (conf, new SecurityManager(conf).getSSLOptions("ui")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 9ac1e86fc82cb..c7f10e569fa4d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -45,7 +45,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { // Used for generating new query answer files by saving private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" - private val goldenSQLPath = "src/test/resources/sqlgen/" + private val goldenSQLPath = getTestResourcePath("sqlgen") protected override def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 29317e2887861..d3873cf6c8231 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -152,7 +152,8 @@ class HiveSparkSubmitSuite case v if v.startsWith("2.10") || v.startsWith("2.11") => v.substring(0, 4) case x => throw new Exception(s"Unsupported Scala Version: $x") } - val testJar = s"sql/hive/src/test/resources/regression-test-SPARK-8489/test-$version.jar" + val jarDir = getTestResourcePath("regression-test-SPARK-8489") + val testJar = s"$jarDir/test-$version.jar" val args = Seq( "--conf", "spark.ui.enabled=false", "--conf", "spark.master.rest.enabled=false", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 6c77a0deb52a4..6f2a16662bf10 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -66,13 +66,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import spark.implicits._ test("script") { + val scriptFilePath = getTestResourcePath("test_script.sh") if (testCommandAvailable("bash") && testCommandAvailable("echo | sed")) { val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") df.createOrReplaceTempView("script_table") val query1 = sql( - """ + s""" |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table - |REDUCE c1, c2, c3 USING 'bash src/test/resources/test_script.sh' AS + |REDUCE c1, c2, c3 USING 'bash $scriptFilePath' AS |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) checkAnswer(query1, Row("x1_y1") :: Row("x2_y2") :: Nil) } @@ -1290,11 +1291,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { .selectExpr("id AS a", "id AS b") .createOrReplaceTempView("test") + val scriptFilePath = getTestResourcePath("data") checkAnswer( sql( - """FROM( + s"""FROM( | FROM test SELECT TRANSFORM(a, b) - | USING 'python src/test/resources/data/scripts/test_transform.py "\t"' + | USING 'python $scriptFilePath/scripts/test_transform.py "\t"' | AS (c STRING, d STRING) |) t |SELECT c @@ -1308,12 +1310,13 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { .selectExpr("id AS a", "id AS b") .createOrReplaceTempView("test") + val scriptFilePath = getTestResourcePath("data") val df = sql( - """FROM test + s"""FROM test |SELECT TRANSFORM(a, b) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES('field.delim' = '|') - |USING 'python src/test/resources/data/scripts/test_transform.py "|"' + |USING 'python $scriptFilePath/scripts/test_transform.py "|"' |AS (c STRING, d STRING) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES('field.delim' = '|') From 26fbca480604ba258f97b9590cfd6dda1ecd31db Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Sun, 9 Oct 2016 21:52:46 -0700 Subject: [PATCH 030/177] [SPARK-17832][SQL] TableIdentifier.quotedString creates un-parseable names when name contains a backtick ## What changes were proposed in this pull request? The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example: ``` import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1")) parseTableIdentifier(complexName.unquotedString) // Does not work parseTableIdentifier(complexName.quotedString) // Does not work parseExpression(complexName.unquotedString) // Does not work parseExpression(complexName.quotedString) // Does not work ``` We should handle the backtick properly to make `quotedString` parseable. ## How was this patch tested? Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`. Author: jiangxingbo Closes #15403 from jiangxb1987/backtick. --- .../org/apache/spark/sql/catalyst/identifiers.scala | 11 +++++++++-- .../sql/catalyst/parser/ExpressionParserSuite.scala | 11 ++++++++++- .../catalyst/parser/TableIdentifierParserSuite.scala | 10 ++++++++++ 3 files changed, 29 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index d7b48ceca591a..834897b85023d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst - /** * An identifier that optionally specifies a database. * @@ -29,8 +28,16 @@ sealed trait IdentifierWithDatabase { def database: Option[String] + /* + * Escapes back-ticks within the identifier name with double-back-ticks. + */ + private def quoteIdentifier(name: String): String = name.replace("`", "``") + def quotedString: String = { - if (database.isDefined) s"`${database.get}`.`$identifier`" else s"`$identifier`" + val replacedId = quoteIdentifier(identifier) + val replacedDb = database.map(quoteIdentifier(_)) + + if (replacedDb.isDefined) s"`${replacedDb.get}`.`$replacedId`" else s"`$replacedId`" } def unquotedString: String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 0fb1138478a9b..17cfc8158803b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -535,4 +535,13 @@ class ExpressionParserSuite extends PlanTest { // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL assertEqual("a.123BD_column", UnresolvedAttribute("a.123BD_column")) } + + test("SPARK-17832 function identifier contains backtick") { + val complexName = FunctionIdentifier("`ba`r", Some("`fo`o")) + assertEqual(complexName.quotedString, UnresolvedAttribute("`fo`o.`ba`r")) + intercept(complexName.unquotedString, "mismatched input") + // Function identifier contains countious backticks should be treated correctly. + val complexName2 = FunctionIdentifier("ba``r", Some("fo``o")) + assertEqual(complexName2.quotedString, UnresolvedAttribute("fo``o.ba``r")) + } } 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 793be8953d07a..7d46011b410e2 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 @@ -104,4 +104,14 @@ class TableIdentifierParserSuite extends SparkFunSuite { // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL assert(parseTableIdentifier("a.123BD_LIST") == TableIdentifier("123BD_LIST", Some("a"))) } + + test("SPARK-17832 table identifier - contains backtick") { + val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1")) + assert(complexName === parseTableIdentifier("```d``b``1`.```weird``table``name`")) + assert(complexName === parseTableIdentifier(complexName.quotedString)) + intercept[ParseException](parseTableIdentifier(complexName.unquotedString)) + // Table identifier contains countious backticks should be treated correctly. + val complexName2 = TableIdentifier("x``y", Some("d``b")) + assert(complexName2 === parseTableIdentifier(complexName2.quotedString)) + } } From 16590030c15b32e83b584283697b6f783cffe043 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Sun, 9 Oct 2016 22:00:54 -0700 Subject: [PATCH 031/177] [SPARK-17741][SQL] Grammar to parse top level and nested data fields separately ## What changes were proposed in this pull request? Currently we use the same rule to parse top level and nested data fields. For example: ``` create table tbl_x( id bigint, nested struct ) ``` Shows both syntaxes. In this PR we split this rule in a top-level and nested rule. Before this PR, ``` sql("CREATE TABLE my_tab(column1: INT)") ``` works fine. After this PR, it will throw a `ParseException`: ``` scala> sql("CREATE TABLE my_tab(column1: INT)") org.apache.spark.sql.catalyst.parser.ParseException: no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27) ``` ## How was this patch tested? Add new testcases in `SparkSqlParserSuite`. Author: jiangxingbo Closes #15346 from jiangxb1987/cdt. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 12 +- .../sql/catalyst/parser/AstBuilder.scala | 32 +++- .../catalyst/parser/DataTypeParserSuite.scala | 14 +- .../spark/sql/execution/SparkSqlParser.scala | 4 +- .../sql/execution/SparkSqlParserSuite.scala | 152 +++++++++++++++++- .../sql/execution/command/DDLSuite.scala | 2 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 2 +- 7 files changed, 195 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 6a94def65f360..a3bbaceca371b 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 @@ -584,7 +584,7 @@ intervalValue dataType : complex=ARRAY '<' dataType '>' #complexDataType | complex=MAP '<' dataType ',' dataType '>' #complexDataType - | complex=STRUCT ('<' colTypeList? '>' | NEQ) #complexDataType + | complex=STRUCT ('<' complexColTypeList? '>' | NEQ) #complexDataType | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')? #primitiveDataType ; @@ -593,7 +593,15 @@ colTypeList ; colType - : identifier ':'? dataType (COMMENT STRING)? + : identifier dataType (COMMENT STRING)? + ; + +complexColTypeList + : complexColType (',' complexColType)* + ; + +complexColType + : identifier ':' dataType (COMMENT STRING)? ; whenClause diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index bf3f30279a6fe..929c1c4f2d9e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -316,7 +316,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // Create the attributes. val (attributes, schemaLess) = if (colTypeList != null) { // Typed return columns. - (createStructType(colTypeList).toAttributes, false) + (createSchema(colTypeList).toAttributes, false) } else if (identifierSeq != null) { // Untyped return columns. val attrs = visitIdentifierSeq(identifierSeq).map { name => @@ -1450,14 +1450,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { case SqlBaseParser.MAP => MapType(typedVisit(ctx.dataType(0)), typedVisit(ctx.dataType(1))) case SqlBaseParser.STRUCT => - createStructType(ctx.colTypeList()) + createStructType(ctx.complexColTypeList()) } } /** - * Create a [[StructType]] from a sequence of [[StructField]]s. + * Create top level table schema. */ - protected def createStructType(ctx: ColTypeListContext): StructType = { + protected def createSchema(ctx: ColTypeListContext): StructType = { StructType(Option(ctx).toSeq.flatMap(visitColTypeList)) } @@ -1476,4 +1476,28 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val structField = StructField(identifier.getText, typedVisit(dataType), nullable = true) if (STRING == null) structField else structField.withComment(string(STRING)) } + + /** + * Create a [[StructType]] from a sequence of [[StructField]]s. + */ + protected def createStructType(ctx: ComplexColTypeListContext): StructType = { + StructType(Option(ctx).toSeq.flatMap(visitComplexColTypeList)) + } + + /** + * Create a [[StructType]] from a number of column definitions. + */ + override def visitComplexColTypeList( + ctx: ComplexColTypeListContext): Seq[StructField] = withOrigin(ctx) { + ctx.complexColType().asScala.map(visitComplexColType) + } + + /** + * Create a [[StructField]] from a column definition. + */ + override def visitComplexColType(ctx: ComplexColTypeContext): StructField = withOrigin(ctx) { + import ctx._ + val structField = StructField(identifier.getText, typedVisit(dataType), nullable = true) + if (STRING == null) structField else structField.withComment(string(STRING)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala index 020fb16f6f3d5..3964fa3924b24 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala @@ -116,6 +116,7 @@ class DataTypeParserSuite extends SparkFunSuite { unsupported("it is not a data type") unsupported("struct") unsupported("struct") // DataType parser accepts certain reserved keywords. checkDataType( @@ -125,16 +126,11 @@ class DataTypeParserSuite extends SparkFunSuite { StructField("DATE", BooleanType, true) :: Nil) ) - // Define struct columns without ':' - checkDataType( - "struct", - (new StructType).add("x", IntegerType).add("y", StringType)) - - checkDataType( - "struct<`x``y` int>", - (new StructType).add("x`y", IntegerType)) - // Use SQL keywords. checkDataType("struct", (new StructType).add("end", LongType).add("select", IntegerType).add("from", StringType)) + + // DataType parser accepts comments. + checkDataType("Struct", + (new StructType).add("x", IntegerType).add("y", StringType, true, "test")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 085bb9fc3c6cc..5f87b71210d31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -340,7 +340,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (provider.toLowerCase == "hive") { throw new AnalysisException("Cannot create hive serde table with CREATE TABLE USING") } - val schema = Option(ctx.colTypeList()).map(createStructType) + val schema = Option(ctx.colTypeList()).map(createSchema) val partitionColumnNames = Option(ctx.partitionColumnNames) .map(visitIdentifierList(_).toArray) @@ -399,7 +399,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: CreateTempViewUsingContext): LogicalPlan = withOrigin(ctx) { CreateTempViewUsing( tableIdent = visitTableIdentifier(ctx.tableIdentifier()), - userSpecifiedSchema = Option(ctx.colTypeList()).map(createStructType), + userSpecifiedSchema = Option(ctx.colTypeList()).map(createSchema), replace = ctx.REPLACE != null, provider = ctx.tableProvider.qualifiedName.getText, options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 6712d32924890..e0976ae95001e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -17,13 +17,17 @@ package org.apache.spark.sql.execution +import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.{DescribeFunctionCommand, DescribeTableCommand, ShowFunctionsCommand} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.execution.datasources.{CreateTable, CreateTempViewUsing} +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} /** * Parser test cases for rules defined in [[SparkSqlParser]]. @@ -35,8 +39,23 @@ class SparkSqlParserSuite extends PlanTest { private lazy val parser = new SparkSqlParser(new SQLConf) + /** + * Normalizes plans: + * - CreateTable the createTime in tableDesc will replaced by -1L. + */ + private def normalizePlan(plan: LogicalPlan): LogicalPlan = { + plan match { + case CreateTable(tableDesc, mode, query) => + val newTableDesc = tableDesc.copy(createTime = -1L) + CreateTable(newTableDesc, mode, query) + case _ => plan // Don't transform + } + } + private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { - comparePlans(parser.parsePlan(sqlCommand), plan) + val normalized1 = normalizePlan(parser.parsePlan(sqlCommand)) + val normalized2 = normalizePlan(plan) + comparePlans(normalized1, normalized2) } private def intercept(sqlCommand: String, messages: String*): Unit = { @@ -68,9 +87,134 @@ class SparkSqlParserSuite extends PlanTest { DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = true)) assertEqual("describe function foo.bar", DescribeFunctionCommand( - FunctionIdentifier("bar", database = Option("foo")), isExtended = false)) + FunctionIdentifier("bar", database = Some("foo")), isExtended = false)) assertEqual("describe function extended f.bar", - DescribeFunctionCommand(FunctionIdentifier("bar", database = Option("f")), isExtended = true)) + DescribeFunctionCommand(FunctionIdentifier("bar", database = Some("f")), isExtended = true)) + } + + private def createTableUsing( + table: String, + database: Option[String] = None, + tableType: CatalogTableType = CatalogTableType.MANAGED, + storage: CatalogStorageFormat = CatalogStorageFormat.empty, + schema: StructType = new StructType, + provider: Option[String] = Some("parquet"), + partitionColumnNames: Seq[String] = Seq.empty, + bucketSpec: Option[BucketSpec] = None, + mode: SaveMode = SaveMode.ErrorIfExists, + query: Option[LogicalPlan] = None): CreateTable = { + CreateTable( + CatalogTable( + identifier = TableIdentifier(table, database), + tableType = tableType, + storage = storage, + schema = schema, + provider = provider, + partitionColumnNames = partitionColumnNames, + bucketSpec = bucketSpec + ), mode, query + ) + } + + private def createTempViewUsing( + table: String, + database: Option[String] = None, + schema: Option[StructType] = None, + replace: Boolean = true, + provider: String = "parquet", + options: Map[String, String] = Map.empty): LogicalPlan = { + CreateTempViewUsing(TableIdentifier(table, database), schema, replace, provider, options) + } + + private def createTable( + table: String, + database: Option[String] = None, + tableType: CatalogTableType = CatalogTableType.MANAGED, + storage: CatalogStorageFormat = CatalogStorageFormat.empty.copy( + inputFormat = HiveSerDe.sourceToSerDe("textfile").get.inputFormat, + outputFormat = HiveSerDe.sourceToSerDe("textfile").get.outputFormat), + schema: StructType = new StructType, + provider: Option[String] = Some("hive"), + partitionColumnNames: Seq[String] = Seq.empty, + comment: Option[String] = None, + mode: SaveMode = SaveMode.ErrorIfExists, + query: Option[LogicalPlan] = None): CreateTable = { + CreateTable( + CatalogTable( + identifier = TableIdentifier(table, database), + tableType = tableType, + storage = storage, + schema = schema, + provider = provider, + partitionColumnNames = partitionColumnNames, + comment = comment + ), mode, query + ) + } + + test("create table - schema") { + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING)", + createTable( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + ) + ) + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) " + + "PARTITIONED BY (c INT, d STRING COMMENT 'test2')", + createTable( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + .add("c", IntegerType) + .add("d", StringType, nullable = true, "test2"), + partitionColumnNames = Seq("c", "d") + ) + ) + assertEqual("CREATE TABLE my_tab(id BIGINT, nested STRUCT)", + createTable( + table = "my_tab", + schema = (new StructType) + .add("id", LongType) + .add("nested", (new StructType) + .add("col1", StringType) + .add("col2", IntegerType) + ) + ) + ) + // Partitioned by a StructType should be accepted by `SparkSqlParser` but will fail an analyze + // rule in `AnalyzeCreateTable`. + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) " + + "PARTITIONED BY (nested STRUCT)", + createTable( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + .add("nested", (new StructType) + .add("col1", StringType) + .add("col2", IntegerType) + ), + partitionColumnNames = Seq("nested") + ) + ) + intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING)", + "no viable alternative at input") + } + + test("create table using - schema") { + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet", + createTableUsing( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + ) + ) + intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet", + "no viable alternative at input") } test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b5499f2884c61..1bcb810a1564f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -642,7 +642,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val csvFile = Thread.currentThread().getContextClassLoader.getResource("test-data/cars.csv").toString withView("testview") { - sql(s"CREATE OR REPLACE TEMPORARY VIEW testview (c1: String, c2: String) USING " + + sql(s"CREATE OR REPLACE TEMPORARY VIEW testview (c1 String, c2 String) USING " + "org.apache.spark.sql.execution.datasources.csv.CSVFileFormat " + s"OPTIONS (PATH '$csvFile')") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 54e27b6f73502..9ce3338647398 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -243,7 +243,7 @@ class HiveDDLCommandSuite extends PlanTest { .asInstanceOf[ScriptTransformation].copy(ioschema = null) val plan2 = parser.parsePlan("map a, b using 'func' as c, d from e") .asInstanceOf[ScriptTransformation].copy(ioschema = null) - val plan3 = parser.parsePlan("reduce a, b using 'func' as (c: int, d decimal(10, 0)) from e") + val plan3 = parser.parsePlan("reduce a, b using 'func' as (c int, d decimal(10, 0)) from e") .asInstanceOf[ScriptTransformation].copy(ioschema = null) val p = ScriptTransformation( From 23ddff4b2b2744c3dc84d928e144c541ad5df376 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 10 Oct 2016 15:48:57 +0800 Subject: [PATCH 032/177] [SPARK-17338][SQL] add global temp view ## What changes were proposed in this pull request? Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1. changes for `SessionCatalog`: 1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name. 2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved. 3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved. 4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views. 5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view. 6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views. 7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views. changes for SQL commands: 1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views 2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views. 3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc. changes for other public API 1. add a new method `dropGlobalTempView` in `Catalog` 2. `Catalog.findTable` can find global temp view 3. add a new method `createGlobalTempView` in `Dataset` ## How was this patch tested? new tests in `SQLViewSuite` Author: Wenchen Fan Closes #14897 from cloud-fan/global-temp-view. --- .../spark/internal/config/package.scala | 7 + docs/sql-programming-guide.md | 45 ++++- .../examples/sql/JavaSparkSQLExample.java | 30 ++- examples/src/main/python/sql/basic.py | 25 +++ .../spark/examples/sql/SparkSQLExample.scala | 25 +++ project/MimaExcludes.scala | 4 +- python/pyspark/sql/catalog.py | 18 +- python/pyspark/sql/context.py | 2 +- python/pyspark/sql/dataframe.py | 25 ++- .../spark/sql/catalyst/parser/SqlBase.g4 | 8 +- .../sql/catalyst/analysis/Analyzer.scala | 10 +- .../catalog/GlobalTempViewManager.scala | 121 +++++++++++ .../sql/catalyst/catalog/SessionCatalog.scala | 189 ++++++++++++++---- .../scala/org/apache/spark/sql/Dataset.scala | 48 ++++- .../apache/spark/sql/catalog/Catalog.scala | 20 +- .../spark/sql/execution/QueryExecution.scala | 8 +- .../spark/sql/execution/SparkSqlParser.scala | 19 +- .../spark/sql/execution/command/ddl.scala | 25 ++- .../spark/sql/execution/command/tables.scala | 11 +- .../spark/sql/execution/command/views.scala | 150 +++++++------- .../spark/sql/execution/datasources/ddl.scala | 20 +- .../spark/sql/internal/CatalogImpl.scala | 26 ++- .../spark/sql/internal/SessionState.scala | 1 + .../spark/sql/internal/SharedState.scala | 75 ++++--- .../apache/spark/sql/SQLContextSuite.scala | 11 +- .../sql/execution/GlobalTempViewSuite.scala | 168 ++++++++++++++++ .../sql/execution/command/DDLSuite.scala | 10 +- .../spark/sql/hive/HiveSessionCatalog.scala | 4 +- .../spark/sql/hive/HiveSessionState.scala | 1 + .../hive/HiveContextCompatibilitySuite.scala | 4 +- .../spark/sql/hive/ListTablesSuite.scala | 8 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../sql/hive/execution/HiveCommandSuite.scala | 10 +- .../sql/hive/execution/SQLViewSuite.scala | 6 +- 34 files changed, 906 insertions(+), 230 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index d536cc5097b2d..0896e68eca7dc 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -98,6 +98,13 @@ package object config { .checkValues(Set("hive", "in-memory")) .createWithDefault("in-memory") + // Note: This is a SQL config but needs to be in core because it's cross-session and can not put + // in SQLConf. + private[spark] val GLOBAL_TEMP_DATABASE = ConfigBuilder("spark.sql.globalTempDatabase") + .internal() + .stringConf + .createWithDefault("global_temp") + private[spark] val LISTENER_BUS_EVENT_QUEUE_SIZE = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.size") .intConf diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 71bdd19c16dbb..835cb6981f5bd 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -220,6 +220,41 @@ The `sql` function enables applications to run SQL queries programmatically and +## Global Temporary View + +Temporay views in Spark SQL are session-scoped and will disappear if the session that creates it +terminates. If you want to have a temporary view that is shared among all sessions and keep alive +until the Spark application terminiates, you can create a global temporary view. Global temporary +view is tied to a system preserved database `global_temp`, and we must use the qualified name to +refer it, e.g. `SELECT * FROM global_temp.view1`. + +
+
+{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
+ +
+{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
+ +
+{% include_example global_temp_view python/sql/basic.py %} +
+ +
+ +{% highlight sql %} + +CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl + +SELECT * FROM global_temp.temp_view + +{% endhighlight %} + +
+
+ + ## Creating Datasets Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use @@ -1058,14 +1093,14 @@ the Data Sources API. The following options are supported: The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). - + truncate - This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g. indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. + This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g. indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. - + createTableOptions @@ -1101,11 +1136,11 @@ USING org.apache.spark.sql.jdbc OPTIONS ( url "jdbc:postgresql:dbserver", dbtable "schema.tablename", - user 'username', + user 'username', password 'password' ) -INSERT INTO TABLE jdbcTable +INSERT INTO TABLE jdbcTable SELECT * FROM resultTable {% endhighlight %} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java index cff9032f52b5a..c5770d147a6b5 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java @@ -54,6 +54,7 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; // $example off:programmatic_schema$ +import org.apache.spark.sql.AnalysisException; // $example on:untyped_ops$ // col("...") is preferable to df.col("...") @@ -84,7 +85,7 @@ public void setAge(int age) { } // $example off:create_ds$ - public static void main(String[] args) { + public static void main(String[] args) throws AnalysisException { // $example on:init_session$ SparkSession spark = SparkSession .builder() @@ -101,7 +102,7 @@ public static void main(String[] args) { spark.stop(); } - private static void runBasicDataFrameExample(SparkSession spark) { + private static void runBasicDataFrameExample(SparkSession spark) throws AnalysisException { // $example on:create_df$ Dataset df = spark.read().json("examples/src/main/resources/people.json"); @@ -176,6 +177,31 @@ private static void runBasicDataFrameExample(SparkSession spark) { // | 19| Justin| // +----+-------+ // $example off:run_sql$ + + // $example on:global_temp_view$ + // Register the DataFrame as a global temporary view + df.createGlobalTempView("people"); + + // Global temporary view is tied to a system preserved database `global_temp` + spark.sql("SELECT * FROM global_temp.people").show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + + // Global temporary view is cross-session + spark.newSession().sql("SELECT * FROM global_temp.people").show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:global_temp_view$ } private static void runDatasetCreationExample(SparkSession spark) { diff --git a/examples/src/main/python/sql/basic.py b/examples/src/main/python/sql/basic.py index fdc017aed97c1..ebcf66995b477 100644 --- a/examples/src/main/python/sql/basic.py +++ b/examples/src/main/python/sql/basic.py @@ -114,6 +114,31 @@ def basic_df_example(spark): # +----+-------+ # $example off:run_sql$ + # $example on:global_temp_view$ + # Register the DataFrame as a global temporary view + df.createGlobalTempView("people") + + # Global temporary view is tied to a system preserved database `global_temp` + spark.sql("SELECT * FROM global_temp.people").show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + + # Global temporary view is cross-session + spark.newSession().sql("SELECT * FROM global_temp.people").show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + # $example off:global_temp_view$ + def schema_inference_example(spark): # $example on:schema_inferring$ diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index 129b81d5fbbf3..f27c403c5b388 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -135,6 +135,31 @@ object SparkSQLExample { // | 19| Justin| // +----+-------+ // $example off:run_sql$ + + // $example on:global_temp_view$ + // Register the DataFrame as a global temporary view + df.createGlobalTempView("people") + + // Global temporary view is tied to a system preserved database `global_temp` + spark.sql("SELECT * FROM global_temp.people").show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + + // Global temporary view is cross-session + spark.newSession().sql("SELECT * FROM global_temp.people").show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:global_temp_view$ } private def runDatasetCreationExample(spark: SparkSession): Unit = { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 163e3f2fdea40..e3d9a17469a35 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -55,7 +55,9 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getFunction"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.databaseExists"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.tableExists"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"), + // [SPARK-17338][SQL] add global temp view + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView") ) } diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 3c5030722f307..df3bf4254d4d3 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -167,7 +167,7 @@ def createExternalTable(self, tableName, path=None, source=None, schema=None, ** @since(2.0) def dropTempView(self, viewName): - """Drops the temporary view with the given view name in the catalog. + """Drops the local temporary view with the given view name in the catalog. If the view has been cached before, then it will also be uncached. >>> spark.createDataFrame([(1, 1)]).createTempView("my_table") @@ -181,6 +181,22 @@ def dropTempView(self, viewName): """ self._jcatalog.dropTempView(viewName) + @since(2.1) + def dropGlobalTempView(self, viewName): + """Drops the global temporary view with the given view name in the catalog. + If the view has been cached before, then it will also be uncached. + + >>> spark.createDataFrame([(1, 1)]).createGlobalTempView("my_table") + >>> spark.table("global_temp.my_table").collect() + [Row(_1=1, _2=1)] + >>> spark.catalog.dropGlobalTempView("my_table") + >>> spark.table("global_temp.my_table") # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + AnalysisException: ... + """ + self._jcatalog.dropGlobalTempView(viewName) + @ignore_unicode_prefix @since(2.0) def registerFunction(self, name, f, returnType=StringType()): diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 7482be8bda5c4..8264dcf8a97d2 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -386,7 +386,7 @@ def tables(self, dbName=None): >>> sqlContext.registerDataFrameAsTable(df, "table1") >>> df2 = sqlContext.tables() >>> df2.filter("tableName = 'table1'").first() - Row(tableName=u'table1', isTemporary=True) + Row(database=u'', tableName=u'table1', isTemporary=True) """ if dbName is None: return DataFrame(self._ssql_ctx.tables(), self) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 0ac481a8a8b56..14e80ea4615ef 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -131,7 +131,7 @@ def registerTempTable(self, name): @since(2.0) def createTempView(self, name): - """Creates a temporary view with this DataFrame. + """Creates a local temporary view with this DataFrame. The lifetime of this temporary table is tied to the :class:`SparkSession` that was used to create this :class:`DataFrame`. @@ -153,7 +153,7 @@ def createTempView(self, name): @since(2.0) def createOrReplaceTempView(self, name): - """Creates or replaces a temporary view with this DataFrame. + """Creates or replaces a local temporary view with this DataFrame. The lifetime of this temporary table is tied to the :class:`SparkSession` that was used to create this :class:`DataFrame`. @@ -169,6 +169,27 @@ def createOrReplaceTempView(self, name): """ self._jdf.createOrReplaceTempView(name) + @since(2.1) + def createGlobalTempView(self, name): + """Creates a global temporary view with this DataFrame. + + The lifetime of this temporary view is tied to this Spark application. + throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the + catalog. + + >>> df.createGlobalTempView("people") + >>> df2 = spark.sql("select * from global_temp.people") + >>> sorted(df.collect()) == sorted(df2.collect()) + True + >>> df.createGlobalTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + AnalysisException: u"Temporary table 'people' already exists;" + >>> spark.catalog.dropGlobalTempView("people") + + """ + self._jdf.createGlobalTempView(name) + @property @since(1.4) def write(self): 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 a3bbaceca371b..b599a884957a8 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 @@ -111,11 +111,12 @@ statement | ALTER TABLE tableIdentifier RECOVER PARTITIONS #recoverPartitions | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? tableIdentifier #dropTable - | CREATE (OR REPLACE)? TEMPORARY? VIEW (IF NOT EXISTS)? tableIdentifier + | CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)? + VIEW (IF NOT EXISTS)? tableIdentifier identifierCommentList? (COMMENT STRING)? (PARTITIONED ON identifierList)? (TBLPROPERTIES tablePropertyList)? AS query #createView - | CREATE (OR REPLACE)? TEMPORARY VIEW + | CREATE (OR REPLACE)? GLOBAL? TEMPORARY VIEW tableIdentifier ('(' colTypeList ')')? tableProvider (OPTIONS tablePropertyList)? #createTempViewUsing | ALTER VIEW tableIdentifier AS? query #alterViewQuery @@ -676,7 +677,7 @@ nonReserved | MAP | ARRAY | STRUCT | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED - | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | TEMPORARY | OPTIONS + | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | GLOBAL | TEMPORARY | OPTIONS | GROUPING | CUBE | ROLLUP | EXPLAIN | FORMAT | LOGICAL | FORMATTED | CODEGEN | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF @@ -864,6 +865,7 @@ CACHE: 'CACHE'; UNCACHE: 'UNCACHE'; LAZY: 'LAZY'; FORMATTED: 'FORMATTED'; +GLOBAL: 'GLOBAL'; TEMPORARY: 'TEMPORARY' | 'TEMP'; OPTIONS: 'OPTIONS'; UNSET: 'UNSET'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ae8869ff25f2d..536d38777f89d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -458,12 +458,12 @@ class Analyzer( i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u))) case u: UnresolvedRelation => val table = u.tableIdentifier - if (table.database.isDefined && conf.runSQLonFile && + if (table.database.isDefined && conf.runSQLonFile && !catalog.isTemporaryTable(table) && (!catalog.databaseExists(table.database.get) || !catalog.tableExists(table))) { - // If the table does not exist, and the database part is specified, and we support - // running SQL directly on files, then let's just return the original UnresolvedRelation. - // It is possible we are matching a query like "select * from parquet.`/path/to/query`". - // The plan will get resolved later. + // If the database part is specified, and we support running SQL directly on files, and + // it's not a temporary view, and the table does not exist, then let's just return the + // original UnresolvedRelation. It is possible we are matching a query like "select * + // from parquet.`/path/to/query`". The plan will get resolved later. // Note that we are testing (!db_exists || !table_exists) because the catalog throws // an exception from tableExists if the database does not exist. u diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala new file mode 100644 index 0000000000000..6095ac0bc9c50 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala @@ -0,0 +1,121 @@ +/* + * 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.catalog + +import javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.StringUtils + + +/** + * A thread-safe manager for global temporary views, providing atomic operations to manage them, + * e.g. create, update, remove, etc. + * + * Note that, the view name is always case-sensitive here, callers are responsible to format the + * view name w.r.t. case-sensitive config. + * + * @param database The system preserved virtual database that keeps all the global temporary views. + */ +class GlobalTempViewManager(val database: String) { + + /** List of view definitions, mapping from view name to logical plan. */ + @GuardedBy("this") + private val viewDefinitions = new mutable.HashMap[String, LogicalPlan] + + /** + * Returns the global view definition which matches the given name, or None if not found. + */ + def get(name: String): Option[LogicalPlan] = synchronized { + viewDefinitions.get(name) + } + + /** + * Creates a global temp view, or issue an exception if the view already exists and + * `overrideIfExists` is false. + */ + def create( + name: String, + viewDefinition: LogicalPlan, + overrideIfExists: Boolean): Unit = synchronized { + if (!overrideIfExists && viewDefinitions.contains(name)) { + throw new TempTableAlreadyExistsException(name) + } + viewDefinitions.put(name, viewDefinition) + } + + /** + * Updates the global temp view if it exists, returns true if updated, false otherwise. + */ + def update( + name: String, + viewDefinition: LogicalPlan): Boolean = synchronized { + if (viewDefinitions.contains(name)) { + viewDefinitions.put(name, viewDefinition) + true + } else { + false + } + } + + /** + * Removes the global temp view if it exists, returns true if removed, false otherwise. + */ + def remove(name: String): Boolean = synchronized { + viewDefinitions.remove(name).isDefined + } + + /** + * Renames the global temp view if the source view exists and the destination view not exists, or + * issue an exception if the source view exists but the destination view already exists. Returns + * true if renamed, false otherwise. + */ + def rename(oldName: String, newName: String): Boolean = synchronized { + if (viewDefinitions.contains(oldName)) { + if (viewDefinitions.contains(newName)) { + throw new AnalysisException( + s"rename temporary view from '$oldName' to '$newName': destination view already exists") + } + + val viewDefinition = viewDefinitions(oldName) + viewDefinitions.remove(oldName) + viewDefinitions.put(newName, viewDefinition) + true + } else { + false + } + } + + /** + * Lists the names of all global temporary views. + */ + def listViewNames(pattern: String): Seq[String] = synchronized { + StringUtils.filterPattern(viewDefinitions.keys.toSeq, pattern) + } + + /** + * Clears all the global temporary views. + */ + def clear(): Unit = synchronized { + viewDefinitions.clear() + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 8c01c7a3f2bd5..e44e30ec648f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} @@ -47,6 +48,7 @@ object SessionCatalog { */ class SessionCatalog( externalCatalog: ExternalCatalog, + globalTempViewManager: GlobalTempViewManager, functionResourceLoader: FunctionResourceLoader, functionRegistry: FunctionRegistry, conf: CatalystConf, @@ -61,6 +63,7 @@ class SessionCatalog( conf: CatalystConf) { this( externalCatalog, + new GlobalTempViewManager(GLOBAL_TEMP_DATABASE.defaultValueString), DummyFunctionResourceLoader, functionRegistry, conf, @@ -142,8 +145,13 @@ class SessionCatalog( // ---------------------------------------------------------------------------- def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = { - val qualifiedPath = makeQualifiedPath(dbDefinition.locationUri).toString val dbName = formatDatabaseName(dbDefinition.name) + if (dbName == globalTempViewManager.database) { + throw new AnalysisException( + s"${globalTempViewManager.database} is a system preserved database, " + + "you cannot create a database with this name.") + } + val qualifiedPath = makeQualifiedPath(dbDefinition.locationUri).toString externalCatalog.createDatabase( dbDefinition.copy(name = dbName, locationUri = qualifiedPath), ignoreIfExists) @@ -154,7 +162,7 @@ class SessionCatalog( if (dbName == DEFAULT_DATABASE) { throw new AnalysisException(s"Can not drop default database") } else if (dbName == getCurrentDatabase) { - throw new AnalysisException(s"Can not drop current database `${dbName}`") + throw new AnalysisException(s"Can not drop current database `$dbName`") } externalCatalog.dropDatabase(dbName, ignoreIfNotExists, cascade) } @@ -188,6 +196,13 @@ class SessionCatalog( def setCurrentDatabase(db: String): Unit = { val dbName = formatDatabaseName(db) + if (dbName == globalTempViewManager.database) { + throw new AnalysisException( + s"${globalTempViewManager.database} is a system preserved database, " + + "you cannot use it as current database. To access global temporary views, you should " + + "use qualified name with the GLOBAL_TEMP_DATABASE, e.g. SELECT * FROM " + + s"${globalTempViewManager.database}.viewName.") + } requireDbExists(dbName) synchronized { currentDb = dbName } } @@ -329,7 +344,7 @@ class SessionCatalog( // ---------------------------------------------- /** - * Create a temporary table. + * Create a local temporary view. */ def createTempView( name: String, @@ -343,19 +358,65 @@ class SessionCatalog( } /** - * Return a temporary view exactly as it was stored. + * Create a global temporary view. + */ + def createGlobalTempView( + name: String, + viewDefinition: LogicalPlan, + overrideIfExists: Boolean): Unit = { + globalTempViewManager.create(formatTableName(name), viewDefinition, overrideIfExists) + } + + /** + * Alter the definition of a local/global temp view matching the given name, returns true if a + * temp view is matched and altered, false otherwise. + */ + def alterTempViewDefinition( + name: TableIdentifier, + viewDefinition: LogicalPlan): Boolean = synchronized { + val viewName = formatTableName(name.table) + if (name.database.isEmpty) { + if (tempTables.contains(viewName)) { + createTempView(viewName, viewDefinition, overrideIfExists = true) + true + } else { + false + } + } else if (formatDatabaseName(name.database.get) == globalTempViewManager.database) { + globalTempViewManager.update(viewName, viewDefinition) + } else { + false + } + } + + /** + * Return a local temporary view exactly as it was stored. */ def getTempView(name: String): Option[LogicalPlan] = synchronized { tempTables.get(formatTableName(name)) } /** - * Drop a temporary view. + * Return a global temporary view exactly as it was stored. + */ + def getGlobalTempView(name: String): Option[LogicalPlan] = { + globalTempViewManager.get(formatTableName(name)) + } + + /** + * Drop a local temporary view. */ def dropTempView(name: String): Unit = synchronized { tempTables.remove(formatTableName(name)) } + /** + * Drop a global temporary view. + */ + def dropGlobalTempView(name: String): Boolean = { + globalTempViewManager.remove(formatTableName(name)) + } + // ------------------------------------------------------------- // | Methods that interact with temporary and metastore tables | // ------------------------------------------------------------- @@ -371,9 +432,7 @@ class SessionCatalog( */ def getTempViewOrPermanentTableMetadata(name: TableIdentifier): CatalogTable = synchronized { val table = formatTableName(name.table) - if (name.database.isDefined) { - getTableMetadata(name) - } else { + if (name.database.isEmpty) { getTempView(table).map { plan => CatalogTable( identifier = TableIdentifier(table), @@ -381,6 +440,16 @@ class SessionCatalog( storage = CatalogStorageFormat.empty, schema = plan.output.toStructType) }.getOrElse(getTableMetadata(name)) + } else if (formatDatabaseName(name.database.get) == globalTempViewManager.database) { + globalTempViewManager.get(table).map { plan => + CatalogTable( + identifier = TableIdentifier(table, Some(globalTempViewManager.database)), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = plan.output.toStructType) + }.getOrElse(throw new NoSuchTableException(globalTempViewManager.database, table)) + } else { + getTableMetadata(name) } } @@ -393,21 +462,25 @@ class SessionCatalog( */ def renameTable(oldName: TableIdentifier, newName: String): Unit = synchronized { val db = formatDatabaseName(oldName.database.getOrElse(currentDb)) - requireDbExists(db) val oldTableName = formatTableName(oldName.table) val newTableName = formatTableName(newName) - if (oldName.database.isDefined || !tempTables.contains(oldTableName)) { - requireTableExists(TableIdentifier(oldTableName, Some(db))) - requireTableNotExists(TableIdentifier(newTableName, Some(db))) - externalCatalog.renameTable(db, oldTableName, newTableName) + if (db == globalTempViewManager.database) { + globalTempViewManager.rename(oldTableName, newTableName) } else { - if (tempTables.contains(newTableName)) { - throw new AnalysisException( - s"RENAME TEMPORARY TABLE from '$oldName' to '$newName': destination table already exists") + requireDbExists(db) + if (oldName.database.isDefined || !tempTables.contains(oldTableName)) { + requireTableExists(TableIdentifier(oldTableName, Some(db))) + requireTableNotExists(TableIdentifier(newTableName, Some(db))) + externalCatalog.renameTable(db, oldTableName, newTableName) + } else { + if (tempTables.contains(newTableName)) { + throw new AnalysisException(s"RENAME TEMPORARY TABLE from '$oldName' to '$newName': " + + "destination table already exists") + } + val table = tempTables(oldTableName) + tempTables.remove(oldTableName) + tempTables.put(newTableName, table) } - val table = tempTables(oldTableName) - tempTables.remove(oldTableName) - tempTables.put(newTableName, table) } } @@ -424,17 +497,24 @@ class SessionCatalog( purge: Boolean): Unit = synchronized { val db = formatDatabaseName(name.database.getOrElse(currentDb)) val table = formatTableName(name.table) - if (name.database.isDefined || !tempTables.contains(table)) { - requireDbExists(db) - // When ignoreIfNotExists is false, no exception is issued when the table does not exist. - // Instead, log it as an error message. - if (tableExists(TableIdentifier(table, Option(db)))) { - externalCatalog.dropTable(db, table, ignoreIfNotExists = true, purge = purge) - } else if (!ignoreIfNotExists) { - throw new NoSuchTableException(db = db, table = table) + if (db == globalTempViewManager.database) { + val viewExists = globalTempViewManager.remove(table) + if (!viewExists && !ignoreIfNotExists) { + throw new NoSuchTableException(globalTempViewManager.database, table) } } else { - tempTables.remove(table) + if (name.database.isDefined || !tempTables.contains(table)) { + requireDbExists(db) + // When ignoreIfNotExists is false, no exception is issued when the table does not exist. + // Instead, log it as an error message. + if (tableExists(TableIdentifier(table, Option(db)))) { + externalCatalog.dropTable(db, table, ignoreIfNotExists = true, purge = purge) + } else if (!ignoreIfNotExists) { + throw new NoSuchTableException(db = db, table = table) + } + } else { + tempTables.remove(table) + } } } @@ -445,6 +525,9 @@ class SessionCatalog( * If no database is specified, this will first attempt to return a temporary table/view with * the same name, then, if that does not exist, return the table/view from the current database. * + * Note that, the global temp view database is also valid here, this will return the global temp + * view matching the given name. + * * If the relation is a view, the relation will be wrapped in a [[SubqueryAlias]] which will * track the name of the view. */ @@ -453,7 +536,11 @@ class SessionCatalog( val db = formatDatabaseName(name.database.getOrElse(currentDb)) val table = formatTableName(name.table) val relationAlias = alias.getOrElse(table) - if (name.database.isDefined || !tempTables.contains(table)) { + if (db == globalTempViewManager.database) { + globalTempViewManager.get(table).map { viewDef => + SubqueryAlias(relationAlias, viewDef, Some(name)) + }.getOrElse(throw new NoSuchTableException(db, table)) + } else if (name.database.isDefined || !tempTables.contains(table)) { val metadata = externalCatalog.getTable(db, table) val view = Option(metadata.tableType).collect { case CatalogTableType.VIEW => name @@ -472,27 +559,48 @@ class SessionCatalog( * explicitly specified. */ def isTemporaryTable(name: TableIdentifier): Boolean = synchronized { - name.database.isEmpty && tempTables.contains(formatTableName(name.table)) + val table = formatTableName(name.table) + if (name.database.isEmpty) { + tempTables.contains(table) + } else if (formatDatabaseName(name.database.get) == globalTempViewManager.database) { + globalTempViewManager.get(table).isDefined + } else { + false + } } /** - * List all tables in the specified database, including temporary tables. + * List all tables in the specified database, including local temporary tables. + * + * Note that, if the specified database is global temporary view database, we will list global + * temporary views. */ def listTables(db: String): Seq[TableIdentifier] = listTables(db, "*") /** - * List all matching tables in the specified database, including temporary tables. + * List all matching tables in the specified database, including local temporary tables. + * + * Note that, if the specified database is global temporary view database, we will list global + * temporary views. */ def listTables(db: String, pattern: String): Seq[TableIdentifier] = { val dbName = formatDatabaseName(db) - requireDbExists(dbName) - val dbTables = - externalCatalog.listTables(dbName, pattern).map { t => TableIdentifier(t, Some(dbName)) } - synchronized { - val _tempTables = StringUtils.filterPattern(tempTables.keys.toSeq, pattern) - .map { t => TableIdentifier(t) } - dbTables ++ _tempTables + val dbTables = if (dbName == globalTempViewManager.database) { + globalTempViewManager.listViewNames(pattern).map { name => + TableIdentifier(name, Some(globalTempViewManager.database)) + } + } else { + requireDbExists(dbName) + externalCatalog.listTables(dbName, pattern).map { name => + TableIdentifier(name, Some(dbName)) + } + } + val localTempViews = synchronized { + StringUtils.filterPattern(tempTables.keys.toSeq, pattern).map { name => + TableIdentifier(name) + } } + dbTables ++ localTempViews } /** @@ -504,6 +612,8 @@ class SessionCatalog( // If the database is not defined, there is a good chance this is a temp table. if (name.database.isEmpty) { tempTables.get(formatTableName(name.table)).foreach(_.refresh()) + } else if (formatDatabaseName(name.database.get) == globalTempViewManager.database) { + globalTempViewManager.get(formatTableName(name.table)).foreach(_.refresh()) } } @@ -919,6 +1029,7 @@ class SessionCatalog( } } tempTables.clear() + globalTempViewManager.clear() functionRegistry.clear() // restore built-in functions FunctionRegistry.builtin.listFunction().foreach { f => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 9cfbdffd02582..4b52508740bfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} -import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand} +import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython @@ -2433,9 +2433,13 @@ class Dataset[T] private[sql]( } /** - * Creates a temporary view using the given name. The lifetime of this + * Creates a local temporary view using the given name. The lifetime of this * temporary view is tied to the [[SparkSession]] that was used to create this Dataset. * + * Local temporary view is session-scoped. Its lifetime is the lifetime of the session that + * created it, i.e. it will be automatically dropped when the session terminates. It's not + * tied to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. + * * @throws AnalysisException if the view name already exists * * @group basic @@ -2443,21 +2447,51 @@ class Dataset[T] private[sql]( */ @throws[AnalysisException] def createTempView(viewName: String): Unit = withPlan { - createViewCommand(viewName, replace = false) + createTempViewCommand(viewName, replace = false, global = false) } + + /** - * Creates a temporary view using the given name. The lifetime of this + * Creates a local temporary view using the given name. The lifetime of this * temporary view is tied to the [[SparkSession]] that was used to create this Dataset. * * @group basic * @since 2.0.0 */ def createOrReplaceTempView(viewName: String): Unit = withPlan { - createViewCommand(viewName, replace = true) + createTempViewCommand(viewName, replace = true, global = false) } - private def createViewCommand(viewName: String, replace: Boolean): CreateViewCommand = { + /** + * Creates a global temporary view using the given name. The lifetime of this + * temporary view is tied to this Spark application. + * + * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, + * i.e. it will be automatically dropped when the application terminates. It's tied to a system + * preserved database `_global_temp`, and we must use the qualified name to refer a global temp + * view, e.g. `SELECT * FROM _global_temp.view1`. + * + * @throws TempTableAlreadyExistsException if the view name already exists + * + * @group basic + * @since 2.1.0 + */ + @throws[AnalysisException] + def createGlobalTempView(viewName: String): Unit = withPlan { + createTempViewCommand(viewName, replace = false, global = true) + } + + private def createTempViewCommand( + viewName: String, + replace: Boolean, + global: Boolean): CreateViewCommand = { + val viewType = if (global) { + GlobalTempView + } else { + LocalTempView + } + CreateViewCommand( name = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName), userSpecifiedColumns = Nil, @@ -2467,7 +2501,7 @@ class Dataset[T] private[sql]( child = logicalPlan, allowExisting = false, replace = replace, - isTemporary = true) + viewType = viewType) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 7f2762c7dac92..717fb291901bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -262,14 +262,32 @@ abstract class Catalog { options: Map[String, String]): DataFrame /** - * Drops the temporary view with the given view name in the catalog. + * Drops the local temporary view with the given view name in the catalog. * If the view has been cached before, then it will also be uncached. * + * Local temporary view is session-scoped. Its lifetime is the lifetime of the session that + * created it, i.e. it will be automatically dropped when the session terminates. It's not + * tied to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. + * * @param viewName the name of the view to be dropped. * @since 2.0.0 */ def dropTempView(viewName: String): Unit + /** + * Drops the global temporary view with the given view name in the catalog. + * If the view has been cached before, then it will also be uncached. + * + * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, + * i.e. it will be automatically dropped when the application terminates. It's tied to a system + * preserved database `_global_temp`, and we must use the qualified name to refer a global temp + * view, e.g. `SELECT * FROM _global_temp.view1`. + * + * @param viewName the name of the view to be dropped. + * @since 2.1.0 + */ + def dropGlobalTempView(viewName: String): Boolean + /** * Returns true if the table is currently cached in-memory. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 383b3a233fc27..cb45a6d78b9b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -21,15 +21,14 @@ import java.nio.charset.StandardCharsets import java.sql.Timestamp import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext} +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec} +import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _} import org.apache.spark.util.Utils @@ -125,6 +124,9 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { .mkString("\t") } } + // SHOW TABLES in Hive only output table names, while ours outputs database, table name, isTemp. + case command: ExecutedCommandExec if command.cmd.isInstanceOf[ShowTablesCommand] => + command.executeCollect().map(_.getString(1)) case command: ExecutedCommandExec => command.executeCollect().map(_.getString(0)) case other => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 5f87b71210d31..be2eddbb0e423 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -29,9 +29,9 @@ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, ScriptInputOutputSchema} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{CreateTable, CreateTempViewUsing, _} +import org.apache.spark.sql.execution.datasources.{CreateTable, _} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType /** * Concrete parser for Spark SQL statements. @@ -385,7 +385,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { logWarning(s"CREATE TEMPORARY TABLE ... USING ... is deprecated, please use " + "CREATE TEMPORARY VIEW ... USING ... instead") - CreateTempViewUsing(table, schema, replace = true, provider, options) + CreateTempViewUsing(table, schema, replace = true, global = false, provider, options) } else { CreateTable(tableDesc, mode, None) } @@ -401,6 +401,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { tableIdent = visitTableIdentifier(ctx.tableIdentifier()), userSpecifiedSchema = Option(ctx.colTypeList()).map(createSchema), replace = ctx.REPLACE != null, + global = ctx.GLOBAL != null, provider = ctx.tableProvider.qualifiedName.getText, options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } @@ -1269,7 +1270,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * For example: * {{{ - * CREATE [OR REPLACE] [TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name + * CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name * [(column_name [COMMENT column_comment], ...) ] * [COMMENT view_comment] * [TBLPROPERTIES (property_name = property_value, ...)] @@ -1286,6 +1287,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } } + val viewType = if (ctx.TEMPORARY == null) { + PersistedView + } else if (ctx.GLOBAL != null) { + GlobalTempView + } else { + LocalTempView + } + CreateViewCommand( name = visitTableIdentifier(ctx.tableIdentifier), userSpecifiedColumns = userSpecifiedColumns, @@ -1295,7 +1304,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { child = plan(ctx.query), allowExisting = ctx.EXISTS != null, replace = ctx.REPLACE != null, - isTemporary = ctx.TEMPORARY != null) + viewType = viewType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 01ac89868d100..45fa293e58951 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -183,17 +183,20 @@ case class DropTableCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view - // issue an exception. - catalog.getTableMetadataOption(tableName).map(_.tableType match { - case CatalogTableType.VIEW if !isView => - throw new AnalysisException( - "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead") - case o if o != CatalogTableType.VIEW && isView => - throw new AnalysisException( - s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead") - case _ => - }) + + if (!catalog.isTemporaryTable(tableName) && catalog.tableExists(tableName)) { + // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view + // issue an exception. + catalog.getTableMetadata(tableName).tableType match { + case CatalogTableType.VIEW if !isView => + throw new AnalysisException( + "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead") + case o if o != CatalogTableType.VIEW && isView => + throw new AnalysisException( + s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead") + case _ => + } + } try { sparkSession.sharedState.cacheManager.uncacheQuery( sparkSession.table(tableName.quotedString)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 08de6cd4242c5..424ef58d76c5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -579,9 +579,10 @@ case class ShowTablesCommand( databaseName: Option[String], tableIdentifierPattern: Option[String]) extends RunnableCommand { - // The result of SHOW TABLES has two columns, tableName and isTemporary. + // The result of SHOW TABLES has three columns: database, tableName and isTemporary. override val output: Seq[Attribute] = { - AttributeReference("tableName", StringType, nullable = false)() :: + AttributeReference("database", StringType, nullable = false)() :: + AttributeReference("tableName", StringType, nullable = false)() :: AttributeReference("isTemporary", BooleanType, nullable = false)() :: Nil } @@ -592,9 +593,9 @@ case class ShowTablesCommand( val db = databaseName.getOrElse(catalog.getCurrentDatabase) val tables = tableIdentifierPattern.map(catalog.listTables(db, _)).getOrElse(catalog.listTables(db)) - tables.map { t => - val isTemp = t.database.isEmpty - Row(t.table, isTemp) + tables.map { tableIdent => + val isTemp = catalog.isTemporaryTable(tableIdent) + Row(tableIdent.database.getOrElse(""), tableIdent.table, isTemp) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 15340ee921f68..bbcd9c4ef564c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -19,13 +19,46 @@ package org.apache.spark.sql.execution.command import scala.util.control.NonFatal -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} +import org.apache.spark.sql.types.{MetadataBuilder, StructType} + + +/** + * ViewType is used to specify the expected view type when we want to create or replace a view in + * [[CreateViewCommand]]. + */ +sealed trait ViewType + +/** + * LocalTempView means session-scoped local temporary views. Its lifetime is the lifetime of the + * session that created it, i.e. it will be automatically dropped when the session terminates. It's + * not tied to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. + */ +object LocalTempView extends ViewType + +/** + * GlobalTempView means cross-session global temporary views. Its lifetime is the lifetime of the + * Spark application, i.e. it will be automatically dropped when the application terminates. It's + * tied to a system preserved database `_global_temp`, and we must use the qualified name to refer a + * global temp view, e.g. SELECT * FROM _global_temp.view1. + */ +object GlobalTempView extends ViewType + +/** + * PersistedView means cross-session persisted views. Persisted views stay until they are + * explicitly dropped by user command. It's always tied to a database, default to the current + * database if not specified. + * + * Note that, Existing persisted view with the same name are not visible to the current session + * while the local temporary view exists, unless the view name is qualified by database. + */ +object PersistedView extends ViewType /** @@ -46,10 +79,7 @@ import org.apache.spark.sql.types.StructType * already exists, throws analysis exception. * @param replace if true, and if the view already exists, updates it; if false, and if the view * already exists, throws analysis exception. - * @param isTemporary if true, the view is created as a temporary view. Temporary views are dropped - * at the end of current Spark session. Existing permanent relations with the same - * name are not visible to the current session while the temporary view exists, - * unless they are specified with full qualified table name with database prefix. + * @param viewType the expected view type to be created with this command. */ case class CreateViewCommand( name: TableIdentifier, @@ -60,20 +90,21 @@ case class CreateViewCommand( child: LogicalPlan, allowExisting: Boolean, replace: Boolean, - isTemporary: Boolean) + viewType: ViewType) extends RunnableCommand { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) - if (!isTemporary) { - require(originalText.isDefined, - "The table to created with CREATE VIEW must have 'originalText'.") + if (viewType == PersistedView) { + require(originalText.isDefined, "'originalText' must be provided to create permanent view") } if (allowExisting && replace) { throw new AnalysisException("CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed.") } + private def isTemporary = viewType == LocalTempView || viewType == GlobalTempView + // Disallows 'CREATE TEMPORARY VIEW IF NOT EXISTS' to be consistent with 'CREATE TEMPORARY TABLE' if (allowExisting && isTemporary) { throw new AnalysisException( @@ -99,72 +130,53 @@ case class CreateViewCommand( s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " + s"specified by CREATE VIEW (num: `${userSpecifiedColumns.length}`).") } - val sessionState = sparkSession.sessionState - - if (isTemporary) { - createTemporaryView(sparkSession, analyzedPlan) - } else { - // Adds default database for permanent table if it doesn't exist, so that tableExists() - // only check permanent tables. - val database = name.database.getOrElse(sessionState.catalog.getCurrentDatabase) - val qualifiedName = name.copy(database = Option(database)) - - if (sessionState.catalog.tableExists(qualifiedName)) { - val tableMetadata = sessionState.catalog.getTableMetadata(qualifiedName) - if (allowExisting) { - // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view - // already exists. - } else if (tableMetadata.tableType != CatalogTableType.VIEW) { - throw new AnalysisException(s"$qualifiedName is not a view") - } else if (replace) { - // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` - sessionState.catalog.alterTable(prepareTable(sparkSession, analyzedPlan)) - } else { - // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already - // exists. - throw new AnalysisException( - s"View $qualifiedName already exists. If you want to update the view definition, " + - "please use ALTER VIEW AS or CREATE OR REPLACE VIEW AS") - } - } else { - // Create the view if it doesn't exist. - sessionState.catalog.createTable( - prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) - } - } - Seq.empty[Row] - } - - private def createTemporaryView(sparkSession: SparkSession, analyzedPlan: LogicalPlan): Unit = { - val catalog = sparkSession.sessionState.catalog - // Projects column names to alias names - val logicalPlan = if (userSpecifiedColumns.isEmpty) { + val aliasedPlan = if (userSpecifiedColumns.isEmpty) { analyzedPlan } else { val projectList = analyzedPlan.output.zip(userSpecifiedColumns).map { - case (attr, (colName, _)) => Alias(attr, colName)() + case (attr, (colName, None)) => Alias(attr, colName)() + case (attr, (colName, Some(colComment))) => + val meta = new MetadataBuilder().putString("comment", colComment).build() + Alias(attr, colName)(explicitMetadata = Some(meta)) } sparkSession.sessionState.executePlan(Project(projectList, analyzedPlan)).analyzed } - catalog.createTempView(name.table, logicalPlan, replace) + val catalog = sparkSession.sessionState.catalog + if (viewType == LocalTempView) { + catalog.createTempView(name.table, aliasedPlan, overrideIfExists = replace) + } else if (viewType == GlobalTempView) { + catalog.createGlobalTempView(name.table, aliasedPlan, overrideIfExists = replace) + } else if (catalog.tableExists(name)) { + val tableMetadata = catalog.getTableMetadata(name) + if (allowExisting) { + // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view + // already exists. + } else if (tableMetadata.tableType != CatalogTableType.VIEW) { + throw new AnalysisException(s"$name is not a view") + } else if (replace) { + // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` + catalog.alterTable(prepareTable(sparkSession, aliasedPlan)) + } else { + // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already + // exists. + throw new AnalysisException( + s"View $name already exists. If you want to update the view definition, " + + "please use ALTER VIEW AS or CREATE OR REPLACE VIEW AS") + } + } else { + // Create the view if it doesn't exist. + catalog.createTable(prepareTable(sparkSession, aliasedPlan), ignoreIfExists = false) + } + Seq.empty[Row] } /** * Returns a [[CatalogTable]] that can be used to save in the catalog. This comment canonicalize * SQL based on the analyzed plan, and also creates the proper schema for the view. */ - private def prepareTable(sparkSession: SparkSession, analyzedPlan: LogicalPlan): CatalogTable = { - val aliasedPlan = if (userSpecifiedColumns.isEmpty) { - analyzedPlan - } else { - val projectList = analyzedPlan.output.zip(userSpecifiedColumns).map { - case (attr, (colName, _)) => Alias(attr, colName)() - } - sparkSession.sessionState.executePlan(Project(projectList, analyzedPlan)).analyzed - } - + private def prepareTable(sparkSession: SparkSession, aliasedPlan: LogicalPlan): CatalogTable = { val viewSQL: String = new SQLBuilder(aliasedPlan).toSQL // Validate the view SQL - make sure we can parse it and analyze it. @@ -176,19 +188,11 @@ case class CreateViewCommand( throw new RuntimeException(s"Failed to analyze the canonicalized SQL: $viewSQL", e) } - val viewSchema = if (userSpecifiedColumns.isEmpty) { - aliasedPlan.schema - } else { - StructType(aliasedPlan.schema.zip(userSpecifiedColumns).map { - case (field, (_, comment)) => comment.map(field.withComment).getOrElse(field) - }) - } - CatalogTable( identifier = name, tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, - schema = viewSchema, + schema = aliasedPlan.schema, properties = properties, viewOriginalText = originalText, viewText = Some(viewSQL), @@ -222,8 +226,8 @@ case class AlterViewAsCommand( qe.assertAnalyzed() val analyzedPlan = qe.analyzed - if (session.sessionState.catalog.isTemporaryTable(name)) { - session.sessionState.catalog.createTempView(name.table, analyzedPlan, overrideIfExists = true) + if (session.sessionState.catalog.alterTempViewDefinition(name, analyzedPlan)) { + // a local/global temp view has been altered, we are done. } else { alterPermanentView(session, analyzedPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index fa95af2648cf9..59fb48ffea598 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -40,16 +40,20 @@ case class CreateTable( override def innerChildren: Seq[QueryPlan[_]] = query.toSeq } +/** + * Create or replace a local/global temporary view with given data source. + */ case class CreateTempViewUsing( tableIdent: TableIdentifier, userSpecifiedSchema: Option[StructType], replace: Boolean, + global: Boolean, provider: String, options: Map[String, String]) extends RunnableCommand { if (tableIdent.database.isDefined) { throw new AnalysisException( - s"Temporary table '$tableIdent' should not have specified a database") + s"Temporary view '$tableIdent' should not have specified a database") } def run(sparkSession: SparkSession): Seq[Row] = { @@ -58,10 +62,16 @@ case class CreateTempViewUsing( userSpecifiedSchema = userSpecifiedSchema, className = provider, options = options) - sparkSession.sessionState.catalog.createTempView( - tableIdent.table, - Dataset.ofRows(sparkSession, LogicalRelation(dataSource.resolveRelation())).logicalPlan, - replace) + + val catalog = sparkSession.sessionState.catalog + val viewDefinition = Dataset.ofRows( + sparkSession, LogicalRelation(dataSource.resolveRelation())).logicalPlan + + if (global) { + catalog.createGlobalTempView(tableIdent.table, viewDefinition, replace) + } else { + catalog.createTempView(tableIdent.table, viewDefinition, replace) + } Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index e412e1b4b302a..c05bda3f1b526 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -94,20 +94,19 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { */ @throws[AnalysisException]("database does not exist") override def listTables(dbName: String): Dataset[Table] = { - requireDatabaseExists(dbName) val tables = sessionCatalog.listTables(dbName).map(makeTable) CatalogImpl.makeDataset(tables, sparkSession) } private def makeTable(tableIdent: TableIdentifier): Table = { val metadata = sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdent) - val database = metadata.identifier.database + val isTemp = sessionCatalog.isTemporaryTable(tableIdent) new Table( name = tableIdent.table, - database = database.orNull, + database = metadata.identifier.database.orNull, description = metadata.comment.orNull, - tableType = if (database.isEmpty) "TEMPORARY" else metadata.tableType.name, - isTemporary = database.isEmpty) + tableType = if (isTemp) "TEMPORARY" else metadata.tableType.name, + isTemporary = isTemp) } /** @@ -365,7 +364,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { } /** - * Drops the temporary view with the given view name in the catalog. + * Drops the local temporary view with the given view name in the catalog. * If the view has been cached/persisted before, it's also unpersisted. * * @param viewName the name of the view to be dropped. @@ -379,6 +378,21 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { } } + /** + * Drops the global temporary view with the given view name in the catalog. + * If the view has been cached/persisted before, it's also unpersisted. + * + * @param viewName the name of the view to be dropped. + * @group ddl_ops + * @since 2.1.0 + */ + override def dropGlobalTempView(viewName: String): Boolean = { + sparkSession.sessionState.catalog.getGlobalTempView(viewName).exists { viewDef => + sparkSession.sharedState.cacheManager.uncacheQuery(Dataset.ofRows(sparkSession, viewDef)) + sessionCatalog.dropGlobalTempView(viewName) + } + } + /** * Returns true if the table is currently cached in-memory. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 9f7d0019c6b92..8759dfe39ce1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -95,6 +95,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { */ lazy val catalog = new SessionCatalog( sparkSession.sharedState.externalCatalog, + sparkSession.sharedState.globalTempViewManager, functionResourceLoader, functionRegistry, conf, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 6387f0150631c..c555a43cd2581 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -22,11 +22,11 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.internal.config._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SQLContext} -import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} +import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, GlobalTempViewManager, InMemoryCatalog} import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.util.{MutableURLClassLoader, Utils} @@ -37,39 +37,14 @@ import org.apache.spark.util.{MutableURLClassLoader, Utils} */ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { - /** - * Class for caching query results reused in future executions. - */ - val cacheManager: CacheManager = new CacheManager - - /** - * A listener for SQL-specific [[org.apache.spark.scheduler.SparkListenerEvent]]s. - */ - val listener: SQLListener = createListenerAndUI(sparkContext) - + // Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on + // the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf. { val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") if (configFile != null) { sparkContext.hadoopConfiguration.addResource(configFile) } - } - - /** - * A catalog that interacts with external systems. - */ - lazy val externalCatalog: ExternalCatalog = - SharedState.reflect[ExternalCatalog, SparkConf, Configuration]( - SharedState.externalCatalogClassName(sparkContext.conf), - sparkContext.conf, - sparkContext.hadoopConfiguration) - - /** - * A classloader used to load all user-added jar. - */ - val jarClassLoader = new NonClosableMutableURLClassLoader( - org.apache.spark.util.Utils.getContextOrSparkClassLoader) - { // Set the Hive metastore warehouse path to the one we use val tempConf = new SQLConf sparkContext.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } @@ -93,6 +68,48 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { logInfo(s"Warehouse path is '${tempConf.warehousePath}'.") } + /** + * Class for caching query results reused in future executions. + */ + val cacheManager: CacheManager = new CacheManager + + /** + * A listener for SQL-specific [[org.apache.spark.scheduler.SparkListenerEvent]]s. + */ + val listener: SQLListener = createListenerAndUI(sparkContext) + + /** + * A catalog that interacts with external systems. + */ + val externalCatalog: ExternalCatalog = + SharedState.reflect[ExternalCatalog, SparkConf, Configuration]( + SharedState.externalCatalogClassName(sparkContext.conf), + sparkContext.conf, + sparkContext.hadoopConfiguration) + + /** + * A manager for global temporary views. + */ + val globalTempViewManager = { + // System preserved database should not exists in metastore. However it's hard to guarantee it + // for every session, because case-sensitivity differs. Here we always lowercase it to make our + // life easier. + val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE).toLowerCase + if (externalCatalog.databaseExists(globalTempDB)) { + throw new SparkException( + s"$globalTempDB is a system preserved database, please rename your existing database " + + "to resolve the name conflict, or set a different value for " + + s"${GLOBAL_TEMP_DATABASE.key}, and launch your Spark application again.") + } + new GlobalTempViewManager(globalTempDB) + } + + /** + * A classloader used to load all user-added jar. + */ + val jarClassLoader = new NonClosableMutableURLClassLoader( + org.apache.spark.util.Utils.getContextOrSparkClassLoader) + /** * Create a SQLListener then add it into SparkContext, and create a SQLTab if there is SparkUI. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 001c1a1d85313..2b35db411e2ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -88,11 +88,11 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { df.createOrReplaceTempView("listtablessuitetable") assert( sqlContext.tables().filter("tableName = 'listtablessuitetable'").collect().toSeq == - Row("listtablessuitetable", true) :: Nil) + Row("", "listtablessuitetable", true) :: Nil) assert( sqlContext.sql("SHOW tables").filter("tableName = 'listtablessuitetable'").collect().toSeq == - Row("listtablessuitetable", true) :: Nil) + Row("", "listtablessuitetable", true) :: Nil) sqlContext.sessionState.catalog.dropTable( TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true, purge = false) @@ -105,11 +105,11 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { df.createOrReplaceTempView("listtablessuitetable") assert( sqlContext.tables("default").filter("tableName = 'listtablessuitetable'").collect().toSeq == - Row("listtablessuitetable", true) :: Nil) + Row("", "listtablessuitetable", true) :: Nil) assert( sqlContext.sql("show TABLES in default").filter("tableName = 'listtablessuitetable'") - .collect().toSeq == Row("listtablessuitetable", true) :: Nil) + .collect().toSeq == Row("", "listtablessuitetable", true) :: Nil) sqlContext.sessionState.catalog.dropTable( TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true, purge = false) @@ -122,7 +122,8 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { df.createOrReplaceTempView("listtablessuitetable") val expectedSchema = StructType( - StructField("tableName", StringType, false) :: + StructField("database", StringType, false) :: + StructField("tableName", StringType, false) :: StructField("isTemporary", BooleanType, false) :: Nil) Seq(sqlContext.tables(), sqlContext.sql("SHOW TABLes")).foreach { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala new file mode 100644 index 0000000000000..391bcb8b35d02 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalog.Table +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType + +class GlobalTempViewSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + override protected def beforeAll(): Unit = { + super.beforeAll() + globalTempDB = spark.sharedState.globalTempViewManager.database + } + + private var globalTempDB: String = _ + + test("basic semantic") { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 'a'") + + // If there is no database in table name, we should try local temp view first, if not found, + // try table/view in current database, which is "default" in this case. So we expect + // NoSuchTableException here. + intercept[NoSuchTableException](spark.table("src")) + + // Use qualified name to refer to the global temp view explicitly. + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a")) + + // Table name without database will never refer to a global temp view. + intercept[NoSuchTableException](sql("DROP VIEW src")) + + sql(s"DROP VIEW $globalTempDB.src") + // The global temp view should be dropped successfully. + intercept[NoSuchTableException](spark.table(s"$globalTempDB.src")) + + // We can also use Dataset API to create global temp view + Seq(1 -> "a").toDF("i", "j").createGlobalTempView("src") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a")) + + // Use qualified name to rename a global temp view. + sql(s"ALTER VIEW $globalTempDB.src RENAME TO src2") + intercept[NoSuchTableException](spark.table(s"$globalTempDB.src")) + checkAnswer(spark.table(s"$globalTempDB.src2"), Row(1, "a")) + + // Use qualified name to alter a global temp view. + sql(s"ALTER VIEW $globalTempDB.src2 AS SELECT 2, 'b'") + checkAnswer(spark.table(s"$globalTempDB.src2"), Row(2, "b")) + + // We can also use Catalog API to drop global temp view + spark.catalog.dropGlobalTempView("src2") + intercept[NoSuchTableException](spark.table(s"$globalTempDB.src2")) + } + + test("global temp view is shared among all sessions") { + try { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 2") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, 2)) + val newSession = spark.newSession() + checkAnswer(newSession.table(s"$globalTempDB.src"), Row(1, 2)) + } finally { + spark.catalog.dropGlobalTempView("src") + } + } + + test("global temp view database should be preserved") { + val e = intercept[AnalysisException](sql(s"CREATE DATABASE $globalTempDB")) + assert(e.message.contains("system preserved database")) + + val e2 = intercept[AnalysisException](sql(s"USE $globalTempDB")) + assert(e2.message.contains("system preserved database")) + } + + test("CREATE GLOBAL TEMP VIEW USING") { + withTempPath { path => + try { + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getAbsolutePath) + sql(s"CREATE GLOBAL TEMP VIEW src USING parquet OPTIONS (PATH '${path.getAbsolutePath}')") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a")) + sql(s"INSERT INTO $globalTempDB.src SELECT 2, 'b'") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a") :: Row(2, "b") :: Nil) + } finally { + spark.catalog.dropGlobalTempView("src") + } + } + } + + test("CREATE TABLE LIKE should work for global temp view") { + try { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1 AS a, '2' AS b") + sql(s"CREATE TABLE cloned LIKE ${globalTempDB}.src") + val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier("cloned")) + assert(tableMeta.schema == new StructType().add("a", "int", false).add("b", "string", false)) + } finally { + spark.catalog.dropGlobalTempView("src") + sql("DROP TABLE default.cloned") + } + } + + test("list global temp views") { + try { + sql("CREATE GLOBAL TEMP VIEW v1 AS SELECT 3, 4") + sql("CREATE TEMP VIEW v2 AS SELECT 1, 2") + + checkAnswer(sql(s"SHOW TABLES IN $globalTempDB"), + Row(globalTempDB, "v1", true) :: + Row("", "v2", true) :: Nil) + + assert(spark.catalog.listTables(globalTempDB).collect().toSeq.map(_.name) == Seq("v1", "v2")) + } finally { + spark.catalog.dropTempView("v1") + spark.catalog.dropGlobalTempView("v2") + } + } + + test("should lookup global temp view if and only if global temp db is specified") { + try { + sql("CREATE GLOBAL TEMP VIEW same_name AS SELECT 3, 4") + sql("CREATE TEMP VIEW same_name AS SELECT 1, 2") + + checkAnswer(sql("SELECT * FROM same_name"), Row(1, 2)) + + // we never lookup global temp views if database is not specified in table name + spark.catalog.dropTempView("same_name") + intercept[AnalysisException](sql("SELECT * FROM same_name")) + + // Use qualified name to lookup a global temp view. + checkAnswer(sql(s"SELECT * FROM $globalTempDB.same_name"), Row(3, 4)) + } finally { + spark.catalog.dropTempView("same_name") + spark.catalog.dropGlobalTempView("same_name") + } + } + + test("public Catalog should recognize global temp view") { + try { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 2") + + assert(spark.catalog.tableExists(globalTempDB, "src")) + assert(spark.catalog.getTable(globalTempDB, "src").toString == new Table( + name = "src", + database = globalTempDB, + description = null, + tableType = "TEMPORARY", + isTemporary = true).toString) + } finally { + spark.catalog.dropGlobalTempView("src") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 1bcb810a1564f..19885156cc722 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -969,17 +969,17 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { """.stripMargin) checkAnswer( sql("SHOW TABLES IN default 'show1*'"), - Row("show1a", true) :: Nil) + Row("", "show1a", true) :: Nil) checkAnswer( sql("SHOW TABLES IN default 'show1*|show2*'"), - Row("show1a", true) :: - Row("show2b", true) :: Nil) + Row("", "show1a", true) :: + Row("", "show2b", true) :: Nil) checkAnswer( sql("SHOW TABLES 'show1*|show2*'"), - Row("show1a", true) :: - Row("show2b", true) :: Nil) + Row("", "show1a", true) :: + Row("", "show2b", true) :: Nil) assert( sql("SHOW TABLES").count() >= 2) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 85c509847d8ef..85ecf0ce70756 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder -import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, GlobalTempViewManager, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.rules.Rule @@ -41,6 +41,7 @@ import org.apache.spark.util.Utils private[sql] class HiveSessionCatalog( externalCatalog: HiveExternalCatalog, + globalTempViewManager: GlobalTempViewManager, sparkSession: SparkSession, functionResourceLoader: FunctionResourceLoader, functionRegistry: FunctionRegistry, @@ -48,6 +49,7 @@ private[sql] class HiveSessionCatalog( hadoopConf: Configuration) extends SessionCatalog( externalCatalog, + globalTempViewManager, functionResourceLoader, functionRegistry, conf, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index eb10c11382e83..6d4fe1a941a98 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -45,6 +45,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) override lazy val catalog = { new HiveSessionCatalog( sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog], + sparkSession.sharedState.globalTempViewManager, sparkSession, functionResourceLoader, functionRegistry, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala index 57363b7259c61..939fd71b4f1ed 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -87,11 +87,11 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac assert( hc.sql("SELECT * FROM moo_table order by name").collect().toSeq == df.collect().toSeq.sortBy(_.getString(0))) - val tables = hc.sql("SHOW TABLES IN mee_db").collect().map(_.getString(0)) + val tables = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) assert(tables.toSet == Set("moo_table", "mee_table")) hc.sql("DROP TABLE moo_table") hc.sql("DROP TABLE mee_table") - val tables2 = hc.sql("SHOW TABLES IN mee_db").collect().map(_.getString(0)) + val tables2 = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) assert(tables2.isEmpty) hc.sql("USE default") hc.sql("DROP DATABASE mee_db CASCADE") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala index 6eeb67510c735..15ba61646d03f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala @@ -58,10 +58,10 @@ class ListTablesSuite extends QueryTest with TestHiveSingleton with BeforeAndAft // We are using default DB. checkAnswer( allTables.filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) + Row("", "listtablessuitetable", true)) checkAnswer( allTables.filter("tableName = 'hivelisttablessuitetable'"), - Row("hivelisttablessuitetable", false)) + Row("default", "hivelisttablessuitetable", false)) assert(allTables.filter("tableName = 'hiveindblisttablessuitetable'").count() === 0) } } @@ -71,11 +71,11 @@ class ListTablesSuite extends QueryTest with TestHiveSingleton with BeforeAndAft case allTables => checkAnswer( allTables.filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) + Row("", "listtablessuitetable", true)) assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0) checkAnswer( allTables.filter("tableName = 'hiveindblisttablessuitetable'"), - Row("hiveindblisttablessuitetable", false)) + Row("listtablessuitedb", "hiveindblisttablessuitetable", false)) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 8ae6868c9848a..51670649ad1d4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -984,7 +984,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv checkAnswer( spark.sql("show TABLES in testdb8156").filter("tableName = 'ttt3'"), - Row("ttt3", false)) + Row("testdb8156", "ttt3", false)) spark.sql("""use default""") spark.sql("""drop database if exists testdb8156 CASCADE""") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index b2103b3bfc36c..2c772ce2155ef 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -94,15 +94,15 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto sql("CREATE TABLE show2b(c2 int)") checkAnswer( sql("SHOW TABLES IN default 'show1*'"), - Row("show1a", false) :: Nil) + Row("default", "show1a", false) :: Nil) checkAnswer( sql("SHOW TABLES IN default 'show1*|show2*'"), - Row("show1a", false) :: - Row("show2b", false) :: Nil) + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) checkAnswer( sql("SHOW TABLES 'show1*|show2*'"), - Row("show1a", false) :: - Row("show2b", false) :: Nil) + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) assert( sql("SHOW TABLES").count() >= 2) assert( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index f5c605fe5e2fa..2af935da689c9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -62,15 +62,15 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { var e = intercept[AnalysisException] { sql("CREATE OR REPLACE VIEW tab1 AS SELECT * FROM jt") }.getMessage - assert(e.contains("`default`.`tab1` is not a view")) + assert(e.contains("`tab1` is not a view")) e = intercept[AnalysisException] { sql("CREATE VIEW tab1 AS SELECT * FROM jt") }.getMessage - assert(e.contains("`default`.`tab1` is not a view")) + assert(e.contains("`tab1` is not a view")) e = intercept[AnalysisException] { sql("ALTER VIEW tab1 AS SELECT * FROM jt") }.getMessage - assert(e.contains("`default`.`tab1` is not a view")) + assert(e.contains("`tab1` is not a view")) } } From 7e16c94f18ec07e4de63e66e06ad757b9e2550b9 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Mon, 10 Oct 2016 13:49:25 +0100 Subject: [PATCH 033/177] [HOT-FIX][SQL][TESTS] Remove unused function in `SparkSqlParserSuite` ## What changes were proposed in this pull request? The function `SparkSqlParserSuite.createTempViewUsing` is not used for now and causes build failure, this PR simply removes it. ## How was this patch tested? N/A Author: jiangxingbo Closes #15418 from jiangxb1987/parserSuite. --- .../spark/sql/execution/SparkSqlParserSuite.scala | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index e0976ae95001e..679150e9ae4c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -116,16 +116,6 @@ class SparkSqlParserSuite extends PlanTest { ) } - private def createTempViewUsing( - table: String, - database: Option[String] = None, - schema: Option[StructType] = None, - replace: Boolean = true, - provider: String = "parquet", - options: Map[String, String] = Map.empty): LogicalPlan = { - CreateTempViewUsing(TableIdentifier(table, database), schema, replace, provider, options) - } - private def createTable( table: String, database: Option[String] = None, From 4bafacaa5f50a3e986c14a38bc8df9bae303f3a0 Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Mon, 10 Oct 2016 10:55:57 -0500 Subject: [PATCH 034/177] [SPARK-17417][CORE] Fix # of partitions for Reliable RDD checkpointing ## What changes were proposed in this pull request? Currently the no. of partition files are limited to 10000 files (%05d format). If there are more than 10000 part files, the logic goes for a toss while recreating the RDD as it sorts them by string. More details can be found in the JIRA desc [here](https://issues.apache.org/jira/browse/SPARK-17417). ## How was this patch tested? I tested this patch by checkpointing a RDD and then manually renaming part files to the old format and tried to access the RDD. It was successfully created from the old format. Also verified loading a sample parquet file and saving it as multiple formats - CSV, JSON, Text, Parquet, ORC and read them successfully back from the saved files. I couldn't launch the unit test from my local box, so will wait for the Jenkins output. Author: Dhruve Ashar Closes #15370 from dhruve/bug/SPARK-17417. --- .../scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index ab6554fd8a7e7..eac901d10067c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -69,10 +69,10 @@ private[spark] class ReliableCheckpointRDD[T: ClassTag]( val inputFiles = fs.listStatus(cpath) .map(_.getPath) .filter(_.getName.startsWith("part-")) - .sortBy(_.toString) + .sortBy(_.getName.stripPrefix("part-").toInt) // Fail fast if input files are invalid inputFiles.zipWithIndex.foreach { case (path, i) => - if (!path.toString.endsWith(ReliableCheckpointRDD.checkpointFileName(i))) { + if (path.getName != ReliableCheckpointRDD.checkpointFileName(i)) { throw new SparkException(s"Invalid checkpoint file: $path") } } From 689de920056ae20fe203c2b6faf5b1462e8ea73c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 10 Oct 2016 11:29:09 -0700 Subject: [PATCH 035/177] [SPARK-17830] Annotate spark.sql package with InterfaceStability ## What changes were proposed in this pull request? This patch annotates the InterfaceStability level for top level classes in o.a.spark.sql and o.a.spark.sql.util packages, to experiment with this new annotation. ## How was this patch tested? N/A Author: Reynold Xin Closes #15392 from rxin/SPARK-17830. --- .../scala/org/apache/spark/sql/Column.scala | 5 +++- .../spark/sql/DataFrameNaFunctions.scala | 3 +- .../apache/spark/sql/DataFrameReader.scala | 2 ++ .../spark/sql/DataFrameStatFunctions.scala | 3 +- .../apache/spark/sql/DataFrameWriter.scala | 3 +- .../scala/org/apache/spark/sql/Dataset.scala | 29 ++++++++++++++++--- .../org/apache/spark/sql/DatasetHolder.scala | 3 ++ .../spark/sql/ExperimentalMethods.scala | 5 ++-- .../org/apache/spark/sql/ForeachWriter.scala | 5 +++- .../spark/sql/KeyValueGroupedDataset.scala | 3 +- .../spark/sql/RelationalGroupedDataset.scala | 4 +-- .../org/apache/spark/sql/RuntimeConfig.scala | 2 ++ .../org/apache/spark/sql/SQLContext.scala | 18 +++++++++++- .../org/apache/spark/sql/SQLImplicits.scala | 2 ++ .../org/apache/spark/sql/SparkSession.scala | 23 ++++++++++++++- .../apache/spark/sql/UDFRegistration.scala | 2 ++ .../org/apache/spark/sql/functions.scala | 8 +++-- .../scala/org/apache/spark/sql/package.scala | 5 ++-- .../sql/util/QueryExecutionListener.scala | 4 ++- 19 files changed, 107 insertions(+), 22 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 63da501f18cca..d22bb17934ce7 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.language.implicitConversions -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} @@ -59,6 +59,7 @@ private[sql] object Column { * * @since 1.6.0 */ +@InterfaceStability.Stable class TypedColumn[-T, U]( expr: Expression, private[sql] val encoder: ExpressionEncoder[U]) @@ -124,6 +125,7 @@ class TypedColumn[-T, U]( * * @since 1.3.0 */ +@InterfaceStability.Stable class Column(protected[sql] val expr: Expression) extends Logging { def this(name: String) = this(name match { @@ -1185,6 +1187,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { * @since 1.3.0 */ @Experimental +@InterfaceStability.Evolving class ColumnName(name: String) extends Column(name) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index ad00966a917ad..65a9c008f9650 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -21,7 +21,7 @@ import java.{lang => jl} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -34,6 +34,7 @@ import org.apache.spark.sql.types._ * @since 1.3.1 */ @Experimental +@InterfaceStability.Evolving final class DataFrameNaFunctions private[sql](df: DataFrame) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b84fb2fb95914..b54e695db3b5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.Partition +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptions} import org.apache.spark.sql.execution.LogicalRDD @@ -38,6 +39,7 @@ import org.apache.spark.sql.types.StructType * * @since 1.4.0 */ +@InterfaceStability.Stable class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index d69be36917360..a212bb6205328 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -21,7 +21,7 @@ import java.{lang => jl, util => ju} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.stat._ import org.apache.spark.sql.types._ @@ -34,6 +34,7 @@ import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch} * @since 1.4.0 */ @Experimental +@InterfaceStability.Evolving final class DataFrameStatFunctions private[sql](df: DataFrame) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 7374a8e045035..35ef050dcb169 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -21,12 +21,12 @@ import java.util.Properties import scala.collection.JavaConverters._ +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.execution.datasources.{CaseInsensitiveMap, CreateTable, DataSource, HadoopFsRelation} -import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} import org.apache.spark.sql.types.StructType /** @@ -35,6 +35,7 @@ import org.apache.spark.sql.types.StructType * * @since 1.4.0 */ +@InterfaceStability.Stable final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private val df = ds.toDF() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 4b52508740bfb..30349ba3cb452 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.commons.lang3.StringUtils -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.function._ import org.apache.spark.api.python.{PythonRDD, SerDeUtil} @@ -149,9 +149,10 @@ private[sql] object Dataset { * * @since 1.6.0 */ +@InterfaceStability.Stable class Dataset[T] private[sql]( @transient val sparkSession: SparkSession, - @DeveloperApi @transient val queryExecution: QueryExecution, + @DeveloperApi @InterfaceStability.Unstable @transient val queryExecution: QueryExecution, encoder: Encoder[T]) extends Serializable { @@ -369,6 +370,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def as[U : Encoder]: Dataset[U] = Dataset[U](sparkSession, logicalPlan) /** @@ -477,6 +479,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def isStreaming: Boolean = logicalPlan.isStreaming /** @@ -798,6 +801,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def joinWith[U](other: Dataset[U], condition: Column, joinType: String): Dataset[(T, U)] = { // Creates a Join node and resolve it first, to get join condition resolved, self-join resolved, // etc. @@ -869,6 +873,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def joinWith[U](other: Dataset[U], condition: Column): Dataset[(T, U)] = { joinWith(other, condition, "inner") } @@ -1071,6 +1076,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def select[U1](c1: TypedColumn[T, U1]): Dataset[U1] = { implicit val encoder = c1.encoder val project = Project(c1.withInputType(exprEnc, logicalPlan.output).named :: Nil, @@ -1105,6 +1111,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def select[U1, U2](c1: TypedColumn[T, U1], c2: TypedColumn[T, U2]): Dataset[(U1, U2)] = selectUntyped(c1, c2).asInstanceOf[Dataset[(U1, U2)]] @@ -1116,6 +1123,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def select[U1, U2, U3]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1130,6 +1138,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def select[U1, U2, U3, U4]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1145,6 +1154,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def select[U1, U2, U3, U4, U5]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1315,6 +1325,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def reduce(func: (T, T) => T): T = rdd.reduce(func) /** @@ -1327,6 +1338,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def reduce(func: ReduceFunction[T]): T = reduce(func.call(_, _)) /** @@ -1338,6 +1350,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def groupByKey[K: Encoder](func: T => K): KeyValueGroupedDataset[K, T] = { val inputPlan = logicalPlan val withGroupingKey = AppendColumns(func, inputPlan) @@ -1360,6 +1373,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def groupByKey[K](func: MapFunction[T, K], encoder: Encoder[K]): KeyValueGroupedDataset[K, T] = groupByKey(func.call(_))(encoder) @@ -2028,6 +2042,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def filter(func: T => Boolean): Dataset[T] = { withTypedPlan(TypedFilter(func, logicalPlan)) } @@ -2041,6 +2056,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def filter(func: FilterFunction[T]): Dataset[T] = { withTypedPlan(TypedFilter(func, logicalPlan)) } @@ -2054,6 +2070,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def map[U : Encoder](func: T => U): Dataset[U] = withTypedPlan { MapElements[T, U](func, logicalPlan) } @@ -2067,6 +2084,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def map[U](func: MapFunction[T, U], encoder: Encoder[U]): Dataset[U] = { implicit val uEnc = encoder withTypedPlan(MapElements[T, U](func, logicalPlan)) @@ -2081,6 +2099,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def mapPartitions[U : Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { new Dataset[U]( sparkSession, @@ -2097,6 +2116,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def mapPartitions[U](f: MapPartitionsFunction[T, U], encoder: Encoder[U]): Dataset[U] = { val func: (Iterator[T]) => Iterator[U] = x => f.call(x.asJava).asScala mapPartitions(func)(encoder) @@ -2127,6 +2147,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def flatMap[U : Encoder](func: T => TraversableOnce[U]): Dataset[U] = mapPartitions(_.flatMap(func)) @@ -2140,6 +2161,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental + @InterfaceStability.Evolving def flatMap[U](f: FlatMapFunction[T, U], encoder: Encoder[U]): Dataset[U] = { val func: (T) => Iterator[U] = x => f.call(x).asScala flatMap(func)(encoder) @@ -2505,13 +2527,11 @@ class Dataset[T] private[sql]( } /** - * :: Experimental :: * Interface for saving the content of the non-streaming Dataset out into external storage. * * @group basic * @since 1.6.0 */ - @Experimental def write: DataFrameWriter[T] = { if (isStreaming) { logicalPlan.failAnalysis( @@ -2528,6 +2548,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def writeStream: DataStreamWriter[T] = { if (!isStreaming) { logicalPlan.failAnalysis( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala b/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala index 47b81c17a31dc..18bccee98f610 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.spark.annotation.InterfaceStability + /** * A container for a [[Dataset]], used for implicit conversions in Scala. * @@ -27,6 +29,7 @@ package org.apache.spark.sql * * @since 1.6.0 */ +@InterfaceStability.Stable case class DatasetHolder[T] private[sql](private val ds: Dataset[T]) { // This is declared with parentheses to prevent the Scala compiler from treating diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index a435734b0caef..1e8ba51e59e33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.rules.Rule * @since 1.3.0 */ @Experimental +@InterfaceStability.Unstable class ExperimentalMethods private[sql]() { /** @@ -41,10 +42,8 @@ class ExperimentalMethods private[sql]() { * * @since 1.3.0 */ - @Experimental @volatile var extraStrategies: Seq[Strategy] = Nil - @Experimental @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala index f56b25b5576f1..1163035e315fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.streaming.StreamingQuery /** @@ -68,8 +68,11 @@ import org.apache.spark.sql.streaming.StreamingQuery * @since 2.0.0 */ @Experimental +@InterfaceStability.Evolving abstract class ForeachWriter[T] extends Serializable { + // TODO: Move this to org.apache.spark.sql.util or consolidate this with batch API. + /** * Called when starting to process one partition of new data in the executor. The `version` is * for data deduplication when there are failures. When recovering from a failure, some data may diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index cea16fba76e47..828eb94efe598 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.api.java.function._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateStruct} @@ -36,6 +36,7 @@ import org.apache.spark.sql.expressions.ReduceAggregator * @since 2.0.0 */ @Experimental +@InterfaceStability.Evolving class KeyValueGroupedDataset[K, V] private[sql]( kEncoder: Encoder[K], vEncoder: Encoder[V], 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 6c3fe07709fa3..f019d1e9daceb 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 @@ -20,10 +20,9 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.language.implicitConversions +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, FlatMapGroupsInR, Pivot} @@ -43,6 +42,7 @@ import org.apache.spark.sql.types.StructType * * @since 2.0.0 */ +@InterfaceStability.Stable class RelationalGroupedDataset protected[sql]( df: DataFrame, groupingExprs: Seq[Expression], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 7e07e0cb84a87..c2baa74ed7d2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} import org.apache.spark.sql.internal.SQLConf @@ -28,6 +29,7 @@ import org.apache.spark.sql.internal.SQLConf * * @since 2.0.0 */ +@InterfaceStability.Stable class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 2edf2e1972053..3c5cf037c578d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,7 +24,7 @@ import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry @@ -55,6 +55,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager * @groupname Ungrouped Support functions for language integrated queries * @since 1.0.0 */ +@InterfaceStability.Stable class SQLContext private[sql](val sparkSession: SparkSession) extends Logging with Serializable { @@ -95,6 +96,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * that listen for execution metrics. */ @Experimental + @InterfaceStability.Evolving def listenerManager: ExecutionListenerManager = sparkSession.listenerManager /** @@ -166,6 +168,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) */ @Experimental @transient + @InterfaceStability.Unstable def experimental: ExperimentalMethods = sparkSession.experimental /** @@ -261,6 +264,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental + @InterfaceStability.Evolving object implicits extends SQLImplicits with Serializable { protected override def _sqlContext: SQLContext = self } @@ -274,6 +278,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental + @InterfaceStability.Evolving def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { sparkSession.createDataFrame(rdd) } @@ -286,6 +291,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental + @InterfaceStability.Evolving def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { sparkSession.createDataFrame(data) } @@ -333,6 +339,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @DeveloperApi + @InterfaceStability.Evolving def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rowRDD, schema) } @@ -376,6 +383,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataset */ @Experimental + @InterfaceStability.Evolving def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { sparkSession.createDataset(data) } @@ -413,6 +421,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataset */ @Experimental + @InterfaceStability.Evolving def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { sparkSession.createDataset(data) } @@ -436,6 +445,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @DeveloperApi + @InterfaceStability.Evolving def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rowRDD, schema) } @@ -450,6 +460,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.6.0 */ @DeveloperApi + @InterfaceStability.Evolving def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rows, schema) } @@ -515,6 +526,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def readStream: DataStreamReader = sparkSession.readStream @@ -632,6 +644,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental + @InterfaceStability.Evolving def range(end: Long): DataFrame = sparkSession.range(end).toDF() /** @@ -643,6 +656,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental + @InterfaceStability.Evolving def range(start: Long, end: Long): DataFrame = sparkSession.range(start, end).toDF() /** @@ -654,6 +668,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental + @InterfaceStability.Evolving def range(start: Long, end: Long, step: Long): DataFrame = { sparkSession.range(start, end, step).toDF() } @@ -668,6 +683,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental + @InterfaceStability.Evolving def range(start: Long, end: Long, step: Long, numPartitions: Int): DataFrame = { sparkSession.range(start, end, step, numPartitions).toDF() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 440952572d8c4..73d16d8a10fd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -28,6 +29,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder * * @since 1.6.0 */ +@InterfaceStability.Evolving abstract class SQLImplicits { protected def _sqlContext: SQLContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 6d7ac0f6c1bb2..d26eea507284c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -26,7 +26,7 @@ import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION @@ -68,6 +68,7 @@ import org.apache.spark.util.Utils * .getOrCreate() * }}} */ +@InterfaceStability.Stable class SparkSession private( @transient val sparkContext: SparkContext, @transient private val existingSharedState: Option[SharedState]) @@ -137,6 +138,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def listenerManager: ExecutionListenerManager = sessionState.listenerManager /** @@ -147,6 +149,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Unstable def experimental: ExperimentalMethods = sessionState.experimentalMethods /** @@ -190,6 +193,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Unstable def streams: StreamingQueryManager = sessionState.streamingQueryManager /** @@ -229,6 +233,7 @@ class SparkSession private( * @return 2.0.0 */ @Experimental + @InterfaceStability.Evolving def emptyDataset[T: Encoder]: Dataset[T] = { val encoder = implicitly[Encoder[T]] new Dataset(self, LocalRelation(encoder.schema.toAttributes), encoder) @@ -241,6 +246,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { SparkSession.setActiveSession(this) val encoder = Encoders.product[A] @@ -254,6 +260,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { SparkSession.setActiveSession(this) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] @@ -293,6 +300,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi + @InterfaceStability.Evolving def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD, schema, needsConversion = true) } @@ -306,6 +314,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi + @InterfaceStability.Evolving def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD.rdd, schema) } @@ -319,6 +328,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi + @InterfaceStability.Evolving def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { Dataset.ofRows(self, LocalRelation.fromExternalRows(schema.toAttributes, rows.asScala)) } @@ -410,6 +420,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -428,6 +439,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { Dataset[T](self, ExternalRDD(data, self)) } @@ -449,6 +461,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } @@ -461,6 +474,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def range(end: Long): Dataset[java.lang.Long] = range(0, end) /** @@ -471,6 +485,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def range(start: Long, end: Long): Dataset[java.lang.Long] = { range(start, end, step = 1, numPartitions = sparkContext.defaultParallelism) } @@ -483,6 +498,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def range(start: Long, end: Long, step: Long): Dataset[java.lang.Long] = { range(start, end, step, numPartitions = sparkContext.defaultParallelism) } @@ -496,6 +512,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[java.lang.Long] = { new Dataset(self, Range(start, end, step, numPartitions), Encoders.LONG) } @@ -596,6 +613,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def readStream: DataStreamReader = new DataStreamReader(self) @@ -614,6 +632,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving object implicits extends SQLImplicits with Serializable { protected override def _sqlContext: SQLContext = SparkSession.this.sqlContext } @@ -670,11 +689,13 @@ class SparkSession private( } +@InterfaceStability.Stable object SparkSession { /** * Builder for [[SparkSession]]. */ + @InterfaceStability.Stable class Builder extends Logging { private[this] val options = new scala.collection.mutable.HashMap[String, String] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index b006236481a29..617a14793697b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.reflect.runtime.universe.TypeTag import scala.util.Try +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.internal.Logging import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry @@ -36,6 +37,7 @@ import org.apache.spark.sql.types.DataType * * @since 1.3.0 */ +@InterfaceStability.Stable class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends Logging { protected[sql] def registerPython(name: String, udf: UserDefinedPythonFunction): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 40f82d895d43b..de4943152720c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -22,7 +22,7 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.{typeTag, TypeTag} import scala.util.Try -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedFunction} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -38,7 +38,7 @@ import org.apache.spark.util.Utils /** * :: Experimental :: - * Functions available for [[DataFrame]]. + * Functions available for DataFrame operations. * * @groupname udf_funcs UDF functions * @groupname agg_funcs Aggregate functions @@ -54,6 +54,7 @@ import org.apache.spark.util.Utils * @since 1.3.0 */ @Experimental +@InterfaceStability.Evolving // scalastyle:off object functions { // scalastyle:on @@ -2730,6 +2731,7 @@ object functions { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def window( timeColumn: Column, windowDuration: String, @@ -2783,6 +2785,7 @@ object functions { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { window(timeColumn, windowDuration, slideDuration, "0 second") } @@ -2821,6 +2824,7 @@ object functions { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def window(timeColumn: Column, windowDuration: String): Column = { window(timeColumn, windowDuration, windowDuration, "0 second") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 28d8bc3de68b8..161e0102f0b43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -17,8 +17,8 @@ package org.apache.spark -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy} +import org.apache.spark.annotation.{DeveloperApi, InterfaceStability} +import org.apache.spark.sql.execution.SparkStrategy /** * Allows the execution of relational queries, including those expressed in SQL using Spark. @@ -40,6 +40,7 @@ package object sql { * [[org.apache.spark.sql.sources]] */ @DeveloperApi + @InterfaceStability.Unstable type Strategy = SparkStrategy type DataFrame = Dataset[Row] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index 3cae5355eecc6..5e93fc469a41f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -22,7 +22,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.mutable.ListBuffer import scala.util.control.NonFatal -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.QueryExecution @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.QueryExecution * multiple different threads. */ @Experimental +@InterfaceStability.Evolving trait QueryExecutionListener { /** @@ -68,6 +69,7 @@ trait QueryExecutionListener { * Manager for [[QueryExecutionListener]]. See [[org.apache.spark.sql.SQLContext.listenerManager]]. */ @Experimental +@InterfaceStability.Evolving class ExecutionListenerManager private[sql] () extends Logging { /** From 3f8a0222e2fa9351a3de09bd2636b000a88da67a Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Mon, 10 Oct 2016 23:16:40 +0200 Subject: [PATCH 036/177] [SPARK-17828][DOCS] Remove unused generate-changelist.py ## What changes were proposed in this pull request? We can remove this file based on discussion at https://issues.apache.org/jira/browse/SPARK-17828 it's evident this file has been redundant for a while, JIRA release notes serves this purpose for us already. For ease of future reference you can find detailed release notes at, for example: http://spark.apache.org/downloads.html -> http://spark.apache.org/releases/spark-release-2-0-1.html -> "Detailed changes" which links to https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315420&version=12336857 ## How was this patch tested? Searched the codebase and saw nothing referencing this, hasn't been used in a while (probably manually invoked a long time ago) Author: Adam Roberts Closes #15419 from a-roberts/patch-7. --- dev/create-release/generate-changelist.py | 148 ---------------------- 1 file changed, 148 deletions(-) delete mode 100755 dev/create-release/generate-changelist.py diff --git a/dev/create-release/generate-changelist.py b/dev/create-release/generate-changelist.py deleted file mode 100755 index 2e1a35a629342..0000000000000 --- a/dev/create-release/generate-changelist.py +++ /dev/null @@ -1,148 +0,0 @@ -#!/usr/bin/python - -# -# 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. -# -# Creates CHANGES.txt from git history. -# -# Usage: -# First set the new release version and old CHANGES.txt version in this file. -# Make sure you have SPARK_HOME set. -# $ python generate-changelist.py - - -import os -import sys -import subprocess -import time -import traceback - -SPARK_HOME = os.environ["SPARK_HOME"] -NEW_RELEASE_VERSION = "1.0.0" -PREV_RELEASE_GIT_TAG = "v0.9.1" - -CHANGELIST = "CHANGES.txt" -OLD_CHANGELIST = "%s.old" % (CHANGELIST) -NEW_CHANGELIST = "%s.new" % (CHANGELIST) -TMP_CHANGELIST = "%s.tmp" % (CHANGELIST) - -# date before first PR in TLP Spark repo -SPARK_REPO_CHANGE_DATE1 = time.strptime("2014-02-26", "%Y-%m-%d") -# date after last PR in incubator Spark repo -SPARK_REPO_CHANGE_DATE2 = time.strptime("2014-03-01", "%Y-%m-%d") -# Threshold PR number that differentiates PRs to TLP -# and incubator repos -SPARK_REPO_PR_NUM_THRESH = 200 - -LOG_FILE_NAME = "changes_%s" % time.strftime("%h_%m_%Y_%I_%M_%S") -LOG_FILE = open(LOG_FILE_NAME, 'w') - - -def run_cmd(cmd): - try: - print >> LOG_FILE, "Running command: %s" % cmd - output = subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) - print >> LOG_FILE, "Output: %s" % output - return output - except: - traceback.print_exc() - cleanup() - sys.exit(1) - - -def append_to_changelist(string): - with open(TMP_CHANGELIST, "a") as f: - print >> f, string - - -def cleanup(ask=True): - if ask is True: - print "OK to delete temporary and log files? (y/N): " - response = raw_input() - if ask is False or (ask is True and response == "y"): - if os.path.isfile(TMP_CHANGELIST): - os.remove(TMP_CHANGELIST) - if os.path.isfile(OLD_CHANGELIST): - os.remove(OLD_CHANGELIST) - LOG_FILE.close() - os.remove(LOG_FILE_NAME) - - -print "Generating new %s for Spark release %s" % (CHANGELIST, NEW_RELEASE_VERSION) -os.chdir(SPARK_HOME) -if os.path.isfile(TMP_CHANGELIST): - os.remove(TMP_CHANGELIST) -if os.path.isfile(OLD_CHANGELIST): - os.remove(OLD_CHANGELIST) - -append_to_changelist("Spark Change Log") -append_to_changelist("----------------") -append_to_changelist("") -append_to_changelist("Release %s" % NEW_RELEASE_VERSION) -append_to_changelist("") - -print "Getting commits between tag %s and HEAD" % PREV_RELEASE_GIT_TAG -hashes = run_cmd("git log %s..HEAD --pretty='%%h'" % PREV_RELEASE_GIT_TAG).split() - -print "Getting details of %s commits" % len(hashes) -for h in hashes: - date = run_cmd("git log %s -1 --pretty='%%ad' --date=iso | head -1" % h).strip() - subject = run_cmd("git log %s -1 --pretty='%%s' | head -1" % h).strip() - body = run_cmd("git log %s -1 --pretty='%%b'" % h) - committer = run_cmd("git log %s -1 --pretty='%%cn <%%ce>' | head -1" % h).strip() - body_lines = body.split("\n") - - if "Merge pull" in subject: - # Parse old format commit message - append_to_changelist(" %s %s" % (h, date)) - append_to_changelist(" %s" % subject) - append_to_changelist(" [%s]" % body_lines[0]) - append_to_changelist("") - - elif "maven-release" not in subject: - # Parse new format commit message - # Get authors from commit message, committer otherwise - authors = [committer] - if "Author:" in body: - authors = [line.split(":")[1].strip() for line in body_lines if "Author:" in line] - - # Generate GitHub PR URL for easy access if possible - github_url = "" - if "Closes #" in body: - pr_num = [line.split()[1].lstrip("#") for line in body_lines if "Closes #" in line][0] - github_url = "github.com/apache/spark/pull/%s" % pr_num - day = time.strptime(date.split()[0], "%Y-%m-%d") - if (day < SPARK_REPO_CHANGE_DATE1 or - (day < SPARK_REPO_CHANGE_DATE2 and pr_num < SPARK_REPO_PR_NUM_THRESH)): - github_url = "github.com/apache/incubator-spark/pull/%s" % pr_num - - append_to_changelist(" %s" % subject) - append_to_changelist(" %s" % ', '.join(authors)) - # for author in authors: - # append_to_changelist(" %s" % author) - append_to_changelist(" %s" % date) - if len(github_url) > 0: - append_to_changelist(" Commit: %s, %s" % (h, github_url)) - else: - append_to_changelist(" Commit: %s" % h) - append_to_changelist("") - -# Append old change list -print "Appending changelist from tag %s" % PREV_RELEASE_GIT_TAG -run_cmd("git show %s:%s | tail -n +3 >> %s" % (PREV_RELEASE_GIT_TAG, CHANGELIST, TMP_CHANGELIST)) -run_cmd("cp %s %s" % (TMP_CHANGELIST, NEW_CHANGELIST)) -print "New change list generated as %s" % NEW_CHANGELIST -cleanup(False) From 29f186bfdf929b1e8ffd8e33ee37b76d5dc5af53 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Mon, 10 Oct 2016 23:20:15 +0200 Subject: [PATCH 037/177] [SPARK-14082][MESOS] Enable GPU support with Mesos ## What changes were proposed in this pull request? Enable GPU resources to be used when running coarse grain mode with Mesos. ## How was this patch tested? Manual test with GPU. Author: Timothy Chen Closes #14644 from tnachen/gpu_mesos. --- docs/running-on-mesos.md | 9 +++ .../MesosCoarseGrainedSchedulerBackend.scala | 30 +++++++-- .../cluster/mesos/MesosSchedulerUtils.scala | 5 ++ ...osCoarseGrainedSchedulerBackendSuite.scala | 61 ++++++++++++++----- .../spark/scheduler/cluster/mesos/Utils.scala | 14 +++-- 5 files changed, 96 insertions(+), 23 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 173961deaadcb..77b06fcf33740 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -498,6 +498,15 @@ See the [configuration page](configuration.html) for information on Spark config in the history server. + + spark.mesos.gpus.max + 0 + + Set the maximum number GPU resources to acquire for this job. Note that executors will still launch when no GPU resources are found + since this configuration is just a upper limit and not a guaranteed amount. + + + diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index a64b5768c57b2..e67bf3e328f94 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -59,6 +59,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt + val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) + private[this] val shutdownTimeoutMS = conf.getTimeAsMs("spark.mesos.coarse.shutdownTimeout", "10s") .ensuring(_ >= 0, "spark.mesos.coarse.shutdownTimeout must be >= 0") @@ -72,7 +74,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // Cores we have acquired with each Mesos task ID val coresByTaskId = new mutable.HashMap[String, Int] + val gpusByTaskId = new mutable.HashMap[String, Int] var totalCoresAcquired = 0 + var totalGpusAcquired = 0 // SlaveID -> Slave // This map accumulates entries for the duration of the job. Slaves are never deleted, because @@ -396,6 +400,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( launchTasks = true val taskId = newMesosTaskId() val offerCPUs = getResource(resources, "cpus").toInt + val taskGPUs = Math.min( + Math.max(0, maxGpus - totalGpusAcquired), getResource(resources, "gpus").toInt) val taskCPUs = executorCores(offerCPUs) val taskMemory = executorMemory(sc) @@ -403,7 +409,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId) val (resourcesLeft, resourcesToUse) = - partitionTaskResources(resources, taskCPUs, taskMemory) + partitionTaskResources(resources, taskCPUs, taskMemory, taskGPUs) val taskBuilder = MesosTaskInfo.newBuilder() .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) @@ -425,6 +431,10 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( remainingResources(offerId) = resourcesLeft.asJava totalCoresAcquired += taskCPUs coresByTaskId(taskId) = taskCPUs + if (taskGPUs > 0) { + totalGpusAcquired += taskGPUs + gpusByTaskId(taskId) = taskGPUs + } } } } @@ -432,21 +442,28 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } /** Extracts task needed resources from a list of available resources. */ - private def partitionTaskResources(resources: JList[Resource], taskCPUs: Int, taskMemory: Int) + private def partitionTaskResources( + resources: JList[Resource], + taskCPUs: Int, + taskMemory: Int, + taskGPUs: Int) : (List[Resource], List[Resource]) = { // partition cpus & mem val (afterCPUResources, cpuResourcesToUse) = partitionResources(resources, "cpus", taskCPUs) val (afterMemResources, memResourcesToUse) = partitionResources(afterCPUResources.asJava, "mem", taskMemory) + val (afterGPUResources, gpuResourcesToUse) = + partitionResources(afterMemResources.asJava, "gpus", taskGPUs) // If user specifies port numbers in SparkConfig then consecutive tasks will not be launched // on the same host. This essentially means one executor per host. // TODO: handle network isolator case val (nonPortResources, portResourcesToUse) = - partitionPortResources(nonZeroPortValuesFromConfig(sc.conf), afterMemResources) + partitionPortResources(nonZeroPortValuesFromConfig(sc.conf), afterGPUResources) - (nonPortResources, cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse) + (nonPortResources, + cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse ++ gpuResourcesToUse) } private def canLaunchTask(slaveId: String, resources: JList[Resource]): Boolean = { @@ -513,6 +530,11 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( totalCoresAcquired -= cores coresByTaskId -= taskId } + // Also remove the gpus we have remembered for this task, if it's in the hashmap + for (gpus <- gpusByTaskId.get(taskId)) { + totalGpusAcquired -= gpus + gpusByTaskId -= taskId + } // If it was a failure, mark the slave as failed for blacklisting purposes if (TaskState.isFailed(state)) { slave.taskFailures += 1 diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 2963d161d6700..73cc241239c4c 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -27,6 +27,7 @@ import scala.util.control.NonFatal import com.google.common.base.Splitter import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDriver} import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} +import org.apache.mesos.Protos.FrameworkInfo.Capability import org.apache.mesos.protobuf.{ByteString, GeneratedMessage} import org.apache.spark.{SparkConf, SparkContext, SparkException} @@ -93,6 +94,10 @@ trait MesosSchedulerUtils extends Logging { conf.getOption("spark.mesos.role").foreach { role => fwInfoBuilder.setRole(role) } + val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) + if (maxGpus > 0) { + fwInfoBuilder.addCapabilities(Capability.newBuilder().setType(Capability.Type.GPU_RESOURCES)) + } if (credBuilder.hasPrincipal) { new MesosSchedulerDriver( scheduler, fwInfoBuilder.build(), masterUrl, credBuilder.build()) diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index c3ab488e2aa69..75ba02e470e27 100644 --- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -67,7 +67,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val minMem = backend.executorMemory(sc) val minCpu = 4 - val offers = List((minMem, minCpu)) + val offers = List(Resources(minMem, minCpu)) // launches a task on a valid offer offerResources(offers) @@ -95,8 +95,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite // launches a task on a valid offer val minMem = backend.executorMemory(sc) + 1024 val minCpu = 4 - val offer1 = (minMem, minCpu) - val offer2 = (minMem, 1) + val offer1 = Resources(minMem, minCpu) + val offer2 = Resources(minMem, 1) offerResources(List(offer1, offer2)) verifyTaskLaunched(driver, "o1") @@ -115,7 +115,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite setBackend(Map("spark.executor.cores" -> executorCores.toString)) val executorMemory = backend.executorMemory(sc) - val offers = List((executorMemory * 2, executorCores + 1)) + val offers = List(Resources(executorMemory * 2, executorCores + 1)) offerResources(offers) val taskInfos = verifyTaskLaunched(driver, "o1") @@ -130,7 +130,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val executorMemory = backend.executorMemory(sc) val offerCores = 10 - offerResources(List((executorMemory * 2, offerCores))) + offerResources(List(Resources(executorMemory * 2, offerCores))) val taskInfos = verifyTaskLaunched(driver, "o1") assert(taskInfos.length == 1) @@ -144,7 +144,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite setBackend(Map("spark.cores.max" -> maxCores.toString)) val executorMemory = backend.executorMemory(sc) - offerResources(List((executorMemory, maxCores + 1))) + offerResources(List(Resources(executorMemory, maxCores + 1))) val taskInfos = verifyTaskLaunched(driver, "o1") assert(taskInfos.length == 1) @@ -153,9 +153,38 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite assert(cpus == maxCores) } + test("mesos does not acquire gpus if not specified") { + setBackend() + + val executorMemory = backend.executorMemory(sc) + offerResources(List(Resources(executorMemory, 1, 1))) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 1) + + val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") + assert(gpus == 0.0) + } + + + test("mesos does not acquire more than spark.mesos.gpus.max") { + val maxGpus = 5 + setBackend(Map("spark.mesos.gpus.max" -> maxGpus.toString)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List(Resources(executorMemory, 1, maxGpus + 1))) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 1) + + val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") + assert(gpus == maxGpus) + } + + test("mesos declines offers that violate attribute constraints") { setBackend(Map("spark.mesos.constraints" -> "x:true")) - offerResources(List((backend.executorMemory(sc), 4))) + offerResources(List(Resources(backend.executorMemory(sc), 4))) verifyDeclinedOffer(driver, createOfferId("o1"), true) } @@ -165,8 +194,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val executorMemory = backend.executorMemory(sc) offerResources(List( - (executorMemory, maxCores + 1), - (executorMemory, maxCores + 1))) + Resources(executorMemory, maxCores + 1), + Resources(executorMemory, maxCores + 1))) verifyTaskLaunched(driver, "o1") verifyDeclinedOffer(driver, createOfferId("o2"), true) @@ -180,8 +209,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val executorMemory = backend.executorMemory(sc) offerResources(List( - (executorMemory * 2, executorCores * 2), - (executorMemory * 2, executorCores * 2))) + Resources(executorMemory * 2, executorCores * 2), + Resources(executorMemory * 2, executorCores * 2))) verifyTaskLaunched(driver, "o1") verifyTaskLaunched(driver, "o2") @@ -193,7 +222,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite // offer with room for two executors val executorMemory = backend.executorMemory(sc) - offerResources(List((executorMemory * 2, executorCores * 2))) + offerResources(List(Resources(executorMemory * 2, executorCores * 2))) // verify two executors were started on a single offer val taskInfos = verifyTaskLaunched(driver, "o1") @@ -397,7 +426,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite setBackend() // launches a task on a valid offer - val offers = List((backend.executorMemory(sc), 1)) + val offers = List(Resources(backend.executorMemory(sc), 1)) offerResources(offers) verifyTaskLaunched(driver, "o1") @@ -434,6 +463,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite assert(launchedTasks.head.getCommand.getUrisList.asScala(0).getValue == url) } + private case class Resources(mem: Int, cpus: Int, gpus: Int = 0) + private def verifyDeclinedOffer(driver: SchedulerDriver, offerId: OfferID, filter: Boolean = false): Unit = { @@ -444,9 +475,9 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } } - private def offerResources(offers: List[(Int, Int)], startId: Int = 1): Unit = { + private def offerResources(offers: List[Resources], startId: Int = 1): Unit = { val mesosOffers = offers.zipWithIndex.map {case (offer, i) => - createOffer(s"o${i + startId}", s"s${i + startId}", offer._1, offer._2)} + createOffer(s"o${i + startId}", s"s${i + startId}", offer.mem, offer.cpus, None, offer.gpus)} backend.resourceOffers(driver, mesosOffers.asJava) } diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala index fa9406f5f0553..7ebb294aa9080 100644 --- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala +++ b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala @@ -32,8 +32,9 @@ object Utils { offerId: String, slaveId: String, mem: Int, - cpu: Int, - ports: Option[(Long, Long)] = None): Offer = { + cpus: Int, + ports: Option[(Long, Long)] = None, + gpus: Int = 0): Offer = { val builder = Offer.newBuilder() builder.addResourcesBuilder() .setName("mem") @@ -42,7 +43,7 @@ object Utils { builder.addResourcesBuilder() .setName("cpus") .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(cpu)) + .setScalar(Scalar.newBuilder().setValue(cpus)) ports.foreach { resourcePorts => builder.addResourcesBuilder() .setName("ports") @@ -50,6 +51,12 @@ object Utils { .setRanges(Ranges.newBuilder().addRange(MesosRange.newBuilder() .setBegin(resourcePorts._1).setEnd(resourcePorts._2).build())) } + if (gpus > 0) { + builder.addResourcesBuilder() + .setName("gpus") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(gpus)) + } builder.setId(createOfferId(offerId)) .setFrameworkId(FrameworkID.newBuilder() .setValue("f1")) @@ -82,4 +89,3 @@ object Utils { TaskID.newBuilder().setValue(taskId).build() } } - From 03c40202f36ea9fc93071b79fed21ed3f2190ba1 Mon Sep 17 00:00:00 2001 From: sethah Date: Mon, 10 Oct 2016 17:04:11 -0700 Subject: [PATCH 038/177] [SPARK-14610][ML] Remove superfluous split for continuous features in decision tree training ## What changes were proposed in this pull request? A nonsensical split is produced from method `findSplitsForContinuousFeature` for decision trees. This PR removes the superfluous split and updates unit tests accordingly. Additionally, an assertion to check that the number of found splits is `> 0` is removed, and instead features with zero possible splits are ignored. ## How was this patch tested? A unit test was added to check that finding splits for a constant feature produces an empty array. Author: sethah Closes #12374 from sethah/SPARK-14610. --- .../spark/ml/tree/impl/RandomForest.scala | 31 +++++++------ .../ml/tree/impl/RandomForestSuite.scala | 44 ++++++++++++++++--- 2 files changed, 52 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 0b7ad92b3cf30..b504f411d256d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -705,14 +705,17 @@ private[spark] object RandomForest extends Logging { node.stats } + val validFeatureSplits = + Range(0, binAggregates.metadata.numFeaturesPerNode).view.map { featureIndexIdx => + featuresForNode.map(features => (featureIndexIdx, features(featureIndexIdx))) + .getOrElse((featureIndexIdx, featureIndexIdx)) + }.withFilter { case (_, featureIndex) => + binAggregates.metadata.numSplits(featureIndex) != 0 + } + // For each (feature, split), calculate the gain, and select the best (feature, split). val (bestSplit, bestSplitStats) = - Range(0, binAggregates.metadata.numFeaturesPerNode).map { featureIndexIdx => - val featureIndex = if (featuresForNode.nonEmpty) { - featuresForNode.get.apply(featureIndexIdx) - } else { - featureIndexIdx - } + validFeatureSplits.map { case (featureIndexIdx, featureIndex) => val numSplits = binAggregates.metadata.numSplits(featureIndex) if (binAggregates.metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. @@ -966,7 +969,7 @@ private[spark] object RandomForest extends Logging { * NOTE: `metadata.numbins` will be changed accordingly * if there are not enough splits to be found * @param featureIndex feature index to find splits - * @return array of splits + * @return array of split thresholds */ private[tree] def findSplitsForContinuousFeature( featureSamples: Iterable[Double], @@ -975,7 +978,9 @@ private[spark] object RandomForest extends Logging { require(metadata.isContinuous(featureIndex), "findSplitsForContinuousFeature can only be used to find splits for a continuous feature.") - val splits = { + val splits = if (featureSamples.isEmpty) { + Array.empty[Double] + } else { val numSplits = metadata.numSplits(featureIndex) // get count for each distinct value @@ -987,9 +992,9 @@ private[spark] object RandomForest extends Logging { val valueCounts = valueCountMap.toSeq.sortBy(_._1).toArray // if possible splits is not enough or just enough, just return all possible splits - val possibleSplits = valueCounts.length + val possibleSplits = valueCounts.length - 1 if (possibleSplits <= numSplits) { - valueCounts.map(_._1) + valueCounts.map(_._1).init } else { // stride between splits val stride: Double = numSamples.toDouble / (numSplits + 1) @@ -1023,12 +1028,6 @@ private[spark] object RandomForest extends Logging { splitsBuilder.result() } } - - // TODO: Do not fail; just ignore the useless feature. - assert(splits.length > 0, - s"DecisionTree could not handle feature $featureIndex since it had only 1 unique value." + - " Please remove this feature and then try again.") - splits } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 79b19ea5ad206..499d386e66413 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -115,7 +115,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { ) val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 3).map(_.toDouble) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits.length === 3) + assert(splits === Array(1.0, 2.0)) // check returned splits are distinct assert(splits.distinct.length === splits.length) } @@ -129,23 +129,53 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { ) val featureSamples = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 4, 5).map(_.toDouble) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits.length === 2) - assert(splits(0) === 2.0) - assert(splits(1) === 3.0) + assert(splits === Array(2.0, 3.0)) } // find splits when most samples close to the maximum { val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, Map(), Set(), - Array(3), Gini, QuantileStrategy.Sort, + Array(2), Gini, QuantileStrategy.Sort, 0, 0, 0.0, 0, 0 ) val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2).map(_.toDouble) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits.length === 1) - assert(splits(0) === 1.0) + assert(splits === Array(1.0)) } + + // find splits for constant feature + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(0, 0, 0).map(_.toDouble) + val featureSamplesEmpty = Array.empty[Double] + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits === Array[Double]()) + val splitsEmpty = + RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, fakeMetadata, 0) + assert(splitsEmpty === Array[Double]()) + } + } + + test("train with constant features") { + val lp = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0)) + val data = Array.fill(5)(lp) + val rdd = sc.parallelize(data) + val strategy = new OldStrategy( + OldAlgo.Classification, + Gini, + maxDepth = 2, + numClasses = 2, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 1, 1 -> 5)) + val Array(tree) = RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None) + assert(tree.rootNode.impurity === -1.0) + assert(tree.depth === 0) + assert(tree.rootNode.prediction === lp.label) } test("Multiclass classification with unordered categorical features: split calculations") { From d5ec4a3e014494a3d991a6350caffbc3b17be0fd Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 10 Oct 2016 19:14:01 -0700 Subject: [PATCH 039/177] [SPARK-17738][TEST] Fix flaky test in ColumnTypeSuite ## What changes were proposed in this pull request? The default buffer size is not big enough for randomly generated MapType. ## How was this patch tested? Ran the tests in 100 times, it never fail (it fail 8 times before the patch). Author: Davies Liu Closes #15395 from davies/flaky_map. --- .../spark/sql/execution/columnar/ColumnTypeSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 8bf9f521e2f06..5f2a3aaff634c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -101,14 +101,15 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { def testColumnType[JvmType](columnType: ColumnType[JvmType]): Unit = { - val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE).order(ByteOrder.nativeOrder()) val proj = UnsafeProjection.create(Array[DataType](columnType.dataType)) val converter = CatalystTypeConverters.createToScalaConverter(columnType.dataType) val seq = (0 until 4).map(_ => proj(makeRandomRow(columnType)).copy()) + val totalSize = seq.map(_.getSizeInBytes).sum + val bufferSize = Math.max(DEFAULT_BUFFER_SIZE, totalSize) test(s"$columnType append/extract") { - buffer.rewind() - seq.foreach(columnType.append(_, 0, buffer)) + val buffer = ByteBuffer.allocate(bufferSize).order(ByteOrder.nativeOrder()) + seq.foreach(r => columnType.append(columnType.getField(r, 0), buffer)) buffer.rewind() seq.foreach { row => From 90217f9deed01ae187e28ef1531491aac8ee50c9 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 11 Oct 2016 10:21:22 +0800 Subject: [PATCH 040/177] [SPARK-16896][SQL] Handle duplicated field names in header consistently with null or empty strings in CSV ## What changes were proposed in this pull request? Currently, CSV datasource allows to load duplicated empty string fields or fields having `nullValue` in the header. It'd be great if this can deal with normal fields as well. This PR proposes handling the duplicates consistently with the existing behaviour with considering case-sensitivity (`spark.sql.caseSensitive`) as below: data below: ``` fieldA,fieldB,,FIELDA,fielda,, 1,2,3,4,5,6,7 ``` is parsed as below: ```scala spark.read.format("csv").option("header", "true").load("test.csv").show() ``` - when `spark.sql.caseSensitive` is `false` (by default). ``` +-------+------+---+-------+-------+---+---+ |fieldA0|fieldB|_c2|FIELDA3|fieldA4|_c5|_c6| +-------+------+---+-------+-------+---+---+ | 1| 2| 3| 4| 5| 6| 7| +-------+------+---+-------+-------+---+---+ ``` - when `spark.sql.caseSensitive` is `true`. ``` +-------+------+---+-------+-------+---+---+ |fieldA0|fieldB|_c2| FIELDA|fieldA4|_c5|_c6| +-------+------+---+-------+-------+---+---+ | 1| 2| 3| 4| 5| 6| 7| +-------+------+---+-------+-------+---+---+ ``` **In more details**, There is a good reference about this problem, `read.csv()` in R. So, I initially wanted to propose the similar behaviour. In case of R, the CSV data below: ``` fieldA,fieldB,,fieldA,fieldA,, 1,2,3,4,5,6,7 ``` is parsed as below: ```r test <- read.csv(file="test.csv",header=TRUE,sep=",") > test fieldA fieldB X fieldA.1 fieldA.2 X.1 X.2 1 1 2 3 4 5 6 7 ``` However, Spark CSV datasource already is handling duplicated empty strings and `nullValue` as field names. So the data below: ``` ,,,fieldA,,fieldB, 1,2,3,4,5,6,7 ``` is parsed as below: ```scala spark.read.format("csv").option("header", "true").load("test.csv").show() ``` ``` +---+---+---+------+---+------+---+ |_c0|_c1|_c2|fieldA|_c4|fieldB|_c6| +---+---+---+------+---+------+---+ | 1| 2| 3| 4| 5| 6| 7| +---+---+---+------+---+------+---+ ``` R starts the number for each duplicate but Spark adds the number for its position for all fields for `nullValue` and empty strings. In terms of case-sensitivity, it seems R is case-sensitive as below: (it seems it is not configurable). ``` a,a,a,A,A 1,2,3,4,5 ``` is parsed as below: ```r test <- read.csv(file="test.csv",header=TRUE,sep=",") > test a a.1 a.2 A A.1 1 1 2 3 4 5 ``` ## How was this patch tested? Unit test in `CSVSuite`. Author: hyukjinkwon Closes #14745 from HyukjinKwon/SPARK-16896. --- .../datasources/csv/CSVFileFormat.scala | 50 +++++++++++++++---- .../execution/datasources/csv/CSVSuite.scala | 33 ++++++++++++ .../datasources/csv/CSVTypeCastSuite.scala | 2 - 3 files changed, 74 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 4e662a52a7bb7..a3691158ee758 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -59,14 +59,8 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { val rdd = baseRdd(sparkSession, csvOptions, paths) val firstLine = findFirstLine(csvOptions, rdd) val firstRow = new CsvReader(csvOptions).parseLine(firstLine) - - val header = if (csvOptions.headerFlag) { - firstRow.zipWithIndex.map { case (value, index) => - if (value == null || value.isEmpty || value == csvOptions.nullValue) s"_c$index" else value - } - } else { - firstRow.zipWithIndex.map { case (value, index) => s"_c$index" } - } + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + val header = makeSafeHeader(firstRow, csvOptions, caseSensitive) val parsedRdd = tokenRdd(sparkSession, csvOptions, header, paths) val schema = if (csvOptions.inferSchemaFlag) { @@ -74,13 +68,51 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { } else { // By default fields are assumed to be StringType val schemaFields = header.map { fieldName => - StructField(fieldName.toString, StringType, nullable = true) + StructField(fieldName, StringType, nullable = true) } StructType(schemaFields) } Some(schema) } + /** + * Generates a header from the given row which is null-safe and duplicate-safe. + */ + private def makeSafeHeader( + row: Array[String], + options: CSVOptions, + caseSensitive: Boolean): Array[String] = { + if (options.headerFlag) { + val duplicates = { + val headerNames = row.filter(_ != null) + .map(name => if (caseSensitive) name else name.toLowerCase) + headerNames.diff(headerNames.distinct).distinct + } + + row.zipWithIndex.map { case (value, index) => + if (value == null || value.isEmpty || value == options.nullValue) { + // When there are empty strings or the values set in `nullValue`, put the + // index as the suffix. + s"_c$index" + } else if (!caseSensitive && duplicates.contains(value.toLowerCase)) { + // When there are case-insensitive duplicates, put the index as the suffix. + s"$value$index" + } else if (duplicates.contains(value)) { + // When there are duplicates, put the index as the suffix. + s"$value$index" + } else { + value + } + } + } else { + row.zipWithIndex.map { case (_, index) => + // Uses default column names, "_c#" where # is its position of fields + // when header option is disabled. + s"_c$index" + } + } + } + override def prepareWrite( sparkSession: SparkSession, job: Job, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 29aac9def6924..f7c22c6c93f7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.SparkException import org.apache.spark.sql.{DataFrame, QueryTest, Row, UDT} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ @@ -856,4 +857,36 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { checkAnswer(stringTimestampsWithFormat, expectedStringTimestampsWithFormat) } } + + test("load duplicated field names consistently with null or empty strings - case sensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTempPath { path => + Seq("a,a,c,A,b,B").toDF().write.text(path.getAbsolutePath) + val actualSchema = spark.read + .format("csv") + .option("header", true) + .load(path.getAbsolutePath) + .schema + val fields = Seq("a0", "a1", "c", "A", "b", "B").map(StructField(_, StringType, true)) + val expectedSchema = StructType(fields) + assert(actualSchema == expectedSchema) + } + } + } + + test("load duplicated field names consistently with null or empty strings - case insensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTempPath { path => + Seq("a,A,c,A,b,B").toDF().write.text(path.getAbsolutePath) + val actualSchema = spark.read + .format("csv") + .option("header", true) + .load(path.getAbsolutePath) + .schema + val fields = Seq("a0", "A1", "c", "A3", "b4", "B5").map(StructField(_, StringType, true)) + val expectedSchema = StructType(fields) + assert(actualSchema == expectedSchema) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala index dae92f626c225..51832a13cfe0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala @@ -18,8 +18,6 @@ package org.apache.spark.sql.execution.datasources.csv import java.math.BigDecimal -import java.sql.{Date, Timestamp} -import java.text.SimpleDateFormat import java.util.Locale import org.apache.spark.SparkFunSuite From 19a5bae47f69929d00d9de43387c7df37a05ee25 Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Mon, 10 Oct 2016 20:41:31 -0700 Subject: [PATCH 041/177] [SPARK-17816][CORE] Fix ConcurrentModificationException issue in BlockStatusesAccumulator ## What changes were proposed in this pull request? Change the BlockStatusesAccumulator to return immutable object when value method is called. ## How was this patch tested? Existing tests plus I verified this change by running a pipeline which consistently repro this issue. This is the stack trace for this exception: ` java.util.ConcurrentModificationException at java.util.ArrayList$Itr.checkForComodification(ArrayList.java:901) at java.util.ArrayList$Itr.next(ArrayList.java:851) at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:43) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.ListBuffer.$plus$plus$eq(ListBuffer.scala:183) at scala.collection.mutable.ListBuffer.$plus$plus$eq(ListBuffer.scala:45) at scala.collection.TraversableLike$class.to(TraversableLike.scala:590) at scala.collection.AbstractTraversable.to(Traversable.scala:104) at scala.collection.TraversableOnce$class.toList(TraversableOnce.scala:294) at scala.collection.AbstractTraversable.toList(Traversable.scala:104) at org.apache.spark.util.JsonProtocol$.accumValueToJson(JsonProtocol.scala:314) at org.apache.spark.util.JsonProtocol$$anonfun$accumulableInfoToJson$5.apply(JsonProtocol.scala:291) at org.apache.spark.util.JsonProtocol$$anonfun$accumulableInfoToJson$5.apply(JsonProtocol.scala:291) at scala.Option.map(Option.scala:146) at org.apache.spark.util.JsonProtocol$.accumulableInfoToJson(JsonProtocol.scala:291) at org.apache.spark.util.JsonProtocol$$anonfun$taskInfoToJson$12.apply(JsonProtocol.scala:283) at org.apache.spark.util.JsonProtocol$$anonfun$taskInfoToJson$12.apply(JsonProtocol.scala:283) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35) at scala.collection.mutable.ListBuffer.foreach(ListBuffer.scala:45) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at org.apache.spark.util.JsonProtocol$.taskInfoToJson(JsonProtocol.scala:283) at org.apache.spark.util.JsonProtocol$.taskEndToJson(JsonProtocol.scala:145) at org.apache.spark.util.JsonProtocol$.sparkEventToJson(JsonProtocol.scala:76) ` Author: Ergin Seyfe Closes #15371 from seyfe/race_cond_jsonprotocal. --- .../apache/spark/executor/TaskMetrics.scala | 42 +------------------ .../org/apache/spark/util/AccumulatorV2.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- 3 files changed, 6 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 2956768c16417..dfd2f818acdac 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,8 +17,6 @@ package org.apache.spark.executor -import java.util.{ArrayList, Collections} - import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} @@ -27,7 +25,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} -import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} +import org.apache.spark.util._ /** @@ -56,7 +54,7 @@ class TaskMetrics private[spark] () extends Serializable { private val _memoryBytesSpilled = new LongAccumulator private val _diskBytesSpilled = new LongAccumulator private val _peakExecutionMemory = new LongAccumulator - private val _updatedBlockStatuses = new BlockStatusesAccumulator + private val _updatedBlockStatuses = new CollectionAccumulator[(BlockId, BlockStatus)] /** * Time taken on the executor to deserialize this task. @@ -323,39 +321,3 @@ private[spark] object TaskMetrics extends Logging { tm } } - - -private[spark] class BlockStatusesAccumulator - extends AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]] { - private val _seq = Collections.synchronizedList(new ArrayList[(BlockId, BlockStatus)]()) - - override def isZero(): Boolean = _seq.isEmpty - - override def copyAndReset(): BlockStatusesAccumulator = new BlockStatusesAccumulator - - override def copy(): BlockStatusesAccumulator = { - val newAcc = new BlockStatusesAccumulator - newAcc._seq.addAll(_seq) - newAcc - } - - override def reset(): Unit = _seq.clear() - - override def add(v: (BlockId, BlockStatus)): Unit = _seq.add(v) - - override def merge( - other: AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]]): Unit = { - other match { - case o: BlockStatusesAccumulator => _seq.addAll(o.value) - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - } - - override def value: java.util.List[(BlockId, BlockStatus)] = _seq - - def setValue(newValue: java.util.List[(BlockId, BlockStatus)]): Unit = { - _seq.clear() - _seq.addAll(newValue) - } -} diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 470d912ecff13..d3ddd39131326 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -444,7 +444,9 @@ class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { override def copy(): CollectionAccumulator[T] = { val newAcc = new CollectionAccumulator[T] - newAcc._list.addAll(_list) + _list.synchronized { + newAcc._list.addAll(_list) + } newAcc } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f4fa7b4061640..c11eb3ffa4601 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -281,7 +281,7 @@ private[spark] object JsonProtocol { ("Finish Time" -> taskInfo.finishTime) ~ ("Failed" -> taskInfo.failed) ~ ("Killed" -> taskInfo.killed) ~ - ("Accumulables" -> JArray(taskInfo.accumulables.map(accumulableInfoToJson).toList)) + ("Accumulables" -> JArray(taskInfo.accumulables.toList.map(accumulableInfoToJson))) } def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { From 0c0ad436ad909364915b910867d08262c62bc95d Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 10 Oct 2016 22:22:41 -0700 Subject: [PATCH 042/177] [SPARK-17719][SPARK-17776][SQL] Unify and tie up options in a single place in JDBC datasource package ## What changes were proposed in this pull request? This PR proposes to fix arbitrary usages among `Map[String, String]`, `Properties` and `JDBCOptions` instances for options in `execution/jdbc` package and make the connection properties exclude Spark-only options. This PR includes some changes as below: - Unify `Map[String, String]`, `Properties` and `JDBCOptions` in `execution/jdbc` package to `JDBCOptions`. - Move `batchsize`, `fetchszie`, `driver` and `isolationlevel` options into `JDBCOptions` instance. - Document `batchSize` and `isolationlevel` with marking both read-only options and write-only options. Also, this includes minor types and detailed explanation for some statements such as url. - Throw exceptions fast by checking arguments first rather than in execution time (e.g. for `fetchsize`). - Exclude Spark-only options in connection properties. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon Closes #15292 from HyukjinKwon/SPARK-17719. --- docs/sql-programming-guide.md | 36 ++++-- .../apache/spark/sql/DataFrameReader.scala | 13 +-- .../datasources/jdbc/JDBCOptions.scala | 110 +++++++++++++++--- .../execution/datasources/jdbc/JDBCRDD.scala | 45 +++---- .../datasources/jdbc/JDBCRelation.scala | 20 ++-- .../jdbc/JdbcRelationProvider.scala | 30 ++--- .../datasources/jdbc/JdbcUtils.scala | 42 ++----- .../spark/sql/jdbc/PostgresDialect.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 11 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 8 +- 10 files changed, 182 insertions(+), 137 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 835cb6981f5bd..d0f43ab0a9cc9 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1049,16 +1049,20 @@ bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9. {% endhighlight %} Tables from the remote database can be loaded as a DataFrame or Spark SQL Temporary table using -the Data Sources API. The following options are supported: +the Data Sources API. Users can specify the JDBC connection properties in the data source options. +user and password are normally provided as connection properties for +logging into the data sources. In addition to the connection properties, Spark also supports +the following case-sensitive options: + + + + + + + + + + +
Property NameMeaning
url - The JDBC URL to connect to. + The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret
dbtable @@ -1083,28 +1087,42 @@ the Data Sources API. The following options are supported: partitionColumn must be a numeric column from the table in question. Notice that lowerBound and upperBound are just used to decide the partition stride, not for filtering the rows in table. So all rows in the table will be - partitioned and returned. + partitioned and returned. This option applies only to reading.
fetchsize - The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). + The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading.
batchsize + The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. +
isolationLevel + The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. +
truncate - This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g. indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. + This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing.
createTableOptions - This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table. For example: CREATE TABLE t (name string) ENGINE=InnoDB. + This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing.
@@ -1328,7 +1346,7 @@ options. - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed - transformations (e.g. `map`, `filter`, and `groupByKey`) and untyped transformations (e.g. + transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in Python and R is not a language feature, the concept of Dataset does not apply to these languages’ APIs. Instead, `DataFrame` remains the primary programing abstraction, which is analogous to the @@ -1377,7 +1395,7 @@ options. - Timestamps are now stored at a precision of 1us, rather than 1ns - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains unchanged. - - The canonical name of SQL/DataFrame functions are now lower case (e.g. sum vs SUM). + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). - JSON data source will not automatically load new files that are created by other applications (i.e. files that are not inserted to the dataset through Spark SQL). For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), @@ -1392,7 +1410,7 @@ options. Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) and writing data out (`DataFrame.write`), -and deprecated the old APIs (e.g. `SQLContext.parquetFile`, `SQLContext.jsonFile`). +and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). See the API docs for `SQLContext.read` (
Scala, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b54e695db3b5e..a716a916b7f7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -29,7 +29,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptions} import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} +import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.InferSchema import org.apache.spark.sql.types.StructType @@ -231,13 +231,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { table: String, parts: Array[Partition], connectionProperties: Properties): DataFrame = { - val props = new Properties() - extraOptions.foreach { case (key, value) => - props.put(key, value) - } - // connectionProperties should override settings in extraOptions - props.putAll(connectionProperties) - val relation = JDBCRelation(url, table, parts, props)(sparkSession) + // connectionProperties should override settings in extraOptions. + val params = extraOptions.toMap ++ connectionProperties.asScala.toMap + val options = new JDBCOptions(url, table, params) + val relation = JDBCRelation(parts, options)(sparkSession) sparkSession.baseRelationToDataFrame(relation) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index bcf65e53afa73..fcd7409159def 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -17,6 +17,11 @@ package org.apache.spark.sql.execution.datasources.jdbc +import java.sql.{Connection, DriverManager} +import java.util.Properties + +import scala.collection.mutable.ArrayBuffer + /** * Options for the JDBC data source. */ @@ -24,40 +29,115 @@ class JDBCOptions( @transient private val parameters: Map[String, String]) extends Serializable { + import JDBCOptions._ + + def this(url: String, table: String, parameters: Map[String, String]) = { + this(parameters ++ Map( + JDBCOptions.JDBC_URL -> url, + JDBCOptions.JDBC_TABLE_NAME -> table)) + } + + val asConnectionProperties: Properties = { + val properties = new Properties() + // We should avoid to pass the options into properties. See SPARK-17776. + parameters.filterKeys(!jdbcOptionNames.contains(_)) + .foreach { case (k, v) => properties.setProperty(k, v) } + properties + } + // ------------------------------------------------------------ // Required parameters // ------------------------------------------------------------ - require(parameters.isDefinedAt("url"), "Option 'url' is required.") - require(parameters.isDefinedAt("dbtable"), "Option 'dbtable' is required.") + require(parameters.isDefinedAt(JDBC_URL), s"Option '$JDBC_URL' is required.") + require(parameters.isDefinedAt(JDBC_TABLE_NAME), s"Option '$JDBC_TABLE_NAME' is required.") // a JDBC URL - val url = parameters("url") + val url = parameters(JDBC_URL) // name of table - val table = parameters("dbtable") + val table = parameters(JDBC_TABLE_NAME) + + // ------------------------------------------------------------ + // Optional parameters + // ------------------------------------------------------------ + val driverClass = { + val userSpecifiedDriverClass = parameters.get(JDBC_DRIVER_CLASS) + userSpecifiedDriverClass.foreach(DriverRegistry.register) + + // Performing this part of the logic on the driver guards against the corner-case where the + // driver returned for a URL is different on the driver and executors due to classpath + // differences. + userSpecifiedDriverClass.getOrElse { + DriverManager.getDriver(url).getClass.getCanonicalName + } + } // ------------------------------------------------------------ - // Optional parameter list + // Optional parameters only for reading // ------------------------------------------------------------ // the column used to partition - val partitionColumn = parameters.getOrElse("partitionColumn", null) + val partitionColumn = parameters.getOrElse(JDBC_PARTITION_COLUMN, null) // the lower bound of partition column - val lowerBound = parameters.getOrElse("lowerBound", null) + val lowerBound = parameters.getOrElse(JDBC_LOWER_BOUND, null) // the upper bound of the partition column - val upperBound = parameters.getOrElse("upperBound", null) + val upperBound = parameters.getOrElse(JDBC_UPPER_BOUND, null) // the number of partitions - val numPartitions = parameters.getOrElse("numPartitions", null) - + val numPartitions = parameters.getOrElse(JDBC_NUM_PARTITIONS, null) require(partitionColumn == null || (lowerBound != null && upperBound != null && numPartitions != null), - "If 'partitionColumn' is specified then 'lowerBound', 'upperBound'," + - " and 'numPartitions' are required.") + s"If '$JDBC_PARTITION_COLUMN' is specified then '$JDBC_LOWER_BOUND', '$JDBC_UPPER_BOUND'," + + s" and '$JDBC_NUM_PARTITIONS' are required.") + val fetchSize = { + val size = parameters.getOrElse(JDBC_BATCH_FETCH_SIZE, "0").toInt + require(size >= 0, + s"Invalid value `${size.toString}` for parameter " + + s"`$JDBC_BATCH_FETCH_SIZE`. The minimum value is 0. When the value is 0, " + + "the JDBC driver ignores the value and does the estimates.") + size + } // ------------------------------------------------------------ - // The options for DataFrameWriter + // Optional parameters only for writing // ------------------------------------------------------------ // if to truncate the table from the JDBC database - val isTruncate = parameters.getOrElse("truncate", "false").toBoolean + val isTruncate = parameters.getOrElse(JDBC_TRUNCATE, "false").toBoolean // the create table option , which can be table_options or partition_options. // E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" // TODO: to reuse the existing partition parameters for those partition specific options - val createTableOptions = parameters.getOrElse("createTableOptions", "") + val createTableOptions = parameters.getOrElse(JDBC_CREATE_TABLE_OPTIONS, "") + val batchSize = { + val size = parameters.getOrElse(JDBC_BATCH_INSERT_SIZE, "1000").toInt + require(size >= 1, + s"Invalid value `${size.toString}` for parameter " + + s"`$JDBC_BATCH_INSERT_SIZE`. The minimum value is 1.") + size + } + val isolationLevel = + parameters.getOrElse(JDBC_TXN_ISOLATION_LEVEL, "READ_UNCOMMITTED") match { + case "NONE" => Connection.TRANSACTION_NONE + case "READ_UNCOMMITTED" => Connection.TRANSACTION_READ_UNCOMMITTED + case "READ_COMMITTED" => Connection.TRANSACTION_READ_COMMITTED + case "REPEATABLE_READ" => Connection.TRANSACTION_REPEATABLE_READ + case "SERIALIZABLE" => Connection.TRANSACTION_SERIALIZABLE + } +} + +object JDBCOptions { + private val jdbcOptionNames = ArrayBuffer.empty[String] + + private def newOption(name: String): String = { + jdbcOptionNames += name + name + } + + val JDBC_URL = newOption("url") + val JDBC_TABLE_NAME = newOption("dbtable") + val JDBC_DRIVER_CLASS = newOption("driver") + val JDBC_PARTITION_COLUMN = newOption("partitionColumn") + val JDBC_LOWER_BOUND = newOption("lowerBound") + val JDBC_UPPER_BOUND = newOption("upperBound") + val JDBC_NUM_PARTITIONS = newOption("numPartitions") + val JDBC_BATCH_FETCH_SIZE = newOption("fetchsize") + val JDBC_TRUNCATE = newOption("truncate") + val JDBC_CREATE_TABLE_OPTIONS = newOption("createTableOptions") + val JDBC_BATCH_INSERT_SIZE = newOption("batchsize") + val JDBC_TXN_ISOLATION_LEVEL = newOption("isolationLevel") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index f10615ebe4bcf..c0fabc81e42a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.jdbc import java.sql.{Connection, Date, PreparedStatement, ResultSet, SQLException, Timestamp} -import java.util.Properties import scala.util.control.NonFatal @@ -46,17 +45,18 @@ object JDBCRDD extends Logging { * Takes a (schema, table) specification and returns the table's Catalyst * schema. * - * @param url - The JDBC url to fetch information from. - * @param table - The table name of the desired table. This may also be a - * SQL query wrapped in parentheses. + * @param options - JDBC options that contains url, table and other information. * * @return A StructType giving the table's Catalyst schema. * @throws SQLException if the table specification is garbage. * @throws SQLException if the table contains an unsupported type. */ - def resolveTable(url: String, table: String, properties: Properties): StructType = { + def resolveTable(options: JDBCOptions): StructType = { + val url = options.url + val table = options.table + val properties = options.asConnectionProperties val dialect = JdbcDialects.get(url) - val conn: Connection = JdbcUtils.createConnectionFactory(url, properties)() + val conn: Connection = JdbcUtils.createConnectionFactory(options)() try { val statement = conn.prepareStatement(dialect.getSchemaQuery(table)) try { @@ -143,43 +143,38 @@ object JDBCRDD extends Logging { }) } - - /** * Build and return JDBCRDD from the given information. * * @param sc - Your SparkContext. * @param schema - The Catalyst schema of the underlying database table. - * @param url - The JDBC url to connect to. - * @param fqTable - The fully-qualified table name (or paren'd SQL query) to use. * @param requiredColumns - The names of the columns to SELECT. * @param filters - The filters to include in all WHERE clauses. * @param parts - An array of JDBCPartitions specifying partition ids and * per-partition WHERE clauses. + * @param options - JDBC options that contains url, table and other information. * * @return An RDD representing "SELECT requiredColumns FROM fqTable". */ def scanTable( sc: SparkContext, schema: StructType, - url: String, - properties: Properties, - fqTable: String, requiredColumns: Array[String], filters: Array[Filter], - parts: Array[Partition]): RDD[InternalRow] = { + parts: Array[Partition], + options: JDBCOptions): RDD[InternalRow] = { + val url = options.url val dialect = JdbcDialects.get(url) val quotedColumns = requiredColumns.map(colName => dialect.quoteIdentifier(colName)) new JDBCRDD( sc, - JdbcUtils.createConnectionFactory(url, properties), + JdbcUtils.createConnectionFactory(options), pruneSchema(schema, requiredColumns), - fqTable, quotedColumns, filters, parts, url, - properties) + options) } } @@ -192,12 +187,11 @@ private[jdbc] class JDBCRDD( sc: SparkContext, getConnection: () => Connection, schema: StructType, - fqTable: String, columns: Array[String], filters: Array[Filter], partitions: Array[Partition], url: String, - properties: Properties) + options: JDBCOptions) extends RDD[InternalRow](sc, Nil) { /** @@ -211,7 +205,7 @@ private[jdbc] class JDBCRDD( private val columnList: String = { val sb = new StringBuilder() columns.foreach(x => sb.append(",").append(x)) - if (sb.length == 0) "1" else sb.substring(1) + if (sb.isEmpty) "1" else sb.substring(1) } /** @@ -286,7 +280,7 @@ private[jdbc] class JDBCRDD( conn = getConnection() val dialect = JdbcDialects.get(url) import scala.collection.JavaConverters._ - dialect.beforeFetch(conn, properties.asScala.toMap) + dialect.beforeFetch(conn, options.asConnectionProperties.asScala.toMap) // H2's JDBC driver does not support the setSchema() method. We pass a // fully-qualified table name in the SELECT statement. I don't know how to @@ -294,15 +288,10 @@ private[jdbc] class JDBCRDD( val myWhereClause = getWhereClause(part) - val sqlText = s"SELECT $columnList FROM $fqTable $myWhereClause" + val sqlText = s"SELECT $columnList FROM ${options.table} $myWhereClause" stmt = conn.prepareStatement(sqlText, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) - val fetchSize = properties.getProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "0").toInt - require(fetchSize >= 0, - s"Invalid value `${fetchSize.toString}` for parameter " + - s"`${JdbcUtils.JDBC_BATCH_FETCH_SIZE}`. The minimum value is 0. When the value is 0, " + - "the JDBC driver ignores the value and does the estimates.") - stmt.setFetchSize(fetchSize) + stmt.setFetchSize(options.fetchSize) rs = stmt.executeQuery() val rowsIterator = JdbcUtils.resultSetToSparkInternalRows(rs, schema, inputMetrics) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 11613dd912eca..672c21c6ac734 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.util.Properties - import scala.collection.mutable.ArrayBuffer import org.apache.spark.internal.Logging @@ -102,10 +100,7 @@ private[sql] object JDBCRelation extends Logging { } private[sql] case class JDBCRelation( - url: String, - table: String, - parts: Array[Partition], - properties: Properties = new Properties())(@transient val sparkSession: SparkSession) + parts: Array[Partition], jdbcOptions: JDBCOptions)(@transient val sparkSession: SparkSession) extends BaseRelation with PrunedFilteredScan with InsertableRelation { @@ -114,7 +109,7 @@ private[sql] case class JDBCRelation( override val needConversion: Boolean = false - override val schema: StructType = JDBCRDD.resolveTable(url, table, properties) + override val schema: StructType = JDBCRDD.resolveTable(jdbcOptions) // Check if JDBCRDD.compileFilter can accept input filters override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { @@ -126,15 +121,16 @@ private[sql] case class JDBCRelation( JDBCRDD.scanTable( sparkSession.sparkContext, schema, - url, - properties, - table, requiredColumns, filters, - parts).asInstanceOf[RDD[Row]] + parts, + jdbcOptions).asInstanceOf[RDD[Row]] } override def insert(data: DataFrame, overwrite: Boolean): Unit = { + val url = jdbcOptions.url + val table = jdbcOptions.table + val properties = jdbcOptions.asConnectionProperties data.write .mode(if (overwrite) SaveMode.Overwrite else SaveMode.Append) .jdbc(url, table, properties) @@ -142,6 +138,6 @@ private[sql] case class JDBCRelation( override def toString: String = { // credentials should not be included in the plan output, table information is sufficient. - s"JDBCRelation(${table})" + s"JDBCRelation(${jdbcOptions.table})" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index b1a061b6f7422..4420b3b18a907 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -17,10 +17,6 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.util.Properties - -import scala.collection.JavaConverters.mapAsJavaMapConverter - import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister, RelationProvider} @@ -46,9 +42,7 @@ class JdbcRelationProvider extends CreatableRelationProvider partitionColumn, lowerBound.toLong, upperBound.toLong, numPartitions.toInt) } val parts = JDBCRelation.columnPartition(partitionInfo) - val properties = new Properties() // Additional properties that we will pass to getConnection - parameters.foreach(kv => properties.setProperty(kv._1, kv._2)) - JDBCRelation(jdbcOptions.url, jdbcOptions.table, parts, properties)(sqlContext.sparkSession) + JDBCRelation(parts, jdbcOptions)(sqlContext.sparkSession) } override def createRelation( @@ -56,15 +50,13 @@ class JdbcRelationProvider extends CreatableRelationProvider mode: SaveMode, parameters: Map[String, String], df: DataFrame): BaseRelation = { - val options = new JDBCOptions(parameters) - val url = options.url - val table = options.table - val createTableOptions = options.createTableOptions - val isTruncate = options.isTruncate - val props = new Properties() - props.putAll(parameters.asJava) + val jdbcOptions = new JDBCOptions(parameters) + val url = jdbcOptions.url + val table = jdbcOptions.table + val createTableOptions = jdbcOptions.createTableOptions + val isTruncate = jdbcOptions.isTruncate - val conn = JdbcUtils.createConnectionFactory(url, props)() + val conn = JdbcUtils.createConnectionFactory(jdbcOptions)() try { val tableExists = JdbcUtils.tableExists(conn, url, table) if (tableExists) { @@ -73,16 +65,16 @@ class JdbcRelationProvider extends CreatableRelationProvider if (isTruncate && isCascadingTruncateTable(url) == Some(false)) { // In this case, we should truncate table and then load. truncateTable(conn, table) - saveTable(df, url, table, props) + saveTable(df, url, table, jdbcOptions) } else { // Otherwise, do not truncate the table, instead drop and recreate it dropTable(conn, table) createTable(df.schema, url, table, createTableOptions, conn) - saveTable(df, url, table, props) + saveTable(df, url, table, jdbcOptions) } case SaveMode.Append => - saveTable(df, url, table, props) + saveTable(df, url, table, jdbcOptions) case SaveMode.ErrorIfExists => throw new AnalysisException( @@ -95,7 +87,7 @@ class JdbcRelationProvider extends CreatableRelationProvider } } else { createTable(df.schema, url, table, createTableOptions, conn) - saveTable(df, url, table, props) + saveTable(df, url, table, jdbcOptions) } } finally { conn.close() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 47549637b5813..e32db73bd6c6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.jdbc import java.sql.{Connection, Driver, DriverManager, PreparedStatement, ResultSet, ResultSetMetaData, SQLException} -import java.util.Properties import scala.collection.JavaConverters._ import scala.util.Try @@ -41,27 +40,13 @@ import org.apache.spark.util.NextIterator * Util functions for JDBC tables. */ object JdbcUtils extends Logging { - - // the property names are case sensitive - val JDBC_BATCH_FETCH_SIZE = "fetchsize" - val JDBC_BATCH_INSERT_SIZE = "batchsize" - val JDBC_TXN_ISOLATION_LEVEL = "isolationLevel" - /** * Returns a factory for creating connections to the given JDBC URL. * - * @param url the JDBC url to connect to. - * @param properties JDBC connection properties. + * @param options - JDBC options that contains url, table and other information. */ - def createConnectionFactory(url: String, properties: Properties): () => Connection = { - val userSpecifiedDriverClass = Option(properties.getProperty("driver")) - userSpecifiedDriverClass.foreach(DriverRegistry.register) - // Performing this part of the logic on the driver guards against the corner-case where the - // driver returned for a URL is different on the driver and executors due to classpath - // differences. - val driverClass: String = userSpecifiedDriverClass.getOrElse { - DriverManager.getDriver(url).getClass.getCanonicalName - } + def createConnectionFactory(options: JDBCOptions): () => Connection = { + val driverClass: String = options.driverClass () => { DriverRegistry.register(driverClass) val driver: Driver = DriverManager.getDrivers.asScala.collectFirst { @@ -71,7 +56,7 @@ object JdbcUtils extends Logging { throw new IllegalStateException( s"Did not find registered driver with class $driverClass") } - driver.connect(url, properties) + driver.connect(options.url, options.asConnectionProperties) } } @@ -550,10 +535,6 @@ object JdbcUtils extends Logging { batchSize: Int, dialect: JdbcDialect, isolationLevel: Int): Iterator[Byte] = { - require(batchSize >= 1, - s"Invalid value `${batchSize.toString}` for parameter " + - s"`$JDBC_BATCH_INSERT_SIZE`. The minimum value is 1.") - val conn = getConnection() var committed = false @@ -676,23 +657,16 @@ object JdbcUtils extends Logging { df: DataFrame, url: String, table: String, - properties: Properties) { + options: JDBCOptions) { val dialect = JdbcDialects.get(url) val nullTypes: Array[Int] = df.schema.fields.map { field => getJdbcType(field.dataType, dialect).jdbcNullType } val rddSchema = df.schema - val getConnection: () => Connection = createConnectionFactory(url, properties) - val batchSize = properties.getProperty(JDBC_BATCH_INSERT_SIZE, "1000").toInt - val isolationLevel = - properties.getProperty(JDBC_TXN_ISOLATION_LEVEL, "READ_UNCOMMITTED") match { - case "NONE" => Connection.TRANSACTION_NONE - case "READ_UNCOMMITTED" => Connection.TRANSACTION_READ_UNCOMMITTED - case "READ_COMMITTED" => Connection.TRANSACTION_READ_COMMITTED - case "REPEATABLE_READ" => Connection.TRANSACTION_REPEATABLE_READ - case "SERIALIZABLE" => Connection.TRANSACTION_SERIALIZABLE - } + val getConnection: () => Connection = createConnectionFactory(options) + val batchSize = options.batchSize + val isolationLevel = options.isolationLevel df.foreachPartition(iterator => savePartition( getConnection, table, iterator, rddSchema, nullTypes, batchSize, dialect, isolationLevel) ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 3f540d6258a0d..4f61a328f47ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.jdbc import java.sql.{Connection, Types} -import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} import org.apache.spark.sql.types._ @@ -94,7 +94,7 @@ private object PostgresDialect extends JdbcDialect { // // See: https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor // - if (properties.getOrElse(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "0").toInt > 0) { + if (properties.getOrElse(JDBCOptions.JDBC_BATCH_FETCH_SIZE, "0").toInt > 0) { connection.setAutoCommit(false) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 7cc3989b791ad..71cf5e6a22916 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -29,8 +29,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD -import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JdbcUtils} import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -84,7 +83,7 @@ class JDBCSuite extends SparkFunSuite |CREATE TEMPORARY TABLE fetchtwo |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass', - | ${JdbcUtils.JDBC_BATCH_FETCH_SIZE} '2') + | ${JDBCOptions.JDBC_BATCH_FETCH_SIZE} '2') """.stripMargin.replaceAll("\n", " ")) sql( @@ -354,8 +353,8 @@ class JDBCSuite extends SparkFunSuite test("Basic API with illegal fetchsize") { val properties = new Properties() - properties.setProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "-1") - val e = intercept[SparkException] { + properties.setProperty(JDBCOptions.JDBC_BATCH_FETCH_SIZE, "-1") + val e = intercept[IllegalArgumentException] { spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", properties).collect() }.getMessage assert(e.contains("Invalid value `-1` for parameter `fetchsize`")) @@ -364,7 +363,7 @@ class JDBCSuite extends SparkFunSuite test("Basic API with FetchSize") { (0 to 4).foreach { size => val properties = new Properties() - properties.setProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, size.toString) + properties.setProperty(JDBCOptions.JDBC_BATCH_FETCH_SIZE, size.toString) assert(spark.read.jdbc( urlWithUserAndPass, "TEST.PEOPLE", properties).collect().length === 3) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 62b29db4d5524..96540ec92da73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SaveMode} -import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -113,8 +113,8 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { (-1 to 0).foreach { size => val properties = new Properties() - properties.setProperty(JdbcUtils.JDBC_BATCH_INSERT_SIZE, size.toString) - val e = intercept[SparkException] { + properties.setProperty(JDBCOptions.JDBC_BATCH_INSERT_SIZE, size.toString) + val e = intercept[IllegalArgumentException] { df.write.mode(SaveMode.Overwrite).jdbc(url, "TEST.BASICCREATETEST", properties) }.getMessage assert(e.contains(s"Invalid value `$size` for parameter `batchsize`")) @@ -126,7 +126,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { (1 to 3).foreach { size => val properties = new Properties() - properties.setProperty(JdbcUtils.JDBC_BATCH_INSERT_SIZE, size.toString) + properties.setProperty(JDBCOptions.JDBC_BATCH_INSERT_SIZE, size.toString) df.write.mode(SaveMode.Overwrite).jdbc(url, "TEST.BASICCREATETEST", properties) assert(2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties()).count()) } From b515768f2668749ad37a3bdf9d265ce45ec447b1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 10 Oct 2016 22:33:20 -0700 Subject: [PATCH 043/177] [SPARK-17844] Simplify DataFrame API for defining frame boundaries in window functions ## What changes were proposed in this pull request? When I was creating the example code for SPARK-10496, I realized it was pretty convoluted to define the frame boundaries for window functions when there is no partition column or ordering column. The reason is that we don't provide a way to create a WindowSpec directly with the frame boundaries. We can trivially improve this by adding rowsBetween and rangeBetween to Window object. As an example, to compute cumulative sum using the natural ordering, before this pr: ``` df.select('key, sum("value").over(Window.partitionBy(lit(1)).rowsBetween(Long.MinValue, 0))) ``` After this pr: ``` df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0))) ``` Note that you could argue there is no point specifying a window frame without partitionBy/orderBy -- but it is strange that only rowsBetween and rangeBetween are not the only two APIs not available. This also fixes https://issues.apache.org/jira/browse/SPARK-17656 (removing _root_.scala). ## How was this patch tested? Added test cases to compute cumulative sum in DataFrameWindowSuite for Scala/Java and tests.py for Python. Author: Reynold Xin Closes #15412 from rxin/SPARK-17844. --- python/pyspark/sql/tests.py | 9 ++++ python/pyspark/sql/window.py | 48 +++++++++++++++++++ .../apache/spark/sql/expressions/Window.scala | 46 ++++++++++++++++-- .../spark/sql/expressions/WindowSpec.scala | 10 ++-- .../apache/spark/sql/expressions/udaf.scala | 4 +- .../spark/sql/DataFrameWindowSuite.scala | 12 +++++ 6 files changed, 119 insertions(+), 10 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index a9e455565a6cd..7b6f9f0ef1c28 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1859,6 +1859,15 @@ def test_window_functions_without_partitionBy(self): for r, ex in zip(rs, expected): self.assertEqual(tuple(r), ex[:len(r)]) + def test_window_functions_cumulative_sum(self): + df = self.spark.createDataFrame([("one", 1), ("two", 2)], ["key", "value"]) + from pyspark.sql import functions as F + sel = df.select(df.key, F.sum(df.value).over(Window.rowsBetween(-sys.maxsize, 0))) + rs = sorted(sel.collect()) + expected = [("one", 1), ("two", 3)] + for r, ex in zip(rs, expected): + self.assertEqual(tuple(r), ex[:len(r)]) + def test_collect_functions(self): df = self.spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"]) from pyspark.sql import functions diff --git a/python/pyspark/sql/window.py b/python/pyspark/sql/window.py index 46663f69a0881..87e9a988987ea 100644 --- a/python/pyspark/sql/window.py +++ b/python/pyspark/sql/window.py @@ -66,6 +66,54 @@ def orderBy(*cols): jspec = sc._jvm.org.apache.spark.sql.expressions.Window.orderBy(_to_java_cols(cols)) return WindowSpec(jspec) + @staticmethod + @since(2.1) + def rowsBetween(start, end): + """ + Creates a :class:`WindowSpec` with the frame boundaries defined, + from `start` (inclusive) to `end` (inclusive). + + Both `start` and `end` are relative positions from the current row. + For example, "0" means "current row", while "-1" means the row before + the current row, and "5" means the fifth row after the current row. + + :param start: boundary start, inclusive. + The frame is unbounded if this is ``-sys.maxsize`` (or lower). + :param end: boundary end, inclusive. + The frame is unbounded if this is ``sys.maxsize`` (or higher). + """ + if start <= -sys.maxsize: + start = WindowSpec._JAVA_MIN_LONG + if end >= sys.maxsize: + end = WindowSpec._JAVA_MAX_LONG + sc = SparkContext._active_spark_context + jspec = sc._jvm.org.apache.spark.sql.expressions.Window.rowsBetween(start, end) + return WindowSpec(jspec) + + @staticmethod + @since(2.1) + def rangeBetween(start, end): + """ + Creates a :class:`WindowSpec` with the frame boundaries defined, + from `start` (inclusive) to `end` (inclusive). + + Both `start` and `end` are relative from the current row. For example, + "0" means "current row", while "-1" means one off before the current row, + and "5" means the five off after the current row. + + :param start: boundary start, inclusive. + The frame is unbounded if this is ``-sys.maxsize`` (or lower). + :param end: boundary end, inclusive. + The frame is unbounded if this is ``sys.maxsize`` (or higher). + """ + if start <= -sys.maxsize: + start = WindowSpec._JAVA_MIN_LONG + if end >= sys.maxsize: + end = WindowSpec._JAVA_MAX_LONG + sc = SparkContext._active_spark_context + jspec = sc._jvm.org.apache.spark.sql.expressions.Window.rangeBetween(start, end) + return WindowSpec(jspec) + class WindowSpec(object): """ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index c29ec6f426789..e8a0c5f43fe46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -42,7 +42,7 @@ object Window { * Creates a [[WindowSpec]] with the partitioning defined. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def partitionBy(colName: String, colNames: String*): WindowSpec = { spec.partitionBy(colName, colNames : _*) } @@ -51,7 +51,7 @@ object Window { * Creates a [[WindowSpec]] with the partitioning defined. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def partitionBy(cols: Column*): WindowSpec = { spec.partitionBy(cols : _*) } @@ -60,7 +60,7 @@ object Window { * Creates a [[WindowSpec]] with the ordering defined. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def orderBy(colName: String, colNames: String*): WindowSpec = { spec.orderBy(colName, colNames : _*) } @@ -69,11 +69,49 @@ object Window { * Creates a [[WindowSpec]] with the ordering defined. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def orderBy(cols: Column*): WindowSpec = { spec.orderBy(cols : _*) } + /** + * Creates a [[WindowSpec]] with the frame boundaries defined, + * from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are relative positions from the current row. For example, "0" means + * "current row", while "-1" means the row before the current row, and "5" means the fifth row + * after the current row. + * + * @param start boundary start, inclusive. + * The frame is unbounded if this is the minimum long value. + * @param end boundary end, inclusive. + * The frame is unbounded if this is the maximum long value. + * @since 2.1.0 + */ + // Note: when updating the doc for this method, also update WindowSpec.rowsBetween. + def rowsBetween(start: Long, end: Long): WindowSpec = { + spec.rowsBetween(start, end) + } + + /** + * Creates a [[WindowSpec]] with the frame boundaries defined, + * from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are relative from the current row. For example, "0" means "current row", + * while "-1" means one off before the current row, and "5" means the five off after the + * current row. + * + * @param start boundary start, inclusive. + * The frame is unbounded if this is the minimum long value. + * @param end boundary end, inclusive. + * The frame is unbounded if this is the maximum long value. + * @since 2.1.0 + */ + // Note: when updating the doc for this method, also update WindowSpec.rangeBetween. + def rangeBetween(start: Long, end: Long): WindowSpec = { + spec.rangeBetween(start, end) + } + private[sql] def spec: WindowSpec = { new WindowSpec(Seq.empty, Seq.empty, UnspecifiedFrame) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index d716da2668675..82bc8f152d6ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -39,7 +39,7 @@ class WindowSpec private[sql]( * Defines the partitioning columns in a [[WindowSpec]]. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def partitionBy(colName: String, colNames: String*): WindowSpec = { partitionBy((colName +: colNames).map(Column(_)): _*) } @@ -48,7 +48,7 @@ class WindowSpec private[sql]( * Defines the partitioning columns in a [[WindowSpec]]. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def partitionBy(cols: Column*): WindowSpec = { new WindowSpec(cols.map(_.expr), orderSpec, frame) } @@ -57,7 +57,7 @@ class WindowSpec private[sql]( * Defines the ordering columns in a [[WindowSpec]]. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def orderBy(colName: String, colNames: String*): WindowSpec = { orderBy((colName +: colNames).map(Column(_)): _*) } @@ -66,7 +66,7 @@ class WindowSpec private[sql]( * Defines the ordering columns in a [[WindowSpec]]. * @since 1.4.0 */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def orderBy(cols: Column*): WindowSpec = { val sortOrder: Seq[SortOrder] = cols.map { col => col.expr match { @@ -92,6 +92,7 @@ class WindowSpec private[sql]( * The frame is unbounded if this is the maximum long value. * @since 1.4.0 */ + // Note: when updating the doc for this method, also update Window.rowsBetween. def rowsBetween(start: Long, end: Long): WindowSpec = { between(RowFrame, start, end) } @@ -109,6 +110,7 @@ class WindowSpec private[sql]( * The frame is unbounded if this is the maximum long value. * @since 1.4.0 */ + // Note: when updating the doc for this method, also update Window.rangeBetween. def rangeBetween(start: Long, end: Long): WindowSpec = { between(RangeFrame, start, end) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index eac658c6176cb..5417a0e481158 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -106,7 +106,7 @@ abstract class UserDefinedAggregateFunction extends Serializable { /** * Creates a [[Column]] for this UDAF using given [[Column]]s as input arguments. */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def apply(exprs: Column*): Column = { val aggregateExpression = AggregateExpression( @@ -120,7 +120,7 @@ abstract class UserDefinedAggregateFunction extends Serializable { * Creates a [[Column]] for this UDAF using the distinct values of the given * [[Column]]s as input arguments. */ - @_root_.scala.annotation.varargs + @scala.annotation.varargs def distinct(exprs: Column*): Column = { val aggregateExpression = AggregateExpression( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala index c2b47cae8f4c4..5bc386f291043 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala @@ -22,6 +22,9 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{DataType, LongType, StructType} +/** + * Window function testing for DataFrame API. + */ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -47,6 +50,15 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { Row(1, "1") :: Row(2, "2") :: Row(null, null) :: Row(null, null) :: Nil) } + test("Window.rowsBetween") { + val df = Seq(("one", 1), ("two", 2)).toDF("key", "value") + // Running (cumulative) sum + checkAnswer( + df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0))), + Row("one", 1) :: Row("two", 3) :: Nil + ) + } + test("lead") { val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") df.createOrReplaceTempView("window_table") From 19401a203b441e3355f0d3fc3fd062b6d5bdee1f Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 10 Oct 2016 22:50:59 -0700 Subject: [PATCH 044/177] [SPARK-15957][ML] RFormula supports forcing to index label ## What changes were proposed in this pull request? ```RFormula``` will index label only when it is string type currently. If the label is numeric type and we use ```RFormula``` to present a classification model, there is no label attributes in label column metadata. The label attributes are useful when making prediction for classification, so we can force to index label by ```StringIndexer``` whether it is numeric or string type for classification. Then SparkR wrappers can extract label attributes from label column metadata successfully. This feature can help us to fix bug similar with [SPARK-15153](https://issues.apache.org/jira/browse/SPARK-15153). For regression, we will still to keep label as numeric type. In this PR, we add a param ```indexLabel``` to control whether to force to index label for ```RFormula```. ## How was this patch tested? Unit tests. Author: Yanbo Liang Closes #13675 from yanboliang/spark-15957. --- .../apache/spark/ml/feature/RFormula.scala | 29 +++++++++++++++++-- .../spark/ml/feature/RFormulaSuite.scala | 27 ++++++++++++++++- 2 files changed, 52 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 2ee899bcca564..389898666eb8e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -26,7 +26,7 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model, Pipeline, PipelineModel, PipelineStage, Transformer} import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.VectorUDT -import org.apache.spark.ml.param.{Param, ParamMap} +import org.apache.spark.ml.param.{BooleanParam, Param, ParamMap} import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} @@ -104,6 +104,27 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) @Since("1.5.0") def setLabelCol(value: String): this.type = set(labelCol, value) + /** + * Force to index label whether it is numeric or string type. + * Usually we index label only when it is string type. + * If the formula was used by classification algorithms, + * we can force to index label even it is numeric type by setting this param with true. + * Default: false. + * @group param + */ + @Since("2.1.0") + val forceIndexLabel: BooleanParam = new BooleanParam(this, "forceIndexLabel", + "Force to index label whether it is numeric or string") + setDefault(forceIndexLabel -> false) + + /** @group getParam */ + @Since("2.1.0") + def getForceIndexLabel: Boolean = $(forceIndexLabel) + + /** @group setParam */ + @Since("2.1.0") + def setForceIndexLabel(value: Boolean): this.type = set(forceIndexLabel, value) + /** Whether the formula specifies fitting an intercept. */ private[ml] def hasIntercept: Boolean = { require(isDefined(formula), "Formula must be defined first.") @@ -167,8 +188,8 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) encoderStages += new VectorAttributeRewriter($(featuresCol), prefixesToRewrite.toMap) encoderStages += new ColumnPruner(tempColumns.toSet) - if (dataset.schema.fieldNames.contains(resolvedFormula.label) && - dataset.schema(resolvedFormula.label).dataType == StringType) { + if ((dataset.schema.fieldNames.contains(resolvedFormula.label) && + dataset.schema(resolvedFormula.label).dataType == StringType) || $(forceIndexLabel)) { encoderStages += new StringIndexer() .setInputCol(resolvedFormula.label) .setOutputCol($(labelCol)) @@ -181,6 +202,8 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) @Since("1.5.0") // optimistic schema; does not contain any ML attributes override def transformSchema(schema: StructType): StructType = { + require(!hasLabelCol(schema) || !$(forceIndexLabel), + "If label column already exists, forceIndexLabel can not be set with true.") if (hasLabelCol(schema)) { StructType(schema.fields :+ StructField($(featuresCol), new VectorUDT, true)) } else { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index 97c268f3d5c97..c664460d7d8bb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -57,7 +57,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul } } - test("label column already exists") { + test("label column already exists and forceIndexLabel was set with false") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") val original = Seq((0, 1.0), (2, 2.0)).toDF("x", "y") val model = formula.fit(original) @@ -66,6 +66,14 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul assert(resultSchema.toString == model.transform(original).schema.toString) } + test("label column already exists but forceIndexLabel was set with true") { + val formula = new RFormula().setFormula("y ~ x").setLabelCol("y").setForceIndexLabel(true) + val original = spark.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") + intercept[IllegalArgumentException] { + formula.fit(original) + } + } + test("label column already exists but is not numeric type") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") val original = Seq((0, true), (2, false)).toDF("x", "y") @@ -137,6 +145,23 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul assert(result.collect() === expected.collect()) } + test("force to index label even it is numeric type") { + val formula = new RFormula().setFormula("id ~ a + b").setForceIndexLabel(true) + val original = spark.createDataFrame( + Seq((1.0, "foo", 4), (1.0, "bar", 4), (0.0, "bar", 5), (1.0, "baz", 5)) + ).toDF("id", "a", "b") + val model = formula.fit(original) + val result = model.transform(original) + val expected = spark.createDataFrame( + Seq( + (1.0, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 0.0), + (1.0, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 0.0), + (0.0, "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 1.0), + (1.0, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 0.0)) + ).toDF("id", "a", "b", "features", "label") + assert(result.collect() === expected.collect()) + } + test("attribute generation") { val formula = new RFormula().setFormula("id ~ a + b") val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) From 658c7147f5bf637f36e8c66b9207d94b1e7c74c5 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Tue, 11 Oct 2016 08:29:52 +0200 Subject: [PATCH 045/177] [SPARK-17808][PYSPARK] Upgraded version of Pyrolite to 4.13 ## What changes were proposed in this pull request? Upgraded to a newer version of Pyrolite which supports serialization of a BinaryType StructField for PySpark.SQL ## How was this patch tested? Added a unit test which fails with a raised ValueError when using the previous version of Pyrolite 4.9 and Python3 Author: Bryan Cutler Closes #15386 from BryanCutler/pyrolite-upgrade-SPARK-17808. --- core/pom.xml | 2 +- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/pyspark/sql/tests.py | 8 ++++++++ 7 files changed, 14 insertions(+), 6 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 9a4f234953a23..205bbc588be09 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -320,7 +320,7 @@ net.razorvine pyrolite - 4.9 + 4.13 net.razorvine diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index f4f92c6d20c23..b30f8c347c0af 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -141,7 +141,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 3db013f1a7585..5b3a7651dd299 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -148,7 +148,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 71710109a16ac..e323efe30f64b 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -148,7 +148,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index cb30fda253c0a..77d97e5365b9f 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -156,7 +156,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 9008aa80bc877..572edfa0cc29e 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -157,7 +157,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 7b6f9f0ef1c28..86c590dae34d7 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1708,6 +1708,14 @@ def test_read_text_file_list(self): count = df.count() self.assertEquals(count, 4) + def test_BinaryType_serialization(self): + # Pyrolite version <= 4.9 could not serialize BinaryType with Python3 SPARK-17808 + schema = StructType([StructField('mybytes', BinaryType())]) + data = [[bytearray(b'here is my data')], + [bytearray(b'and here is some more')]] + df = self.spark.createDataFrame(data, schema=schema) + df.collect() + class HiveSparkSubmitTests(SparkSubmitTests): From 7388ad94d717784a1837ac5a4a9b53219892d080 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 11 Oct 2016 15:21:28 +0800 Subject: [PATCH 046/177] [SPARK-17338][SQL][FOLLOW-UP] add global temp view ## What changes were proposed in this pull request? address post hoc review comments for https://github.com/apache/spark/pull/14897 ## How was this patch tested? N/A Author: Wenchen Fan Closes #15424 from cloud-fan/global-temp-view. --- project/MimaExcludes.scala | 4 +++- python/pyspark/sql/catalog.py | 5 +++++ .../spark/sql/catalyst/catalog/SessionCatalog.scala | 8 ++++++-- .../src/main/scala/org/apache/spark/sql/Dataset.scala | 9 ++------- .../scala/org/apache/spark/sql/catalog/Catalog.scala | 7 ++++++- .../org/apache/spark/sql/internal/CatalogImpl.scala | 4 ++-- 6 files changed, 24 insertions(+), 13 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index e3d9a17469a35..ae72d37a0b61c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -57,7 +57,9 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.tableExists"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"), // [SPARK-17338][SQL] add global temp view - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView") ) } diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index df3bf4254d4d3..a36d02e0db134 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -169,6 +169,10 @@ def createExternalTable(self, tableName, path=None, source=None, schema=None, ** def dropTempView(self, viewName): """Drops the local temporary view with the given view name in the catalog. If the view has been cached before, then it will also be uncached. + Returns true if this view is dropped successfully, false otherwise. + + Note that, the return type of this method was None in Spark 2.0, but changed to Boolean + in Spark 2.1. >>> spark.createDataFrame([(1, 1)]).createTempView("my_table") >>> spark.table("my_table").collect() @@ -185,6 +189,7 @@ def dropTempView(self, viewName): def dropGlobalTempView(self, viewName): """Drops the global temporary view with the given view name in the catalog. If the view has been cached before, then it will also be uncached. + Returns true if this view is dropped successfully, false otherwise. >>> spark.createDataFrame([(1, 1)]).createGlobalTempView("my_table") >>> spark.table("global_temp.my_table").collect() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e44e30ec648f6..5863c6a71cdf9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -405,13 +405,17 @@ class SessionCatalog( /** * Drop a local temporary view. + * + * Returns true if this view is dropped successfully, false otherwise. */ - def dropTempView(name: String): Unit = synchronized { - tempTables.remove(formatTableName(name)) + def dropTempView(name: String): Boolean = synchronized { + tempTables.remove(formatTableName(name)).isDefined } /** * Drop a global temporary view. + * + * Returns true if this view is dropped successfully, false otherwise. */ def dropGlobalTempView(name: String): Boolean = { globalTempViewManager.remove(formatTableName(name)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 30349ba3cb452..a7a84730a6fd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2494,7 +2494,7 @@ class Dataset[T] private[sql]( * preserved database `_global_temp`, and we must use the qualified name to refer a global temp * view, e.g. `SELECT * FROM _global_temp.view1`. * - * @throws TempTableAlreadyExistsException if the view name already exists + * @throws AnalysisException if the view name already exists * * @group basic * @since 2.1.0 @@ -2508,12 +2508,7 @@ class Dataset[T] private[sql]( viewName: String, replace: Boolean, global: Boolean): CreateViewCommand = { - val viewType = if (global) { - GlobalTempView - } else { - LocalTempView - } - + val viewType = if (global) GlobalTempView else LocalTempView CreateViewCommand( name = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName), userSpecifiedColumns = Nil, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 717fb291901bf..18cba8ce28b4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -269,10 +269,14 @@ abstract class Catalog { * created it, i.e. it will be automatically dropped when the session terminates. It's not * tied to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. * + * Note that, the return type of this method was Unit in Spark 2.0, but changed to Boolean + * in Spark 2.1. + * * @param viewName the name of the view to be dropped. + * @return true if the view is dropped successfully, false otherwise. * @since 2.0.0 */ - def dropTempView(viewName: String): Unit + def dropTempView(viewName: String): Boolean /** * Drops the global temporary view with the given view name in the catalog. @@ -284,6 +288,7 @@ abstract class Catalog { * view, e.g. `SELECT * FROM _global_temp.view1`. * * @param viewName the name of the view to be dropped. + * @return true if the view is dropped successfully, false otherwise. * @since 2.1.0 */ def dropGlobalTempView(viewName: String): Boolean diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index c05bda3f1b526..f6c297e91b7c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -371,8 +371,8 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @group ddl_ops * @since 2.0.0 */ - override def dropTempView(viewName: String): Unit = { - sparkSession.sessionState.catalog.getTempView(viewName).foreach { tempView => + override def dropTempView(viewName: String): Boolean = { + sparkSession.sessionState.catalog.getTempView(viewName).exists { tempView => sparkSession.sharedState.cacheManager.uncacheQuery(Dataset.ofRows(sparkSession, tempView)) sessionCatalog.dropTempView(viewName) } From 3694ba48f0db0f47baea4b005cdeef3f454b7329 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 11 Oct 2016 15:35:52 +0800 Subject: [PATCH 047/177] [SPARK-17864][SQL] Mark data type APIs as stable (not DeveloperApi) ## What changes were proposed in this pull request? The data type API has not been changed since Spark 1.3.0, and is ready for graduation. This patch marks them as stable APIs using the new InterfaceStability annotation. This patch also looks at the various files in the catalyst module (not the "package") and marks the remaining few classes appropriately as well. ## How was this patch tested? This is an annotation change. No functional changes. Author: Reynold Xin Closes #15426 from rxin/SPARK-17864. --- .../java/org/apache/spark/sql/RowFactory.java | 6 +++++ .../spark/sql/streaming/OutputMode.java | 2 ++ .../org/apache/spark/sql/types/DataTypes.java | 5 ++++ .../spark/sql/types/SQLUserDefinedType.java | 2 ++ .../apache/spark/sql/AnalysisException.scala | 9 ++++---- .../scala/org/apache/spark/sql/Encoder.scala | 3 ++- .../scala/org/apache/spark/sql/Encoders.scala | 3 ++- .../main/scala/org/apache/spark/sql/Row.scala | 10 ++++++-- .../spark/sql/types/AbstractDataType.scala | 7 +++--- .../apache/spark/sql/types/ArrayType.scala | 14 +++++++---- .../apache/spark/sql/types/BinaryType.scala | 10 ++++---- .../apache/spark/sql/types/BooleanType.scala | 12 ++++++---- .../org/apache/spark/sql/types/ByteType.scala | 12 +++++++--- .../sql/types/CalendarIntervalType.scala | 12 ++++++---- .../org/apache/spark/sql/types/DataType.scala | 11 ++++++--- .../org/apache/spark/sql/types/DateType.scala | 12 ++++++---- .../org/apache/spark/sql/types/Decimal.scala | 5 ++-- .../apache/spark/sql/types/DecimalType.scala | 14 +++++++---- .../apache/spark/sql/types/DoubleType.scala | 11 ++++++--- .../apache/spark/sql/types/FloatType.scala | 12 +++++++--- .../apache/spark/sql/types/IntegerType.scala | 11 ++++++--- .../org/apache/spark/sql/types/LongType.scala | 12 ++++++---- .../org/apache/spark/sql/types/MapType.scala | 10 ++++---- .../org/apache/spark/sql/types/Metadata.scala | 20 +++++++++------- .../org/apache/spark/sql/types/NullType.scala | 11 ++++++--- .../apache/spark/sql/types/ShortType.scala | 11 ++++++--- .../apache/spark/sql/types/StringType.scala | 11 ++++++--- .../apache/spark/sql/types/StructField.scala | 5 ++++ .../apache/spark/sql/types/StructType.scala | 23 +++++++++++-------- .../spark/sql/types/TimestampType.scala | 11 ++++++--- .../spark/sql/types/UserDefinedType.scala | 4 ---- 31 files changed, 207 insertions(+), 94 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java index 5ed60fe78d116..2ce1fdcbf56ae 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java @@ -17,16 +17,22 @@ package org.apache.spark.sql; +import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.catalyst.expressions.GenericRow; /** * A factory class used to construct {@link Row} objects. + * + * @since 1.3.0 */ +@InterfaceStability.Stable public class RowFactory { /** * Create a {@link Row} from the given arguments. Position i in the argument list becomes * position i in the created {@link Row} object. + * + * @since 1.3.0 */ public static Row create(Object ... values) { return new GenericRow(values); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java index 41e2582921198..49a18df2c72c0 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming; import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.InternalOutputModes; /** @@ -29,6 +30,7 @@ * @since 2.0.0 */ @Experimental +@InterfaceStability.Evolving public class OutputMode { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java index 747ab1809fc0a..0f8570fe470bd 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java @@ -19,10 +19,15 @@ import java.util.*; +import org.apache.spark.annotation.InterfaceStability; + /** * To get/create specific data type, users should use singleton objects and factory methods * provided by this class. + * + * @since 1.3.0 */ +@InterfaceStability.Stable public class DataTypes { /** * Gets the StringType object. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java index 110ed460cc8fa..1290614a3207d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java @@ -20,6 +20,7 @@ import java.lang.annotation.*; import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.annotation.InterfaceStability; /** * ::DeveloperApi:: @@ -30,6 +31,7 @@ @DeveloperApi @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.TYPE) +@InterfaceStability.Evolving public @interface SQLUserDefinedType { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 6911843999392..f3003306acc6d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -17,17 +17,16 @@ package org.apache.spark.sql -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -// TODO: don't swallow original stack trace if it exists - /** - * :: DeveloperApi :: * Thrown when a query fails to analyze, usually because the query itself is invalid. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class AnalysisException protected[sql] ( val message: String, val line: Option[Int] = None, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala index 501c1304dbedb..b9f8c46443021 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.annotation.implicitNotFound import scala.reflect.ClassTag -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.types._ @@ -67,6 +67,7 @@ import org.apache.spark.sql.types._ * @since 1.6.0 */ @Experimental +@InterfaceStability.Evolving @implicitNotFound("Unable to find encoder for type stored in a Dataset. Primitive types " + "(Int, String, etc) and Product types (case classes) are supported by importing " + "spark.implicits._ Support for serializing other types will be added in future " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index e72f67c48a296..dc90659a676e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -22,7 +22,7 @@ import java.lang.reflect.Modifier import scala.reflect.{classTag, ClassTag} import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{BoundReference, Cast} @@ -36,6 +36,7 @@ import org.apache.spark.sql.types._ * @since 1.6.0 */ @Experimental +@InterfaceStability.Evolving object Encoders { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index e16850efbea5f..344dcb9bce62b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,9 +20,14 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.util.hashing.MurmurHash3 +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.types.StructType +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable object Row { /** * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: @@ -117,8 +122,9 @@ object Row { * } * }}} * - * @group row + * @since 1.3.0 */ +@InterfaceStability.Stable trait Row extends Serializable { /** Number of elements in the Row. */ def size: Int = length @@ -351,7 +357,7 @@ trait Row extends Serializable { }.toMap } - override def toString(): String = s"[${this.mkString(",")}]" + override def toString: String = s"[${this.mkString(",")}]" /** * Make a copy of the current [[Row]] object. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 1981fd8f0a1b5..76dbb7cf0aec1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.types import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.Expression /** @@ -131,10 +131,11 @@ protected[sql] abstract class AtomicType extends DataType { /** - * :: DeveloperApi :: * Numeric data types. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable abstract class NumericType extends AtomicType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 82a03b0afc002..5d70ef01373f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -21,9 +21,15 @@ import scala.math.Ordering import org.json4s.JsonDSL._ -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.util.ArrayData +/** + * Companion object for ArrayType. + * + * @since 1.3.0 + */ +@InterfaceStability.Stable object ArrayType extends AbstractDataType { /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ def apply(elementType: DataType): ArrayType = ArrayType(elementType, containsNull = true) @@ -37,9 +43,7 @@ object ArrayType extends AbstractDataType { override private[sql] def simpleString: String = "array" } - /** - * :: DeveloperApi :: * The data type for collections of multiple values. * Internally these are represented as columns that contain a ``scala.collection.Seq``. * @@ -51,8 +55,10 @@ object ArrayType extends AbstractDataType { * * @param elementType The data type of values. * @param containsNull Indicates if values have `null` values + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { /** No-arg constructor for kryo. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala index c40e140e8c5c6..a4a358a242c70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala @@ -20,17 +20,16 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.util.TypeUtils /** - * :: DeveloperApi :: * The data type representing `Array[Byte]` values. * Please use the singleton [[DataTypes.BinaryType]]. */ -@DeveloperApi +@InterfaceStability.Stable class BinaryType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. @@ -54,5 +53,8 @@ class BinaryType private() extends AtomicType { private[spark] override def asNullable: BinaryType = this } - +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object BinaryType extends BinaryType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala index 2d8ee3d9bc286..059f89f9cda32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala @@ -20,15 +20,16 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock /** - * :: DeveloperApi :: * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class BooleanType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. @@ -45,5 +46,8 @@ class BooleanType private() extends AtomicType { private[spark] override def asNullable: BooleanType = this } - +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object BooleanType extends BooleanType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala index d37130e27ba5a..bc6251f024e58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala @@ -20,14 +20,15 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock /** - * :: DeveloperApi :: * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class ByteType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. @@ -48,4 +49,9 @@ class ByteType private() extends IntegralType { private[spark] override def asNullable: ByteType = this } + +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object ByteType extends ByteType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index 3565f52c21f69..e121044288e5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -17,19 +17,19 @@ package org.apache.spark.sql.types -import org.apache.spark.annotation.DeveloperApi - +import org.apache.spark.annotation.InterfaceStability /** - * :: DeveloperApi :: * The data type representing calendar time intervals. The calendar time interval is stored * internally in two components: number of months the number of microseconds. * * Note that calendar intervals are not comparable. * * Please use the singleton [[DataTypes.CalendarIntervalType]]. + * + * @since 1.5.0 */ -@DeveloperApi +@InterfaceStability.Stable class CalendarIntervalType private() extends DataType { override def defaultSize: Int = 16 @@ -37,4 +37,8 @@ class CalendarIntervalType private() extends DataType { private[spark] override def asNullable: CalendarIntervalType = this } +/** + * @since 1.5.0 + */ +@InterfaceStability.Stable case object CalendarIntervalType extends CalendarIntervalType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 4fc65cbce15bd..312585df1516b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -22,15 +22,16 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.util.Utils /** - * :: DeveloperApi :: * The base type of all Spark SQL data types. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable abstract class DataType extends AbstractDataType { /** * Enables matching against DataType for expressions: @@ -94,6 +95,10 @@ abstract class DataType extends AbstractDataType { } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala index 2c966230e447e..8d0ecc051f4ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala @@ -20,19 +20,20 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock /** - * :: DeveloperApi :: * A date type, supporting "0001-01-01" through "9999-12-31". * * Please use the singleton [[DataTypes.DateType]]. * * Internally, this is represented as the number of days from 1970-01-01. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class DateType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "DateType$" in byte code. @@ -51,5 +52,8 @@ class DateType private() extends AtomicType { private[spark] override def asNullable: DateType = this } - +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object DateType extends DateType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 70859052872dd..465fb83669a76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import java.lang.{Long => JLong} import java.math.{BigInteger, MathContext, RoundingMode} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability /** * A mutable implementation of BigDecimal that can hold a Long if values are small enough. @@ -30,6 +30,7 @@ import org.apache.spark.annotation.DeveloperApi * - If decimalVal is set, it represents the whole decimal value * - Otherwise, the decimal value is longVal / (10 ** _scale) */ +@InterfaceStability.Unstable final class Decimal extends Ordered[Decimal] with Serializable { import org.apache.spark.sql.types.Decimal._ @@ -185,7 +186,6 @@ final class Decimal extends Ordered[Decimal] with Serializable { override def toString: String = toBigDecimal.toString() - @DeveloperApi def toDebugString: String = { if (decimalVal.ne(null)) { s"Decimal(expanded,$decimalVal,$precision,$scale})" @@ -380,6 +380,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } +@InterfaceStability.Unstable object Decimal { val ROUND_HALF_UP = BigDecimal.RoundingMode.HALF_UP val ROUND_HALF_EVEN = BigDecimal.RoundingMode.HALF_EVEN diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 6500875f95e54..d7ca0cbeedcd3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -19,14 +19,13 @@ package org.apache.spark.sql.types import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.Expression /** - * :: DeveloperApi :: * The data type representing `java.math.BigDecimal` values. * A Decimal that must have fixed precision (the maximum number of digits) and scale (the number * of digits on right side of dot). @@ -36,8 +35,10 @@ import org.apache.spark.sql.catalyst.expressions.Expression * The default precision and scale is (10, 0). * * Please use [[DataTypes.createDecimalType()]] to create a specific instance. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable case class DecimalType(precision: Int, scale: Int) extends FractionalType { if (scale > precision) { @@ -101,7 +102,12 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { } -/** Extra factory methods and pattern matchers for Decimals */ +/** + * Extra factory methods and pattern matchers for Decimals. + * + * @since 1.3.0 + */ +@InterfaceStability.Stable object DecimalType extends AbstractDataType { import scala.math.min diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala index e553f65f3c99d..c21ac0e43eee0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -21,15 +21,16 @@ import scala.math.{Fractional, Numeric, Ordering} import scala.math.Numeric.DoubleAsIfIntegral import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.util.Utils /** - * :: DeveloperApi :: * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class DoubleType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. @@ -51,4 +52,8 @@ class DoubleType private() extends FractionalType { private[spark] override def asNullable: DoubleType = this } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object DoubleType extends DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala index ae9aa9eefaf2a..c5bf8883bad93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -21,15 +21,16 @@ import scala.math.{Fractional, Numeric, Ordering} import scala.math.Numeric.FloatAsIfIntegral import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.util.Utils /** - * :: DeveloperApi :: * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class FloatType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. @@ -51,4 +52,9 @@ class FloatType private() extends FractionalType { private[spark] override def asNullable: FloatType = this } + +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object FloatType extends FloatType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala index 38a7b8ee52651..724e59c0bcbf4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala @@ -20,15 +20,16 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock /** - * :: DeveloperApi :: * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class IntegerType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. @@ -49,4 +50,8 @@ class IntegerType private() extends IntegralType { private[spark] override def asNullable: IntegerType = this } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object IntegerType extends IntegerType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala index 88aff0c87755c..42285a9d0aa29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala @@ -20,14 +20,15 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock /** - * :: DeveloperApi :: * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class LongType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "LongType$" in byte code. @@ -48,5 +49,8 @@ class LongType private() extends IntegralType { private[spark] override def asNullable: LongType = this } - +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object LongType extends LongType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala index 178960929bd83..3a32aa43d1c3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -20,10 +20,9 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability /** - * :: DeveloperApi :: * The data type for Maps. Keys in a map are not allowed to have `null` values. * * Please use [[DataTypes.createMapType()]] to create a specific instance. @@ -32,7 +31,7 @@ import org.apache.spark.annotation.DeveloperApi * @param valueType The data type of map values. * @param valueContainsNull Indicates if map values have `null` values. */ -@DeveloperApi +@InterfaceStability.Stable case class MapType( keyType: DataType, valueType: DataType, @@ -76,7 +75,10 @@ case class MapType( } } - +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable object MapType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = apply(NullType, NullType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 657bd86ce17d9..3aa4bf619f274 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -22,22 +22,22 @@ import scala.collection.mutable import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability /** - * :: DeveloperApi :: - * * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and * Array[Metadata]. JSON is used for serialization. * * The default constructor is private. User should use either [[MetadataBuilder]] or - * [[Metadata.fromJson()]] to create Metadata instances. + * `Metadata.fromJson()` to create Metadata instances. * * @param map an immutable map that stores the data + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable sealed class Metadata private[types] (private[types] val map: Map[String, Any]) extends Serializable { @@ -114,6 +114,10 @@ sealed class Metadata private[types] (private[types] val map: Map[String, Any]) private[sql] def jsonValue: JValue = Metadata.toJsonValue(this) } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable object Metadata { private[this] val _empty = new Metadata(Map.empty) @@ -218,11 +222,11 @@ object Metadata { } /** - * :: DeveloperApi :: - * * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class MetadataBuilder { private val map: mutable.Map[String, Any] = mutable.Map.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala index aa84115c2e42c..bdf9a819d007b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala @@ -17,14 +17,15 @@ package org.apache.spark.sql.types -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability /** - * :: DeveloperApi :: * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class NullType private() extends DataType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "NullType$" in byte code. @@ -34,4 +35,8 @@ class NullType private() extends DataType { private[spark] override def asNullable: NullType = this } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object NullType extends NullType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala index 486cf585284df..3fee299d578cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala @@ -20,14 +20,15 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock /** - * :: DeveloperApi :: * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class ShortType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. @@ -48,4 +49,8 @@ class ShortType private() extends IntegralType { private[spark] override def asNullable: ShortType = this } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object ShortType extends ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala index 44a25361f31c4..5d5a6f52a305b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -20,15 +20,16 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.unsafe.types.UTF8String /** - * :: DeveloperApi :: * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class StringType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. @@ -45,5 +46,9 @@ class StringType private() extends AtomicType { private[spark] override def asNullable: StringType = this } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object StringType extends StringType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index cb8bf616968e5..2c18fdcc497fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ +import org.apache.spark.annotation.InterfaceStability + /** * A field inside a StructType. * @param name The name of this field. @@ -27,7 +29,10 @@ import org.json4s.JsonDSL._ * @param nullable Indicates if values of this field can be `null` values. * @param metadata The metadata of this field. The metadata should be preserved during * transformation if the content of the column is not modified, e.g, in selection. + * + * @since 1.3.0 */ +@InterfaceStability.Stable case class StructField( name: String, dataType: DataType, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index dd4c88c4c43bc..0205c13aa986d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -23,14 +23,13 @@ import scala.util.Try import org.json4s.JsonDSL._ import org.apache.spark.SparkException -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.util.Utils /** - * :: DeveloperApi :: * A [[StructType]] object can be constructed by * {{{ * StructType(fields: Seq[StructField]) @@ -90,8 +89,10 @@ import org.apache.spark.util.Utils * val row = Row(Row(1, 2, true)) * // row: Row = [[1,2,true]] * }}} + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { /** No-arg constructor for kryo. */ @@ -138,7 +139,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * .add("c", StringType) */ def add(name: String, dataType: DataType): StructType = { - StructType(fields :+ new StructField(name, dataType, nullable = true, Metadata.empty)) + StructType(fields :+ StructField(name, dataType, nullable = true, Metadata.empty)) } /** @@ -150,7 +151,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * .add("c", StringType, true) */ def add(name: String, dataType: DataType, nullable: Boolean): StructType = { - StructType(fields :+ new StructField(name, dataType, nullable, Metadata.empty)) + StructType(fields :+ StructField(name, dataType, nullable, Metadata.empty)) } /** @@ -167,7 +168,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru dataType: DataType, nullable: Boolean, metadata: Metadata): StructType = { - StructType(fields :+ new StructField(name, dataType, nullable, metadata)) + StructType(fields :+ StructField(name, dataType, nullable, metadata)) } /** @@ -347,7 +348,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru private[sql] override def simpleString(maxNumberFields: Int): String = { val builder = new StringBuilder val fieldTypes = fields.take(maxNumberFields).map { - case f => s"${f.name}: ${f.dataType.simpleString(maxNumberFields)}" + f => s"${f.name}: ${f.dataType.simpleString(maxNumberFields)}" } builder.append("struct<") builder.append(fieldTypes.mkString(", ")) @@ -393,6 +394,10 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru InterpretedOrdering.forSchema(this.fields.map(_.dataType)) } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable object StructType extends AbstractDataType { /** @@ -469,7 +474,7 @@ object StructType extends AbstractDataType { nullable = leftNullable || rightNullable) } .orElse { - optionalMeta.putBoolean(metadataKeyForOptionalField, true) + optionalMeta.putBoolean(metadataKeyForOptionalField, value = true) Some(leftField.copy(metadata = optionalMeta.build())) } .foreach(newFields += _) @@ -479,7 +484,7 @@ object StructType extends AbstractDataType { rightFields .filterNot(f => leftMapped.get(f.name).nonEmpty) .foreach { f => - optionalMeta.putBoolean(metadataKeyForOptionalField, true) + optionalMeta.putBoolean(metadataKeyForOptionalField, value = true) newFields += f.copy(metadata = optionalMeta.build()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala index 2be9b2d76c9fe..4540d8358acad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala @@ -20,16 +20,17 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.ScalaReflectionLock /** - * :: DeveloperApi :: * The data type representing `java.sql.Timestamp` values. * Please use the singleton [[DataTypes.TimestampType]]. + * + * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable class TimestampType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. @@ -48,4 +49,8 @@ class TimestampType private() extends AtomicType { private[spark] override def asNullable: TimestampType = this } +/** + * @since 1.3.0 + */ +@InterfaceStability.Stable case object TimestampType extends TimestampType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala index 894631382f8ce..c33219c95b50a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala @@ -22,8 +22,6 @@ import java.util.Objects import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ -import org.apache.spark.annotation.DeveloperApi - /** * The data type for User Defined Types (UDTs). * @@ -96,12 +94,10 @@ abstract class UserDefinedType[UserType >: Null] extends DataType with Serializa } /** - * :: DeveloperApi :: * The user defined type in Python. * * Note: This can only be accessed via Python UDF, or accessed as serialized object. */ -@DeveloperApi private[sql] class PythonUserDefinedType( val sqlType: DataType, override val pyUDT: String, From c8c090640ab73624841d0f4abcfd7409a0838725 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 11 Oct 2016 16:06:40 +0800 Subject: [PATCH 048/177] [SPARK-17821][SQL] Support And and Or in Expression Canonicalize ## What changes were proposed in this pull request? Currently `Canonicalize` object doesn't support `And` and `Or`. So we can compare canonicalized form of predicates consistently. We should add the support. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #15388 from viirya/canonicalize-and-or. --- .../catalyst/expressions/Canonicalize.scala | 7 ++ .../expressions/ExpressionSetSuite.scala | 82 +++++++++++++++++++ 2 files changed, 89 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala index 07ba7d5e4a849..e876450c73fde 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala @@ -62,6 +62,13 @@ object Canonicalize extends { case a: Add => orderCommutative(a, { case Add(l, r) => Seq(l, r) }).reduce(Add) case m: Multiply => orderCommutative(m, { case Multiply(l, r) => Seq(l, r) }).reduce(Multiply) + case o: Or => + orderCommutative(o, { case Or(l, r) if l.deterministic && r.deterministic => Seq(l, r) }) + .reduce(Or) + case a: And => + orderCommutative(a, { case And(l, r) if l.deterministic && r.deterministic => Seq(l, r)}) + .reduce(And) + case EqualTo(l, r) if l.hashCode() > r.hashCode() => EqualTo(r, l) case EqualNullSafe(l, r) if l.hashCode() > r.hashCode() => EqualNullSafe(r, l) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala index 60939ee0eda5d..c587d4f632531 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala @@ -80,6 +80,88 @@ class ExpressionSetSuite extends SparkFunSuite { setTest(1, Not(aUpper >= 1), aUpper < 1, Not(Literal(1) <= aUpper), Literal(1) > aUpper) setTest(1, Not(aUpper <= 1), aUpper > 1, Not(Literal(1) >= aUpper), Literal(1) < aUpper) + // Reordering AND/OR expressions + setTest(1, aUpper > bUpper && aUpper <= 10, aUpper <= 10 && aUpper > bUpper) + setTest(1, + aUpper > bUpper && bUpper > 100 && aUpper <= 10, + bUpper > 100 && aUpper <= 10 && aUpper > bUpper) + + setTest(1, aUpper > bUpper || aUpper <= 10, aUpper <= 10 || aUpper > bUpper) + setTest(1, + aUpper > bUpper || bUpper > 100 || aUpper <= 10, + bUpper > 100 || aUpper <= 10 || aUpper > bUpper) + + setTest(1, + (aUpper <= 10 && aUpper > bUpper) || bUpper > 100, + bUpper > 100 || (aUpper <= 10 && aUpper > bUpper)) + + setTest(1, + aUpper >= bUpper || (aUpper > 10 && bUpper < 10), + (bUpper < 10 && aUpper > 10) || aUpper >= bUpper) + + // More complicated cases mixing AND/OR + // Three predicates in the following: + // (bUpper > 100) + // (aUpper < 100 && bUpper <= aUpper) + // (aUpper >= 10 && bUpper >= 50) + // They can be reordered and the sub-predicates contained in each of them can be reordered too. + setTest(1, + (bUpper > 100) || (aUpper < 100 && bUpper <= aUpper) || (aUpper >= 10 && bUpper >= 50), + (aUpper >= 10 && bUpper >= 50) || (bUpper > 100) || (aUpper < 100 && bUpper <= aUpper), + (bUpper >= 50 && aUpper >= 10) || (bUpper <= aUpper && aUpper < 100) || (bUpper > 100)) + + // Two predicates in the following: + // (bUpper > 100 && aUpper < 100 && bUpper <= aUpper) + // (aUpper >= 10 && bUpper >= 50) + setTest(1, + (bUpper > 100 && aUpper < 100 && bUpper <= aUpper) || (aUpper >= 10 && bUpper >= 50), + (aUpper >= 10 && bUpper >= 50) || (aUpper < 100 && bUpper > 100 && bUpper <= aUpper), + (bUpper >= 50 && aUpper >= 10) || (bUpper <= aUpper && aUpper < 100 && bUpper > 100)) + + // Three predicates in the following: + // (aUpper >= 10) + // (bUpper <= 10 && aUpper === bUpper && aUpper < 100) + // (bUpper >= 100) + setTest(1, + (aUpper >= 10) || (bUpper <= 10 && aUpper === bUpper && aUpper < 100) || (bUpper >= 100), + (aUpper === bUpper && aUpper < 100 && bUpper <= 10) || (bUpper >= 100) || (aUpper >= 10), + (aUpper < 100 && bUpper <= 10 && aUpper === bUpper) || (aUpper >= 10) || (bUpper >= 100), + ((bUpper <= 10 && aUpper === bUpper) && aUpper < 100) || ((aUpper >= 10) || (bUpper >= 100))) + + // Don't reorder non-deterministic expression in AND/OR. + setTest(2, Rand(1L) > aUpper && aUpper <= 10, aUpper <= 10 && Rand(1L) > aUpper) + setTest(2, + aUpper > bUpper && bUpper > 100 && Rand(1L) > aUpper, + bUpper > 100 && Rand(1L) > aUpper && aUpper > bUpper) + + setTest(2, Rand(1L) > aUpper || aUpper <= 10, aUpper <= 10 || Rand(1L) > aUpper) + setTest(2, + aUpper > bUpper || aUpper <= Rand(1L) || aUpper <= 10, + aUpper <= Rand(1L) || aUpper <= 10 || aUpper > bUpper) + + // Partial reorder case: we don't reorder non-deterministic expressions, + // but we can reorder sub-expressions in deterministic AND/OR expressions. + // There are two predicates: + // (aUpper > bUpper || bUpper > 100) => we can reorder sub-expressions in it. + // (aUpper === Rand(1L)) + setTest(1, + (aUpper > bUpper || bUpper > 100) && aUpper === Rand(1L), + (bUpper > 100 || aUpper > bUpper) && aUpper === Rand(1L)) + + // There are three predicates: + // (Rand(1L) > aUpper) + // (aUpper <= Rand(1L) && aUpper > bUpper) + // (aUpper > 10 && bUpper > 10) => we can reorder sub-expressions in it. + setTest(1, + Rand(1L) > aUpper || (aUpper <= Rand(1L) && aUpper > bUpper) || (aUpper > 10 && bUpper > 10), + Rand(1L) > aUpper || (aUpper <= Rand(1L) && aUpper > bUpper) || (bUpper > 10 && aUpper > 10)) + + // Same predicates as above, but a negative case when we reorder non-deterministic + // expression in (aUpper <= Rand(1L) && aUpper > bUpper). + setTest(2, + Rand(1L) > aUpper || (aUpper <= Rand(1L) && aUpper > bUpper) || (aUpper > 10 && bUpper > 10), + Rand(1L) > aUpper || (aUpper > bUpper && aUpper <= Rand(1L)) || (aUpper > 10 && bUpper > 10)) + test("add to / remove from set") { val initialSet = ExpressionSet(aUpper + 1 :: Nil) From 75b9e351413dca0930e8545e6283874db09d8482 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 11 Oct 2016 10:53:07 -0700 Subject: [PATCH 049/177] [SPARK-17346][SQL][TESTS] Fix the flaky topic deletion in KafkaSourceStressSuite ## What changes were proposed in this pull request? A follow up Pr for SPARK-17346 to fix flaky `org.apache.spark.sql.kafka010.KafkaSourceStressSuite`. Test log: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/1855/testReport/junit/org.apache.spark.sql.kafka010/KafkaSourceStressSuite/_It_is_not_a_test_/ Looks like deleting the Kafka internal topic `__consumer_offsets` is flaky. This PR just simply ignores internal topics. ## How was this patch tested? Existing tests. Author: Shixiong Zhu Closes #15384 from zsxwing/SPARK-17346-flaky-test. --- .../org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 6c03070398fca..c640b93b0a2ee 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata -import org.scalatest.BeforeAndAfter import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ @@ -344,7 +343,7 @@ class KafkaSourceSuite extends KafkaSourceTest { } -class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { +class KafkaSourceStressSuite extends KafkaSourceTest { import testImplicits._ @@ -358,12 +357,6 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { start + Random.nextInt(start + end - 1) } - after { - for (topic <- testUtils.getAllTopicsAndPartitionSize().toMap.keys) { - testUtils.deleteTopic(topic) - } - } - test("stress test with multiple topics and partitions") { topics.foreach { topic => testUtils.createTopic(topic, partitions = nextInt(1, 6)) From 07508bd01d16f3331be167ff92770d19c8b1f46a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 11 Oct 2016 11:43:24 -0700 Subject: [PATCH 050/177] [SPARK-17817][PYSPARK] PySpark RDD Repartitioning Results in Highly Skewed Partition Sizes ## What changes were proposed in this pull request? Quoted from JIRA description: Calling repartition on a PySpark RDD to increase the number of partitions results in highly skewed partition sizes, with most having 0 rows. The repartition method should evenly spread out the rows across the partitions, and this behavior is correctly seen on the Scala side. Please reference the following code for a reproducible example of this issue: num_partitions = 20000 a = sc.parallelize(range(int(1e6)), 2) # start with 2 even partitions l = a.repartition(num_partitions).glom().map(len).collect() # get length of each partition min(l), max(l), sum(l)/len(l), len(l) # skewed! In Scala's `repartition` code, we will distribute elements evenly across output partitions. However, the RDD from Python is serialized as a single binary data, so the distribution fails. We need to convert the RDD in Python to java object before repartitioning. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #15389 from viirya/pyspark-rdd-repartition. --- python/pyspark/rdd.py | 13 ++++++++++--- python/pyspark/tests.py | 10 ++++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index ed81eb16df3cd..0e2ae19ca39aa 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2017,8 +2017,7 @@ def repartition(self, numPartitions): >>> len(rdd.repartition(10).glom().collect()) 10 """ - jrdd = self._jrdd.repartition(numPartitions) - return RDD(jrdd, self.ctx, self._jrdd_deserializer) + return self.coalesce(numPartitions, shuffle=True) def coalesce(self, numPartitions, shuffle=False): """ @@ -2029,7 +2028,15 @@ def coalesce(self, numPartitions, shuffle=False): >>> sc.parallelize([1, 2, 3, 4, 5], 3).coalesce(1).glom().collect() [[1, 2, 3, 4, 5]] """ - jrdd = self._jrdd.coalesce(numPartitions, shuffle) + if shuffle: + # In Scala's repartition code, we will distribute elements evenly across output + # partitions. However, the RDD from Python is serialized as a single binary data, + # so the distribution fails and produces highly skewed partitions. We need to + # convert it to a RDD of java object before repartitioning. + data_java_rdd = self._to_java_object_rdd().coalesce(numPartitions, shuffle) + jrdd = self.ctx._jvm.SerDeUtil.javaToPython(data_java_rdd) + else: + jrdd = self._jrdd.coalesce(numPartitions, shuffle) return RDD(jrdd, self.ctx, self._jrdd_deserializer) def zip(self, other): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b0756911bfc10..3e0bd16d85ca4 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -914,6 +914,16 @@ def test_repartitionAndSortWithinPartitions(self): self.assertEqual(partitions[0], [(0, 5), (0, 8), (2, 6)]) self.assertEqual(partitions[1], [(1, 3), (3, 8), (3, 8)]) + def test_repartition_no_skewed(self): + num_partitions = 20 + a = self.sc.parallelize(range(int(1000)), 2) + l = a.repartition(num_partitions).glom().map(len).collect() + zeros = len([x for x in l if x == 0]) + self.assertTrue(zeros == 0) + l = a.coalesce(num_partitions, True).glom().map(len).collect() + zeros = len([x for x in l if x == 0]) + self.assertTrue(zeros == 0) + def test_distinct(self): rdd = self.sc.parallelize((1, 2, 3)*10, 10) self.assertEqual(rdd.getNumPartitions(), 10) From 23405f324a8089f86ebcbede9bb32944137508e8 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 11 Oct 2016 12:41:35 -0700 Subject: [PATCH 051/177] [SPARK-15153][ML][SPARKR] Fix SparkR spark.naiveBayes error when label is numeric type ## What changes were proposed in this pull request? Fix SparkR ```spark.naiveBayes``` error when response variable of dataset is numeric type. See details and how to reproduce this bug at [SPARK-15153](https://issues.apache.org/jira/browse/SPARK-15153). ## How was this patch tested? Add unit test. Author: Yanbo Liang Closes #15431 from yanboliang/spark-15153-2. --- R/pkg/inst/tests/testthat/test_mllib.R | 10 ++++++++++ .../org/apache/spark/ml/r/NaiveBayesWrapper.scala | 1 + 2 files changed, 11 insertions(+) diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index a1eaaf20916a2..c99315726a22c 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -481,6 +481,16 @@ test_that("spark.naiveBayes", { expect_error(m <- e1071::naiveBayes(Survived ~ ., data = t1), NA) expect_equal(as.character(predict(m, t1[1, ])), "Yes") } + + # Test numeric response variable + t1$NumericSurvived <- ifelse(t1$Survived == "No", 0, 1) + t2 <- t1[-4] + df <- suppressWarnings(createDataFrame(t2)) + m <- spark.naiveBayes(df, NumericSurvived ~ ., smoothing = 0.0) + s <- summary(m) + expect_equal(as.double(s$apriori[1, 1]), 0.5833333, tolerance = 1e-6) + expect_equal(sum(s$apriori), 1) + expect_equal(as.double(s$tables[1, "Age_Adult"]), 0.5714286, tolerance = 1e-6) }) test_that("spark.survreg", { diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala index d1a39fea76ef8..4fdab2dd94655 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala @@ -59,6 +59,7 @@ private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { def fit(formula: String, data: DataFrame, smoothing: Double): NaiveBayesWrapper = { val rFormula = new RFormula() .setFormula(formula) + .setForceIndexLabel(true) RWrapperUtils.checkDataColumns(rFormula, data) val rFormulaModel = rFormula.fit(data) // get labels and feature names from output schema From 5b77e66dd6a128c5992ab3bde418613f84be7009 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Tue, 11 Oct 2016 14:56:26 -0700 Subject: [PATCH 052/177] [SPARK-17387][PYSPARK] Creating SparkContext() from python without spark-submit ignores user conf ## What changes were proposed in this pull request? The root cause that we would ignore SparkConf when launching JVM is that SparkConf require JVM to be created first. https://github.com/apache/spark/blob/master/python/pyspark/conf.py#L106 In this PR, I would defer the launching of JVM until SparkContext is created so that we can pass SparkConf to JVM correctly. ## How was this patch tested? Use the example code in the description of SPARK-17387, ``` $ SPARK_HOME=$PWD PYTHONPATH=python:python/lib/py4j-0.10.3-src.zip python Python 2.7.12 (default, Jul 1 2016, 15:12:24) [GCC 5.4.0 20160609] on linux2 Type "help", "copyright", "credits" or "license" for more information. >>> from pyspark import SparkContext >>> from pyspark import SparkConf >>> conf = SparkConf().set("spark.driver.memory", "4g") >>> sc = SparkContext(conf=conf) ``` And verify the spark.driver.memory is correctly picked up. ``` ...op/ -Xmx4g org.apache.spark.deploy.SparkSubmit --conf spark.driver.memory=4g pyspark-shell ``` Author: Jeff Zhang Closes #14959 from zjffdu/SPARK-17387. --- python/pyspark/conf.py | 71 +++++++++++++++++++++++++--------- python/pyspark/context.py | 16 ++++++-- python/pyspark/java_gateway.py | 13 ++++++- 3 files changed, 75 insertions(+), 25 deletions(-) diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 924da3eecf214..64b6f238e9c32 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -52,6 +52,14 @@ >>> sorted(conf.getAll(), key=lambda p: p[0]) [(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), \ (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] +>>> conf._jconf.setExecutorEnv("VAR5", "value5") +JavaObject id... +>>> print(conf.toDebugString()) +spark.executorEnv.VAR1=value1 +spark.executorEnv.VAR3=value3 +spark.executorEnv.VAR4=value4 +spark.executorEnv.VAR5=value5 +spark.home=/path """ __all__ = ['SparkConf'] @@ -101,13 +109,24 @@ def __init__(self, loadDefaults=True, _jvm=None, _jconf=None): self._jconf = _jconf else: from pyspark.context import SparkContext - SparkContext._ensure_initialized() _jvm = _jvm or SparkContext._jvm - self._jconf = _jvm.SparkConf(loadDefaults) + + if _jvm is not None: + # JVM is created, so create self._jconf directly through JVM + self._jconf = _jvm.SparkConf(loadDefaults) + self._conf = None + else: + # JVM is not created, so store data in self._conf first + self._jconf = None + self._conf = {} def set(self, key, value): """Set a configuration property.""" - self._jconf.set(key, unicode(value)) + # Try to set self._jconf first if JVM is created, set self._conf if JVM is not created yet. + if self._jconf is not None: + self._jconf.set(key, unicode(value)) + else: + self._conf[key] = unicode(value) return self def setIfMissing(self, key, value): @@ -118,17 +137,17 @@ def setIfMissing(self, key, value): def setMaster(self, value): """Set master URL to connect to.""" - self._jconf.setMaster(value) + self.set("spark.master", value) return self def setAppName(self, value): """Set application name.""" - self._jconf.setAppName(value) + self.set("spark.app.name", value) return self def setSparkHome(self, value): """Set path where Spark is installed on worker nodes.""" - self._jconf.setSparkHome(value) + self.set("spark.home", value) return self def setExecutorEnv(self, key=None, value=None, pairs=None): @@ -136,10 +155,10 @@ def setExecutorEnv(self, key=None, value=None, pairs=None): if (key is not None and pairs is not None) or (key is None and pairs is None): raise Exception("Either pass one key-value pair or a list of pairs") elif key is not None: - self._jconf.setExecutorEnv(key, value) + self.set("spark.executorEnv." + key, value) elif pairs is not None: for (k, v) in pairs: - self._jconf.setExecutorEnv(k, v) + self.set("spark.executorEnv." + k, v) return self def setAll(self, pairs): @@ -149,35 +168,49 @@ def setAll(self, pairs): :param pairs: list of key-value pairs to set """ for (k, v) in pairs: - self._jconf.set(k, v) + self.set(k, v) return self def get(self, key, defaultValue=None): """Get the configured value for some key, or return a default otherwise.""" if defaultValue is None: # Py4J doesn't call the right get() if we pass None - if not self._jconf.contains(key): - return None - return self._jconf.get(key) + if self._jconf is not None: + if not self._jconf.contains(key): + return None + return self._jconf.get(key) + else: + if key not in self._conf: + return None + return self._conf[key] else: - return self._jconf.get(key, defaultValue) + if self._jconf is not None: + return self._jconf.get(key, defaultValue) + else: + return self._conf.get(key, defaultValue) def getAll(self): """Get all values as a list of key-value pairs.""" - pairs = [] - for elem in self._jconf.getAll(): - pairs.append((elem._1(), elem._2())) - return pairs + if self._jconf is not None: + return [(elem._1(), elem._2()) for elem in self._jconf.getAll()] + else: + return self._conf.items() def contains(self, key): """Does this configuration contain a given key?""" - return self._jconf.contains(key) + if self._jconf is not None: + return self._jconf.contains(key) + else: + return key in self._conf def toDebugString(self): """ Returns a printable version of the configuration, as a list of key=value pairs, one per line. """ - return self._jconf.toDebugString() + if self._jconf is not None: + return self._jconf.toDebugString() + else: + return '\n'.join('%s=%s' % (k, v) for k, v in self._conf.items()) def _test(): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a3dd1950a522f..1b2e199c395be 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -109,7 +109,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ValueError:... """ self._callsite = first_spark_call() or CallSite(None, None, None) - SparkContext._ensure_initialized(self, gateway=gateway) + SparkContext._ensure_initialized(self, gateway=gateway, conf=conf) try: self._do_init(master, appName, sparkHome, pyFiles, environment, batchSize, serializer, conf, jsc, profiler_cls) @@ -121,7 +121,15 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, serializer, conf, jsc, profiler_cls): self.environment = environment or {} - self._conf = conf or SparkConf(_jvm=self._jvm) + # java gateway must have been launched at this point. + if conf is not None and conf._jconf is not None: + # conf has been initialized in JVM properly, so use conf directly. This represent the + # scenario that JVM has been launched before SparkConf is created (e.g. SparkContext is + # created and then stopped, and we create a new SparkConf and new SparkContext again) + self._conf = conf + else: + self._conf = SparkConf(_jvm=SparkContext._jvm) + self._batchSize = batchSize # -1 represents an unlimited batch size self._unbatched_serializer = serializer if batchSize == 0: @@ -232,14 +240,14 @@ def _initialize_context(self, jconf): return self._jvm.JavaSparkContext(jconf) @classmethod - def _ensure_initialized(cls, instance=None, gateway=None): + def _ensure_initialized(cls, instance=None, gateway=None, conf=None): """ Checks whether a SparkContext is initialized or not. Throws error if a SparkContext is already running. """ with SparkContext._lock: if not SparkContext._gateway: - SparkContext._gateway = gateway or launch_gateway() + SparkContext._gateway = gateway or launch_gateway(conf) SparkContext._jvm = SparkContext._gateway.jvm if instance: diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index f76cadcf62438..c1cf843d84388 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -32,7 +32,12 @@ from pyspark.serializers import read_int -def launch_gateway(): +def launch_gateway(conf=None): + """ + launch jvm gateway + :param conf: spark configuration passed to spark-submit + :return: + """ if "PYSPARK_GATEWAY_PORT" in os.environ: gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) else: @@ -41,13 +46,17 @@ def launch_gateway(): # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" script = "./bin/spark-submit.cmd" if on_windows else "./bin/spark-submit" + command = [os.path.join(SPARK_HOME, script)] + if conf: + for k, v in conf.getAll(): + command += ['--conf', '%s=%s' % (k, v)] submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell") if os.environ.get("SPARK_TESTING"): submit_args = ' '.join([ "--conf spark.ui.enabled=false", submit_args ]) - command = [os.path.join(SPARK_HOME, script)] + shlex.split(submit_args) + command = command + shlex.split(submit_args) # Start a socket that will be used by PythonGatewayServer to communicate its port to us callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) From b9a147181d5e38d9abed0c7215f4c5cb695f579c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 11 Oct 2016 20:27:08 -0700 Subject: [PATCH 053/177] [SPARK-17720][SQL] introduce static SQL conf ## What changes were proposed in this pull request? SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897. Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf. ## How was this patch tested? new tests in SQLConfSuite Author: Wenchen Fan Closes #15295 from cloud-fan/global-conf. --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- .../spark/internal/config/package.scala | 14 ----- python/pyspark/sql/session.py | 2 +- .../scala/org/apache/spark/repl/Main.scala | 2 +- .../org/apache/spark/repl/ReplSuite.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../org/apache/spark/sql/RuntimeConfig.scala | 11 +++- .../org/apache/spark/sql/SparkSession.scala | 8 +-- .../org/apache/spark/sql/api/r/SQLUtils.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 56 +++++++++++++------ .../spark/sql/internal/SharedState.scala | 1 + .../sql/execution/command/DDLSuite.scala | 2 +- .../spark/sql/internal/SQLConfSuite.scala | 23 +++++++- .../spark/sql/hive/HiveExternalCatalog.scala | 7 +-- .../org/apache/spark/sql/hive/HiveUtils.scala | 3 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 45 ++++++++------- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 18 files changed, 111 insertions(+), 78 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 6d8cfad5c1f93..61554248ee8f8 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2609,7 +2609,7 @@ test_that("enableHiveSupport on SparkSession", { unsetHiveContext() # if we are still here, it must be built with hive conf <- callJMethod(sparkSession, "conf") - value <- callJMethod(conf, "get", "spark.sql.catalogImplementation", "") + value <- callJMethod(conf, "get", "spark.sql.catalogImplementation") expect_equal(value, "hive") }) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 0896e68eca7dc..5a710158db89f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -91,20 +91,6 @@ package object config { .toSequence .createWithDefault(Nil) - // Note: This is a SQL config but needs to be in core because the REPL depends on it - private[spark] val CATALOG_IMPLEMENTATION = ConfigBuilder("spark.sql.catalogImplementation") - .internal() - .stringConf - .checkValues(Set("hive", "in-memory")) - .createWithDefault("in-memory") - - // Note: This is a SQL config but needs to be in core because it's cross-session and can not put - // in SQLConf. - private[spark] val GLOBAL_TEMP_DATABASE = ConfigBuilder("spark.sql.globalTempDatabase") - .internal() - .stringConf - .createWithDefault("global_temp") - private[spark] val LISTENER_BUS_EVENT_QUEUE_SIZE = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.size") .intConf diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 8418abf99c8d5..1e40b9c39fc4f 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -176,7 +176,7 @@ def getOrCreate(self): sc._conf.set(key, value) session = SparkSession(sc) for key, value in self._options.items(): - session.conf.set(key, value) + session._jsparkSession.sessionState().conf().setConfString(key, value) for key, value in self._options.items(): session.sparkContext._conf.set(key, value) return session diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 5dfe18ad49822..fec4d49379591 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -22,9 +22,9 @@ import java.io.File import scala.tools.nsc.GenericRunnerSettings import org.apache.spark._ -import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.util.Utils object Main extends Logging { diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index f7d7a4f041315..9262e938c2a60 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -24,8 +24,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.lang3.StringEscapeUtils import org.apache.log4j.{Level, LogManager} import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.internal.config._ import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.util.Utils class ReplSuite extends SparkFunSuite { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 5863c6a71cdf9..fe41c41a6eb20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} @@ -63,7 +62,7 @@ class SessionCatalog( conf: CatalystConf) { this( externalCatalog, - new GlobalTempViewManager(GLOBAL_TEMP_DATABASE.defaultValueString), + new GlobalTempViewManager("global_temp"), DummyFunctionResourceLoader, functionRegistry, conf, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index c2baa74ed7d2e..9108d19d0a0c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import org.apache.spark.annotation.InterfaceStability import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} /** @@ -38,6 +38,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * @since 2.0.0 */ def set(key: String, value: String): Unit = { + requireNonStaticConf(key) sqlConf.setConfString(key, value) } @@ -47,6 +48,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * @since 2.0.0 */ def set(key: String, value: Boolean): Unit = { + requireNonStaticConf(key) set(key, value.toString) } @@ -56,6 +58,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * @since 2.0.0 */ def set(key: String, value: Long): Unit = { + requireNonStaticConf(key) set(key, value.toString) } @@ -124,6 +127,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * @since 2.0.0 */ def unset(key: String): Unit = { + requireNonStaticConf(key) sqlConf.unsetConf(key) } @@ -134,4 +138,9 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { sqlConf.contains(key) } + private def requireNonStaticConf(key: String): Unit = { + if (StaticSQLConf.globalConfKeys.contains(key)) { + throw new AnalysisException(s"Cannot modify the value of a static config: $key") + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index d26eea507284c..137c426b4b88d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -29,7 +29,6 @@ import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog @@ -41,6 +40,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState} +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.streaming._ import org.apache.spark.sql.types.{DataType, LongType, StructType} @@ -812,7 +812,7 @@ object SparkSession { // Get the session from current thread's active session. var session = activeThreadSession.get() if ((session ne null) && !session.sparkContext.isStopped) { - options.foreach { case (k, v) => session.conf.set(k, v) } + options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } if (options.nonEmpty) { logWarning("Use an existing SparkSession, some configuration may not take effect.") } @@ -824,7 +824,7 @@ object SparkSession { // If the current thread does not have an active session, get it from the global session. session = defaultSession.get() if ((session ne null) && !session.sparkContext.isStopped) { - options.foreach { case (k, v) => session.conf.set(k, v) } + options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } if (options.nonEmpty) { logWarning("Use an existing SparkSession, some configuration may not take effect.") } @@ -850,7 +850,7 @@ object SparkSession { sc } session = new SparkSession(sparkContext) - options.foreach { case (k, v) => session.conf.set(k, v) } + options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } defaultSession.set(session) // Register a successfully instantiated context to the singleton. This should be at the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 7d8ea03a27910..9de6510c634b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -28,11 +28,11 @@ import org.apache.spark.SparkContext import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.execution.command.ShowTablesCommand +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.types._ private[sql] object SQLUtils extends Logging { @@ -64,7 +64,7 @@ private[sql] object SQLUtils extends Logging { spark: SparkSession, sparkConfigMap: JMap[Object, Object]): Unit = { for ((name, value) <- sparkConfigMap.asScala) { - spark.conf.set(name.toString, value.toString) + spark.sessionState.conf.setConfString(name.toString, value.toString) } for ((name, value) <- sparkConfigMap.asScala) { spark.sparkContext.conf.set(name.toString, value.toString) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index fecdf792fd14a..8cbfc4c7628f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -41,7 +41,7 @@ object SQLConf { private val sqlConfEntries = java.util.Collections.synchronizedMap( new java.util.HashMap[String, ConfigEntry[_]]()) - private def register(entry: ConfigEntry[_]): Unit = sqlConfEntries.synchronized { + private[sql] def register(entry: ConfigEntry[_]): Unit = sqlConfEntries.synchronized { require(!sqlConfEntries.containsKey(entry.key), s"Duplicate SQLConfigEntry. ${entry.key} has been registered") sqlConfEntries.put(entry.key, entry) @@ -326,18 +326,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - // This is used to control the when we will split a schema's JSON string to multiple pieces - // in order to fit the JSON string in metastore's table property (by default, the value has - // a length restriction of 4000 characters). We will split the JSON string of a schema - // to its length exceeds the threshold. - val SCHEMA_STRING_LENGTH_THRESHOLD = - SQLConfigBuilder("spark.sql.sources.schemaStringLengthThreshold") - .doc("The maximum length allowed in a single cell when " + - "storing additional schema information in Hive's metastore.") - .internal() - .intConf - .createWithDefault(4000) - val PARTITION_COLUMN_TYPE_INFERENCE = SQLConfigBuilder("spark.sql.sources.partitionColumnTypeInference.enabled") .doc("When true, automatically infer the data types for partitioned columns.") @@ -736,10 +724,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def bucketingEnabled: Boolean = getConf(SQLConf.BUCKETING_ENABLED) - // Do not use a value larger than 4000 as the default value of this property. - // See the comments of SCHEMA_STRING_LENGTH_THRESHOLD above for more information. - def schemaStringLengthThreshold: Int = getConf(SCHEMA_STRING_LENGTH_THRESHOLD) - def dataFrameEagerAnalysis: Boolean = getConf(DATAFRAME_EAGER_ANALYSIS) def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = @@ -886,3 +870,41 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { } } +/** + * Static SQL configuration is a cross-session, immutable Spark configuration. External users can + * see the static sql configs via `SparkSession.conf`, but can NOT set/unset them. + */ +object StaticSQLConf { + val globalConfKeys = java.util.Collections.synchronizedSet(new java.util.HashSet[String]()) + + private def buildConf(key: String): ConfigBuilder = { + ConfigBuilder(key).onCreate { entry => + globalConfKeys.add(entry.key) + SQLConf.register(entry) + } + } + + val CATALOG_IMPLEMENTATION = buildConf("spark.sql.catalogImplementation") + .internal() + .stringConf + .checkValues(Set("hive", "in-memory")) + .createWithDefault("in-memory") + + val GLOBAL_TEMP_DATABASE = buildConf("spark.sql.globalTempDatabase") + .internal() + .stringConf + .createWithDefault("global_temp") + + // This is used to control when we will split a schema's JSON string to multiple pieces + // in order to fit the JSON string in metastore's table property (by default, the value has + // a length restriction of 4000 characters, so do not use a value larger than 4000 as the default + // value of this property). We will split the JSON string of a schema to its length exceeds the + // threshold. Note that, this conf is only read in HiveExternalCatalog which is cross-session, + // that's why this conf has to be a static SQL conf. + val SCHEMA_STRING_LENGTH_THRESHOLD = buildConf("spark.sql.sources.schemaStringLengthThreshold") + .doc("The maximum length allowed in a single cell when " + + "storing additional schema information in Hive's metastore.") + .internal() + .intConf + .createWithDefault(4000) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index c555a43cd2581..c6083b372a2db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, GlobalTempViewManager, InMemoryCatalog} import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} +import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.util.{MutableURLClassLoader, Utils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 19885156cc722..097dc2441351f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -22,7 +22,6 @@ import java.io.File import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach -import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} @@ -31,6 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 3c60b233c2b04..f545de0e10a6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -19,11 +19,14 @@ package org.apache.spark.sql.internal import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} +import org.apache.spark.sql._ import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} class SQLConfSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + private val testKey = "test.key.0" private val testVal = "test.val.0" @@ -250,4 +253,22 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } } + + test("global SQL conf comes from SparkConf") { + val newSession = SparkSession.builder() + .config(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000") + .getOrCreate() + + assert(newSession.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD.key) == "2000") + checkAnswer( + newSession.sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}"), + Row(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000")) + } + + test("cannot set/unset global SQL conf") { + val e1 = intercept[AnalysisException](sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}=10")) + assert(e1.message.contains("Cannot modify the value of a static config")) + val e2 = intercept[AnalysisException](spark.conf.unset(SCHEMA_STRING_LENGTH_THRESHOLD.key)) + assert(e2.message.contains("Cannot modify the value of a static config")) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 261cc6feff090..e1c0cad907b98 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.execution.command.{ColumnStatStruct, DDLUtils} import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.HiveSerDe +import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD import org.apache.spark.sql.types.{DataType, StructType} @@ -201,11 +202,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // Serialized JSON schema string may be too long to be stored into a single metastore table // property. In this case, we split the JSON string and store each part as a separate table // property. - // TODO: the threshold should be set by `spark.sql.sources.schemaStringLengthThreshold`, - // however the current SQLConf is session isolated, which is not applicable to external - // catalog. We should re-enable this conf instead of hard code the value here, after we have - // global SQLConf. - val threshold = 4000 + val threshold = conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) val schemaJsonString = tableDefinition.schema.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 39d71e164bf51..a5ef8723c8b6f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -23,7 +23,6 @@ import java.nio.charset.StandardCharsets import java.sql.Timestamp import java.util.concurrent.TimeUnit -import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import scala.language.implicitConversions @@ -36,11 +35,11 @@ import org.apache.hadoop.util.VersionInfo import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.sql._ import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.types._ import org.apache.spark.util.Utils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 163f210802b53..6eb571b91ffab 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -30,7 +30,6 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -40,6 +39,7 @@ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.CacheTableCommand import org.apache.spark.sql.hive._ import org.apache.spark.sql.internal.{SharedState, SQLConf} +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.util.{ShutdownHookManager, Utils} // SPARK-3729: Test key required to check for initialization errors with config. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 51670649ad1d4..0477122fc6a27 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -26,12 +26,12 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -699,28 +699,27 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } test("SPARK-6024 wide schema support") { - withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD.key -> "4000") { - withTable("wide_schema") { - withTempDir { tempDir => - // We will need 80 splits for this schema if the threshold is 4000. - val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType))) - - val tableDesc = CatalogTable( - identifier = TableIdentifier("wide_schema"), - tableType = CatalogTableType.EXTERNAL, - storage = CatalogStorageFormat.empty.copy( - properties = Map("path" -> tempDir.getCanonicalPath) - ), - schema = schema, - provider = Some("json") - ) - spark.sessionState.catalog.createTable(tableDesc, ignoreIfExists = false) - - sessionState.refreshTable("wide_schema") - - val actualSchema = table("wide_schema").schema - assert(schema === actualSchema) - } + assert(spark.sparkContext.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) == 4000) + withTable("wide_schema") { + withTempDir { tempDir => + // We will need 80 splits for this schema if the threshold is 4000. + val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType))) + + val tableDesc = CatalogTable( + identifier = TableIdentifier("wide_schema"), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat.empty.copy( + properties = Map("path" -> tempDir.getCanonicalPath) + ), + schema = schema, + provider = Some("json") + ) + spark.sessionState.catalog.createTable(tableDesc, ignoreIfExists = false) + + sessionState.refreshTable("wide_schema") + + val actualSchema = table("wide_schema").schema + assert(schema === actualSchema) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 751e976c7b908..8bff6de008fdb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -22,7 +22,6 @@ import java.io.File import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach -import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTableType} @@ -32,6 +31,7 @@ import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SQLTestUtils class HiveDDLSuite From 299eb04ba05038c7dbb3ecf74a35d4bbfa456643 Mon Sep 17 00:00:00 2001 From: Alexander Pivovarov Date: Tue, 11 Oct 2016 22:31:21 -0700 Subject: [PATCH 054/177] Fix hadoop.version in building-spark.md Couple of mvn build examples use `-Dhadoop.version=VERSION` instead of actual version number Author: Alexander Pivovarov Closes #15440 from apivovarov/patch-1. --- docs/building-spark.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index da7eeb8348378..f5acee6b90059 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -91,13 +91,13 @@ Examples: ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package # Apache Hadoop 2.4.X or 2.5.X - ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package # Apache Hadoop 2.6.X ./build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package # Apache Hadoop 2.7.X and later - ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.0 -DskipTests clean package # Different versions of HDFS and YARN. ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package From b512f04f8e546843d5a3f35dcc6b675b5f4f5bc0 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 11 Oct 2016 22:36:57 -0700 Subject: [PATCH 055/177] [SPARK-17880][DOC] The url linking to `AccumulatorV2` in the document is incorrect. ## What changes were proposed in this pull request? In `programming-guide.md`, the url which links to `AccumulatorV2` says `api/scala/index.html#org.apache.spark.AccumulatorV2` but `api/scala/index.html#org.apache.spark.util.AccumulatorV2` is correct. ## How was this patch tested? manual test. Author: Kousuke Saruta Closes #15439 from sarutak/SPARK-17880. --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 74d5ee1ca6b3f..20b4bee0f58e1 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1373,7 +1373,7 @@ res2: Long = 10 {% endhighlight %} While this code used the built-in support for accumulators of type Long, programmers can also -create their own types by subclassing [AccumulatorV2](api/scala/index.html#org.apache.spark.AccumulatorV2). +create their own types by subclassing [AccumulatorV2](api/scala/index.html#org.apache.spark.util.AccumulatorV2). The AccumulatorV2 abstract class has several methods which need to override: `reset` for resetting the accumulator to zero, and `add` for add anothor value into the accumulator, `merge` for merging another same-type accumulator into this one. Other methods need to override can refer to scala API document. For example, supposing we had a `MyVector` class representing mathematical vectors, we could write: From c264ef9b1918256a5018c7a42a1a2b42308ea3f7 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 12 Oct 2016 00:40:47 -0700 Subject: [PATCH 056/177] [SPARK-17853][STREAMING][KAFKA][DOC] make it clear that reusing group.id is bad ## What changes were proposed in this pull request? Documentation fix to make it clear that reusing group id for different streams is super duper bad, just like it is with the underlying Kafka consumer. ## How was this patch tested? I built jekyll doc and made sure it looked ok. Author: cody koeninger Closes #15442 from koeninger/SPARK-17853. --- docs/streaming-kafka-0-10-integration.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index 44c39e39446de..456b8453383db 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -27,7 +27,7 @@ For Scala/Java applications using SBT/Maven project definitions, link your strea "bootstrap.servers" -> "localhost:9092,anotherhost:9092", "key.deserializer" -> classOf[StringDeserializer], "value.deserializer" -> classOf[StringDeserializer], - "group.id" -> "example", + "group.id" -> "use_a_separate_group_id_for_each_stream", "auto.offset.reset" -> "latest", "enable.auto.commit" -> (false: java.lang.Boolean) ) @@ -48,7 +48,7 @@ Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javad For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). -Note that enable.auto.commit is disabled, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. +Note that the example sets enable.auto.commit to false, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. ### LocationStrategies The new Kafka consumer API will pre-fetch messages into buffers. Therefore it is important for performance reasons that the Spark integration keep cached consumers on executors (rather than recreating them for each batch), and prefer to schedule partitions on the host locations that have the appropriate consumers. @@ -57,6 +57,9 @@ In most cases, you should use `LocationStrategies.PreferConsistent` as shown abo The cache for consumers has a default maximum size of 64. If you expect to be handling more than (64 * number of executors) Kafka partitions, you can change this setting via `spark.streaming.kafka.consumer.cache.maxCapacity` +The cache is keyed by topicpartition and group.id, so use a **separate** `group.id` for each call to `createDirectStream`. + + ### ConsumerStrategies The new Kafka consumer API has a number of different ways to specify topics, some of which require considerable post-object-instantiation setup. `ConsumerStrategies` provides an abstraction that allows Spark to obtain properly configured consumers even after restart from checkpoint. From 8d33e1e5bfde6d2d1270058e49772427383312b3 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 12 Oct 2016 10:00:53 +0100 Subject: [PATCH 057/177] [SPARK-11560][MLLIB] Optimize KMeans implementation / remove 'runs' ## What changes were proposed in this pull request? This is a revival of https://github.com/apache/spark/pull/14948 and related to https://github.com/apache/spark/pull/14937. This removes the 'runs' parameter, which has already been disabled, from the K-means implementation and further deprecates API methods that involve it. This also happens to resolve the issue that K-means should not return duplicate centers, meaning that it may return less than k centroids if not enough data is available. ## How was this patch tested? Existing tests Author: Sean Owen Closes #15342 from srowen/SPARK-11560. --- .../spark/mllib/clustering/KMeans.scala | 296 ++++++++---------- 1 file changed, 132 insertions(+), 164 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 23141aaf42b49..68a7b3b6763af 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -43,18 +43,17 @@ import org.apache.spark.util.random.XORShiftRandom class KMeans private ( private var k: Int, private var maxIterations: Int, - private var runs: Int, private var initializationMode: String, private var initializationSteps: Int, private var epsilon: Double, private var seed: Long) extends Serializable with Logging { /** - * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, + * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, * initializationMode: "k-means||", initializationSteps: 2, epsilon: 1e-4, seed: random}. */ @Since("0.8.0") - def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 2, 1e-4, Utils.random.nextLong()) + def this() = this(2, 20, KMeans.K_MEANS_PARALLEL, 2, 1e-4, Utils.random.nextLong()) /** * Number of clusters to create (k). @@ -112,15 +111,17 @@ class KMeans private ( * This function has no effect since Spark 2.0.0. */ @Since("1.4.0") + @deprecated("This has no effect and always returns 1", "2.1.0") def getRuns: Int = { logWarning("Getting number of runs has no effect since Spark 2.0.0.") - runs + 1 } /** * This function has no effect since Spark 2.0.0. */ @Since("0.8.0") + @deprecated("This has no effect", "2.1.0") def setRuns(runs: Int): this.type = { logWarning("Setting number of runs has no effect since Spark 2.0.0.") this @@ -239,17 +240,9 @@ class KMeans private ( val initStartTime = System.nanoTime() - // Only one run is allowed when initialModel is given - val numRuns = if (initialModel.nonEmpty) { - if (runs > 1) logWarning("Ignoring runs; one run is allowed when initialModel is given.") - 1 - } else { - runs - } - val centers = initialModel match { case Some(kMeansCenters) => - Array(kMeansCenters.clusterCenters.map(s => new VectorWithNorm(s))) + kMeansCenters.clusterCenters.map(new VectorWithNorm(_)) case None => if (initializationMode == KMeans.RANDOM) { initRandom(data) @@ -258,89 +251,62 @@ class KMeans private ( } } val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - logInfo(s"Initialization with $initializationMode took " + "%.3f".format(initTimeInSeconds) + - " seconds.") + logInfo(f"Initialization with $initializationMode took $initTimeInSeconds%.3f seconds.") - val active = Array.fill(numRuns)(true) - val costs = Array.fill(numRuns)(0.0) - - var activeRuns = new ArrayBuffer[Int] ++ (0 until numRuns) + var converged = false + var cost = 0.0 var iteration = 0 val iterationStartTime = System.nanoTime() - instr.foreach(_.logNumFeatures(centers(0)(0).vector.size)) + instr.foreach(_.logNumFeatures(centers.head.vector.size)) - // Execute iterations of Lloyd's algorithm until all runs have converged - while (iteration < maxIterations && !activeRuns.isEmpty) { - type WeightedPoint = (Vector, Long) - def mergeContribs(x: WeightedPoint, y: WeightedPoint): WeightedPoint = { - axpy(1.0, x._1, y._1) - (y._1, x._2 + y._2) - } - - val activeCenters = activeRuns.map(r => centers(r)).toArray - val costAccums = activeRuns.map(_ => sc.doubleAccumulator) - - val bcActiveCenters = sc.broadcast(activeCenters) + // Execute iterations of Lloyd's algorithm until converged + while (iteration < maxIterations && !converged) { + val costAccum = sc.doubleAccumulator + val bcCenters = sc.broadcast(centers) // Find the sum and count of points mapping to each center val totalContribs = data.mapPartitions { points => - val thisActiveCenters = bcActiveCenters.value - val runs = thisActiveCenters.length - val k = thisActiveCenters(0).length - val dims = thisActiveCenters(0)(0).vector.size + val thisCenters = bcCenters.value + val dims = thisCenters.head.vector.size - val sums = Array.fill(runs, k)(Vectors.zeros(dims)) - val counts = Array.fill(runs, k)(0L) + val sums = Array.fill(thisCenters.length)(Vectors.zeros(dims)) + val counts = Array.fill(thisCenters.length)(0L) points.foreach { point => - (0 until runs).foreach { i => - val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) - costAccums(i).add(cost) - val sum = sums(i)(bestCenter) - axpy(1.0, point.vector, sum) - counts(i)(bestCenter) += 1 - } + val (bestCenter, cost) = KMeans.findClosest(thisCenters, point) + costAccum.add(cost) + val sum = sums(bestCenter) + axpy(1.0, point.vector, sum) + counts(bestCenter) += 1 } - val contribs = for (i <- 0 until runs; j <- 0 until k) yield { - ((i, j), (sums(i)(j), counts(i)(j))) - } - contribs.iterator - }.reduceByKey(mergeContribs).collectAsMap() - - bcActiveCenters.destroy(blocking = false) - - // Update the cluster centers and costs for each active run - for ((run, i) <- activeRuns.zipWithIndex) { - var changed = false - var j = 0 - while (j < k) { - val (sum, count) = totalContribs((i, j)) - if (count != 0) { - scal(1.0 / count, sum) - val newCenter = new VectorWithNorm(sum) - if (KMeans.fastSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { - changed = true - } - centers(run)(j) = newCenter - } - j += 1 - } - if (!changed) { - active(run) = false - logInfo("Run " + run + " finished in " + (iteration + 1) + " iterations") + counts.indices.filter(counts(_) > 0).map(j => (j, (sums(j), counts(j)))).iterator + }.reduceByKey { case ((sum1, count1), (sum2, count2)) => + axpy(1.0, sum2, sum1) + (sum1, count1 + count2) + }.collectAsMap() + + bcCenters.destroy(blocking = false) + + // Update the cluster centers and costs + converged = true + totalContribs.foreach { case (j, (sum, count)) => + scal(1.0 / count, sum) + val newCenter = new VectorWithNorm(sum) + if (converged && KMeans.fastSquaredDistance(newCenter, centers(j)) > epsilon * epsilon) { + converged = false } - costs(run) = costAccums(i).value + centers(j) = newCenter } - activeRuns = activeRuns.filter(active(_)) + cost = costAccum.value iteration += 1 } val iterationTimeInSeconds = (System.nanoTime() - iterationStartTime) / 1e9 - logInfo(s"Iterations took " + "%.3f".format(iterationTimeInSeconds) + " seconds.") + logInfo(f"Iterations took $iterationTimeInSeconds%.3f seconds.") if (iteration == maxIterations) { logInfo(s"KMeans reached the max number of iterations: $maxIterations.") @@ -348,59 +314,43 @@ class KMeans private ( logInfo(s"KMeans converged in $iteration iterations.") } - val (minCost, bestRun) = costs.zipWithIndex.min + logInfo(s"The cost is $cost.") - logInfo(s"The cost for the best run is $minCost.") - - new KMeansModel(centers(bestRun).map(_.vector)) + new KMeansModel(centers.map(_.vector)) } /** - * Initialize `runs` sets of cluster centers at random. + * Initialize a set of cluster centers at random. */ - private def initRandom(data: RDD[VectorWithNorm]) - : Array[Array[VectorWithNorm]] = { - // Sample all the cluster centers in one pass to avoid repeated scans - val sample = data.takeSample(true, runs * k, new XORShiftRandom(this.seed).nextInt()).toSeq - Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v => - new VectorWithNorm(Vectors.dense(v.vector.toArray), v.norm) - }.toArray) + private def initRandom(data: RDD[VectorWithNorm]): Array[VectorWithNorm] = { + data.takeSample(true, k, new XORShiftRandom(this.seed).nextInt()).map(_.toDense) } /** - * Initialize `runs` sets of cluster centers using the k-means|| algorithm by Bahmani et al. + * Initialize a set of cluster centers using the k-means|| algorithm by Bahmani et al. * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries - * to find with dissimilar cluster centers by starting with a random center and then doing + * to find dissimilar cluster centers by starting with a random center and then doing * passes where more centers are chosen with probability proportional to their squared distance * to the current cluster set. It results in a provable approximation to an optimal clustering. * * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. */ - private def initKMeansParallel(data: RDD[VectorWithNorm]) - : Array[Array[VectorWithNorm]] = { + private def initKMeansParallel(data: RDD[VectorWithNorm]): Array[VectorWithNorm] = { // Initialize empty centers and point costs. - val centers = Array.tabulate(runs)(r => ArrayBuffer.empty[VectorWithNorm]) - var costs = data.map(_ => Array.fill(runs)(Double.PositiveInfinity)) + var costs = data.map(_ => Double.PositiveInfinity) - // Initialize each run's first center to a random point. + // Initialize the first center to a random point. val seed = new XORShiftRandom(this.seed).nextInt() - val sample = data.takeSample(true, runs, seed).toSeq + val sample = data.takeSample(false, 1, seed) // Could be empty if data is empty; fail with a better message early: - require(sample.size >= runs, s"Required $runs samples but got ${sample.size} from $data") - val newCenters = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDense)) - - /** Merges new centers to centers. */ - def mergeNewCenters(): Unit = { - var r = 0 - while (r < runs) { - centers(r) ++= newCenters(r) - newCenters(r).clear() - r += 1 - } - } + require(sample.nonEmpty, s"No samples available from $data") + + val centers = ArrayBuffer[VectorWithNorm]() + var newCenters = Seq(sample.head.toDense) + centers ++= newCenters - // On each step, sample 2 * k points on average for each run with probability proportional - // to their squared distance from that run's centers. Note that only distances between points + // On each step, sample 2 * k points on average with probability proportional + // to their squared distance from the centers. Note that only distances between points // and new centers are computed in each iteration. var step = 0 var bcNewCentersList = ArrayBuffer[Broadcast[_]]() @@ -409,74 +359,39 @@ class KMeans private ( bcNewCentersList += bcNewCenters val preCosts = costs costs = data.zip(preCosts).map { case (point, cost) => - Array.tabulate(runs) { r => - math.min(KMeans.pointCost(bcNewCenters.value(r), point), cost(r)) - } - }.persist(StorageLevel.MEMORY_AND_DISK) - val sumCosts = costs - .aggregate(new Array[Double](runs))( - seqOp = (s, v) => { - // s += v - var r = 0 - while (r < runs) { - s(r) += v(r) - r += 1 - } - s - }, - combOp = (s0, s1) => { - // s0 += s1 - var r = 0 - while (r < runs) { - s0(r) += s1(r) - r += 1 - } - s0 - } - ) + math.min(KMeans.pointCost(bcNewCenters.value, point), cost) + }.persist(StorageLevel.MEMORY_AND_DISK) + val sumCosts = costs.sum() bcNewCenters.unpersist(blocking = false) preCosts.unpersist(blocking = false) - val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointsWithCosts) => + val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointCosts) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) - pointsWithCosts.flatMap { case (p, c) => - val rs = (0 until runs).filter { r => - rand.nextDouble() < 2.0 * c(r) * k / sumCosts(r) - } - if (rs.nonEmpty) Some((p, rs)) else None - } + pointCosts.filter { case (_, c) => rand.nextDouble() < 2.0 * c * k / sumCosts }.map(_._1) }.collect() - mergeNewCenters() - chosen.foreach { case (p, rs) => - rs.foreach(newCenters(_) += p.toDense) - } + newCenters = chosen.map(_.toDense) + centers ++= newCenters step += 1 } - mergeNewCenters() costs.unpersist(blocking = false) bcNewCentersList.foreach(_.destroy(false)) - // Finally, we might have a set of more than k candidate centers for each run; weigh each - // candidate by the number of points in the dataset mapping to it and run a local k-means++ - // on the weighted centers to pick just k of them - val bcCenters = data.context.broadcast(centers) - val weightMap = data.flatMap { p => - Iterator.tabulate(runs) { r => - ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0) - } - }.reduceByKey(_ + _).collectAsMap() + if (centers.size == k) { + centers.toArray + } else { + // Finally, we might have a set of more or less than k candidate centers; weight each + // candidate by the number of points in the dataset mapping to it and run a local k-means++ + // on the weighted centers to pick k of them + val bcCenters = data.context.broadcast(centers) + val countMap = data.map(KMeans.findClosest(bcCenters.value, _)._1).countByValue() - bcCenters.destroy(blocking = false) + bcCenters.destroy(blocking = false) - val finalCenters = (0 until runs).par.map { r => - val myCenters = centers(r).toArray - val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray - LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) + val myWeights = centers.indices.map(countMap.getOrElse(_, 0L).toDouble).toArray + LocalKMeans.kMeansPlusPlus(0, centers.toArray, myWeights, k, 30) } - - finalCenters.toArray } } @@ -493,6 +408,52 @@ object KMeans { @Since("0.8.0") val K_MEANS_PARALLEL = "k-means||" + /** + * Trains a k-means model using the given set of parameters. + * + * @param data Training points as an `RDD` of `Vector` types. + * @param k Number of clusters to create. + * @param maxIterations Maximum number of iterations allowed. + * @param initializationMode The initialization algorithm. This can either be "random" or + * "k-means||". (default: "k-means||") + * @param seed Random seed for cluster initialization. Default is to generate seed based + * on system time. + */ + @Since("2.1.0") + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + initializationMode: String, + seed: Long): KMeansModel = { + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .setInitializationMode(initializationMode) + .setSeed(seed) + .run(data) + } + + /** + * Trains a k-means model using the given set of parameters. + * + * @param data Training points as an `RDD` of `Vector` types. + * @param k Number of clusters to create. + * @param maxIterations Maximum number of iterations allowed. + * @param initializationMode The initialization algorithm. This can either be "random" or + * "k-means||". (default: "k-means||") + */ + @Since("2.1.0") + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + initializationMode: String): KMeansModel = { + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .setInitializationMode(initializationMode) + .run(data) + } + /** * Trains a k-means model using the given set of parameters. * @@ -506,6 +467,7 @@ object KMeans { * on system time. */ @Since("1.3.0") + @deprecated("Use train method without 'runs'", "2.1.0") def train( data: RDD[Vector], k: Int, @@ -531,6 +493,7 @@ object KMeans { * "k-means||". (default: "k-means||") */ @Since("0.8.0") + @deprecated("Use train method without 'runs'", "2.1.0") def train( data: RDD[Vector], k: Int, @@ -551,19 +514,24 @@ object KMeans { data: RDD[Vector], k: Int, maxIterations: Int): KMeansModel = { - train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .run(data) } /** * Trains a k-means model using specified parameters and the default values for unspecified. */ @Since("0.8.0") + @deprecated("Use train method without 'runs'", "2.1.0") def train( data: RDD[Vector], k: Int, maxIterations: Int, runs: Int): KMeansModel = { - train(data, k, maxIterations, runs, K_MEANS_PARALLEL) + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .run(data) } /** From 8880fd13ef2b581f9c7190e7e3e6d24bc11b4ef7 Mon Sep 17 00:00:00 2001 From: Bijay Pathak Date: Wed, 12 Oct 2016 10:09:49 -0700 Subject: [PATCH 058/177] [SPARK-14761][SQL] Reject invalid join methods when join columns are not specified in PySpark DataFrame join. ## What changes were proposed in this pull request? In PySpark, the invalid join type will not throw error for the following join: ```df1.join(df2, how='not-a-valid-join-type')``` The signature of the join is: ```def join(self, other, on=None, how=None):``` The existing code completely ignores the `how` parameter when `on` is `None`. This patch will process the arguments passed to join and pass in to JVM Spark SQL Analyzer, which will validate the join type passed. ## How was this patch tested? Used manual and existing test suites. Author: Bijay Pathak Closes #15409 from bkpathak/SPARK-14761. --- python/pyspark/sql/dataframe.py | 31 +++++++++++++++---------------- python/pyspark/sql/tests.py | 6 ++++++ 2 files changed, 21 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 14e80ea4615ef..ce277eb204d13 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -661,25 +661,24 @@ def join(self, other, on=None, how=None): if on is not None and not isinstance(on, list): on = [on] - if on is None or len(on) == 0: - jdf = self._jdf.crossJoin(other._jdf) - elif isinstance(on[0], basestring): - if how is None: - jdf = self._jdf.join(other._jdf, self._jseq(on), "inner") + if on is not None: + if isinstance(on[0], basestring): + on = self._jseq(on) else: - assert isinstance(how, basestring), "how should be basestring" - jdf = self._jdf.join(other._jdf, self._jseq(on), how) + assert isinstance(on[0], Column), "on should be Column or list of Column" + if len(on) > 1: + on = reduce(lambda x, y: x.__and__(y), on) + else: + on = on[0] + on = on._jc + + if on is None and how is None: + jdf = self._jdf.crossJoin(other._jdf) else: - assert isinstance(on[0], Column), "on should be Column or list of Column" - if len(on) > 1: - on = reduce(lambda x, y: x.__and__(y), on) - else: - on = on[0] if how is None: - jdf = self._jdf.join(other._jdf, on._jc, "inner") - else: - assert isinstance(how, basestring), "how should be basestring" - jdf = self._jdf.join(other._jdf, on._jc, how) + how = "inner" + assert isinstance(how, basestring), "how should be basestring" + jdf = self._jdf.join(other._jdf, on, how) return DataFrame(jdf, self.sql_ctx) @since(1.6) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 86c590dae34d7..61674a8a7ed65 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1508,6 +1508,12 @@ def test_toDF_with_schema_string(self): self.assertEqual(df.schema.simpleString(), "struct") self.assertEqual(df.collect(), [Row(key=i) for i in range(100)]) + # Regression test for invalid join methods when on is None, Spark-14761 + def test_invalid_join_method(self): + df1 = self.spark.createDataFrame([("Alice", 5), ("Bob", 8)], ["name", "age"]) + df2 = self.spark.createDataFrame([("Alice", 80), ("Bob", 90)], ["name", "height"]) + self.assertRaises(IllegalArgumentException, lambda: df1.join(df2, how="invalid-join-type")) + def test_conf(self): spark = self.spark spark.conf.set("bogo", "sipeo") From d5580ebaa086b9feb72d5428f24c5b60cd7da745 Mon Sep 17 00:00:00 2001 From: prigarg Date: Wed, 12 Oct 2016 10:14:45 -0700 Subject: [PATCH 059/177] [SPARK-17884][SQL] To resolve Null pointer exception when casting from empty string to interval type. ## What changes were proposed in this pull request? This change adds a check in castToInterval method of Cast expression , such that if converted value is null , then isNull variable should be set to true. Earlier, the expression Cast(Literal(), CalendarIntervalType) was throwing NullPointerException because of the above mentioned reason. ## How was this patch tested? Added test case in CastSuite.scala jira entry for detail: https://issues.apache.org/jira/browse/SPARK-17884 Author: prigarg Closes #15449 from priyankagargnitk/SPARK-17884. --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 7 ++++++- .../apache/spark/sql/catalyst/expressions/CastSuite.scala | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1314c416510dc..58fd65f62ffe7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -657,7 +657,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] def castToIntervalCode(from: DataType): CastFunction = from match { case StringType => (c, evPrim, evNull) => - s"$evPrim = CalendarInterval.fromString($c.toString());" + s"""$evPrim = CalendarInterval.fromString($c.toString()); + if(${evPrim} == null) { + ${evNull} = true; + } + """.stripMargin + } private[this] def decimalToTimestampCode(d: String): String = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 5c35baacef2fa..b748595fc4f2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -767,6 +767,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { test("cast between string and interval") { import org.apache.spark.unsafe.types.CalendarInterval + checkEvaluation(Cast(Literal(""), CalendarIntervalType), null) checkEvaluation(Cast(Literal("interval -3 month 7 hours"), CalendarIntervalType), new CalendarInterval(-3, 7 * CalendarInterval.MICROS_PER_HOUR)) checkEvaluation(Cast(Literal.create( From 5cc503f4fe9737a4c7947a80eecac053780606df Mon Sep 17 00:00:00 2001 From: Hossein Date: Wed, 12 Oct 2016 10:32:38 -0700 Subject: [PATCH 060/177] [SPARK-17790][SPARKR] Support for parallelizing R data.frame larger than 2GB ## What changes were proposed in this pull request? If the R data structure that is being parallelized is larger than `INT_MAX` we use files to transfer data to JVM. The serialization protocol mimics Python pickling. This allows us to simply call `PythonRDD.readRDDFromFile` to create the RDD. I tested this on my MacBook. Following code works with this patch: ```R intMax <- .Machine$integer.max largeVec <- 1:intMax rdd <- SparkR:::parallelize(sc, largeVec, 2) ``` ## How was this patch tested? * [x] Unit tests Author: Hossein Closes #15375 from falaki/SPARK-17790. --- R/pkg/R/context.R | 45 ++++++++++++++++++- R/pkg/inst/tests/testthat/test_sparkSQL.R | 11 +++++ .../apache/spark/api/r/RBackendHandler.scala | 2 +- .../scala/org/apache/spark/api/r/RRDD.scala | 13 ++++++ 4 files changed, 68 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index fe2f3e3d10a9b..438d77a388f0e 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -87,6 +87,10 @@ objectFile <- function(sc, path, minPartitions = NULL) { #' in the list are split into \code{numSlices} slices and distributed to nodes #' in the cluster. #' +#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MB), the function +#' will write it to disk and send the file name to JVM. Also to make sure each slice is not +#' larger than that limit, number of slices may be increased. +#' #' @param sc SparkContext to use #' @param coll collection to parallelize #' @param numSlices number of partitions to create in the RDD @@ -120,6 +124,11 @@ parallelize <- function(sc, coll, numSlices = 1) { coll <- as.list(coll) } + sizeLimit <- getMaxAllocationLimit(sc) + objectSize <- object.size(coll) + + # For large objects we make sure the size of each slice is also smaller than sizeLimit + numSlices <- max(numSlices, ceiling(objectSize / sizeLimit)) if (numSlices > length(coll)) numSlices <- length(coll) @@ -130,12 +139,44 @@ parallelize <- function(sc, coll, numSlices = 1) { # 2-tuples of raws serializedSlices <- lapply(slices, serialize, connection = NULL) - jrdd <- callJStatic("org.apache.spark.api.r.RRDD", - "createRDDFromArray", sc, serializedSlices) + # The PRC backend cannot handle arguments larger than 2GB (INT_MAX) + # If serialized data is safely less than that threshold we send it over the PRC channel. + # Otherwise, we write it to a file and send the file name + if (objectSize < sizeLimit) { + jrdd <- callJStatic("org.apache.spark.api.r.RRDD", "createRDDFromArray", sc, serializedSlices) + } else { + fileName <- writeToTempFile(serializedSlices) + jrdd <- tryCatch(callJStatic( + "org.apache.spark.api.r.RRDD", "createRDDFromFile", sc, fileName, as.integer(numSlices)), + finally = { + file.remove(fileName) + }) + } RDD(jrdd, "byte") } +getMaxAllocationLimit <- function(sc) { + conf <- callJMethod(sc, "getConf") + as.numeric( + callJMethod(conf, + "get", + "spark.r.maxAllocationLimit", + toString(.Machine$integer.max / 10) # Default to a safe value: 200MB + )) +} + +writeToTempFile <- function(serializedSlices) { + fileName <- tempfile() + conn <- file(fileName, "wb") + for (slice in serializedSlices) { + writeBin(as.integer(length(slice)), conn, endian = "big") + writeBin(slice, conn, endian = "big") + } + close(conn) + fileName +} + #' Include this specified package on all workers #' #' This function can be used to include a package on all workers before the diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 61554248ee8f8..af81d0586e0a6 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -208,6 +208,17 @@ test_that("create DataFrame from RDD", { unsetHiveContext() }) +test_that("createDataFrame uses files for large objects", { + # To simulate a large file scenario, we set spark.r.maxAllocationLimit to a smaller value + conf <- callJMethod(sparkSession, "conf") + callJMethod(conf, "set", "spark.r.maxAllocationLimit", "100") + df <- createDataFrame(iris) + + # Resetting the conf back to default value + callJMethod(conf, "set", "spark.r.maxAllocationLimit", toString(.Machine$integer.max / 10)) + expect_equal(dim(df), dim(iris)) +}) + test_that("read/write csv as DataFrame", { csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") mockLinesCsv <- c("year,make,model,comment,blank", diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 7d5348266bf6e..1422ef888fd4a 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -168,7 +168,7 @@ private[r] class RBackendHandler(server: RBackend) } } catch { case e: Exception => - logError(s"$methodName on $objId failed") + logError(s"$methodName on $objId failed", e) writeInt(dos, -1) // Writing the error message of the cause for the exception. This will be returned // to user in the R process. diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 59c8429c80172..a1a5eb8cf55e8 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.python.PythonRDD import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -140,4 +141,16 @@ private[r] object RRDD { def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = { JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length)) } + + /** + * Create an RRDD given a temporary file name. This is used to create RRDD when parallelize is + * called on large R objects. + * + * @param fileName name of temporary file on driver machine + * @param parallelism number of slices defaults to 4 + */ + def createRDDFromFile(jsc: JavaSparkContext, fileName: String, parallelism: Int): + JavaRDD[Array[Byte]] = { + PythonRDD.readRDDFromFile(jsc, fileName, parallelism) + } } From f8062b63fc5e07a6bf4c153a74a966602865fa6e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 12 Oct 2016 11:14:03 -0700 Subject: [PATCH 061/177] [SPARK-17840][DOCS] Add some pointers for wiki/CONTRIBUTING.md in README.md and some warnings in PULL_REQUEST_TEMPLATE ## What changes were proposed in this pull request? Link to contributing wiki in PR template, README.md ## How was this patch tested? Doc-only change, tested by Jekyll Author: Sean Owen Closes #15429 from srowen/SPARK-17840. --- .github/PULL_REQUEST_TEMPLATE | 4 +--- README.md | 5 +++++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE b/.github/PULL_REQUEST_TEMPLATE index 989e95ccd0135..0e41cf1826453 100644 --- a/.github/PULL_REQUEST_TEMPLATE +++ b/.github/PULL_REQUEST_TEMPLATE @@ -2,11 +2,9 @@ (Please fill in changes proposed in this fix) - ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) - - (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) +Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. diff --git a/README.md b/README.md index c77c429e577cd..dd7d0e22495b3 100644 --- a/README.md +++ b/README.md @@ -97,3 +97,8 @@ building for particular Hive and Hive Thriftserver distributions. Please refer to the [Configuration Guide](http://spark.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. + +## Contributing + +Please review the [Contribution to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) +wiki for information on how to get started contributing to the project. From eb69335cdbce54f943ae6168aed39687f40e53ed Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 12 Oct 2016 11:59:01 -0700 Subject: [PATCH 062/177] [BUILD] Closing stale PRs Closes #15303 Closes #15078 Closes #15080 Closes #15135 Closes #14565 Closes #12355 Closes #15404 Author: Sean Owen Closes #15451 from srowen/CloseStalePRs. From 47776e7c0c68590fe446cef910900b1aaead06f9 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 12 Oct 2016 13:51:53 -0700 Subject: [PATCH 063/177] [SPARK-17850][CORE] Add a flag to ignore corrupt files ## What changes were proposed in this pull request? Add a flag to ignore corrupt files. For Spark core, the configuration is `spark.files.ignoreCorruptFiles`. For Spark SQL, it's `spark.sql.files.ignoreCorruptFiles`. ## How was this patch tested? The added unit tests Author: Shixiong Zhu Closes #15422 from zsxwing/SPARK-17850. --- .../spark/internal/config/package.scala | 5 ++ .../org/apache/spark/rdd/HadoopRDD.scala | 8 ++- .../org/apache/spark/rdd/NewHadoopRDD.scala | 10 ++- .../scala/org/apache/spark/FileSuite.scala | 62 ++++++++++++++++++- .../execution/datasources/FileScanRDD.scala | 30 ++++++++- .../apache/spark/sql/internal/SQLConf.scala | 8 +++ .../datasources/FileSourceStrategySuite.scala | 37 ++++++++++- 7 files changed, 153 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5a710158db89f..517fc3e9e9c77 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -156,4 +156,9 @@ package object config { .doc("Port to use for the block managed on the driver.") .fallbackConf(BLOCK_MANAGER_PORT) + private[spark] val IGNORE_CORRUPT_FILES = ConfigBuilder("spark.files.ignoreCorruptFiles") + .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " + + "encountering corrupt files and contents that have been read will still be returned.") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 4640b5dc2f654..e1cf3938de098 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.io.EOFException +import java.io.IOException import java.text.SimpleDateFormat import java.util.Date @@ -43,6 +43,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} import org.apache.spark.storage.StorageLevel @@ -139,6 +140,8 @@ class HadoopRDD[K, V]( private val shouldCloneJobConf = sparkContext.conf.getBoolean("spark.hadoop.cloneConf", false) + private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES) + // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { val conf: Configuration = broadcastedConf.value.value @@ -253,8 +256,7 @@ class HadoopRDD[K, V]( try { finished = !reader.next(key, value) } catch { - case eof: EOFException => - finished = true + case e: IOException if ignoreCorruptFiles => finished = true } if (!finished) { inputMetrics.incRecordsRead(1) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 1c7aec919bdc4..baf31fb658870 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.rdd +import java.io.IOException import java.text.SimpleDateFormat import java.util.Date @@ -33,6 +34,7 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager} @@ -85,6 +87,8 @@ class NewHadoopRDD[K, V]( private val shouldCloneJobConf = sparkContext.conf.getBoolean("spark.hadoop.cloneConf", false) + private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES) + def getConf: Configuration = { val conf: Configuration = confBroadcast.value.value if (shouldCloneJobConf) { @@ -179,7 +183,11 @@ class NewHadoopRDD[K, V]( override def hasNext: Boolean = { if (!finished && !havePair) { - finished = !reader.nextKeyValue + try { + finished = !reader.nextKeyValue + } catch { + case e: IOException if ignoreCorruptFiles => finished = true + } if (finished) { // Close and release the reader here; close() will also be called when the task // completes, but for tasks that read from many files, it helps to release the diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 993834f8d7d42..cc52bb1d23cd5 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark -import java.io.{File, FileWriter} +import java.io._ +import java.util.zip.GZIPOutputStream import scala.io.Source @@ -29,6 +30,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInp import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} import org.apache.spark.input.PortableDataStream +import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -541,4 +543,62 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { }.collect() assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001")) } + + test("spark.files.ignoreCorruptFiles should work both HadoopRDD and NewHadoopRDD") { + val inputFile = File.createTempFile("input-", ".gz") + try { + // Create a corrupt gzip file + val byteOutput = new ByteArrayOutputStream() + val gzip = new GZIPOutputStream(byteOutput) + try { + gzip.write(Array[Byte](1, 2, 3, 4)) + } finally { + gzip.close() + } + val bytes = byteOutput.toByteArray + val o = new FileOutputStream(inputFile) + try { + // It's corrupt since we only write half of bytes into the file. + o.write(bytes.take(bytes.length / 2)) + } finally { + o.close() + } + + // Reading a corrupt gzip file should throw EOFException + sc = new SparkContext("local", "test") + // Test HadoopRDD + var e = intercept[SparkException] { + sc.textFile(inputFile.toURI.toString).collect() + } + assert(e.getCause.isInstanceOf[EOFException]) + assert(e.getCause.getMessage === "Unexpected end of input stream") + // Test NewHadoopRDD + e = intercept[SparkException] { + sc.newAPIHadoopFile( + inputFile.toURI.toString, + classOf[NewTextInputFormat], + classOf[LongWritable], + classOf[Text]).collect() + } + assert(e.getCause.isInstanceOf[EOFException]) + assert(e.getCause.getMessage === "Unexpected end of input stream") + sc.stop() + + val conf = new SparkConf().set(IGNORE_CORRUPT_FILES, true) + sc = new SparkContext("local", "test", conf) + // Test HadoopRDD + assert(sc.textFile(inputFile.toURI.toString).collect().isEmpty) + // Test NewHadoopRDD + assert { + sc.newAPIHadoopFile( + inputFile.toURI.toString, + classOf[NewTextInputFormat], + classOf[LongWritable], + classOf[Text]).collect().isEmpty + } + } finally { + inputFile.delete() + } + } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index c66da3a83198d..89944570df662 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources +import java.io.IOException + import scala.collection.mutable import org.apache.spark.{Partition => RDDPartition, TaskContext} @@ -25,6 +27,7 @@ import org.apache.spark.rdd.{InputFileNameHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.vectorized.ColumnarBatch +import org.apache.spark.util.NextIterator /** * A part (i.e. "block") of a single file that should be read, along with partition column values @@ -62,6 +65,8 @@ class FileScanRDD( @transient val filePartitions: Seq[FilePartition]) extends RDD[InternalRow](sparkSession.sparkContext, Nil) { + private val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles + override def compute(split: RDDPartition, context: TaskContext): Iterator[InternalRow] = { val iterator = new Iterator[Object] with AutoCloseable { private val inputMetrics = context.taskMetrics().inputMetrics @@ -119,7 +124,30 @@ class FileScanRDD( InputFileNameHolder.setInputFileName(currentFile.filePath) try { - currentIterator = readFunction(currentFile) + if (ignoreCorruptFiles) { + currentIterator = new NextIterator[Object] { + private val internalIter = readFunction(currentFile) + + override def getNext(): AnyRef = { + try { + if (internalIter.hasNext) { + internalIter.next() + } else { + finished = true + null + } + } catch { + case e: IOException => + finished = true + null + } + } + + override def close(): Unit = {} + } + } else { + currentIterator = readFunction(currentFile) + } } catch { case e: java.io.FileNotFoundException => throw new java.io.FileNotFoundException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8cbfc4c7628f7..9e7c1ec211893 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -576,6 +576,12 @@ object SQLConf { .doubleConf .createWithDefault(0.05) + val IGNORE_CORRUPT_FILES = SQLConfigBuilder("spark.sql.files.ignoreCorruptFiles") + .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " + + "encountering corrupt files and contents that have been read will still be returned.") + .booleanConf + .createWithDefault(false) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -743,6 +749,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString + def ignoreCorruptFiles: Boolean = getConf(IGNORE_CORRUPT_FILES) + override def orderByOrdinal: Boolean = getConf(ORDER_BY_ORDINAL) override def groupByOrdinal: Boolean = getConf(GROUP_BY_ORDINAL) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 45411fa0656cd..c5deb31fec183 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.execution.datasources -import java.io.File +import java.io._ import java.util.concurrent.atomic.AtomicInteger +import java.util.zip.GZIPOutputStream import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path, RawLocalFileSystem} @@ -441,6 +442,40 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } } + test("spark.files.ignoreCorruptFiles should work in SQL") { + val inputFile = File.createTempFile("input-", ".gz") + try { + // Create a corrupt gzip file + val byteOutput = new ByteArrayOutputStream() + val gzip = new GZIPOutputStream(byteOutput) + try { + gzip.write(Array[Byte](1, 2, 3, 4)) + } finally { + gzip.close() + } + val bytes = byteOutput.toByteArray + val o = new FileOutputStream(inputFile) + try { + // It's corrupt since we only write half of bytes into the file. + o.write(bytes.take(bytes.length / 2)) + } finally { + o.close() + } + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { + val e = intercept[SparkException] { + spark.read.text(inputFile.toURI.toString).collect() + } + assert(e.getCause.isInstanceOf[EOFException]) + assert(e.getCause.getMessage === "Unexpected end of input stream") + } + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") { + assert(spark.read.text(inputFile.toURI.toString).collect().isEmpty) + } + } finally { + inputFile.delete() + } + } + // Helpers for checking the arguments passed to the FileFormat. protected val checkPartitionSchema = From 9ce7d3e542e786c62f047c13f3001e178f76e06a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 12 Oct 2016 16:43:03 -0500 Subject: [PATCH 064/177] [SPARK-17675][CORE] Expand Blacklist for TaskSets ## What changes were proposed in this pull request? This is a step along the way to SPARK-8425. To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for * (task, executor) pairs (this already exists via an undocumented config) * (task, node) * (taskset, executor) * (taskset, node) Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster. Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be). ## How was this patch tested? Added unit tests, run tests via jenkins. Author: Imran Rashid Author: mwws Closes #15249 from squito/taskset_blacklist_only. --- .../scala/org/apache/spark/SparkConf.scala | 4 +- .../org/apache/spark/TaskEndReason.scala | 11 + .../spark/internal/config/package.scala | 45 +++ .../spark/scheduler/BlacklistTracker.scala | 114 ++++++++ .../scheduler/ExecutorFailuresInTaskSet.scala | 50 ++++ .../spark/scheduler/TaskSchedulerImpl.scala | 31 +- .../spark/scheduler/TaskSetBlacklist.scala | 128 ++++++++ .../spark/scheduler/TaskSetManager.scala | 276 +++++++++--------- .../scheduler/BlacklistIntegrationSuite.scala | 52 ++-- .../scheduler/BlacklistTrackerSuite.scala | 81 +++++ .../scheduler/SchedulerIntegrationSuite.scala | 4 +- .../scheduler/TaskSchedulerImplSuite.scala | 22 +- .../scheduler/TaskSetBlacklistSuite.scala | 163 +++++++++++ .../spark/scheduler/TaskSetManagerSuite.scala | 131 ++++++++- .../KryoSerializerDistributedSuite.scala | 4 +- docs/configuration.md | 43 +++ .../sql/execution/ui/SQLListenerSuite.scala | 3 +- 17 files changed, 964 insertions(+), 198 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 51a699f41d15d..c9c342df82c97 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -636,7 +636,9 @@ private[spark] object SparkConf extends Logging { "Please use spark.kryoserializer.buffer instead. The default value for " + "spark.kryoserializer.buffer.mb was previously specified as '0.064'. Fractional values " + "are no longer accepted. To specify the equivalent now, one may use '64k'."), - DeprecatedConfig("spark.rpc", "2.0", "Not used any more.") + DeprecatedConfig("spark.rpc", "2.0", "Not used any more."), + DeprecatedConfig("spark.scheduler.executorTaskBlacklistTime", "2.1.0", + "Please use the new blacklisting options, spark.blacklist.*") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 42690844f9610..7ca3c103dbf5b 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -92,6 +92,16 @@ case class FetchFailed( s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " + s"message=\n$message\n)" } + + /** + * Fetch failures lead to a different failure handling path: (1) we don't abort the stage after + * 4 task failures, instead we immediately go back to the stage which generated the map output, + * and regenerate the missing data. (2) we don't count fetch failures for blacklisting, since + * presumably its not the fault of the executor where the task ran, but the executor which + * stored the data. This is especially important because we we might rack up a bunch of + * fetch-failures in rapid succession, on all nodes of the cluster, due to one bad node. + */ + override def countTowardsTaskFailures: Boolean = false } /** @@ -204,6 +214,7 @@ case object TaskResultLost extends TaskFailedReason { @DeveloperApi case object TaskKilled extends TaskFailedReason { override def toErrorString: String = "TaskKilled (killed intentionally)" + override def countTowardsTaskFailures: Boolean = false } /** diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 517fc3e9e9c77..497ca92c7bc60 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -17,6 +17,8 @@ package org.apache.spark.internal +import java.util.concurrent.TimeUnit + import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.ByteUnit import org.apache.spark.util.Utils @@ -91,6 +93,49 @@ package object config { .toSequence .createWithDefault(Nil) + private[spark] val MAX_TASK_FAILURES = + ConfigBuilder("spark.task.maxFailures") + .intConf + .createWithDefault(4) + + // Blacklist confs + private[spark] val BLACKLIST_ENABLED = + ConfigBuilder("spark.blacklist.enabled") + .booleanConf + .createOptional + + private[spark] val MAX_TASK_ATTEMPTS_PER_EXECUTOR = + ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerExecutor") + .intConf + .createWithDefault(1) + + private[spark] val MAX_TASK_ATTEMPTS_PER_NODE = + ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerNode") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILURES_PER_EXEC_STAGE = + ConfigBuilder("spark.blacklist.stage.maxFailedTasksPerExecutor") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE = + ConfigBuilder("spark.blacklist.stage.maxFailedExecutorsPerNode") + .intConf + .createWithDefault(2) + + private[spark] val BLACKLIST_TIMEOUT_CONF = + ConfigBuilder("spark.blacklist.timeout") + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + + private[spark] val BLACKLIST_LEGACY_TIMEOUT_CONF = + ConfigBuilder("spark.scheduler.executorTaskBlacklistTime") + .internal() + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + // End blacklist confs + private[spark] val LISTENER_BUS_EVENT_QUEUE_SIZE = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.size") .intConf diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala new file mode 100644 index 0000000000000..fca4c6d37e446 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.util.Utils + +private[scheduler] object BlacklistTracker extends Logging { + + private val DEFAULT_TIMEOUT = "1h" + + /** + * Returns true if the blacklist is enabled, based on checking the configuration in the following + * order: + * 1. Is it specifically enabled or disabled? + * 2. Is it enabled via the legacy timeout conf? + * 3. Default is off + */ + def isBlacklistEnabled(conf: SparkConf): Boolean = { + conf.get(config.BLACKLIST_ENABLED) match { + case Some(enabled) => + enabled + case None => + // if they've got a non-zero setting for the legacy conf, always enable the blacklist, + // otherwise, use the default. + val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key + conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).exists { legacyTimeout => + if (legacyTimeout == 0) { + logWarning(s"Turning off blacklisting due to legacy configuration: $legacyKey == 0") + false + } else { + logWarning(s"Turning on blacklisting due to legacy configuration: $legacyKey > 0") + true + } + } + } + } + + def getBlacklistTimeout(conf: SparkConf): Long = { + conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse { + conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse { + Utils.timeStringAsMs(DEFAULT_TIMEOUT) + } + } + } + + /** + * Verify that blacklist configurations are consistent; if not, throw an exception. Should only + * be called if blacklisting is enabled. + * + * The configuration for the blacklist is expected to adhere to a few invariants. Default + * values follow these rules of course, but users may unwittingly change one configuration + * without making the corresponding adjustment elsewhere. This ensures we fail-fast when + * there are such misconfigurations. + */ + def validateBlacklistConfs(conf: SparkConf): Unit = { + + def mustBePos(k: String, v: String): Unit = { + throw new IllegalArgumentException(s"$k was $v, but must be > 0.") + } + + Seq( + config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, + config.MAX_TASK_ATTEMPTS_PER_NODE, + config.MAX_FAILURES_PER_EXEC_STAGE, + config.MAX_FAILED_EXEC_PER_NODE_STAGE + ).foreach { config => + val v = conf.get(config) + if (v <= 0) { + mustBePos(config.key, v.toString) + } + } + + val timeout = getBlacklistTimeout(conf) + if (timeout <= 0) { + // first, figure out where the timeout came from, to include the right conf in the message. + conf.get(config.BLACKLIST_TIMEOUT_CONF) match { + case Some(t) => + mustBePos(config.BLACKLIST_TIMEOUT_CONF.key, timeout.toString) + case None => + mustBePos(config.BLACKLIST_LEGACY_TIMEOUT_CONF.key, timeout.toString) + } + } + + val maxTaskFailures = conf.get(config.MAX_TASK_FAILURES) + val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + + if (maxNodeAttempts >= maxTaskFailures) { + throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_FAILURES.key} " + + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + + s"Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala new file mode 100644 index 0000000000000..20ab27d127aba --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler + +import scala.collection.mutable.HashMap + +/** + * Small helper for tracking failed tasks for blacklisting purposes. Info on all failures on one + * executor, within one task set. + */ +private[scheduler] class ExecutorFailuresInTaskSet(val node: String) { + /** + * Mapping from index of the tasks in the taskset, to the number of times it has failed on this + * executor. + */ + val taskToFailureCount = HashMap[Int, Int]() + + def updateWithFailure(taskIndex: Int): Unit = { + val prevFailureCount = taskToFailureCount.getOrElse(taskIndex, 0) + taskToFailureCount(taskIndex) = prevFailureCount + 1 + } + + def numUniqueTasksWithFailures: Int = taskToFailureCount.size + + /** + * Return the number of times this executor has failed on the given task index. + */ + def getNumTaskFailures(index: Int): Int = { + taskToFailureCount.getOrElse(index, 0) + } + + override def toString(): String = { + s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " + + s"tasksToFailureCount = $taskToFailureCount" + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 0ad4730fe20a6..3e3f1ad031e66 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -22,14 +22,14 @@ import java.util.{Timer, TimerTask} import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet +import scala.collection.Set +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.scheduler.local.LocalSchedulerBackend @@ -57,7 +57,7 @@ private[spark] class TaskSchedulerImpl( isLocal: Boolean = false) extends TaskScheduler with Logging { - def this(sc: SparkContext) = this(sc, sc.conf.getInt("spark.task.maxFailures", 4)) + def this(sc: SparkContext) = this(sc, sc.conf.get(config.MAX_TASK_FAILURES)) val conf = sc.conf @@ -100,7 +100,7 @@ private[spark] class TaskSchedulerImpl( // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host - protected val executorsByHost = new HashMap[String, HashSet[String]] + protected val hostToExecutors = new HashMap[String, HashSet[String]] protected val hostsByRack = new HashMap[String, HashSet[String]] @@ -243,8 +243,8 @@ private[spark] class TaskSchedulerImpl( } } manager.parent.removeSchedulable(manager) - logInfo("Removed TaskSet %s, whose tasks have all completed, from pool %s" - .format(manager.taskSet.id, manager.parent.name)) + logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + + s" ${manager.parent.name}") } private def resourceOfferSingleTaskSet( @@ -291,11 +291,11 @@ private[spark] class TaskSchedulerImpl( // Also track if new executor is added var newExecAvail = false for (o <- offers) { - if (!executorsByHost.contains(o.host)) { - executorsByHost(o.host) = new HashSet[String]() + if (!hostToExecutors.contains(o.host)) { + hostToExecutors(o.host) = new HashSet[String]() } if (!executorIdToTaskCount.contains(o.executorId)) { - executorsByHost(o.host) += o.executorId + hostToExecutors(o.host) += o.executorId executorAdded(o.executorId, o.host) executorIdToHost(o.executorId) = o.host executorIdToTaskCount(o.executorId) = 0 @@ -334,7 +334,7 @@ private[spark] class TaskSchedulerImpl( } while (launchedTaskAtCurrentMaxLocality) } if (!launchedAnyTask) { - taskSet.abortIfCompletelyBlacklisted(executorIdToHost.keys) + taskSet.abortIfCompletelyBlacklisted(hostToExecutors) } } @@ -542,10 +542,10 @@ private[spark] class TaskSchedulerImpl( executorIdToTaskCount -= executorId val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) + val execs = hostToExecutors.getOrElse(host, new HashSet) execs -= executorId if (execs.isEmpty) { - executorsByHost -= host + hostToExecutors -= host for (rack <- getRackForHost(host); hosts <- hostsByRack.get(rack)) { hosts -= host if (hosts.isEmpty) { @@ -565,11 +565,11 @@ private[spark] class TaskSchedulerImpl( } def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { - executorsByHost.get(host).map(_.toSet) + hostToExecutors.get(host).map(_.toSet) } def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { - executorsByHost.contains(host) + hostToExecutors.contains(host) } def hasHostAliveOnRack(rack: String): Boolean = synchronized { @@ -662,5 +662,4 @@ private[spark] object TaskSchedulerImpl { retval.toList } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala new file mode 100644 index 0000000000000..f4b0f55b7686a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.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.scheduler + +import scala.collection.mutable.{HashMap, HashSet} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.util.Clock + +/** + * Handles blacklisting executors and nodes within a taskset. This includes blacklisting specific + * (task, executor) / (task, nodes) pairs, and also completely blacklisting executors and nodes + * for the entire taskset. + * + * THREADING: This class is a helper to [[TaskSetManager]]; as with the methods in + * [[TaskSetManager]] this class is designed only to be called from code with a lock on the + * TaskScheduler (e.g. its event handlers). It should not be called from other threads. + */ +private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, val clock: Clock) + extends Logging { + + private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR) + private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) + private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) + + /** + * A map from each executor to the task failures on that executor. + */ + val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]() + + /** + * Map from node to all executors on it with failures. Needed because we want to know about + * executors on a node even after they have died. (We don't want to bother tracking the + * node -> execs mapping in the usual case when there aren't any failures). + */ + private val nodeToExecsWithFailures = new HashMap[String, HashSet[String]]() + private val nodeToBlacklistedTaskIndexes = new HashMap[String, HashSet[Int]]() + private val blacklistedExecs = new HashSet[String]() + private val blacklistedNodes = new HashSet[String]() + + /** + * Return true if this executor is blacklisted for the given task. This does *not* + * need to return true if the executor is blacklisted for the entire stage. + * That is to keep this method as fast as possible in the inner-loop of the + * scheduler, where those filters will have already been applied. + */ + def isExecutorBlacklistedForTask(executorId: String, index: Int): Boolean = { + execToFailures.get(executorId).exists { execFailures => + execFailures.getNumTaskFailures(index) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR + } + } + + def isNodeBlacklistedForTask(node: String, index: Int): Boolean = { + nodeToBlacklistedTaskIndexes.get(node).exists(_.contains(index)) + } + + /** + * Return true if this executor is blacklisted for the given stage. Completely ignores + * anything to do with the node the executor is on. That + * is to keep this method as fast as possible in the inner-loop of the scheduler, where those + * filters will already have been applied. + */ + def isExecutorBlacklistedForTaskSet(executorId: String): Boolean = { + blacklistedExecs.contains(executorId) + } + + def isNodeBlacklistedForTaskSet(node: String): Boolean = { + blacklistedNodes.contains(node) + } + + private[scheduler] def updateBlacklistForFailedTask( + host: String, + exec: String, + index: Int): Unit = { + val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) + execFailures.updateWithFailure(index) + + // check if this task has also failed on other executors on the same host -- if its gone + // over the limit, blacklist this task from the entire host. + val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) + execsWithFailuresOnNode += exec + val failuresOnHost = execsWithFailuresOnNode.toIterator.flatMap { exec => + execToFailures.get(exec).map { failures => + // We count task attempts here, not the number of unique executors with failures. This is + // because jobs are aborted based on the number task attempts; if we counted unique + // executors, it would be hard to config to ensure that you try another + // node before hitting the max number of task failures. + failures.getNumTaskFailures(index) + } + }.sum + if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { + nodeToBlacklistedTaskIndexes.getOrElseUpdate(host, new HashSet()) += index + } + + // Check if enough tasks have failed on the executor to blacklist it for the entire stage. + if (execFailures.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) { + if (blacklistedExecs.add(exec)) { + logInfo(s"Blacklisting executor ${exec} for stage $stageId") + // This executor has been pushed into the blacklist for this stage. Let's check if it + // pushes the whole node into the blacklist. + val blacklistedExecutorsOnNode = + execsWithFailuresOnNode.filter(blacklistedExecs.contains(_)) + if (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { + if (blacklistedNodes.add(host)) { + logInfo(s"Blacklisting ${host} for stage $stageId") + } + } + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 226bed284a40a..9491bc7a0497e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -22,9 +22,7 @@ import java.nio.ByteBuffer import java.util.Arrays import java.util.concurrent.ConcurrentLinkedQueue -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.math.{max, min} import scala.util.control.NonFatal @@ -53,19 +51,9 @@ private[spark] class TaskSetManager( sched: TaskSchedulerImpl, val taskSet: TaskSet, val maxTaskFailures: Int, - clock: Clock = new SystemClock()) - extends Schedulable with Logging { + clock: Clock = new SystemClock()) extends Schedulable with Logging { - val conf = sched.sc.conf - - /* - * Sometimes if an executor is dead or in an otherwise invalid state, the driver - * does not realize right away leading to repeated task failures. If enabled, - * this temporarily prevents a task from re-launching on an executor where - * it just failed. - */ - private val EXECUTOR_TASK_BLACKLIST_TIMEOUT = - conf.getLong("spark.scheduler.executorTaskBlacklistTime", 0L) + private val conf = sched.sc.conf // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) @@ -83,8 +71,6 @@ private[spark] class TaskSetManager( val copiesRunning = new Array[Int](numTasks) val successful = new Array[Boolean](numTasks) private val numFailures = new Array[Int](numTasks) - // key is taskId (aka TaskInfo.index), value is a Map of executor id to when it failed - private val failedExecutors = new HashMap[Int, HashMap[String, Long]]() val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksSuccessful = 0 @@ -98,6 +84,14 @@ private[spark] class TaskSetManager( var totalResultSize = 0L var calculatedTasks = 0 + private val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = { + if (BlacklistTracker.isBlacklistEnabled(conf)) { + Some(new TaskSetBlacklist(conf, stageId, clock)) + } else { + None + } + } + val runningTasksSet = new HashSet[Long] override def runningTasks: Int = runningTasksSet.size @@ -245,12 +239,15 @@ private[spark] class TaskSetManager( * This method also cleans up any tasks in the list that have already * been launched, since we want that to happen lazily. */ - private def dequeueTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { + private def dequeueTaskFromList( + execId: String, + host: String, + list: ArrayBuffer[Int]): Option[Int] = { var indexOffset = list.size while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!executorIsBlacklisted(execId, index)) { + if (!isTaskBlacklistedOnExecOrNode(index, execId, host)) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -266,19 +263,11 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } - /** - * Is this re-execution of a failed task on an executor it already failed in before - * EXECUTOR_TASK_BLACKLIST_TIMEOUT has elapsed ? - */ - private[scheduler] def executorIsBlacklisted(execId: String, taskId: Int): Boolean = { - if (failedExecutors.contains(taskId)) { - val failed = failedExecutors.get(taskId).get - - return failed.contains(execId) && - clock.getTimeMillis() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT + private def isTaskBlacklistedOnExecOrNode(index: Int, execId: String, host: String): Boolean = { + taskSetBlacklistHelperOpt.exists { blacklist => + blacklist.isNodeBlacklistedForTask(host, index) || + blacklist.isExecutorBlacklistedForTask(execId, index) } - - false } /** @@ -292,8 +281,10 @@ private[spark] class TaskSetManager( { speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set - def canRunOnHost(index: Int): Boolean = - !hasAttemptOnHost(index, host) && !executorIsBlacklisted(execId, index) + def canRunOnHost(index: Int): Boolean = { + !hasAttemptOnHost(index, host) && + !isTaskBlacklistedOnExecOrNode(index, execId, host) + } if (!speculatableTasks.isEmpty) { // Check for process-local tasks; note that tasks can be process-local @@ -366,19 +357,19 @@ private[spark] class TaskSetManager( private def dequeueTask(execId: String, host: String, maxLocality: TaskLocality.Value) : Option[(Int, TaskLocality.Value, Boolean)] = { - for (index <- dequeueTaskFromList(execId, getPendingTasksForExecutor(execId))) { + for (index <- dequeueTaskFromList(execId, host, getPendingTasksForExecutor(execId))) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) { - for (index <- dequeueTaskFromList(execId, getPendingTasksForHost(host))) { + for (index <- dequeueTaskFromList(execId, host, getPendingTasksForHost(host))) { return Some((index, TaskLocality.NODE_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NO_PREF)) { // Look for noPref tasks after NODE_LOCAL for minimize cross-rack traffic - for (index <- dequeueTaskFromList(execId, pendingTasksWithNoPrefs)) { + for (index <- dequeueTaskFromList(execId, host, pendingTasksWithNoPrefs)) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } } @@ -386,14 +377,14 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) - index <- dequeueTaskFromList(execId, getPendingTasksForRack(rack)) + index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack)) } { return Some((index, TaskLocality.RACK_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) { - for (index <- dequeueTaskFromList(execId, allPendingTasks)) { + for (index <- dequeueTaskFromList(execId, host, allPendingTasks)) { return Some((index, TaskLocality.ANY, false)) } } @@ -421,7 +412,11 @@ private[spark] class TaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - if (!isZombie) { + val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => + blacklist.isNodeBlacklistedForTaskSet(host) || + blacklist.isExecutorBlacklistedForTaskSet(execId) + } + if (!isZombie && !offerBlacklisted) { val curTime = clock.getTimeMillis() var allowedLocality = maxLocality @@ -434,60 +429,59 @@ private[spark] class TaskSetManager( } } - dequeueTask(execId, host, allowedLocality) match { - case Some((index, taskLocality, speculative)) => - // Found a task; do some bookkeeping and return a task description - val task = tasks(index) - val taskId = sched.newTaskId() - // Do various bookkeeping - copiesRunning(index) += 1 - val attemptNum = taskAttempts(index).size - val info = new TaskInfo(taskId, index, attemptNum, curTime, - execId, host, taskLocality, speculative) - taskInfos(taskId) = info - taskAttempts(index) = info :: taskAttempts(index) - // Update our locality level for delay scheduling - // NO_PREF will not affect the variables related to delay scheduling - if (maxLocality != TaskLocality.NO_PREF) { - currentLocalityIndex = getLocalityIndex(taskLocality) - lastLaunchTime = curTime - } - // Serialize and return the task - val startTime = clock.getTimeMillis() - val serializedTask: ByteBuffer = try { - Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) - } catch { - // If the task cannot be serialized, then there's no point to re-attempt the task, - // as it will always fail. So just abort the whole task-set. - case NonFatal(e) => - val msg = s"Failed to serialize task $taskId, not attempting to retry it." - logError(msg, e) - abort(s"$msg Exception during serialization: $e") - throw new TaskNotSerializableException(e) - } - if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && - !emittedTaskSizeWarning) { - emittedTaskSizeWarning = true - logWarning(s"Stage ${task.stageId} contains a task of very large size " + - s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + - s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") - } - addRunningTask(taskId) - - // We used to log the time it takes to serialize the task, but task size is already - // a good proxy to task serialization time. - // val timeTaken = clock.getTime() - startTime - val taskName = s"task ${info.id} in stage ${taskSet.id}" - logInfo(s"Starting $taskName (TID $taskId, $host, partition ${task.partitionId}," + - s" $taskLocality, ${serializedTask.limit} bytes)") - - sched.dagScheduler.taskStarted(task, info) - return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, - taskName, index, serializedTask)) - case _ => + dequeueTask(execId, host, allowedLocality).map { case ((index, taskLocality, speculative)) => + // Found a task; do some bookkeeping and return a task description + val task = tasks(index) + val taskId = sched.newTaskId() + // Do various bookkeeping + copiesRunning(index) += 1 + val attemptNum = taskAttempts(index).size + val info = new TaskInfo(taskId, index, attemptNum, curTime, + execId, host, taskLocality, speculative) + taskInfos(taskId) = info + taskAttempts(index) = info :: taskAttempts(index) + // Update our locality level for delay scheduling + // NO_PREF will not affect the variables related to delay scheduling + if (maxLocality != TaskLocality.NO_PREF) { + currentLocalityIndex = getLocalityIndex(taskLocality) + lastLaunchTime = curTime + } + // Serialize and return the task + val startTime = clock.getTimeMillis() + val serializedTask: ByteBuffer = try { + Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + } catch { + // If the task cannot be serialized, then there's no point to re-attempt the task, + // as it will always fail. So just abort the whole task-set. + case NonFatal(e) => + val msg = s"Failed to serialize task $taskId, not attempting to retry it." + logError(msg, e) + abort(s"$msg Exception during serialization: $e") + throw new TaskNotSerializableException(e) + } + if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && + !emittedTaskSizeWarning) { + emittedTaskSizeWarning = true + logWarning(s"Stage ${task.stageId} contains a task of very large size " + + s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + + s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") + } + addRunningTask(taskId) + + // We used to log the time it takes to serialize the task, but task size is already + // a good proxy to task serialization time. + // val timeTaken = clock.getTime() - startTime + val taskName = s"task ${info.id} in stage ${taskSet.id}" + logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit} bytes)") + + sched.dagScheduler.taskStarted(task, info) + new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, + taskName, index, serializedTask) } + } else { + None } - None } private def maybeFinishTaskSet() { @@ -589,37 +583,56 @@ private[spark] class TaskSetManager( * the hang as quickly as we could have, but we'll always detect the hang eventually, and the * method is faster in the typical case. In the worst case, this method can take * O(maxTaskFailures + numTasks) time, but it will be faster when there haven't been any task - * failures (this is because the method picks on unscheduled task, and then iterates through each - * executor until it finds one that the task hasn't failed on already). + * failures (this is because the method picks one unscheduled task, and then iterates through each + * executor until it finds one that the task isn't blacklisted on). */ - private[scheduler] def abortIfCompletelyBlacklisted(executors: Iterable[String]): Unit = { - - val pendingTask: Option[Int] = { - // usually this will just take the last pending task, but because of the lazy removal - // from each list, we may need to go deeper in the list. We poll from the end because - // failed tasks are put back at the end of allPendingTasks, so we're more likely to find - // an unschedulable task this way. - val indexOffset = allPendingTasks.lastIndexWhere { indexInTaskSet => - copiesRunning(indexInTaskSet) == 0 && !successful(indexInTaskSet) - } - if (indexOffset == -1) { - None - } else { - Some(allPendingTasks(indexOffset)) - } - } + private[scheduler] def abortIfCompletelyBlacklisted( + hostToExecutors: HashMap[String, HashSet[String]]): Unit = { + taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => + // Only look for unschedulable tasks when at least one executor has registered. Otherwise, + // task sets will be (unnecessarily) aborted in cases when no executors have registered yet. + if (hostToExecutors.nonEmpty) { + // find any task that needs to be scheduled + val pendingTask: Option[Int] = { + // usually this will just take the last pending task, but because of the lazy removal + // from each list, we may need to go deeper in the list. We poll from the end because + // failed tasks are put back at the end of allPendingTasks, so we're more likely to find + // an unschedulable task this way. + val indexOffset = allPendingTasks.lastIndexWhere { indexInTaskSet => + copiesRunning(indexInTaskSet) == 0 && !successful(indexInTaskSet) + } + if (indexOffset == -1) { + None + } else { + Some(allPendingTasks(indexOffset)) + } + } - // If no executors have registered yet, don't abort the stage, just wait. We probably - // got here because a task set was added before the executors registered. - if (executors.nonEmpty) { - // take any task that needs to be scheduled, and see if we can find some executor it *could* - // run on - pendingTask.foreach { taskId => - if (executors.forall(executorIsBlacklisted(_, taskId))) { - val execs = executors.toIndexedSeq.sorted.mkString("(", ",", ")") - val partition = tasks(taskId).partitionId - abort(s"Aborting ${taskSet} because task $taskId (partition $partition)" + - s" has already failed on executors $execs, and no other executors are available.") + pendingTask.foreach { indexInTaskSet => + // try to find some executor this task can run on. Its possible that some *other* + // task isn't schedulable anywhere, but we will discover that in some later call, + // when that unschedulable task is the last task remaining. + val blacklistedEverywhere = hostToExecutors.forall { case (host, execsOnHost) => + // Check if the task can run on the node + val nodeBlacklisted = + taskSetBlacklist.isNodeBlacklistedForTaskSet(host) || + taskSetBlacklist.isNodeBlacklistedForTask(host, indexInTaskSet) + if (nodeBlacklisted) { + true + } else { + // Check if the task can run on any of the executors + execsOnHost.forall { exec => + taskSetBlacklist.isExecutorBlacklistedForTaskSet(exec) || + taskSetBlacklist.isExecutorBlacklistedForTask(exec, indexInTaskSet) + } + } + } + if (blacklistedEverywhere) { + val partition = tasks(indexInTaskSet).partitionId + abort(s"Aborting $taskSet because task $indexInTaskSet (partition $partition) " + + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior " + + s"can be configured via spark.blacklist.*.") + } } } } @@ -677,8 +690,9 @@ private[spark] class TaskSetManager( } if (!successful(index)) { tasksSuccessful += 1 - logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (%d/%d)".format( - info.id, taskSet.id, info.taskId, info.duration, info.host, tasksSuccessful, numTasks)) + logInfo(s"Finished task ${info.id} in stage ${taskSet.id} (TID ${info.taskId}) in" + + s" ${info.duration} ms on ${info.host} (executor ${info.executorId})" + + s" ($tasksSuccessful/$numTasks)") // Mark successful and stop if all the tasks have succeeded. successful(index) = true if (tasksSuccessful == numTasks) { @@ -688,7 +702,6 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } - failedExecutors.remove(index) maybeFinishTaskSet() } @@ -706,8 +719,8 @@ private[spark] class TaskSetManager( val index = info.index copiesRunning(index) -= 1 var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty - val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + - reason.toErrorString + val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}," + + s" executor ${info.executorId}): ${reason.toErrorString}" val failureException: Option[Throwable] = reason match { case fetchFailed: FetchFailed => logWarning(failureReason) @@ -715,7 +728,6 @@ private[spark] class TaskSetManager( successful(index) = true tasksSuccessful += 1 } - // Not adding to failed executors for FetchFailed. isZombie = true None @@ -751,8 +763,8 @@ private[spark] class TaskSetManager( logWarning(failureReason) } else { logInfo( - s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid) on executor ${info.host}: " + - s"${ef.className} (${ef.description}) [duplicate $dupCount]") + s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid) on ${info.host}, executor" + + s" ${info.executorId}: ${ef.className} (${ef.description}) [duplicate $dupCount]") } ef.exception @@ -766,9 +778,7 @@ private[spark] class TaskSetManager( logWarning(failureReason) None } - // always add to failed executors - failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). - put(info.executorId, clock.getTimeMillis()) + sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) if (successful(index)) { @@ -780,7 +790,9 @@ private[spark] class TaskSetManager( addPendingTask(index) } - if (!isZombie && state != TaskState.KILLED && reason.countTowardsTaskFailures) { + if (!isZombie && reason.countTowardsTaskFailures) { + taskSetBlacklistHelperOpt.foreach(_.updateBlacklistForFailedTask( + info.host, info.executorId, index)) assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 14c8b664d4d8b..f6015cd51c2bd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.scheduler -import scala.concurrent.Await import scala.concurrent.duration._ import org.apache.spark._ +import org.apache.spark.internal.config class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ @@ -42,7 +42,10 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM // Test demonstrating the issue -- without a config change, the scheduler keeps scheduling // according to locality preferences, and so the job fails - testScheduler("If preferred node is bad, without blacklist job will fail") { + testScheduler("If preferred node is bad, without blacklist job will fail", + extraConfs = Seq( + config.BLACKLIST_ENABLED.key -> "false" + )) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) withBackend(badHostBackend _) { val jobFuture = submit(rdd, (0 until 10).toArray) @@ -51,37 +54,38 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = false) } - // even with the blacklist turned on, if maxTaskFailures is not more than the number - // of executors on the bad node, then locality preferences will lead to us cycling through - // the executors on the bad node, and still failing the job testScheduler( - "With blacklist on, job will still fail if there are too many bad executors on bad host", + "With default settings, job can succeed despite multiple bad executors on node", extraConfs = Seq( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - ("spark.scheduler.executorTaskBlacklistTime", "10000000") + config.BLACKLIST_ENABLED.key -> "true", + config.MAX_TASK_FAILURES.key -> "4", + "spark.testing.nHosts" -> "2", + "spark.testing.nExecutorsPerHost" -> "5", + "spark.testing.nCoresPerExecutor" -> "10" ) ) { - val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) + // To reliably reproduce the failure that would occur without blacklisting, we have to use 1 + // task. That way, we ensure this 1 task gets rotated through enough bad executors on the host + // to fail the taskSet, before we have a bunch of different tasks fail in the executors so we + // blacklist them. + // But the point here is -- without blacklisting, we would never schedule anything on the good + // host-1 before we hit too many failures trying our preferred host-0. + val rdd = new MockRDDWithLocalityPrefs(sc, 1, Nil, badHost) withBackend(badHostBackend _) { - val jobFuture = submit(rdd, (0 until 10).toArray) + val jobFuture = submit(rdd, (0 until 1).toArray) awaitJobTermination(jobFuture, duration) } - assertDataStructuresEmpty(noFailure = false) + assertDataStructuresEmpty(noFailure = true) } - // Here we run with the blacklist on, and maxTaskFailures high enough that we'll eventually - // schedule on a good node and succeed the job + // Here we run with the blacklist on, and the default config takes care of having this + // robust to one bad node. testScheduler( "Bad node with multiple executors, job will still succeed with the right confs", extraConfs = Seq( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - ("spark.scheduler.executorTaskBlacklistTime", "10000000"), - // this has to be higher than the number of executors on the bad host - ("spark.task.maxFailures", "5"), + config.BLACKLIST_ENABLED.key -> "true", // just to avoid this test taking too long - ("spark.locality.wait", "10ms") + "spark.locality.wait" -> "10ms" ) ) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) @@ -98,9 +102,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - "spark.scheduler.executorTaskBlacklistTime" -> "10000000", + config.BLACKLIST_ENABLED.key -> "true", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "1", "spark.testing.nCoresPerExecutor" -> "1" @@ -112,9 +114,9 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM } withBackend(runBackend _) { val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) - Await.ready(jobFuture, duration) + awaitJobTermination(jobFuture, duration) val pattern = ("Aborting TaskSet 0.0 because task .* " + - "already failed on executors \\(.*\\), and no other executors are available").r + "cannot run anywhere due to node and executor blacklist").r assert(pattern.findFirstIn(failure.getMessage).isDefined, s"Couldn't find $pattern in ${failure.getMessage()}") } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala new file mode 100644 index 0000000000000..b2e7ec5df015c --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config + +class BlacklistTrackerSuite extends SparkFunSuite { + + test("blacklist still respects legacy configs") { + val conf = new SparkConf().setMaster("local") + assert(!BlacklistTracker.isBlacklistEnabled(conf)) + conf.set(config.BLACKLIST_LEGACY_TIMEOUT_CONF, 5000L) + assert(BlacklistTracker.isBlacklistEnabled(conf)) + assert(5000 === BlacklistTracker.getBlacklistTimeout(conf)) + // the new conf takes precedence, though + conf.set(config.BLACKLIST_TIMEOUT_CONF, 1000L) + assert(1000 === BlacklistTracker.getBlacklistTimeout(conf)) + + // if you explicitly set the legacy conf to 0, that also would disable blacklisting + conf.set(config.BLACKLIST_LEGACY_TIMEOUT_CONF, 0L) + assert(!BlacklistTracker.isBlacklistEnabled(conf)) + // but again, the new conf takes precendence + conf.set(config.BLACKLIST_ENABLED, true) + assert(BlacklistTracker.isBlacklistEnabled(conf)) + assert(1000 === BlacklistTracker.getBlacklistTimeout(conf)) + } + + test("check blacklist configuration invariants") { + val conf = new SparkConf().setMaster("yarn-cluster") + Seq( + (2, 2), + (2, 3) + ).foreach { case (maxTaskFailures, maxNodeAttempts) => + conf.set(config.MAX_TASK_FAILURES, maxTaskFailures) + conf.set(config.MAX_TASK_ATTEMPTS_PER_NODE.key, maxNodeAttempts.toString) + val excMsg = intercept[IllegalArgumentException] { + BlacklistTracker.validateBlacklistConfs(conf) + }.getMessage() + assert(excMsg === s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_FAILURES.key} " + + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + + s"Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") + } + + conf.remove(config.MAX_TASK_FAILURES) + conf.remove(config.MAX_TASK_ATTEMPTS_PER_NODE) + + Seq( + config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, + config.MAX_TASK_ATTEMPTS_PER_NODE, + config.MAX_FAILURES_PER_EXEC_STAGE, + config.MAX_FAILED_EXEC_PER_NODE_STAGE, + config.BLACKLIST_TIMEOUT_CONF + ).foreach { config => + conf.set(config.key, "0") + val excMsg = intercept[IllegalArgumentException] { + BlacklistTracker.validateBlacklistConfs(conf) + }.getMessage() + assert(excMsg.contains(s"${config.key} was 0, but must be > 0.")) + conf.remove(config) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 5cd548bbc72d9..c28aa06623a60 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -620,9 +620,9 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) } + assertDataStructuresEmpty() assert(results === (0 until 10).map { idx => idx -> (42 + idx) }.toMap) assert(stageToAttempts === Map(0 -> Set(0, 1), 1 -> Set(0, 1))) - assertDataStructuresEmpty() } testScheduler("job failure after 4 attempts") { @@ -634,7 +634,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) - failure.getMessage.contains("test task failure") + assert(failure.getMessage.contains("test task failure")) } assertDataStructuresEmpty(noFailure = false) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 61787b54f824f..f5f1947661d9a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import org.scalatest.BeforeAndAfterEach import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.internal.Logging class FakeSchedulerBackend extends SchedulerBackend { @@ -32,7 +33,6 @@ class FakeSchedulerBackend extends SchedulerBackend { class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach with Logging { - var failedTaskSetException: Option[Throwable] = None var failedTaskSetReason: String = null var failedTaskSet = false @@ -60,10 +60,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } def setupScheduler(confs: (String, String)*): TaskSchedulerImpl = { - sc = new SparkContext("local", "TaskSchedulerImplSuite") + val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") confs.foreach { case (k, v) => - sc.conf.set(k, v) + conf.set(k, v) } + sc = new SparkContext(conf) taskScheduler = new TaskSchedulerImpl(sc) taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. @@ -287,9 +288,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // schedulable on another executor. However, that executor may fail later on, leaving the // first task with no place to run. val taskScheduler = setupScheduler( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - "spark.scheduler.executorTaskBlacklistTime" -> "10000000" + config.BLACKLIST_ENABLED.key -> "true" ) val taskSet = FakeTask.createTaskSet(2) @@ -328,8 +327,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(tsm.isZombie) assert(failedTaskSet) val idx = failedTask.index - assert(failedTaskSetReason == s"Aborting TaskSet 0.0 because task $idx (partition $idx) has " + - s"already failed on executors (executor0), and no other executors are available.") + assert(failedTaskSetReason === s"Aborting TaskSet 0.0 because task $idx (partition $idx) " + + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior can be " + + s"configured via spark.blacklist.*.") } test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") { @@ -339,9 +339,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // available and not bail on the job val taskScheduler = setupScheduler( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - "spark.scheduler.executorTaskBlacklistTime" -> "10000000" + config.BLACKLIST_ENABLED.key -> "true" ) val taskSet = FakeTask.createTaskSet(2, (0 until 2).map { _ => Seq(TaskLocation("host0")) }: _*) @@ -377,7 +375,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val taskScheduler = setupScheduler() taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0, - (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2"))}: _* + (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2")) }: _* )) val taskDescs = taskScheduler.resourceOffers(IndexedSeq( diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala new file mode 100644 index 0000000000000..8c902af5685ff --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config +import org.apache.spark.util.{ManualClock, SystemClock} + +class TaskSetBlacklistSuite extends SparkFunSuite { + + test("Blacklisting tasks, executors, and nodes") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + val clock = new ManualClock + + val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, clock = clock) + clock.setTime(0) + // We will mark task 0 & 1 failed on both executor 1 & 2. + // We should blacklist all executors on that host, for all tasks for the stage. Note the API + // will return false for isExecutorBacklistedForTaskSet even when the node is blacklisted, so + // the executor is implicitly blacklisted (this makes sense with how the scheduler uses the + // blacklist) + + // First, mark task 0 as failed on exec1. + // task 0 should be blacklisted on exec1, and nowhere else + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 0) + for { + executor <- (1 to 4).map(_.toString) + index <- 0 until 10 + } { + val shouldBeBlacklisted = (executor == "exec1" && index == 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === shouldBeBlacklisted) + } + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Mark task 1 failed on exec1 -- this pushes the executor into the blacklist + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // Mark one task as failed on exec2 -- not enough for any further blacklisting yet. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // Mark another task as failed on exec2 -- now we blacklist exec2, which also leads to + // blacklisting the entire node. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // Make sure the blacklist has the correct per-task && per-executor responses, over a wider + // range of inputs. + for { + executor <- (1 to 4).map(e => s"exec$e") + index <- 0 until 10 + } { + withClue(s"exec = $executor; index = $index") { + val badExec = (executor == "exec1" || executor == "exec2") + val badIndex = (index == 0 || index == 1) + assert( + // this ignores whether the executor is blacklisted entirely for the taskset -- that is + // intentional, it keeps it fast and is sufficient for usage in the scheduler. + taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet(executor) === badExec) + } + } + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + val execToFailures = taskSetBlacklist.execToFailures + assert(execToFailures.keySet === Set("exec1", "exec2")) + + Seq("exec1", "exec2").foreach { exec => + assert( + execToFailures(exec).taskToFailureCount === Map( + 0 -> 1, + 1 -> 1 + ) + ) + } + } + + test("multiple attempts for the same task count once") { + // Make sure that for blacklisting tasks, the node counts task attempts, not executors. But for + // stage-level blacklisting, we count unique tasks. The reason for this difference is, with + // task-attempt blacklisting, we want to make it easy to configure so that you ensure a node + // is blacklisted before the taskset is completely aborted because of spark.task.maxFailures. + // But with stage-blacklisting, we want to make sure we're not just counting one bad task + // that has failed many times. + + val conf = new SparkConf().setMaster("local").setAppName("test") + .set(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, 2) + .set(config.MAX_TASK_ATTEMPTS_PER_NODE, 3) + .set(config.MAX_FAILURES_PER_EXEC_STAGE, 2) + .set(config.MAX_FAILED_EXEC_PER_NODE_STAGE, 3) + val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock()) + // Fail a task twice on hostA, exec:1 + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0)) + assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail the same task once more on hostA, exec:2 + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) + assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail another task on hostA, exec:1. Now that executor has failures on two different tasks, + // so its blacklisted + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail a fourth & fifth task on hostA, exec:3. Now we've got three executors that are + // blacklisted for the taskset, so blacklist the whole node. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3")) + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + } + + test("only blacklist nodes for the task set when all the blacklisted executors are all on " + + "same host") { + // we blacklist executors on two different hosts within one taskSet -- make sure that doesn't + // lead to any node blacklisting + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock()) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostB")) + } + +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 7d6ad08036cb4..69edcf3347243 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.Mockito.{mock, verify} import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.internal.Logging import org.apache.spark.util.{AccumulatorV2, ManualClock} @@ -103,7 +104,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val host = executorIdToHost.get(execId) assert(host != None) val hostId = host.get - val executorsOnHost = executorsByHost(hostId) + val executorsOnHost = hostToExecutors(hostId) executorsOnHost -= execId for (rack <- getRackForHost(hostId); hosts <- hostsByRack.get(rack)) { hosts -= hostId @@ -125,7 +126,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex def addExecutor(execId: String, host: String) { executors.put(execId, host) - val executorsOnHost = executorsByHost.getOrElseUpdate(host, new mutable.HashSet[String]) + val executorsOnHost = hostToExecutors.getOrElseUpdate(host, new mutable.HashSet[String]) executorsOnHost += execId executorIdToHost += execId -> host for (rack <- getRackForHost(host)) { @@ -411,7 +412,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("executors should be blacklisted after task failure, in spite of locality preferences") { val rescheduleDelay = 300L val conf = new SparkConf(). - set("spark.scheduler.executorTaskBlacklistTime", rescheduleDelay.toString). + set(config.BLACKLIST_ENABLED, true). + set(config.BLACKLIST_TIMEOUT_CONF, rescheduleDelay). // don't wait to jump locality levels in this test set("spark.locality.wait", "0") @@ -475,19 +477,24 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) } - // After reschedule delay, scheduling on exec1 should be possible. + // Despite advancing beyond the time for expiring executors from within the blacklist, + // we *never* expire from *within* the stage blacklist clock.advance(rescheduleDelay) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) - assert(offerResult.isDefined, "Expect resource offer to return a task") + assert(offerResult.isEmpty) + } + { + val offerResult = manager.resourceOffer("exec3", "host3", ANY) + assert(offerResult.isDefined) assert(offerResult.get.index === 0) - assert(offerResult.get.executorId === "exec1") + assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) - // Cause exec1 to fail : failure 4 + // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) } @@ -859,6 +866,114 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.endedTasks(3) === Success) } + test("Killing speculative tasks does not count towards aborting the taskset") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = FakeTask.createTaskSet(5) + // Set the speculation multiplier to be 0 so speculative tasks are launched immediately + sc.conf.set("spark.speculation.multiplier", "0.0") + sc.conf.set("spark.speculation.quantile", "0.6") + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => + task.metrics.internalAccums + } + // Offer resources for 5 tasks to start + val tasks = new ArrayBuffer[TaskDescription]() + for ((k, v) <- List( + "exec1" -> "host1", + "exec1" -> "host1", + "exec1" -> "host1", + "exec2" -> "host2", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(k, v, NO_PREF) + assert(taskOption.isDefined) + val task = taskOption.get + assert(task.executorId === k) + tasks += task + } + assert(sched.startedTasks.toSet === (0 until 5).toSet) + // Complete 3 tasks and leave 2 tasks in running + for (id <- Set(0, 1, 2)) { + manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) + assert(sched.endedTasks(id) === Success) + } + + def runningTaskForIndex(index: Int): TaskDescription = { + tasks.find { task => + task.index == index && !sched.endedTasks.contains(task.taskId) + }.getOrElse { + throw new RuntimeException(s"couldn't find index $index in " + + s"tasks: ${tasks.map{t => t.index -> t.taskId}} with endedTasks:" + + s" ${sched.endedTasks.keys}") + } + } + + // have each of the running tasks fail 3 times (not enough to abort the stage) + (0 until 3).foreach { attempt => + Seq(3, 4).foreach { index => + val task = runningTaskForIndex(index) + logInfo(s"failing task $task") + val endReason = ExceptionFailure("a", "b", Array(), "c", None) + manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) + sched.endedTasks(task.taskId) = endReason + assert(!manager.isZombie) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF) + assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") + tasks += nextTask.get + } + } + + // we can't be sure which one of our running tasks will get another speculative copy + val originalTasks = Seq(3, 4).map { index => index -> runningTaskForIndex(index) }.toMap + + // checkSpeculatableTasks checks that the task runtime is greater than the threshold for + // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for + // > 0ms, so advance the clock by 1ms here. + clock.advance(1) + assert(manager.checkSpeculatableTasks(0)) + // Offer resource to start the speculative attempt for the running task + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption5.isDefined) + val speculativeTask = taskOption5.get + assert(speculativeTask.index === 3 || speculativeTask.index === 4) + assert(speculativeTask.taskId === 11) + assert(speculativeTask.executorId === "exec1") + assert(speculativeTask.attemptNumber === 4) + sched.backend = mock(classOf[SchedulerBackend]) + // Complete the speculative attempt for the running task + manager.handleSuccessfulTask(speculativeTask.taskId, createTaskResult(3, accumUpdatesByTask(3))) + // Verify that it kills other running attempt + val origTask = originalTasks(speculativeTask.index) + verify(sched.backend).killTask(origTask.taskId, "exec2", true) + // Because the SchedulerBackend was a mock, the 2nd copy of the task won't actually be + // killed, so the FakeTaskScheduler is only told about the successful completion + // of the speculated task. + assert(sched.endedTasks(3) === Success) + // also because the scheduler is a mock, our manager isn't notified about the task killed event, + // so we do that manually + manager.handleFailedTask(origTask.taskId, TaskState.KILLED, TaskKilled) + // this task has "failed" 4 times, but one of them doesn't count, so keep running the stage + assert(manager.tasksSuccessful === 4) + assert(!manager.isZombie) + + // now run another speculative task + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOpt6.isDefined) + val speculativeTask2 = taskOpt6.get + assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) + assert(speculativeTask2.index !== speculativeTask.index) + assert(speculativeTask2.attemptNumber === 4) + // Complete the speculative attempt for the running task + manager.handleSuccessfulTask(speculativeTask2.taskId, + createTaskResult(3, accumUpdatesByTask(3))) + // Verify that it kills other running attempt + val origTask2 = originalTasks(speculativeTask2.index) + verify(sched.backend).killTask(origTask2.taskId, "exec2", true) + assert(manager.tasksSuccessful === 5) + assert(manager.isZombie) + } + private def createTaskResult( id: Int, accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index c1484b0afa85f..46aa9c37986cc 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.serializer import com.esotericsoftware.kryo.Kryo import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.serializer.KryoDistributedTest._ import org.apache.spark.util.Utils @@ -29,7 +30,8 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex val conf = new SparkConf(false) .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) - .set("spark.task.maxFailures", "1") + .set(config.MAX_TASK_FAILURES, 1) + .set(config.BLACKLIST_ENABLED, false) val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) conf.setJars(List(jar.getPath)) diff --git a/docs/configuration.md b/docs/configuration.md index 82ce232b336d9..373e22d71a872 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1245,6 +1245,49 @@ Apart from these, the following properties are also available, and may be useful The interval length for the scheduler to revive the worker resource offers to run tasks. + + spark.blacklist.enabled + + false + + + If set to "true", prevent Spark from scheduling tasks on executors that have been blacklisted + due to too many task failures. The blacklisting algorithm can be further controlled by the + other "spark.blacklist" configuration options. + + + + spark.blacklist.task.maxTaskAttemptsPerExecutor + 1 + + (Experimental) For a given task, how many times it can be retried on one executor before the + executor is blacklisted for that task. + + + + spark.blacklist.task.maxTaskAttemptsPerNode + 2 + + (Experimental) For a given task, how many times it can be retried on one node, before the entire + node is blacklisted for that task. + + + + spark.blacklist.stage.maxFailedTasksPerExecutor + 2 + + (Experimental) How many different tasks must fail on one executor, within one stage, before the + executor is blacklisted for that stage. + + + + spark.blacklist.stage.maxFailedExecutorsPerNode + 2 + + (Experimental) How many different executors are marked as blacklisted for a given stage, before + the entire node is marked as failed for the stage. + + spark.speculation false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 6e60b0e4fad15..19b6d2603129c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -23,6 +23,7 @@ import org.mockito.Mockito.mock import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.config import org.apache.spark.rdd.RDD import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SparkSession} @@ -446,7 +447,7 @@ class SQLListenerMemoryLeakSuite extends SparkFunSuite { val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.task.maxFailures", "1") // Don't retry the tasks to run this test quickly + .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly val sc = new SparkContext(conf) try { From f9a56a153e0579283160519065c7f3620d12da3e Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 12 Oct 2016 15:22:06 -0700 Subject: [PATCH 065/177] [SPARK-17782][STREAMING][KAFKA] alternative eliminate race condition of poll twice ## What changes were proposed in this pull request? Alternative approach to https://github.com/apache/spark/pull/15387 Author: cody koeninger Closes #15401 from koeninger/SPARK-17782-alt. --- .../streaming/kafka010/ConsumerStrategy.scala | 4 ++++ .../kafka010/DirectKafkaInputDStream.scala | 23 +++++++++++++++++-- .../kafka010/DirectKafkaStreamSuite.scala | 12 ++++++---- 3 files changed, 32 insertions(+), 7 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index 60255fc655e5f..778c06ea16a2b 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -104,6 +104,8 @@ private case class Subscribe[K, V]( toSeek.asScala.foreach { case (topicPartition, offset) => consumer.seek(topicPartition, offset) } + // we've called poll, we must pause or next poll may consume messages and set position + consumer.pause(consumer.assignment()) } consumer @@ -154,6 +156,8 @@ private case class SubscribePattern[K, V]( toSeek.asScala.foreach { case (topicPartition, offset) => consumer.seek(topicPartition, offset) } + // we've called poll, we must pause or next poll may consume messages and set position + consumer.pause(consumer.assignment()) } consumer diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index 13827f68f2cb5..432537ebf05b2 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -161,12 +161,31 @@ private[spark] class DirectKafkaInputDStream[K, V]( } } + /** + * The concern here is that poll might consume messages despite being paused, + * which would throw off consumer position. Fix position if this happens. + */ + private def paranoidPoll(c: Consumer[K, V]): Unit = { + val msgs = c.poll(0) + if (!msgs.isEmpty) { + // position should be minimum offset per topicpartition + msgs.asScala.foldLeft(Map[TopicPartition, Long]()) { (acc, m) => + val tp = new TopicPartition(m.topic, m.partition) + val off = acc.get(tp).map(o => Math.min(o, m.offset)).getOrElse(m.offset) + acc + (tp -> off) + }.foreach { case (tp, off) => + logInfo(s"poll(0) returned messages, seeking $tp to $off to compensate") + c.seek(tp, off) + } + } + } + /** * Returns the latest (highest) available offsets, taking new partitions into account. */ protected def latestOffsets(): Map[TopicPartition, Long] = { val c = consumer - c.poll(0) + paranoidPoll(c) val parts = c.assignment().asScala // make sure new partitions are reflected in currentOffsets @@ -223,7 +242,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( override def start(): Unit = { val c = consumer - c.poll(0) + paranoidPoll(c) if (currentOffsets.isEmpty) { currentOffsets = c.assignment().asScala.map { tp => tp -> c.position(tp) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index e04f35eceb1b4..02aec43c3b34f 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -159,17 +159,19 @@ class DirectKafkaStreamSuite } test("pattern based subscription") { - val topics = List("pat1", "pat2", "advanced3") - // Should match 2 out of 3 topics + val topics = List("pat1", "pat2", "pat3", "advanced3") + // Should match 3 out of 4 topics val pat = """pat\d""".r.pattern val data = Map("a" -> 7, "b" -> 9) topics.foreach { t => kafkaTestUtils.createTopic(t) kafkaTestUtils.sendMessages(t, data) } - val offsets = Map(new TopicPartition("pat2", 0) -> 3L) - // 2 matching topics, one of which starts 3 messages later - val expectedTotal = (data.values.sum * 2) - 3 + val offsets = Map( + new TopicPartition("pat2", 0) -> 3L, + new TopicPartition("pat3", 0) -> 4L) + // 3 matching topics, two of which start a total of 7 messages later + val expectedTotal = (data.values.sum * 3) - 7 val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") ssc = new StreamingContext(sparkConf, Milliseconds(1000)) From 6f20a92ca30f9c367009c4556939ea4de4284cb9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 12 Oct 2016 16:45:10 -0700 Subject: [PATCH 066/177] [SPARK-17845] [SQL] More self-evident window function frame boundary API ## What changes were proposed in this pull request? This patch improves the window function frame boundary API to make it more obvious to read and to use. The two high level changes are: 1. Create Window.currentRow, Window.unboundedPreceding, Window.unboundedFollowing to indicate the special values in frame boundaries. These methods map to the special integral values so we are not breaking backward compatibility here. This change makes the frame boundaries more self-evident (instead of Long.MinValue, it becomes Window.unboundedPreceding). 2. In Python, for any value less than or equal to JVM's Long.MinValue, treat it as Window.unboundedPreceding. For any value larger than or equal to JVM's Long.MaxValue, treat it as Window.unboundedFollowing. Before this change, if the user specifies any value that is less than Long.MinValue but not -sys.maxsize (e.g. -sys.maxsize + 1), the number we pass over to the JVM would overflow, resulting in a frame that does not make sense. Code example required to specify a frame before this patch: ``` Window.rowsBetween(-Long.MinValue, 0) ``` While the above code should still work, the new way is more obvious to read: ``` Window.rowsBetween(Window.unboundedPreceding, Window.currentRow) ``` ## How was this patch tested? - Updated DataFrameWindowSuite (for Scala/Java) - Updated test_window_functions_cumulative_sum (for Python) - Renamed DataFrameWindowSuite DataFrameWindowFunctionsSuite to better reflect its purpose Author: Reynold Xin Closes #15438 from rxin/SPARK-17845. --- python/pyspark/sql/tests.py | 25 +++++- python/pyspark/sql/window.py | 89 +++++++++++++------ .../apache/spark/sql/expressions/Window.scala | 62 +++++++++++-- .../spark/sql/expressions/WindowSpec.scala | 24 +++-- ...la => DataFrameWindowFunctionsSuite.scala} | 11 ++- 5 files changed, 160 insertions(+), 51 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/{DataFrameWindowSuite.scala => DataFrameWindowFunctionsSuite.scala} (97%) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 61674a8a7ed65..51d5e7ab0568e 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1876,12 +1876,35 @@ def test_window_functions_without_partitionBy(self): def test_window_functions_cumulative_sum(self): df = self.spark.createDataFrame([("one", 1), ("two", 2)], ["key", "value"]) from pyspark.sql import functions as F - sel = df.select(df.key, F.sum(df.value).over(Window.rowsBetween(-sys.maxsize, 0))) + + # Test cumulative sum + sel = df.select( + df.key, + F.sum(df.value).over(Window.rowsBetween(Window.unboundedPreceding, 0))) + rs = sorted(sel.collect()) + expected = [("one", 1), ("two", 3)] + for r, ex in zip(rs, expected): + self.assertEqual(tuple(r), ex[:len(r)]) + + # Test boundary values less than JVM's Long.MinValue and make sure we don't overflow + sel = df.select( + df.key, + F.sum(df.value).over(Window.rowsBetween(Window.unboundedPreceding - 1, 0))) rs = sorted(sel.collect()) expected = [("one", 1), ("two", 3)] for r, ex in zip(rs, expected): self.assertEqual(tuple(r), ex[:len(r)]) + # Test boundary values greater than JVM's Long.MaxValue and make sure we don't overflow + frame_end = Window.unboundedFollowing + 1 + sel = df.select( + df.key, + F.sum(df.value).over(Window.rowsBetween(Window.currentRow, frame_end))) + rs = sorted(sel.collect()) + expected = [("one", 3), ("two", 2)] + for r, ex in zip(rs, expected): + self.assertEqual(tuple(r), ex[:len(r)]) + def test_collect_functions(self): df = self.spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"]) from pyspark.sql import functions diff --git a/python/pyspark/sql/window.py b/python/pyspark/sql/window.py index 87e9a988987ea..c345e623f1cb1 100644 --- a/python/pyspark/sql/window.py +++ b/python/pyspark/sql/window.py @@ -36,8 +36,8 @@ class Window(object): For example: - >>> # PARTITION BY country ORDER BY date ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW - >>> window = Window.partitionBy("country").orderBy("date").rowsBetween(-sys.maxsize, 0) + >>> # ORDER BY date ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + >>> window = Window.orderBy("date").rowsBetween(Window.unboundedPreceding, Window.currentRow) >>> # PARTITION BY country ORDER BY date RANGE BETWEEN 3 PRECEDING AND 3 FOLLOWING >>> window = Window.orderBy("date").partitionBy("country").rangeBetween(-3, 3) @@ -46,6 +46,16 @@ class Window(object): .. versionadded:: 1.4 """ + + _JAVA_MIN_LONG = -(1 << 63) # -9223372036854775808 + _JAVA_MAX_LONG = (1 << 63) - 1 # 9223372036854775807 + + unboundedPreceding = _JAVA_MIN_LONG + + unboundedFollowing = _JAVA_MAX_LONG + + currentRow = 0 + @staticmethod @since(1.4) def partitionBy(*cols): @@ -77,15 +87,21 @@ def rowsBetween(start, end): For example, "0" means "current row", while "-1" means the row before the current row, and "5" means the fifth row after the current row. + We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, + and ``Window.currentRow`` to specify special boundary values, rather than using integral + values directly. + :param start: boundary start, inclusive. - The frame is unbounded if this is ``-sys.maxsize`` (or lower). + The frame is unbounded if this is ``Window.unboundedPreceding``, or + any value less than or equal to -9223372036854775808. :param end: boundary end, inclusive. - The frame is unbounded if this is ``sys.maxsize`` (or higher). + The frame is unbounded if this is ``Window.unboundedFollowing``, or + any value greater than or equal to 9223372036854775807. """ - if start <= -sys.maxsize: - start = WindowSpec._JAVA_MIN_LONG - if end >= sys.maxsize: - end = WindowSpec._JAVA_MAX_LONG + if start <= Window._JAVA_MIN_LONG: + start = Window.unboundedPreceding + if end >= Window._JAVA_MAX_LONG: + end = Window.unboundedFollowing sc = SparkContext._active_spark_context jspec = sc._jvm.org.apache.spark.sql.expressions.Window.rowsBetween(start, end) return WindowSpec(jspec) @@ -101,15 +117,21 @@ def rangeBetween(start, end): "0" means "current row", while "-1" means one off before the current row, and "5" means the five off after the current row. + We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, + and ``Window.currentRow`` to specify special boundary values, rather than using integral + values directly. + :param start: boundary start, inclusive. - The frame is unbounded if this is ``-sys.maxsize`` (or lower). + The frame is unbounded if this is ``Window.unboundedPreceding``, or + any value less than or equal to -9223372036854775808. :param end: boundary end, inclusive. - The frame is unbounded if this is ``sys.maxsize`` (or higher). + The frame is unbounded if this is ``Window.unboundedFollowing``, or + any value greater than or equal to 9223372036854775807. """ - if start <= -sys.maxsize: - start = WindowSpec._JAVA_MIN_LONG - if end >= sys.maxsize: - end = WindowSpec._JAVA_MAX_LONG + if start <= Window._JAVA_MIN_LONG: + start = Window.unboundedPreceding + if end >= Window._JAVA_MAX_LONG: + end = Window.unboundedFollowing sc = SparkContext._active_spark_context jspec = sc._jvm.org.apache.spark.sql.expressions.Window.rangeBetween(start, end) return WindowSpec(jspec) @@ -127,9 +149,6 @@ class WindowSpec(object): .. versionadded:: 1.4 """ - _JAVA_MAX_LONG = (1 << 63) - 1 - _JAVA_MIN_LONG = - (1 << 63) - def __init__(self, jspec): self._jspec = jspec @@ -160,15 +179,21 @@ def rowsBetween(self, start, end): For example, "0" means "current row", while "-1" means the row before the current row, and "5" means the fifth row after the current row. + We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, + and ``Window.currentRow`` to specify special boundary values, rather than using integral + values directly. + :param start: boundary start, inclusive. - The frame is unbounded if this is ``-sys.maxsize`` (or lower). + The frame is unbounded if this is ``Window.unboundedPreceding``, or + any value less than or equal to -9223372036854775808. :param end: boundary end, inclusive. - The frame is unbounded if this is ``sys.maxsize`` (or higher). + The frame is unbounded if this is ``Window.unboundedFollowing``, or + any value greater than or equal to 9223372036854775807. """ - if start <= -sys.maxsize: - start = self._JAVA_MIN_LONG - if end >= sys.maxsize: - end = self._JAVA_MAX_LONG + if start <= Window._JAVA_MIN_LONG: + start = Window.unboundedPreceding + if end >= Window._JAVA_MAX_LONG: + end = Window.unboundedFollowing return WindowSpec(self._jspec.rowsBetween(start, end)) @since(1.4) @@ -180,15 +205,21 @@ def rangeBetween(self, start, end): "0" means "current row", while "-1" means one off before the current row, and "5" means the five off after the current row. + We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, + and ``Window.currentRow`` to specify special boundary values, rather than using integral + values directly. + :param start: boundary start, inclusive. - The frame is unbounded if this is ``-sys.maxsize`` (or lower). + The frame is unbounded if this is ``Window.unboundedPreceding``, or + any value less than or equal to -9223372036854775808. :param end: boundary end, inclusive. - The frame is unbounded if this is ``sys.maxsize`` (or higher). + The frame is unbounded if this is ``Window.unboundedFollowing``, or + any value greater than or equal to 9223372036854775807. """ - if start <= -sys.maxsize: - start = self._JAVA_MIN_LONG - if end >= sys.maxsize: - end = self._JAVA_MAX_LONG + if start <= Window._JAVA_MIN_LONG: + start = Window.unboundedPreceding + if end >= Window._JAVA_MAX_LONG: + end = Window.unboundedFollowing return WindowSpec(self._jspec.rangeBetween(start, end)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index e8a0c5f43fe46..3c1f6e897ea62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.catalyst.expressions._ * * {{{ * // PARTITION BY country ORDER BY date ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW - * Window.partitionBy("country").orderBy("date").rowsBetween(Long.MinValue, 0) + * Window.partitionBy("country").orderBy("date") + * .rowsBetween(Window.unboundedPreceding, Window.currentRow) * * // PARTITION BY country ORDER BY date ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING * Window.partitionBy("country").orderBy("date").rowsBetween(-3, 3) @@ -74,6 +75,41 @@ object Window { spec.orderBy(cols : _*) } + /** + * Value representing the last row in the partition, equivalent to "UNBOUNDED PRECEDING" in SQL. + * This can be used to specify the frame boundaries: + * + * {{{ + * Window.rowsBetween(Window.unboundedPreceding, Window.currentRow) + * }}} + * + * @since 2.1.0 + */ + def unboundedPreceding: Long = Long.MinValue + + /** + * Value representing the last row in the partition, equivalent to "UNBOUNDED FOLLOWING" in SQL. + * This can be used to specify the frame boundaries: + * + * {{{ + * Window.rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing) + * }}} + * + * @since 2.1.0 + */ + def unboundedFollowing: Long = Long.MaxValue + + /** + * Value representing the current row. This can be used to specify the frame boundaries: + * + * {{{ + * Window.rowsBetween(Window.unboundedPreceding, Window.currentRow) + * }}} + * + * @since 2.1.0 + */ + def currentRow: Long = 0 + /** * Creates a [[WindowSpec]] with the frame boundaries defined, * from `start` (inclusive) to `end` (inclusive). @@ -82,10 +118,14 @@ object Window { * "current row", while "-1" means the row before the current row, and "5" means the fifth row * after the current row. * - * @param start boundary start, inclusive. - * The frame is unbounded if this is the minimum long value. - * @param end boundary end, inclusive. - * The frame is unbounded if this is the maximum long value. + * We recommend users use [[Window.unboundedPreceding]], [[Window.unboundedFollowing]], + * and [[Window.currentRow]] to specify special boundary values, rather than using integral + * values directly. + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value ([[Window.unboundedPreceding]]). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value ([[Window.unboundedFollowing]]). * @since 2.1.0 */ // Note: when updating the doc for this method, also update WindowSpec.rowsBetween. @@ -101,10 +141,14 @@ object Window { * while "-1" means one off before the current row, and "5" means the five off after the * current row. * - * @param start boundary start, inclusive. - * The frame is unbounded if this is the minimum long value. - * @param end boundary end, inclusive. - * The frame is unbounded if this is the maximum long value. + * We recommend users use [[Window.unboundedPreceding]], [[Window.unboundedFollowing]], + * and [[Window.currentRow]] to specify special boundary values, rather than using integral + * values directly. + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value ([[Window.unboundedPreceding]]). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value ([[Window.unboundedFollowing]]). * @since 2.1.0 */ // Note: when updating the doc for this method, also update WindowSpec.rangeBetween. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index 82bc8f152d6ea..8ebed399bf2d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -86,10 +86,14 @@ class WindowSpec private[sql]( * "current row", while "-1" means the row before the current row, and "5" means the fifth row * after the current row. * - * @param start boundary start, inclusive. - * The frame is unbounded if this is the minimum long value. - * @param end boundary end, inclusive. - * The frame is unbounded if this is the maximum long value. + * We recommend users use [[Window.unboundedPreceding]], [[Window.unboundedFollowing]], + * and [[Window.currentRow]] to specify special boundary values, rather than using integral + * values directly. + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value ([[Window.unboundedPreceding]]). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value ([[Window.unboundedFollowing]]). * @since 1.4.0 */ // Note: when updating the doc for this method, also update Window.rowsBetween. @@ -104,10 +108,14 @@ class WindowSpec private[sql]( * while "-1" means one off before the current row, and "5" means the five off after the * current row. * - * @param start boundary start, inclusive. - * The frame is unbounded if this is the minimum long value. - * @param end boundary end, inclusive. - * The frame is unbounded if this is the maximum long value. + * We recommend users use [[Window.unboundedPreceding]], [[Window.unboundedFollowing]], + * and [[Window.currentRow]] to specify special boundary values, rather than using integral + * values directly. + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value ([[Window.unboundedPreceding]]). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value ([[Window.unboundedFollowing]]). * @since 1.4.0 */ // Note: when updating the doc for this method, also update Window.rangeBetween. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala similarity index 97% rename from sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 5bc386f291043..1255c49104718 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types.{DataType, LongType, StructType} /** * Window function testing for DataFrame API. */ -class DataFrameWindowSuite extends QueryTest with SharedSQLContext { +class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("reuse window partitionBy") { @@ -54,7 +54,8 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { val df = Seq(("one", 1), ("two", 2)).toDF("key", "value") // Running (cumulative) sum checkAnswer( - df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0))), + df.select('key, sum("value").over( + Window.rowsBetween(Window.unboundedPreceding, Window.currentRow))), Row("one", 1) :: Row("two", 3) :: Nil ) } @@ -156,9 +157,11 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { df.select( $"key", last("key").over( - Window.partitionBy($"value").orderBy($"key").rowsBetween(0, Long.MaxValue)), + Window.partitionBy($"value").orderBy($"key") + .rowsBetween(Window.currentRow, Window.unboundedFollowing)), last("key").over( - Window.partitionBy($"value").orderBy($"key").rowsBetween(Long.MinValue, 0)), + Window.partitionBy($"value").orderBy($"key") + .rowsBetween(Window.unboundedPreceding, Window.currentRow)), last("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 1))), Seq(Row(1, 1, 1, 1), Row(2, 3, 2, 3), Row(3, 3, 3, 3), Row(1, 4, 1, 2), Row(2, 4, 2, 4), Row(4, 4, 4, 4))) From 0d4a695279c514c76aa0e9288c70ac7aaef91b03 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 12 Oct 2016 19:52:57 -0700 Subject: [PATCH 067/177] [SPARK-17745][ML][PYSPARK] update NB python api - add weight col parameter ## What changes were proposed in this pull request? update python api for NaiveBayes: add weight col parameter. ## How was this patch tested? doctests added. Author: WeichenXu Closes #15406 from WeichenXu123/nb_python_update. --- python/pyspark/ml/classification.py | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index ea60fab029582..3f763a10d4066 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -981,7 +981,7 @@ def trees(self): @inherit_doc class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, - HasRawPredictionCol, HasThresholds, JavaMLWritable, JavaMLReadable): + HasRawPredictionCol, HasThresholds, HasWeightCol, JavaMLWritable, JavaMLReadable): """ Naive Bayes Classifiers. It supports both Multinomial and Bernoulli NB. `Multinomial NB @@ -995,23 +995,23 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H >>> from pyspark.sql import Row >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([ - ... Row(label=0.0, features=Vectors.dense([0.0, 0.0])), - ... Row(label=0.0, features=Vectors.dense([0.0, 1.0])), - ... Row(label=1.0, features=Vectors.dense([1.0, 0.0]))]) - >>> nb = NaiveBayes(smoothing=1.0, modelType="multinomial") + ... Row(label=0.0, weight=0.1, features=Vectors.dense([0.0, 0.0])), + ... Row(label=0.0, weight=0.5, features=Vectors.dense([0.0, 1.0])), + ... Row(label=1.0, weight=1.0, features=Vectors.dense([1.0, 0.0]))]) + >>> nb = NaiveBayes(smoothing=1.0, modelType="multinomial", weightCol="weight") >>> model = nb.fit(df) >>> model.pi - DenseVector([-0.51..., -0.91...]) + DenseVector([-0.81..., -0.58...]) >>> model.theta - DenseMatrix(2, 2, [-1.09..., -0.40..., -0.40..., -1.09...], 1) + DenseMatrix(2, 2, [-0.91..., -0.51..., -0.40..., -1.09...], 1) >>> test0 = sc.parallelize([Row(features=Vectors.dense([1.0, 0.0]))]).toDF() >>> result = model.transform(test0).head() >>> result.prediction 1.0 >>> result.probability - DenseVector([0.42..., 0.57...]) + DenseVector([0.32..., 0.67...]) >>> result.rawPrediction - DenseVector([-1.60..., -1.32...]) + DenseVector([-1.72..., -0.99...]) >>> test1 = sc.parallelize([Row(features=Vectors.sparse(2, [0], [1.0]))]).toDF() >>> model.transform(test1).head().prediction 1.0 @@ -1045,11 +1045,11 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, - modelType="multinomial", thresholds=None): + modelType="multinomial", thresholds=None, weightCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, \ - modelType="multinomial", thresholds=None) + modelType="multinomial", thresholds=None, weightCol=None) """ super(NaiveBayes, self).__init__() self._java_obj = self._new_java_obj( @@ -1062,11 +1062,11 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred @since("1.5.0") def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, - modelType="multinomial", thresholds=None): + modelType="multinomial", thresholds=None, weightCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, \ - modelType="multinomial", thresholds=None) + modelType="multinomial", thresholds=None, weightCol=None) Sets params for Naive Bayes. """ kwargs = self.setParams._input_kwargs From 21cb59f1cd137d96b2596f1abe691b544581cf59 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 12 Oct 2016 19:56:40 -0700 Subject: [PATCH 068/177] [SPARK-17835][ML][MLLIB] Optimize NaiveBayes mllib wrapper to eliminate extra pass on data ## What changes were proposed in this pull request? [SPARK-14077](https://issues.apache.org/jira/browse/SPARK-14077) copied the ```NaiveBayes``` implementation from mllib to ml and left mllib as a wrapper. However, there are some difference between mllib and ml to handle labels: * mllib allow input labels as {-1, +1}, however, ml assumes the input labels in range [0, numClasses). * mllib ```NaiveBayesModel``` expose ```labels``` but ml did not due to the assumption mention above. During the copy in [SPARK-14077](https://issues.apache.org/jira/browse/SPARK-14077), we use ```val labels = data.map(_.label).distinct().collect().sorted``` to get the distinct labels firstly, and then encode the labels for training. It involves extra Spark job compared with the original implementation. Since ```NaiveBayes``` only do one pass aggregation during training, adding another one seems less efficient. We can get the labels in a single pass along with ```NaiveBayes``` training and send them to MLlib side. ## How was this patch tested? Existing tests. Author: Yanbo Liang Closes #15402 from yanboliang/spark-17835. --- .../spark/ml/classification/NaiveBayes.scala | 46 +++++++++++++++---- .../mllib/classification/NaiveBayes.scala | 15 +++--- 2 files changed, 43 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index e565a6fd3ece2..994ed993c99df 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -110,16 +110,28 @@ class NaiveBayes @Since("1.5.0") ( @Since("2.1.0") def setWeightCol(value: String): this.type = set(weightCol, value) - override protected def train(dataset: Dataset[_]): NaiveBayesModel = { - val numClasses = getNumClasses(dataset) + /** + * ml assumes input labels in range [0, numClasses). But this implementation + * is also called by mllib NaiveBayes which allows other kinds of input labels + * such as {-1, +1}. Here we use this parameter to switch between different processing logic. + * It should be removed when we remove mllib NaiveBayes. + */ + private[spark] var isML: Boolean = true - if (isDefined(thresholds)) { - require($(thresholds).length == numClasses, this.getClass.getSimpleName + - ".train() called with non-matching numClasses and thresholds.length." + - s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") - } + private[spark] def setIsML(isML: Boolean): this.type = { + this.isML = isML + this + } - val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size + override protected def train(dataset: Dataset[_]): NaiveBayesModel = { + if (isML) { + val numClasses = getNumClasses(dataset) + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + } val requireNonnegativeValues: Vector => Unit = (v: Vector) => { val values = v match { @@ -153,6 +165,7 @@ class NaiveBayes @Since("1.5.0") ( } } + val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) // Aggregates term frequencies per label. @@ -176,6 +189,7 @@ class NaiveBayes @Since("1.5.0") ( val numLabels = aggregated.length val numDocuments = aggregated.map(_._2._1).sum + val labelArray = new Array[Double](numLabels) val piArray = new Array[Double](numLabels) val thetaArray = new Array[Double](numLabels * numFeatures) @@ -183,6 +197,7 @@ class NaiveBayes @Since("1.5.0") ( val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 aggregated.foreach { case (label, (n, sumTermFreqs)) => + labelArray(i) = label piArray(i) = math.log(n + lambda) - piLogDenom val thetaLogDenom = $(modelType) match { case Multinomial => math.log(sumTermFreqs.values.sum + numFeatures * lambda) @@ -201,7 +216,7 @@ class NaiveBayes @Since("1.5.0") ( val pi = Vectors.dense(piArray) val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi, theta) + new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) } @Since("1.5.0") @@ -239,6 +254,19 @@ class NaiveBayesModel private[ml] ( import NaiveBayes.{Bernoulli, Multinomial} + /** + * mllib NaiveBayes is a wrapper of ml implementation currently. + * Input labels of mllib could be {-1, +1} and mllib NaiveBayesModel exposes labels, + * both of which are different from ml, so we should store the labels sequentially + * to be called by mllib. This should be removed when we remove mllib NaiveBayes. + */ + private[spark] var oldLabels: Array[Double] = null + + private[spark] def setOldLabels(labels: Array[Double]): this.type = { + this.oldLabels = labels + this + } + /** * Bernoulli scoring requires log(condprob) if 1, log(1-condprob) if 0. * This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 32d6968a4e85f..33561be4b5bc1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -364,15 +364,10 @@ class NaiveBayes private ( val nb = new NewNaiveBayes() .setModelType(modelType) .setSmoothing(lambda) + .setIsML(false) - val labels = data.map(_.label).distinct().collect().sorted - - // Input labels for [[org.apache.spark.ml.classification.NaiveBayes]] must be - // in range [0, numClasses). - val dataset = data.map { - case LabeledPoint(label, features) => - (labels.indexOf(label).toDouble, features.asML) - }.toDF("label", "features") + val dataset = data.map { case LabeledPoint(label, features) => (label, features.asML) } + .toDF("label", "features") val newModel = nb.fit(dataset) @@ -383,7 +378,9 @@ class NaiveBayes private ( theta(i)(j) = v } - new NaiveBayesModel(labels, pi, theta, modelType) + require(newModel.oldLabels != null, + "The underlying ML NaiveBayes training does not produce labels.") + new NaiveBayesModel(newModel.oldLabels, pi, theta, modelType) } } From edeb51a39d76d64196d7635f52be1b42c7ec4341 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 12 Oct 2016 21:40:45 -0700 Subject: [PATCH 069/177] [SPARK-17876] Write StructuredStreaming WAL to a stream instead of materializing all at once ## What changes were proposed in this pull request? The CompactibleFileStreamLog materializes the whole metadata log in memory as a String. This can cause issues when there are lots of files that are being committed, especially during a compaction batch. You may come across stacktraces that look like: ``` java.lang.OutOfMemoryError: Requested array size exceeds VM limit at java.lang.StringCoding.encode(StringCoding.java:350) at java.lang.String.getBytes(String.java:941) at org.apache.spark.sql.execution.streaming.FileStreamSinkLog.serialize(FileStreamSinkLog.scala:127) ``` The safer way is to write to an output stream so that we don't have to materialize a huge string. ## How was this patch tested? Existing unit tests Author: Burak Yavuz Closes #15437 from brkyvz/ser-to-stream. --- .../streaming/CompactibleFileStreamLog.scala | 22 +++++++++----- .../execution/streaming/HDFSMetadataLog.scala | 29 ++++++++++--------- .../streaming/FileStreamSinkLogSuite.scala | 14 +++++---- 3 files changed, 38 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala index 027b5bbfab8d6..c14feea91ed7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.streaming -import java.io.IOException +import java.io.{InputStream, IOException, OutputStream} import java.nio.charset.StandardCharsets.UTF_8 +import scala.io.{Source => IOSource} import scala.reflect.ClassTag import org.apache.hadoop.fs.{Path, PathFilter} @@ -93,20 +94,25 @@ abstract class CompactibleFileStreamLog[T: ClassTag]( } } - override def serialize(logData: Array[T]): Array[Byte] = { - (metadataLogVersion +: logData.map(serializeData)).mkString("\n").getBytes(UTF_8) + override def serialize(logData: Array[T], out: OutputStream): Unit = { + // called inside a try-finally where the underlying stream is closed in the caller + out.write(metadataLogVersion.getBytes(UTF_8)) + logData.foreach { data => + out.write('\n') + out.write(serializeData(data).getBytes(UTF_8)) + } } - override def deserialize(bytes: Array[Byte]): Array[T] = { - val lines = new String(bytes, UTF_8).split("\n") - if (lines.length == 0) { + override def deserialize(in: InputStream): Array[T] = { + val lines = IOSource.fromInputStream(in, UTF_8.name()).getLines() + if (!lines.hasNext) { throw new IllegalStateException("Incomplete log file") } - val version = lines(0) + val version = lines.next() if (version != metadataLogVersion) { throw new IllegalStateException(s"Unknown log version: ${version}") } - lines.slice(1, lines.length).map(deserializeData) + lines.map(deserializeData).toArray } override def add(batchId: Long, logs: Array[T]): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 39a0f3341389c..c7235320fd6bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.execution.streaming -import java.io.{FileNotFoundException, IOException} -import java.nio.ByteBuffer +import java.io.{FileNotFoundException, InputStream, IOException, OutputStream} import java.util.{ConcurrentModificationException, EnumSet, UUID} import scala.reflect.ClassTag @@ -29,7 +28,6 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.internal.Logging -import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.JavaSerializer import org.apache.spark.sql.SparkSession import org.apache.spark.util.UninterruptibleThread @@ -88,12 +86,16 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) } } - protected def serialize(metadata: T): Array[Byte] = { - JavaUtils.bufferToArray(serializer.serialize(metadata)) + protected def serialize(metadata: T, out: OutputStream): Unit = { + // called inside a try-finally where the underlying stream is closed in the caller + val outStream = serializer.serializeStream(out) + outStream.writeObject(metadata) } - protected def deserialize(bytes: Array[Byte]): T = { - serializer.deserialize[T](ByteBuffer.wrap(bytes)) + protected def deserialize(in: InputStream): T = { + // called inside a try-finally where the underlying stream is closed in the caller + val inStream = serializer.deserializeStream(in) + inStream.readObject[T]() } /** @@ -114,7 +116,7 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) // Only write metadata when the batch has not yet been written Thread.currentThread match { case ut: UninterruptibleThread => - ut.runUninterruptibly { writeBatch(batchId, serialize(metadata)) } + ut.runUninterruptibly { writeBatch(batchId, metadata, serialize) } case _ => throw new IllegalStateException( "HDFSMetadataLog.add() must be executed on a o.a.spark.util.UninterruptibleThread") @@ -129,7 +131,7 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) * There may be multiple [[HDFSMetadataLog]] using the same metadata path. Although it is not a * valid behavior, we still need to prevent it from destroying the files. */ - private def writeBatch(batchId: Long, bytes: Array[Byte]): Unit = { + private def writeBatch(batchId: Long, metadata: T, writer: (T, OutputStream) => Unit): Unit = { // Use nextId to create a temp file var nextId = 0 while (true) { @@ -137,9 +139,9 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) try { val output = fileManager.create(tempPath) try { - output.write(bytes) + writer(metadata, output) } finally { - output.close() + IOUtils.closeQuietly(output) } try { // Try to commit the batch @@ -193,10 +195,9 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) if (fileManager.exists(batchMetadataFile)) { val input = fileManager.open(batchMetadataFile) try { - val bytes = IOUtils.toByteArray(input) - Some(deserialize(bytes)) + Some(deserialize(input)) } finally { - input.close() + IOUtils.closeQuietly(input) } } else { logDebug(s"Unable to find batch $batchMetadataFile") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala index 41a8cc2400dff..e1bc674a28071 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.streaming +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.nio.charset.StandardCharsets.UTF_8 import org.apache.spark.SparkFunSuite @@ -133,9 +134,12 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { |{"path":"/a/b/y","size":200,"isDir":false,"modificationTime":2000,"blockReplication":2,"blockSize":20000,"action":"delete"} |{"path":"/a/b/z","size":300,"isDir":false,"modificationTime":3000,"blockReplication":3,"blockSize":30000,"action":"add"}""".stripMargin // scalastyle:on - assert(expected === new String(sinkLog.serialize(logs), UTF_8)) - - assert(VERSION === new String(sinkLog.serialize(Array()), UTF_8)) + val baos = new ByteArrayOutputStream() + sinkLog.serialize(logs, baos) + assert(expected === baos.toString(UTF_8.name())) + baos.reset() + sinkLog.serialize(Array(), baos) + assert(VERSION === baos.toString(UTF_8.name())) } } @@ -174,9 +178,9 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { blockSize = 30000L, action = FileStreamSinkLog.ADD_ACTION)) - assert(expected === sinkLog.deserialize(logs.getBytes(UTF_8))) + assert(expected === sinkLog.deserialize(new ByteArrayInputStream(logs.getBytes(UTF_8)))) - assert(Nil === sinkLog.deserialize(VERSION.getBytes(UTF_8))) + assert(Nil === sinkLog.deserialize(new ByteArrayInputStream(VERSION.getBytes(UTF_8)))) } } From 064d6650e93ed6515a1309079c361e20404843cc Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 13 Oct 2016 13:27:57 +0800 Subject: [PATCH 070/177] [SPARK-17866][SPARK-17867][SQL] Fix Dataset.dropduplicates ## What changes were proposed in this pull request? Two issues regarding Dataset.dropduplicates: 1. Dataset.dropDuplicates should consider the columns with same column name We find and get the first resolved attribute from output with the given column name in `Dataset.dropDuplicates`. When we have the more than one columns with the same name. Other columns are put into aggregation columns, instead of grouping columns. 2. Dataset.dropDuplicates should not change the output of child plan We create new `Alias` with new exprId in `Dataset.dropDuplicates` now. However it causes problem when we want to select the columns as follows: val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() // ds("_2") will cause analysis exception ds.dropDuplicates("_1").select(ds("_1").as[String], ds("_2").as[Int]) Because the two issues are both related to `Dataset.dropduplicates` and the code changes are not big, so submitting them together as one PR. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #15427 from viirya/fix-dropduplicates. --- .../scala/org/apache/spark/sql/Dataset.scala | 16 ++++++++++++---- .../org/apache/spark/sql/DatasetSuite.scala | 17 +++++++++++++++++ 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index a7a84730a6fd9..e59a483075c94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1892,17 +1892,25 @@ class Dataset[T] private[sql]( def dropDuplicates(colNames: Seq[String]): Dataset[T] = withTypedPlan { val resolver = sparkSession.sessionState.analyzer.resolver val allColumns = queryExecution.analyzed.output - val groupCols = colNames.map { colName => - allColumns.find(col => resolver(col.name, colName)).getOrElse( + val groupCols = colNames.flatMap { colName => + // It is possibly there are more than one columns with the same name, + // so we call filter instead of find. + val cols = allColumns.filter(col => resolver(col.name, colName)) + if (cols.isEmpty) { throw new AnalysisException( - s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""")) + s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") + } + cols } val groupColExprIds = groupCols.map(_.exprId) val aggCols = logicalPlan.output.map { attr => if (groupColExprIds.contains(attr.exprId)) { attr } else { - Alias(new First(attr).toAggregateExpression(), attr.name)() + // Removing duplicate rows should not change output attributes. We should keep + // the original exprId of the attribute. Otherwise, to select a column in original + // dataset will cause analysis exception due to unresolved attribute. + Alias(new First(attr).toAggregateExpression(), attr.name)(exprId = attr.exprId) } } Aggregate(groupCols, aggCols, logicalPlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 3243f352a5337..5fce9b4fe97ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -872,6 +872,23 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("a", 1), ("a", 2), ("b", 1)) } + test("dropDuplicates: columns with same column name") { + val ds1 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + val ds2 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + // The dataset joined has two columns of the same name "_2". + val joined = ds1.join(ds2, "_1").select(ds1("_2").as[Int], ds2("_2").as[Int]) + checkDataset( + joined.dropDuplicates(), + (1, 2), (1, 1), (2, 1), (2, 2)) + } + + test("dropDuplicates should not change child plan output") { + val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + checkDataset( + ds.dropDuplicates("_1").select(ds("_1").as[String], ds("_2").as[Int]), + ("a", 1), ("b", 1)) + } + test("SPARK-16097: Encoders.tuple should handle null object correctly") { val enc = Encoders.tuple(Encoders.tuple(Encoders.STRING, Encoders.STRING), Encoders.STRING) val data = Seq((("a", "b"), "c"), (null, "d")) From 7222a25a11790fa9d9d1428c84b6f827a785c9e8 Mon Sep 17 00:00:00 2001 From: buzhihuojie Date: Wed, 12 Oct 2016 22:51:54 -0700 Subject: [PATCH 071/177] minor doc fix for Row.scala ## What changes were proposed in this pull request? minor doc fix for "getAnyValAs" in class Row ## How was this patch tested? None. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: buzhihuojie Closes #15452 from david-weiluo-ren/minorDocFixForRow. --- sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 344dcb9bce62b..65f91429648c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -462,7 +462,7 @@ trait Row extends Serializable { def mkString(start: String, sep: String, end: String): String = toSeq.mkString(start, sep, end) /** - * Returns the value of a given fieldName. + * Returns the value at position i. * * @throws UnsupportedOperationException when schema is not defined. * @throws ClassCastException when data type does not match. From 6f2fa6c54a11caccd446d5560d2014c645fcf7cc Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Thu, 13 Oct 2016 03:24:37 -0400 Subject: [PATCH 072/177] [SPARK-11272][WEB UI] Add support for downloading event logs from HistoryServer UI ## What changes were proposed in this pull request? This is a reworked PR based on feedback in #9238 after it was closed and not reopened. As suggested in that PR I've only added the download feature. This functionality already exists in the api and this allows easier access to download event logs to share with others. I've attached a screenshot of the committed version, but I will also include alternate options with screen shots in the comments below. I'm personally not sure which option is best. ## How was this patch tested? Manual testing ![screen shot 2016-10-07 at 6 11 12 pm](https://cloud.githubusercontent.com/assets/13952758/19209213/832fe48e-8cba-11e6-9840-749b1be4d399.png) Author: Alex Bozarth Closes #15400 from ajbozarth/spark11272. --- .../org/apache/spark/ui/static/historypage-template.html | 7 ++++++- .../resources/org/apache/spark/ui/static/historypage.js | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html index a2b3826dd324b..1fd6ef4a71253 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html @@ -59,7 +59,11 @@ Last Updated - + + + Event Log + + {{#applications}} @@ -73,6 +77,7 @@ {{duration}} {{sparkUser}} {{lastUpdated}} + Download {{/attempts}} {{/applications}} diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index c8094005c65dd..2a32e18672a22 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -133,6 +133,7 @@ $(document).ready(function() { {name: 'sixth', type: "title-numeric"}, {name: 'seventh'}, {name: 'eighth'}, + {name: 'ninth'}, ], "autoWidth": false, "order": [[ 4, "desc" ]] From db8784feaa605adcbd37af4bc8b7146479b631f8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 13 Oct 2016 03:26:29 -0400 Subject: [PATCH 073/177] [SPARK-17899][SQL] add a debug mode to keep raw table properties in HiveExternalCatalog ## What changes were proposed in this pull request? Currently `HiveExternalCatalog` will filter out the Spark SQL internal table properties, e.g. `spark.sql.sources.provider`, `spark.sql.sources.schema`, etc. This is reasonable for external users as they don't want to see these internal properties in `DESC TABLE`. However, as a Spark developer, sometimes we do wanna see the raw table properties. This PR adds a new internal SQL conf, `spark.sql.debug`, to enable debug mode and keep these raw table properties. This config can also be used in similar places where we wanna retain debug information in the future. ## How was this patch tested? new test in MetastoreDataSourcesSuite Author: Wenchen Fan Closes #15458 from cloud-fan/debug. --- .../apache/spark/sql/internal/SQLConf.scala | 5 ++++ .../spark/sql/internal/SQLConfSuite.scala | 24 +++++++++++-------- .../spark/sql/hive/HiveExternalCatalog.scala | 9 +++++-- .../sql/hive/MetastoreDataSourcesSuite.scala | 17 ++++++++++++- 4 files changed, 42 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9e7c1ec211893..192083e2ea5f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -915,4 +915,9 @@ object StaticSQLConf { .internal() .intConf .createWithDefault(4000) + + val DEBUG_MODE = buildConf("spark.sql.debug") + .internal() + .booleanConf + .createWithDefault(false) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index f545de0e10a6b..df640ffab91de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.internal import org.apache.hadoop.fs.Path +import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.internal.StaticSQLConf._ @@ -254,18 +255,21 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } - test("global SQL conf comes from SparkConf") { - val newSession = SparkSession.builder() - .config(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000") - .getOrCreate() - - assert(newSession.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD.key) == "2000") - checkAnswer( - newSession.sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}"), - Row(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000")) + test("static SQL conf comes from SparkConf") { + val previousValue = sparkContext.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) + try { + sparkContext.conf.set(SCHEMA_STRING_LENGTH_THRESHOLD, 2000) + val newSession = new SparkSession(sparkContext) + assert(newSession.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) == 2000) + checkAnswer( + newSession.sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}"), + Row(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000")) + } finally { + sparkContext.conf.set(SCHEMA_STRING_LENGTH_THRESHOLD, previousValue) + } } - test("cannot set/unset global SQL conf") { + test("cannot set/unset static SQL conf") { val e1 = intercept[AnalysisException](sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}=10")) assert(e1.message.contains("Cannot modify the value of a static config")) val e2 = intercept[AnalysisException](spark.conf.unset(SCHEMA_STRING_LENGTH_THRESHOLD.key)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index e1c0cad907b98..ed189724a2dbd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.command.{ColumnStatStruct, DDLUtils} import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.HiveSerDe -import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD +import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.types.{DataType, StructType} @@ -461,13 +461,18 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } else { table.storage } + val tableProps = if (conf.get(DEBUG_MODE)) { + table.properties + } else { + getOriginalTableProperties(table) + } table.copy( storage = storage, schema = getSchemaFromTableProperties(table), provider = Some(provider), partitionColumnNames = getPartitionColumnsFromTableProperties(table), bucketSpec = getBucketSpecFromTableProperties(table), - properties = getOriginalTableProperties(table)) + properties = tableProps) } getOrElse { table.copy(provider = Some("hive")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 0477122fc6a27..7cc6179d44977 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path +import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} @@ -31,7 +32,7 @@ import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD +import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -1324,4 +1325,18 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv hiveClient.dropTable("default", "t", ignoreIfNotExists = true, purge = true) } } + + test("should keep data source entries in table properties when debug mode is on") { + val previousValue = sparkSession.sparkContext.conf.get(DEBUG_MODE) + try { + sparkSession.sparkContext.conf.set(DEBUG_MODE, true) + val newSession = sparkSession.newSession() + newSession.sql("CREATE TABLE abc(i int) USING json") + val tableMeta = newSession.sessionState.catalog.getTableMetadata(TableIdentifier("abc")) + assert(tableMeta.properties(DATASOURCE_SCHEMA_NUMPARTS).toInt == 1) + assert(tableMeta.properties(DATASOURCE_PROVIDER) == "json") + } finally { + sparkSession.sparkContext.conf.set(DEBUG_MODE, previousValue) + } + } } From 7bf8a4049866b2ec7fdf0406b1ad0c3a12488645 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 13 Oct 2016 03:29:14 -0400 Subject: [PATCH 074/177] [SPARK-17686][CORE] Support printing out scala and java version with spark-submit --version command ## What changes were proposed in this pull request? In our universal gateway service we need to specify different jars to Spark according to scala version. For now only after launching Spark application can we know which version of Scala it depends on. It makes hard for us to support different Scala + Spark versions to pick the right jars. So here propose to print out Scala version according to Spark version in "spark-submit --version", so that user could leverage this output to make the choice without needing to launching application. ## How was this patch tested? Manually verified in local environment. Author: jerryshao Closes #15456 from jerryshao/SPARK-17686. --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 80611658a1640..5c052286099f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -24,6 +24,7 @@ import java.security.PrivilegedExceptionAction import scala.annotation.tailrec import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import scala.util.Properties import org.apache.commons.lang3.StringUtils import org.apache.hadoop.fs.Path @@ -47,7 +48,6 @@ import org.apache.spark.deploy.rest._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} - /** * Whether to submit, kill, or request the status of an application. * The latter two operations are currently supported only for standalone and Mesos cluster modes. @@ -104,6 +104,8 @@ object SparkSubmit { /___/ .__/\_,_/_/ /_/\_\ version %s /_/ """.format(SPARK_VERSION)) + printStream.println("Using Scala %s, %s, %s".format( + Properties.versionString, Properties.javaVmName, Properties.javaVersion)) printStream.println("Branch %s".format(SPARK_BRANCH)) printStream.println("Compiled by user %s on %s".format(SPARK_BUILD_USER, SPARK_BUILD_DATE)) printStream.println("Revision %s".format(SPARK_REVISION)) From 0a8e51a5e4cfd3275eff12e9fbbeb3fb487990aa Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 13 Oct 2016 21:36:39 +0800 Subject: [PATCH 075/177] [SPARK-17657][SQL] Disallow Users to Change Table Type ### What changes were proposed in this pull request? Hive allows users to change the table type from `Managed` to `External` or from `External` to `Managed` by altering table's property `EXTERNAL`. See the JIRA: https://issues.apache.org/jira/browse/HIVE-1329 So far, Spark SQL does not correctly support it, although users can do it. Many assumptions are broken in the implementation. Thus, this PR is to disallow users to change it. In addition, we also do not allow users to set the property `EXTERNAL` when creating a table. ### How was this patch tested? Added test cases Author: gatorsmile Closes #15230 from gatorsmile/alterTableSetExternal. --- .../spark/sql/hive/HiveExternalCatalog.scala | 5 +++ .../sql/hive/execution/HiveDDLSuite.scala | 32 +++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index ed189724a2dbd..237b829da882f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -112,6 +112,11 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat s"as table property keys may not start with '$DATASOURCE_PREFIX' or '$STATISTICS_PREFIX':" + s" ${invalidKeys.mkString("[", ", ", "]")}") } + // External users are not allowed to set/switch the table type. In Hive metastore, the table + // type can be switched by changing the value of a case-sensitive table property `EXTERNAL`. + if (table.properties.contains("EXTERNAL")) { + throw new AnalysisException("Cannot set or change the preserved property key: 'EXTERNAL'") + } } // -------------------------------------------------------------------------- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 8bff6de008fdb..3d1712e4354c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -315,6 +315,38 @@ class HiveDDLSuite assert(message.contains("Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) } + test("create table - SET TBLPROPERTIES EXTERNAL to TRUE") { + val tabName = "tab1" + withTable(tabName) { + val message = intercept[AnalysisException] { + sql(s"CREATE TABLE $tabName (height INT, length INT) TBLPROPERTIES('EXTERNAL'='TRUE')") + }.getMessage + assert(message.contains("Cannot set or change the preserved property key: 'EXTERNAL'")) + } + } + + test("alter table - SET TBLPROPERTIES EXTERNAL to TRUE") { + val tabName = "tab1" + withTable(tabName) { + val catalog = spark.sessionState.catalog + sql(s"CREATE TABLE $tabName (height INT, length INT)") + assert( + catalog.getTableMetadata(TableIdentifier(tabName)).tableType == CatalogTableType.MANAGED) + val message = intercept[AnalysisException] { + sql(s"ALTER TABLE $tabName SET TBLPROPERTIES ('EXTERNAL' = 'TRUE')") + }.getMessage + assert(message.contains("Cannot set or change the preserved property key: 'EXTERNAL'")) + // The table type is not changed to external + assert( + catalog.getTableMetadata(TableIdentifier(tabName)).tableType == CatalogTableType.MANAGED) + // The table property is case sensitive. Thus, external is allowed + sql(s"ALTER TABLE $tabName SET TBLPROPERTIES ('external' = 'TRUE')") + // The table type is not changed to external + assert( + catalog.getTableMetadata(TableIdentifier(tabName)).tableType == CatalogTableType.MANAGED) + } + } + test("alter views and alter table - misuse") { val tabName = "tab1" withTable(tabName) { From 04d417a7ca8ef694658b26fb697a035717414731 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 13 Oct 2016 11:12:30 -0700 Subject: [PATCH 076/177] [SPARK-17830][SQL] Annotate remaining SQL APIs with InterfaceStability ## What changes were proposed in this pull request? This patch annotates all the remaining APIs in SQL (excluding streaming) with InterfaceStability. ## How was this patch tested? N/A - just annotation change. Author: Reynold Xin Closes #15457 from rxin/SPARK-17830-2. --- .../java/org/apache/spark/sql/SaveMode.java | 3 +++ .../org/apache/spark/sql/api/java/UDF1.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF10.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF11.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF12.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF13.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF14.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF15.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF16.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF17.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF18.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF19.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF2.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF20.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF21.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF22.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF3.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF4.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF5.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF6.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF7.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF8.java | 8 +++--- .../org/apache/spark/sql/api/java/UDF9.java | 8 +++--- .../spark/sql/expressions/javalang/typed.java | 2 ++ .../apache/spark/sql/catalog/Catalog.scala | 9 ++++++- .../apache/spark/sql/catalog/interface.scala | 5 ++++ .../spark/sql/expressions/Aggregator.scala | 3 ++- .../sql/expressions/UserDefinedFunction.scala | 3 ++- .../apache/spark/sql/expressions/Window.scala | 4 ++- .../spark/sql/expressions/WindowSpec.scala | 7 ++--- .../sql/expressions/scalalang/typed.scala | 3 ++- .../apache/spark/sql/expressions/udaf.scala | 8 +++++- .../apache/spark/sql/jdbc/JdbcDialects.scala | 5 +++- .../apache/spark/sql/sources/filters.scala | 18 +++++++++++++ .../apache/spark/sql/sources/interfaces.scala | 26 +++++++++++++++++-- 35 files changed, 150 insertions(+), 122 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java index 9665c3c46f901..1c3c9794fb6bb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java +++ b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java @@ -16,11 +16,14 @@ */ package org.apache.spark.sql; +import org.apache.spark.annotation.InterfaceStability; + /** * SaveMode is used to specify the expected behavior of saving a DataFrame to a data source. * * @since 1.3.0 */ +@InterfaceStability.Stable public enum SaveMode { /** * Append mode means that when saving a DataFrame to a data source, if data/table already exists, diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java index ef959e35e1027..1460daf27dc20 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 1 arguments. */ +@InterfaceStability.Stable public interface UDF1 extends Serializable { - public R call(T1 t1) throws Exception; + R call(T1 t1) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java index 96ab3a96c3d5e..7c4f1e4897084 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 10 arguments. */ +@InterfaceStability.Stable public interface UDF10 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java index 58ae8edd6d817..26a05106aebd6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 11 arguments. */ +@InterfaceStability.Stable public interface UDF11 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java index d9da0f6eddd94..8ef7a99042025 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 12 arguments. */ +@InterfaceStability.Stable public interface UDF12 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java index 095fc1a8076b5..5c3b2ec1222e2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 13 arguments. */ +@InterfaceStability.Stable public interface UDF13 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java index eb27eaa180086..97e744d843466 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 14 arguments. */ +@InterfaceStability.Stable public interface UDF14 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java index 1fbcff56332b6..7ddbf914fc11a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 15 arguments. */ +@InterfaceStability.Stable public interface UDF15 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java index 1133561787a69..0ae5dc7195ad6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 16 arguments. */ +@InterfaceStability.Stable public interface UDF16 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java index dfae7922c9b63..03543a556c614 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 17 arguments. */ +@InterfaceStability.Stable public interface UDF17 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java index e9d1c6d52d4ea..46740d3443916 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 18 arguments. */ +@InterfaceStability.Stable public interface UDF18 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java index 46b9d2d3c9457..33fefd8ecaf1d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 19 arguments. */ +@InterfaceStability.Stable public interface UDF19 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java index cd3fde8da419e..9822f19217d76 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 2 arguments. */ +@InterfaceStability.Stable public interface UDF2 extends Serializable { - public R call(T1 t1, T2 t2) throws Exception; + R call(T1 t1, T2 t2) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java index 113d3d26be4a7..8c5e90182da1c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 20 arguments. */ +@InterfaceStability.Stable public interface UDF20 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java index 74118f2cf8da7..e3b09f5167cff 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 21 arguments. */ +@InterfaceStability.Stable public interface UDF21 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java index 0e7cc40be45ec..dc6cfa9097bab 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 22 arguments. */ +@InterfaceStability.Stable public interface UDF22 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java index 6a880f16be47a..7c264b69ba195 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 3 arguments. */ +@InterfaceStability.Stable public interface UDF3 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3) throws Exception; + R call(T1 t1, T2 t2, T3 t3) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java index fcad2febb18e6..58df38fc3c911 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 4 arguments. */ +@InterfaceStability.Stable public interface UDF4 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java index ce0cef43a2144..4146f96e2eed5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 5 arguments. */ +@InterfaceStability.Stable public interface UDF5 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java index f56b806684e61..25d39654c1095 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 6 arguments. */ +@InterfaceStability.Stable public interface UDF6 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java index 25bd6d3241bd4..ce63b6a91adbb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 7 arguments. */ +@InterfaceStability.Stable public interface UDF7 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java index a3b7ac5f94ce7..0e00209ef6b9f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 8 arguments. */ +@InterfaceStability.Stable public interface UDF8 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java index 205e72a1522fc..077981bb3e3ee 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java @@ -19,14 +19,12 @@ import java.io.Serializable; -// ************************************************** -// THIS FILE IS AUTOGENERATED BY CODE IN -// org.apache.spark.sql.api.java.FunctionRegistration -// ************************************************** +import org.apache.spark.annotation.InterfaceStability; /** * A Spark SQL UDF that has 9 arguments. */ +@InterfaceStability.Stable public interface UDF9 extends Serializable { - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; + R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java index 247e94b86c349..ec9c107b1c119 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java +++ b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java @@ -18,6 +18,7 @@ package org.apache.spark.sql.expressions.javalang; import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.TypedColumn; import org.apache.spark.sql.execution.aggregate.TypedAverage; @@ -34,6 +35,7 @@ * @since 2.0.0 */ @Experimental +@InterfaceStability.Evolving public class typed { // Note: make sure to keep in sync with typed.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 18cba8ce28b4d..889b8a02784d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalog -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -27,6 +27,7 @@ import org.apache.spark.sql.types.StructType * * @since 2.0.0 */ +@InterfaceStability.Stable abstract class Catalog { /** @@ -193,6 +194,7 @@ abstract class Catalog { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createExternalTable(tableName: String, path: String): DataFrame /** @@ -203,6 +205,7 @@ abstract class Catalog { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createExternalTable(tableName: String, path: String, source: String): DataFrame /** @@ -213,6 +216,7 @@ abstract class Catalog { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createExternalTable( tableName: String, source: String, @@ -227,6 +231,7 @@ abstract class Catalog { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createExternalTable( tableName: String, source: String, @@ -240,6 +245,7 @@ abstract class Catalog { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createExternalTable( tableName: String, source: String, @@ -255,6 +261,7 @@ abstract class Catalog { * @since 2.0.0 */ @Experimental + @InterfaceStability.Evolving def createExternalTable( tableName: String, source: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala index 33032f07f7bea..c0c5ebc2ba2d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalog import javax.annotation.Nullable +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.DefinedByConstructorParams @@ -33,6 +34,7 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams * @param locationUri path (in the form of a uri) to data files. * @since 2.0.0 */ +@InterfaceStability.Stable class Database( val name: String, @Nullable val description: String, @@ -59,6 +61,7 @@ class Database( * @param isTemporary whether the table is a temporary table. * @since 2.0.0 */ +@InterfaceStability.Stable class Table( val name: String, @Nullable val database: String, @@ -90,6 +93,7 @@ class Table( * @param isBucket whether the column is a bucket column. * @since 2.0.0 */ +@InterfaceStability.Stable class Column( val name: String, @Nullable val description: String, @@ -122,6 +126,7 @@ class Column( * @param isTemporary whether the function is a temporary function or not. * @since 2.0.0 */ +@InterfaceStability.Stable class Function( val name: String, @Nullable val database: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala index 51179a528c503..eea98414003ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.{Dataset, Encoder, TypedColumn} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} @@ -51,6 +51,7 @@ import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression * @since 1.6.0 */ @Experimental +@InterfaceStability.Evolving abstract class Aggregator[-IN, BUF, OUT] extends Serializable { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 49fdec57558e8..2e0e937e4aff7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.catalyst.expressions.ScalaUDF import org.apache.spark.sql.Column import org.apache.spark.sql.functions @@ -40,6 +40,7 @@ import org.apache.spark.sql.types.DataType * @since 1.3.0 */ @Experimental +@InterfaceStability.Evolving case class UserDefinedFunction protected[sql] ( f: AnyRef, dataType: DataType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index 3c1f6e897ea62..07ef60183f6fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions._ @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.expressions._ * @since 1.4.0 */ @Experimental +@InterfaceStability.Evolving object Window { /** @@ -177,4 +178,5 @@ object Window { * @since 1.4.0 */ @Experimental +@InterfaceStability.Evolving class Window private() // So we can see Window in JavaDoc. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index 8ebed399bf2d0..18778c8d1c294 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.{catalyst, Column} +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions._ /** @@ -30,10 +30,11 @@ import org.apache.spark.sql.catalyst.expressions._ * @since 1.4.0 */ @Experimental +@InterfaceStability.Evolving class WindowSpec private[sql]( partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], - frame: catalyst.expressions.WindowFrame) { + frame: WindowFrame) { /** * Defines the partitioning columns in a [[WindowSpec]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala index 60d7b7d0894d0..aa71cb9e3bc85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions.scalalang -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql._ import org.apache.spark.sql.execution.aggregate._ @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.aggregate._ * @since 2.0.0 */ @Experimental +@InterfaceStability.Evolving // scalastyle:off object typed { // scalastyle:on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index 5417a0e481158..ef7c09c72b82d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.{Column, Row} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.ScalaUDAF @@ -26,8 +26,11 @@ import org.apache.spark.sql.types._ /** * :: Experimental :: * The base class for implementing user-defined aggregate functions (UDAF). + * + * @since 1.5.0 */ @Experimental +@InterfaceStability.Evolving abstract class UserDefinedAggregateFunction extends Serializable { /** @@ -136,8 +139,11 @@ abstract class UserDefinedAggregateFunction extends Serializable { * A [[Row]] representing a mutable aggregation buffer. * * This is not meant to be extended outside of Spark. + * + * @since 1.5.0 */ @Experimental +@InterfaceStability.Evolving abstract class MutableAggregationBuffer extends Row { /** Update the ith value of this buffer. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 8dd4b8f662713..dec316be7aea1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.jdbc import java.sql.Connection -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.{DeveloperApi, InterfaceStability, Since} import org.apache.spark.sql.types._ /** @@ -31,6 +31,7 @@ import org.apache.spark.sql.types._ * send a null value to the database. */ @DeveloperApi +@InterfaceStability.Evolving case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) /** @@ -53,6 +54,7 @@ case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) * for the given Catalyst type. */ @DeveloperApi +@InterfaceStability.Evolving abstract class JdbcDialect extends Serializable { /** * Check if this dialect instance can handle a certain jdbc url. @@ -142,6 +144,7 @@ abstract class JdbcDialect extends Serializable { * sure to register your dialects first. */ @DeveloperApi +@InterfaceStability.Evolving object JdbcDialects { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala index 13c0766219a8e..e0494dfd9343b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.sources +import org.apache.spark.annotation.InterfaceStability + //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines all the filters that we can push down to the data sources. //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -26,6 +28,7 @@ package org.apache.spark.sql.sources * * @since 1.3.0 */ +@InterfaceStability.Stable abstract class Filter { /** * List of columns that are referenced by this filter. @@ -45,6 +48,7 @@ abstract class Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class EqualTo(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -56,6 +60,7 @@ case class EqualTo(attribute: String, value: Any) extends Filter { * * @since 1.5.0 */ +@InterfaceStability.Stable case class EqualNullSafe(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -66,6 +71,7 @@ case class EqualNullSafe(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class GreaterThan(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -76,6 +82,7 @@ case class GreaterThan(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -86,6 +93,7 @@ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class LessThan(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -96,6 +104,7 @@ case class LessThan(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class LessThanOrEqual(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -105,6 +114,7 @@ case class LessThanOrEqual(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class In(attribute: String, values: Array[Any]) extends Filter { override def hashCode(): Int = { var h = attribute.hashCode @@ -131,6 +141,7 @@ case class In(attribute: String, values: Array[Any]) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class IsNull(attribute: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -140,6 +151,7 @@ case class IsNull(attribute: String) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class IsNotNull(attribute: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -149,6 +161,7 @@ case class IsNotNull(attribute: String) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class And(left: Filter, right: Filter) extends Filter { override def references: Array[String] = left.references ++ right.references } @@ -158,6 +171,7 @@ case class And(left: Filter, right: Filter) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class Or(left: Filter, right: Filter) extends Filter { override def references: Array[String] = left.references ++ right.references } @@ -167,6 +181,7 @@ case class Or(left: Filter, right: Filter) extends Filter { * * @since 1.3.0 */ +@InterfaceStability.Stable case class Not(child: Filter) extends Filter { override def references: Array[String] = child.references } @@ -177,6 +192,7 @@ case class Not(child: Filter) extends Filter { * * @since 1.3.1 */ +@InterfaceStability.Stable case class StringStartsWith(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -187,6 +203,7 @@ case class StringStartsWith(attribute: String, value: String) extends Filter { * * @since 1.3.1 */ +@InterfaceStability.Stable case class StringEndsWith(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -197,6 +214,7 @@ case class StringEndsWith(attribute: String, value: String) extends Filter { * * @since 1.3.1 */ +@InterfaceStability.Stable case class StringContains(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 6484c782b5d15..3172d5ded9504 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -37,6 +37,7 @@ import org.apache.spark.sql.types.StructType * @since 1.5.0 */ @DeveloperApi +@InterfaceStability.Evolving trait DataSourceRegister { /** @@ -68,6 +69,7 @@ trait DataSourceRegister { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving trait RelationProvider { /** * Returns a new base relation with the given parameters. @@ -99,6 +101,7 @@ trait RelationProvider { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving trait SchemaRelationProvider { /** * Returns a new base relation with the given parameters and user defined schema. @@ -114,17 +117,26 @@ trait SchemaRelationProvider { /** * ::Experimental:: * Implemented by objects that can produce a streaming [[Source]] for a specific format or system. + * + * @since 2.0.0 */ @Experimental +@InterfaceStability.Unstable trait StreamSourceProvider { - /** Returns the name and schema of the source that can be used to continually read data. */ + /** + * Returns the name and schema of the source that can be used to continually read data. + * @since 2.0.0 + */ def sourceSchema( sqlContext: SQLContext, schema: Option[StructType], providerName: String, parameters: Map[String, String]): (String, StructType) + /** + * @since 2.0.0 + */ def createSource( sqlContext: SQLContext, metadataPath: String, @@ -136,8 +148,11 @@ trait StreamSourceProvider { /** * ::Experimental:: * Implemented by objects that can produce a streaming [[Sink]] for a specific format or system. + * + * @since 2.0.0 */ @Experimental +@InterfaceStability.Unstable trait StreamSinkProvider { def createSink( sqlContext: SQLContext, @@ -150,6 +165,7 @@ trait StreamSinkProvider { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving trait CreatableRelationProvider { /** * Save the DataFrame to the destination and return a relation with the given parameters based on @@ -186,6 +202,7 @@ trait CreatableRelationProvider { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving abstract class BaseRelation { def sqlContext: SQLContext def schema: StructType @@ -237,6 +254,7 @@ abstract class BaseRelation { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving trait TableScan { def buildScan(): RDD[Row] } @@ -249,6 +267,7 @@ trait TableScan { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving trait PrunedScan { def buildScan(requiredColumns: Array[String]): RDD[Row] } @@ -268,6 +287,7 @@ trait PrunedScan { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving trait PrunedFilteredScan { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } @@ -291,6 +311,7 @@ trait PrunedFilteredScan { * @since 1.3.0 */ @DeveloperApi +@InterfaceStability.Evolving trait InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit } @@ -306,6 +327,7 @@ trait InsertableRelation { * @since 1.3.0 */ @Experimental +@InterfaceStability.Unstable trait CatalystScan { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } From 84f149e414475c2e60863898992001c21cfc13b2 Mon Sep 17 00:00:00 2001 From: Pete Robbins Date: Thu, 13 Oct 2016 11:26:30 -0700 Subject: [PATCH 077/177] [SPARK-17827][SQL] maxColLength type should be Int for String and Binary ## What changes were proposed in this pull request? correct the expected type from Length function to be Int ## How was this patch tested? Test runs on little endian and big endian platforms Author: Pete Robbins Closes #15464 from robbinspg/SPARK-17827. --- .../spark/sql/catalyst/plans/logical/Statistics.scala | 4 ++-- .../org/apache/spark/sql/StatisticsColumnSuite.scala | 8 ++++---- .../scala/org/apache/spark/sql/hive/StatisticsSuite.scala | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index 43455c989c0f4..f3e2147b8f974 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -98,7 +98,7 @@ case class StringColumnStat(statRow: InternalRow) { // The indices here must be consistent with `ColumnStatStruct.stringColumnStat`. val numNulls: Long = statRow.getLong(0) val avgColLen: Double = statRow.getDouble(1) - val maxColLen: Long = statRow.getLong(2) + val maxColLen: Long = statRow.getInt(2) val ndv: Long = statRow.getLong(3) } @@ -106,7 +106,7 @@ case class BinaryColumnStat(statRow: InternalRow) { // The indices here must be consistent with `ColumnStatStruct.binaryColumnStat`. val numNulls: Long = statRow.getLong(0) val avgColLen: Double = statRow.getDouble(1) - val maxColLen: Long = statRow.getLong(2) + val maxColLen: Long = statRow.getInt(2) } case class BooleanColumnStat(statRow: InternalRow) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala index 0ee0547c45591..f1a201abd8da6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala @@ -150,7 +150,7 @@ class StatisticsColumnSuite extends StatisticsTest { val colStat = ColumnStat(InternalRow( values.count(_.isEmpty).toLong, nonNullValues.map(_.length).sum / nonNullValues.length.toDouble, - nonNullValues.map(_.length).max.toLong, + nonNullValues.map(_.length).max.toInt, nonNullValues.distinct.length.toLong)) (f, colStat) } @@ -165,7 +165,7 @@ class StatisticsColumnSuite extends StatisticsTest { val colStat = ColumnStat(InternalRow( values.count(_.isEmpty).toLong, nonNullValues.map(_.length).sum / nonNullValues.length.toDouble, - nonNullValues.map(_.length).max.toLong)) + nonNullValues.map(_.length).max.toInt)) (f, colStat) } checkColStats(df, expectedColStatsSeq) @@ -255,10 +255,10 @@ class StatisticsColumnSuite extends StatisticsTest { doubleSeq.distinct.length.toLong)) case StringType => ColumnStat(InternalRow(0L, stringSeq.map(_.length).sum / stringSeq.length.toDouble, - stringSeq.map(_.length).max.toLong, stringSeq.distinct.length.toLong)) + stringSeq.map(_.length).max.toInt, stringSeq.distinct.length.toLong)) case BinaryType => ColumnStat(InternalRow(0L, binarySeq.map(_.length).sum / binarySeq.length.toDouble, - binarySeq.map(_.length).max.toLong)) + binarySeq.map(_.length).max.toInt)) case BooleanType => ColumnStat(InternalRow(0L, booleanSeq.count(_.equals(true)).toLong, booleanSeq.count(_.equals(false)).toLong)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 99dd080683d40..85228bb00123d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -378,7 +378,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils ColumnStat(InternalRow(0L, intSeq.max, intSeq.min, intSeq.distinct.length.toLong)) case StringType => ColumnStat(InternalRow(0L, stringSeq.map(_.length).sum / stringSeq.length.toDouble, - stringSeq.map(_.length).max.toLong, stringSeq.distinct.length.toLong)) + stringSeq.map(_.length).max.toInt, stringSeq.distinct.length.toLong)) case BooleanType => ColumnStat(InternalRow(0L, booleanSeq.count(_.equals(true)).toLong, booleanSeq.count(_.equals(false)).toLong)) From 08eac356095c7faa2b19d52f2fb0cbc47eb7d1d1 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 13 Oct 2016 13:31:50 -0700 Subject: [PATCH 078/177] [SPARK-17834][SQL] Fetch the earliest offsets manually in KafkaSource instead of counting on KafkaConsumer ## What changes were proposed in this pull request? Because `KafkaConsumer.poll(0)` may update the partition offsets, this PR just calls `seekToBeginning` to manually set the earliest offsets for the KafkaSource initial offsets. ## How was this patch tested? Existing tests. Author: Shixiong Zhu Closes #15397 from zsxwing/SPARK-17834. --- .../spark/sql/kafka010/KafkaSource.scala | 55 ++++++++++++------- .../sql/kafka010/KafkaSourceProvider.scala | 19 +++++-- 2 files changed, 48 insertions(+), 26 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 1be70db87497e..4b0bb0a0f725c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -82,6 +82,7 @@ private[kafka010] case class KafkaSource( executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, + startFromEarliestOffset: Boolean, failOnDataLoss: Boolean) extends Source with Logging { @@ -109,7 +110,11 @@ private[kafka010] case class KafkaSource( private lazy val initialPartitionOffsets = { val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) metadataLog.get(0).getOrElse { - val offsets = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = false)) + val offsets = if (startFromEarliestOffset) { + KafkaSourceOffset(fetchEarliestOffsets()) + } else { + KafkaSourceOffset(fetchLatestOffsets()) + } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") offsets @@ -123,7 +128,7 @@ private[kafka010] case class KafkaSource( // Make sure initialPartitionOffsets is initialized initialPartitionOffsets - val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = true)) + val offset = KafkaSourceOffset(fetchLatestOffsets()) logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") Some(offset) } @@ -227,26 +232,34 @@ private[kafka010] case class KafkaSource( override def toString(): String = s"KafkaSource[$consumerStrategy]" /** - * Fetch the offset of a partition, either seek to the latest offsets or use the current offsets - * in the consumer. + * Fetch the earliest offsets of partitions. */ - private def fetchPartitionOffsets( - seekToEnd: Boolean): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) - assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + private def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() consumer.pause(partitions) - logDebug(s"Partitioned assigned to consumer: $partitions") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") - // Get the current or latest offset of each partition - if (seekToEnd) { - consumer.seekToEnd(partitions) - logDebug("Seeked to the end") - } + consumer.seekToBeginning(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for partition : $partitionOffsets") + partitionOffsets + } + + /** + * Fetch the latest offset of partitions. + */ + private def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") + + consumer.seekToEnd(partitions) val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got offsets for partition : $partitionOffsets") + logDebug(s"Got latest offsets for partition : $partitionOffsets") partitionOffsets } @@ -256,22 +269,21 @@ private[kafka010] case class KafkaSource( */ private def fetchNewPartitionEarliestOffsets( newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) - assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + consumer.pause(partitions) logDebug(s"\tPartitioned assigned to consumer: $partitions") // Get the earliest offset of each partition consumer.seekToBeginning(partitions) - val partitionToOffsets = newPartitions.filter { p => + val partitionOffsets = newPartitions.filter { p => // When deleting topics happen at the same time, some partitions may not be in `partitions`. // So we need to ignore them partitions.contains(p) }.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got offsets for new partitions: $partitionToOffsets") - partitionToOffsets + logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") + partitionOffsets } /** @@ -284,6 +296,9 @@ private[kafka010] case class KafkaSource( */ private def withRetriesWithoutInterrupt( body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + synchronized { var result: Option[Map[TopicPartition, Long]] = None var attempt = 1 diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 1b0a2fe955d03..23b1b60f3bcaa 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -77,10 +77,15 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // id. Hence, we should generate a unique id for each query. val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" - val autoOffsetResetValue = caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { - case Some(value) => value.trim() // same values as those supported by auto.offset.reset - case None => "latest" - } + val startFromEarliestOffset = + caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => false + case Some("earliest") => true + case Some(pos) => + // This should not happen since we have already checked the options. + throw new IllegalStateException(s"Invalid $STARTING_OFFSET_OPTION_KEY: $pos") + case None => false + } val kafkaParamsForStrategy = ConfigUpdater("source", specifiedKafkaParams) @@ -90,8 +95,9 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // So that consumers in Kafka source do not mess with any existing group id .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver") - // So that consumers can start from earliest or latest - .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetResetValue) + // Set to "latest" to avoid exceptions. However, KafkaSource will fetch the initial offsets + // by itself instead of counting on KafkaConsumer. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") // So that consumers in the driver does not commit offsets unnecessarily .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -147,6 +153,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider kafkaParamsForExecutors, parameters, metadataPath, + startFromEarliestOffset, failOnDataLoss) } From 7106866c220c73960c6fe2a70e4911516617e21f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 13 Oct 2016 13:36:26 -0700 Subject: [PATCH 079/177] [SPARK-17731][SQL][STREAMING] Metrics for structured streaming ## What changes were proposed in this pull request? Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics. https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing Specifically, this PR adds the following public APIs changes. ### New APIs - `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later) - `StreamingQueryStatus` has the following important fields - inputRate - Current rate (rows/sec) at which data is being generated by all the sources - processingRate - Current rate (rows/sec) at which the query is processing data from all the sources - ~~outputRate~~ - *Does not work with wholestage codegen* - latency - Current average latency between the data being available in source and the sink writing the corresponding output - sourceStatuses: Array[SourceStatus] - Current statuses of the sources - sinkStatus: SinkStatus - Current status of the sink - triggerStatus - Low-level detailed status of the last completed/currently active trigger - latencies - getOffset, getBatch, full trigger, wal writes - timestamps - trigger start, finish, after getOffset, after getBatch - numRows - input, output, state total/updated rows for aggregations - `SourceStatus` has the following important fields - inputRate - Current rate (rows/sec) at which data is being generated by the source - processingRate - Current rate (rows/sec) at which the query is processing data from the source - triggerStatus - Low-level detailed status of the last completed/currently active trigger - Python API for `StreamingQuery.status()` ### Breaking changes to existing APIs **Existing direct public facing APIs** - Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`. - Branch 2.0 should have it deprecated, master should have it removed. **Existing advanced listener APIs** - `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus` - Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status) - Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`. - Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`. - For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java. ## How was this patch tested? Old and new unit tests. - Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite. - New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite. - New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite. - Source-specific tests for making sure input rows are counted are is source-specific test suites. - Additional tests to test minor additions in LocalTableScanExec, StateStore, etc. Metrics also manually tested using Ganglia sink Author: Tathagata Das Closes #15307 from tdas/SPARK-17731. --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 27 ++ project/MimaExcludes.scala | 13 + python/pyspark/sql/streaming.py | 301 +++++++++++++++++ .../spark/sql/catalyst/trees/TreeNode.scala | 7 + .../sql/execution/LocalTableScanExec.scala | 5 +- .../streaming/StatefulAggregate.scala | 31 +- .../execution/streaming/StreamExecution.scala | 307 ++++++++++++++---- .../execution/streaming/StreamMetrics.scala | 242 ++++++++++++++ .../sql/execution/streaming/memory.scala | 7 + .../state/HDFSBackedStateStoreProvider.scala | 2 + .../streaming/state/StateStore.scala | 3 + .../apache/spark/sql/internal/SQLConf.scala | 8 + .../spark/sql/streaming/SinkStatus.scala | 28 +- .../spark/sql/streaming/SourceStatus.scala | 54 ++- .../spark/sql/streaming/StreamingQuery.scala | 13 +- .../sql/streaming/StreamingQueryInfo.scala | 37 --- .../streaming/StreamingQueryListener.scala | 8 +- .../sql/streaming/StreamingQueryStatus.scala | 139 ++++++++ .../execution/metric/SQLMetricsSuite.scala | 17 + .../streaming/StreamMetricsSuite.scala | 213 ++++++++++++ .../streaming/TextSocketStreamSuite.scala | 24 ++ .../streaming/state/StateStoreSuite.scala | 5 + .../sql/streaming/FileStreamSourceSuite.scala | 14 + .../spark/sql/streaming/StreamTest.scala | 72 ++++ .../streaming/StreamingAggregationSuite.scala | 54 +++ .../StreamingQueryListenerSuite.scala | 220 +++++-------- .../sql/streaming/StreamingQuerySuite.scala | 180 +++++++++- 27 files changed, 1758 insertions(+), 273 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index c640b93b0a2ee..8b5296ea135c7 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -264,6 +264,33 @@ class KafkaSourceSuite extends KafkaSourceTest { testUnsupportedConfig("kafka.auto.offset.reset", "latest") } + test("input row metrics") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val kafka = spark + .readStream + .format("kafka") + .option("subscribe", topic) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + AssertOnLastQueryStatus { status => + assert(status.triggerDetails.get("numRows.input.total").toInt > 0) + assert(status.sourceStatuses(0).processingRate > 0.0) + } + ) + } + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ae72d37a0b61c..1349af4219c16 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -56,6 +56,19 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.databaseExists"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.tableExists"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"), + + // [SPARK-17731][SQL][Streaming] Metrics for structured streaming + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.SourceStatus.offsetDesc"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SinkStatus.this"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo"), + // [SPARK-17338][SQL] add global temp view ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 4e438fd5bee22..ce47bd1640fb1 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -189,6 +189,304 @@ def resetTerminated(self): self._jsqm.resetTerminated() +class StreamingQueryStatus(object): + """A class used to report information about the progress of a StreamingQuery. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jsqs): + self._jsqs = jsqs + + def __str__(self): + """ + Pretty string of this query status. + + >>> print(sqs) + StreamingQueryStatus: + Query name: query + Query id: 1 + Status timestamp: 123 + Input rate: 15.5 rows/sec + Processing rate 23.5 rows/sec + Latency: 345.0 ms + Trigger details: + isDataPresentInTrigger: true + isTriggerActive: true + latency.getBatch.total: 20 + latency.getOffset.total: 10 + numRows.input.total: 100 + triggerId: 5 + Source statuses [1 source]: + Source 1: MySource1 + Available offset: #0 + Input rate: 15.5 rows/sec + Processing rate: 23.5 rows/sec + Trigger details: + numRows.input.source: 100 + latency.getOffset.source: 10 + latency.getBatch.source: 20 + Sink status: MySink + Committed offsets: [#1, -] + """ + return self._jsqs.toString() + + @property + @ignore_unicode_prefix + @since(2.1) + def name(self): + """ + Name of the query. This name is unique across all active queries. + + >>> sqs.name + u'query' + """ + return self._jsqs.name() + + @property + @since(2.1) + def id(self): + """ + Id of the query. This id is unique across all queries that have been started in + the current process. + + >>> int(sqs.id) + 1 + """ + return self._jsqs.id() + + @property + @since(2.1) + def timestamp(self): + """ + Timestamp (ms) of when this query was generated. + + >>> int(sqs.timestamp) + 123 + """ + return self._jsqs.timestamp() + + @property + @since(2.1) + def inputRate(self): + """ + Current total rate (rows/sec) at which data is being generated by all the sources. + + >>> sqs.inputRate + 15.5 + """ + return self._jsqs.inputRate() + + @property + @since(2.1) + def processingRate(self): + """ + Current rate (rows/sec) at which the query is processing data from all the sources. + + >>> sqs.processingRate + 23.5 + """ + return self._jsqs.processingRate() + + @property + @since(2.1) + def latency(self): + """ + Current average latency between the data being available in source and the sink + writing the corresponding output. + + >>> sqs.latency + 345.0 + """ + if (self._jsqs.latency().nonEmpty()): + return self._jsqs.latency().get() + else: + return None + + @property + @ignore_unicode_prefix + @since(2.1) + def sourceStatuses(self): + """ + Current statuses of the sources as a list. + + >>> len(sqs.sourceStatuses) + 1 + >>> sqs.sourceStatuses[0].description + u'MySource1' + """ + return [SourceStatus(ss) for ss in self._jsqs.sourceStatuses()] + + @property + @ignore_unicode_prefix + @since(2.1) + def sinkStatus(self): + """ + Current status of the sink. + + >>> sqs.sinkStatus.description + u'MySink' + """ + return SinkStatus(self._jsqs.sinkStatus()) + + @property + @ignore_unicode_prefix + @since(2.1) + def triggerDetails(self): + """ + Low-level details of the currently active trigger (e.g. number of rows processed + in trigger, latency of intermediate steps, etc.). + + If no trigger is currently active, then it will have details of the last completed trigger. + + >>> sqs.triggerDetails + {u'triggerId': u'5', u'latency.getBatch.total': u'20', u'numRows.input.total': u'100', + u'isTriggerActive': u'true', u'latency.getOffset.total': u'10', + u'isDataPresentInTrigger': u'true'} + """ + return self._jsqs.triggerDetails() + + +class SourceStatus(object): + """ + Status and metrics of a streaming Source. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jss): + self._jss = jss + + def __str__(self): + """ + Pretty string of this source status. + + >>> print(sqs.sourceStatuses[0]) + SourceStatus: MySource1 + Available offset: #0 + Input rate: 15.5 rows/sec + Processing rate: 23.5 rows/sec + Trigger details: + numRows.input.source: 100 + latency.getOffset.source: 10 + latency.getBatch.source: 20 + """ + return self._jss.toString() + + @property + @ignore_unicode_prefix + @since(2.1) + def description(self): + """ + Description of the source corresponding to this status. + + >>> sqs.sourceStatuses[0].description + u'MySource1' + """ + return self._jss.description() + + @property + @ignore_unicode_prefix + @since(2.1) + def offsetDesc(self): + """ + Description of the current offset if known. + + >>> sqs.sourceStatuses[0].offsetDesc + u'#0' + """ + return self._jss.offsetDesc() + + @property + @since(2.1) + def inputRate(self): + """ + Current rate (rows/sec) at which data is being generated by the source. + + >>> sqs.sourceStatuses[0].inputRate + 15.5 + """ + return self._jss.inputRate() + + @property + @since(2.1) + def processingRate(self): + """ + Current rate (rows/sec) at which the query is processing data from the source. + + >>> sqs.sourceStatuses[0].processingRate + 23.5 + """ + return self._jss.processingRate() + + @property + @ignore_unicode_prefix + @since(2.1) + def triggerDetails(self): + """ + Low-level details of the currently active trigger (e.g. number of rows processed + in trigger, latency of intermediate steps, etc.). + + If no trigger is currently active, then it will have details of the last completed trigger. + + >>> sqs.sourceStatuses[0].triggerDetails + {u'numRows.input.source': u'100', u'latency.getOffset.source': u'10', + u'latency.getBatch.source': u'20'} + """ + return self._jss.triggerDetails() + + +class SinkStatus(object): + """ + Status and metrics of a streaming Sink. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jss): + self._jss = jss + + def __str__(self): + """ + Pretty string of this source status. + + >>> print(sqs.sinkStatus) + SinkStatus: MySink + Committed offsets: [#1, -] + """ + return self._jss.toString() + + @property + @ignore_unicode_prefix + @since(2.1) + def description(self): + """ + Description of the source corresponding to this status. + + >>> sqs.sinkStatus.description + u'MySink' + """ + return self._jss.description() + + @property + @ignore_unicode_prefix + @since(2.1) + def offsetDesc(self): + """ + Description of the current offsets up to which data has been written by the sink. + + >>> sqs.sinkStatus.offsetDesc + u'[#1, -]' + """ + return self._jss.offsetDesc() + + class Trigger(object): """Used to indicate how often results should be produced by a :class:`StreamingQuery`. @@ -753,11 +1051,14 @@ def _test(): globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) globs['df'] = \ globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') + globs['sqs'] = StreamingQueryStatus( + spark.sparkContext._jvm.org.apache.spark.sql.streaming.StreamingQueryStatus.testStatus()) (failure_count, test_count) = doctest.testmod( pyspark.sql.streaming, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) globs['spark'].stop() + if failure_count: exit(-1) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 83cb375525832..ea8d8fef7bdf1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -164,6 +164,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { ret } + /** + * Returns a Seq containing the leaves in this tree. + */ + def collectLeaves(): Seq[BaseType] = { + this.collect { case p if p.children.isEmpty => p } + } + /** * Finds and returns the first [[TreeNode]] of the tree for which the given partial function * is defined (pre-order), and applies the partial function to it. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index 6598fa381aa3d..e366b9af35c62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -64,10 +64,13 @@ case class LocalTableScanExec( } override def executeCollect(): Array[InternalRow] = { + longMetric("numOutputRows").add(unsafeRows.size) unsafeRows } override def executeTake(limit: Int): Array[InternalRow] = { - unsafeRows.take(limit) + val taken = unsafeRows.take(limit) + longMetric("numOutputRows").add(taken.size) + taken } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index 4d0283fbef1d0..587ea7d02acab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.execution.SparkPlan @@ -56,7 +57,12 @@ case class StateStoreRestoreExec( child: SparkPlan) extends execution.UnaryExecNode with StatefulOperator { + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + override protected def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + child.execute().mapPartitionsWithStateStore( getStateId.checkpointLocation, operatorId = getStateId.operatorId, @@ -69,6 +75,7 @@ case class StateStoreRestoreExec( iter.flatMap { row => val key = getKey(row) val savedState = store.get(key) + numOutputRows += 1 row +: savedState.toSeq } } @@ -86,7 +93,13 @@ case class StateStoreSaveExec( child: SparkPlan) extends execution.UnaryExecNode with StatefulOperator { + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), + "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows")) + override protected def doExecute(): RDD[InternalRow] = { + metrics // force lazy init at driver assert(returnAllStates.nonEmpty, "Incorrect planning in IncrementalExecution, returnAllStates have not been set") val saveAndReturnFunc = if (returnAllStates.get) saveAndReturnAll _ else saveAndReturnUpdated _ @@ -111,6 +124,10 @@ case class StateStoreSaveExec( private def saveAndReturnUpdated( store: StateStore, iter: Iterator[InternalRow]): Iterator[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val numTotalStateRows = longMetric("numTotalStateRows") + val numUpdatedStateRows = longMetric("numUpdatedStateRows") + new Iterator[InternalRow] { private[this] val baseIterator = iter private[this] val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) @@ -118,6 +135,7 @@ case class StateStoreSaveExec( override def hasNext: Boolean = { if (!baseIterator.hasNext) { store.commit() + numTotalStateRows += store.numKeys() false } else { true @@ -128,6 +146,8 @@ case class StateStoreSaveExec( val row = baseIterator.next().asInstanceOf[UnsafeRow] val key = getKey(row) store.put(key.copy(), row.copy()) + numOutputRows += 1 + numUpdatedStateRows += 1 row } } @@ -142,12 +162,21 @@ case class StateStoreSaveExec( store: StateStore, iter: Iterator[InternalRow]): Iterator[InternalRow] = { val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) + val numOutputRows = longMetric("numOutputRows") + val numTotalStateRows = longMetric("numTotalStateRows") + val numUpdatedStateRows = longMetric("numUpdatedStateRows") + while (iter.hasNext) { val row = iter.next().asInstanceOf[UnsafeRow] val key = getKey(row) store.put(key.copy(), row.copy()) + numUpdatedStateRows += 1 } store.commit() - store.iterator().map(_._2.asInstanceOf[InternalRow]) + numTotalStateRows += store.numKeys() + store.iterator().map { case (k, v) => + numOutputRows += 1 + v.asInstanceOf[InternalRow] + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 333239f875bd3..9144736c940f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ @@ -57,6 +57,7 @@ class StreamExecution( extends StreamingQuery with Logging { import org.apache.spark.sql.streaming.StreamingQueryListener._ + import StreamMetrics._ private val pollingDelayMs = sparkSession.sessionState.conf.streamingPollingDelay @@ -105,11 +106,22 @@ class StreamExecution( var lastExecution: QueryExecution = null @volatile - var streamDeathCause: StreamingQueryException = null + private var streamDeathCause: StreamingQueryException = null /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() + /** Metrics for this query */ + private val streamMetrics = + new StreamMetrics(uniqueSources.toSet, triggerClock, s"StructuredStreaming.$name") + + @volatile + private var currentStatus: StreamingQueryStatus = null + + /** Flag that signals whether any error with input metrics have already been logged */ + @volatile + private var metricWarningLogged: Boolean = false + /** * The thread that runs the micro-batches of this stream. Note that this thread must be * [[org.apache.spark.util.UninterruptibleThread]] to avoid potential deadlocks in using @@ -136,16 +148,14 @@ class StreamExecution( /** Whether the query is currently active or not */ override def isActive: Boolean = state == ACTIVE + /** Returns the current status of the query. */ + override def status: StreamingQueryStatus = currentStatus + /** Returns current status of all the sources. */ - override def sourceStatuses: Array[SourceStatus] = { - val localAvailableOffsets = availableOffsets - sources.map(s => - new SourceStatus(s.toString, localAvailableOffsets.get(s).map(_.toString))).toArray - } + override def sourceStatuses: Array[SourceStatus] = currentStatus.sourceStatuses.toArray /** Returns current status of the sink. */ - override def sinkStatus: SinkStatus = - new SinkStatus(sink.toString, committedOffsets.toCompositeOffset(sources).toString) + override def sinkStatus: SinkStatus = currentStatus.sinkStatus /** Returns the [[StreamingQueryException]] if the query was terminated by an exception. */ override def exception: Option[StreamingQueryException] = Option(streamDeathCause) @@ -176,7 +186,11 @@ class StreamExecution( // Mark ACTIVE and then post the event. QueryStarted event is synchronously sent to listeners, // so must mark this as ACTIVE first. state = ACTIVE - postEvent(new QueryStarted(this.toInfo)) // Assumption: Does not throw exception. + if (sparkSession.sessionState.conf.streamingMetricsEnabled) { + sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) + } + updateStatus() + postEvent(new QueryStarted(currentStatus)) // Assumption: Does not throw exception. // Unblock starting thread startLatch.countDown() @@ -185,25 +199,41 @@ class StreamExecution( SparkSession.setActiveSession(sparkSession) triggerExecutor.execute(() => { - if (isActive) { - if (currentBatchId < 0) { - // We'll do this initialization only once - populateStartOffsets() - logDebug(s"Stream running from $committedOffsets to $availableOffsets") + streamMetrics.reportTriggerStarted(currentBatchId) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "Finding new data from sources") + updateStatus() + val isTerminated = reportTimeTaken(TRIGGER_LATENCY) { + if (isActive) { + if (currentBatchId < 0) { + // We'll do this initialization only once + populateStartOffsets() + logDebug(s"Stream running from $committedOffsets to $availableOffsets") + } else { + constructNextBatch() + } + if (dataAvailable) { + streamMetrics.reportTriggerDetail(IS_DATA_PRESENT_IN_TRIGGER, true) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "Processing new data") + updateStatus() + runBatch() + // We'll increase currentBatchId after we complete processing current batch's data + currentBatchId += 1 + } else { + streamMetrics.reportTriggerDetail(IS_DATA_PRESENT_IN_TRIGGER, false) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "No new data") + updateStatus() + Thread.sleep(pollingDelayMs) + } + true } else { - constructNextBatch() + false } - if (dataAvailable) { - runBatch() - // We'll increase currentBatchId after we complete processing current batch's data - currentBatchId += 1 - } else { - Thread.sleep(pollingDelayMs) - } - true - } else { - false } + // Update metrics and notify others + streamMetrics.reportTriggerFinished() + updateStatus() + postEvent(new QueryProgress(currentStatus)) + isTerminated }) } catch { case _: InterruptedException if state == TERMINATED => // interrupted by stop() @@ -221,8 +251,16 @@ class StreamExecution( } } finally { state = TERMINATED + + // Update metrics and status + streamMetrics.stop() + sparkSession.sparkContext.env.metricsSystem.removeSource(streamMetrics) + updateStatus() + + // Notify others sparkSession.streams.notifyQueryTermination(StreamExecution.this) - postEvent(new QueryTerminated(this.toInfo, exception.map(_.cause).map(Utils.exceptionString))) + postEvent( + new QueryTerminated(currentStatus, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } } @@ -248,7 +286,6 @@ class StreamExecution( committedOffsets = lastOffsets.toStreamProgress(sources) logDebug(s"Resuming with committed offsets: $committedOffsets") } - case None => // We are starting this stream for the first time. logInfo(s"Starting new streaming query.") currentBatchId = 0 @@ -278,8 +315,14 @@ class StreamExecution( val hasNewData = { awaitBatchLock.lock() try { - val newData = uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - availableOffsets ++= newData + reportTimeTaken(GET_OFFSET_LATENCY) { + val latestOffsets: Map[Source, Option[Offset]] = uniqueSources.map { s => + reportTimeTaken(s, SOURCE_GET_OFFSET_LATENCY) { + (s, s.getOffset) + } + }.toMap + availableOffsets ++= latestOffsets.filter { case (s, o) => o.nonEmpty }.mapValues(_.get) + } if (dataAvailable) { true @@ -292,16 +335,19 @@ class StreamExecution( } } if (hasNewData) { - assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), - s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") - logInfo(s"Committed offsets for batch $currentBatchId.") - - // Now that we have logged the new batch, no further processing will happen for - // the previous batch, and it is safe to discard the old metadata. - // Note that purge is exclusive, i.e. it purges everything before currentBatchId. - // NOTE: If StreamExecution implements pipeline parallelism (multiple batches in - // flight at the same time), this cleanup logic will need to change. - offsetLog.purge(currentBatchId) + reportTimeTaken(OFFSET_WAL_WRITE_LATENCY) { + assert( + offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), + s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") + logInfo(s"Committed offsets for batch $currentBatchId.") + + // Now that we have logged the new batch, no further processing will happen for + // the previous batch, and it is safe to discard the old metadata. + // Note that purge is exclusive, i.e. it purges everything before currentBatchId. + // NOTE: If StreamExecution implements pipeline parallelism (multiple batches in + // flight at the same time), this cleanup logic will need to change. + offsetLog.purge(currentBatchId) + } } else { awaitBatchLock.lock() try { @@ -311,26 +357,30 @@ class StreamExecution( awaitBatchLock.unlock() } } + reportTimestamp(GET_OFFSET_TIMESTAMP) } /** * Processes any data available between `availableOffsets` and `committedOffsets`. */ private def runBatch(): Unit = { - val startTime = System.nanoTime() - // TODO: Move this to IncrementalExecution. // Request unprocessed data from all sources. - val newData = availableOffsets.flatMap { - case (source, available) + val newData = reportTimeTaken(GET_BATCH_LATENCY) { + availableOffsets.flatMap { + case (source, available) if committedOffsets.get(source).map(_ != available).getOrElse(true) => - val current = committedOffsets.get(source) - val batch = source.getBatch(current, available) - logDebug(s"Retrieving data from $source: $current -> $available") - Some(source -> batch) - case _ => None - }.toMap + val current = committedOffsets.get(source) + val batch = reportTimeTaken(source, SOURCE_GET_BATCH_LATENCY) { + source.getBatch(current, available) + } + logDebug(s"Retrieving data from $source: $current -> $available") + Some(source -> batch) + case _ => None + } + } + reportTimestamp(GET_BATCH_TIMESTAMP) // A list of attributes that will need to be updated. var replacements = new ArrayBuffer[(Attribute, Attribute)] @@ -351,25 +401,24 @@ class StreamExecution( // Rewire the plan to use the new attributes that were returned by the source. val replacementMap = AttributeMap(replacements) - val newPlan = withNewSources transformAllExpressions { + val triggerLogicalPlan = withNewSources transformAllExpressions { case a: Attribute if replacementMap.contains(a) => replacementMap(a) } - val optimizerStart = System.nanoTime() - lastExecution = new IncrementalExecution( - sparkSession, - newPlan, - outputMode, - checkpointFile("state"), - currentBatchId) - - lastExecution.executedPlan - val optimizerTime = (System.nanoTime() - optimizerStart).toDouble / 1000000 - logDebug(s"Optimized batch in ${optimizerTime}ms") + val executedPlan = reportTimeTaken(OPTIMIZER_LATENCY) { + lastExecution = new IncrementalExecution( + sparkSession, + triggerLogicalPlan, + outputMode, + checkpointFile("state"), + currentBatchId) + lastExecution.executedPlan // Force the lazy generation of execution plan + } val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) sink.addBatch(currentBatchId, nextBatch) + reportNumRows(executedPlan, triggerLogicalPlan, newData) awaitBatchLock.lock() try { @@ -379,11 +428,8 @@ class StreamExecution( awaitBatchLock.unlock() } - val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 - logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") // Update committed offsets. committedOffsets ++= availableOffsets - postEvent(new QueryProgress(this.toInfo)) } private def postEvent(event: StreamingQueryListener.Event) { @@ -516,12 +562,131 @@ class StreamExecution( """.stripMargin } - private def toInfo: StreamingQueryInfo = { - new StreamingQueryInfo( - this.name, - this.id, - this.sourceStatuses, - this.sinkStatus) + /** + * Report row metrics of the executed trigger + * @param triggerExecutionPlan Execution plan of the trigger + * @param triggerLogicalPlan Logical plan of the trigger, generated from the query logical plan + * @param sourceToDF Source to DataFrame returned by the source.getBatch in this trigger + */ + private def reportNumRows( + triggerExecutionPlan: SparkPlan, + triggerLogicalPlan: LogicalPlan, + sourceToDF: Map[Source, DataFrame]): Unit = { + // We want to associate execution plan leaves to sources that generate them, so that we match + // the their metrics (e.g. numOutputRows) to the sources. To do this we do the following. + // Consider the translation from the streaming logical plan to the final executed plan. + // + // streaming logical plan (with sources) <==> trigger's logical plan <==> executed plan + // + // 1. We keep track of streaming sources associated with each leaf in the trigger's logical plan + // - Each logical plan leaf will be associated with a single streaming source. + // - There can be multiple logical plan leaves associated with a streaming source. + // - There can be leaves not associated with any streaming source, because they were + // generated from a batch source (e.g. stream-batch joins) + // + // 2. Assuming that the executed plan has same number of leaves in the same order as that of + // the trigger logical plan, we associate executed plan leaves with corresponding + // streaming sources. + // + // 3. For each source, we sum the metrics of the associated execution plan leaves. + // + val logicalPlanLeafToSource = sourceToDF.flatMap { case (source, df) => + df.logicalPlan.collectLeaves().map { leaf => leaf -> source } + } + val allLogicalPlanLeaves = triggerLogicalPlan.collectLeaves() // includes non-streaming sources + val allExecPlanLeaves = triggerExecutionPlan.collectLeaves() + val sourceToNumInputRows: Map[Source, Long] = + if (allLogicalPlanLeaves.size == allExecPlanLeaves.size) { + val execLeafToSource = allLogicalPlanLeaves.zip(allExecPlanLeaves).flatMap { + case (lp, ep) => logicalPlanLeafToSource.get(lp).map { source => ep -> source } + } + val sourceToNumInputRows = execLeafToSource.map { case (execLeaf, source) => + val numRows = execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L) + source -> numRows + } + sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source + } else { + if (!metricWarningLogged) { + def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" + logWarning( + "Could not report metrics as number leaves in trigger logical plan did not match that" + + s" of the execution plan:\n" + + s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + + s"execution plan leaves: ${toString(allExecPlanLeaves)}\n") + metricWarningLogged = true + } + Map.empty + } + val numOutputRows = triggerExecutionPlan.metrics.get("numOutputRows").map(_.value) + val stateNodes = triggerExecutionPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + + streamMetrics.reportNumInputRows(sourceToNumInputRows) + stateNodes.zipWithIndex.foreach { case (s, i) => + streamMetrics.reportTriggerDetail( + NUM_TOTAL_STATE_ROWS(i + 1), + s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) + streamMetrics.reportTriggerDetail( + NUM_UPDATED_STATE_ROWS(i + 1), + s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) + } + updateStatus() + } + + private def reportTimeTaken[T](triggerDetailKey: String)(body: => T): T = { + val startTime = triggerClock.getTimeMillis() + val result = body + val endTime = triggerClock.getTimeMillis() + val timeTaken = math.max(endTime - startTime, 0) + streamMetrics.reportTriggerDetail(triggerDetailKey, timeTaken) + updateStatus() + if (triggerDetailKey == TRIGGER_LATENCY) { + logInfo(s"Completed up to $availableOffsets in $timeTaken ms") + } + result + } + + private def reportTimeTaken[T](source: Source, triggerDetailKey: String)(body: => T): T = { + val startTime = triggerClock.getTimeMillis() + val result = body + val endTime = triggerClock.getTimeMillis() + streamMetrics.reportSourceTriggerDetail( + source, triggerDetailKey, math.max(endTime - startTime, 0)) + updateStatus() + result + } + + private def reportTimestamp(triggerDetailKey: String): Unit = { + streamMetrics.reportTriggerDetail(triggerDetailKey, triggerClock.getTimeMillis) + updateStatus() + } + + private def updateStatus(): Unit = { + val localAvailableOffsets = availableOffsets + val sourceStatuses = sources.map { s => + SourceStatus( + s.toString, + localAvailableOffsets.get(s).map(_.toString).getOrElse("-"), // TODO: use json if available + streamMetrics.currentSourceInputRate(s), + streamMetrics.currentSourceProcessingRate(s), + streamMetrics.currentSourceTriggerDetails(s)) + }.toArray + val sinkStatus = SinkStatus( + sink.toString, + committedOffsets.toCompositeOffset(sources).toString) + + currentStatus = + StreamingQueryStatus( + name = name, + id = id, + timestamp = triggerClock.getTimeMillis(), + inputRate = streamMetrics.currentInputRate(), + processingRate = streamMetrics.currentProcessingRate(), + latency = streamMetrics.currentLatency(), + sourceStatuses = sourceStatuses, + sinkStatus = sinkStatus, + triggerDetails = streamMetrics.currentTriggerDetails()) } trait State diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala new file mode 100644 index 0000000000000..e98d1883e4596 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.{util => ju} + +import scala.collection.mutable + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.{Source => CodahaleSource} +import org.apache.spark.util.Clock + +/** + * Class that manages all the metrics related to a StreamingQuery. It does the following. + * - Calculates metrics (rates, latencies, etc.) based on information reported by StreamExecution. + * - Allows the current metric values to be queried + * - Serves some of the metrics through Codahale/DropWizard metrics + * + * @param sources Unique set of sources in a query + * @param triggerClock Clock used for triggering in StreamExecution + * @param codahaleSourceName Root name for all the Codahale metrics + */ +class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceName: String) + extends CodahaleSource with Logging { + + import StreamMetrics._ + + // Trigger infos + private val triggerDetails = new mutable.HashMap[String, String] + private val sourceTriggerDetails = new mutable.HashMap[Source, mutable.HashMap[String, String]] + + // Rate estimators for sources and sinks + private val inputRates = new mutable.HashMap[Source, RateCalculator] + private val processingRates = new mutable.HashMap[Source, RateCalculator] + + // Number of input rows in the current trigger + private val numInputRows = new mutable.HashMap[Source, Long] + private var currentTriggerStartTimestamp: Long = -1 + private var previousTriggerStartTimestamp: Long = -1 + private var latency: Option[Double] = None + + override val sourceName: String = codahaleSourceName + override val metricRegistry: MetricRegistry = new MetricRegistry + + // =========== Initialization =========== + + // Metric names should not have . in them, so that all the metrics of a query are identified + // together in Ganglia as a single metric group + registerGauge("inputRate-total", currentInputRate) + registerGauge("processingRate-total", () => currentProcessingRate) + registerGauge("latency", () => currentLatency().getOrElse(-1.0)) + + sources.foreach { s => + inputRates.put(s, new RateCalculator) + processingRates.put(s, new RateCalculator) + sourceTriggerDetails.put(s, new mutable.HashMap[String, String]) + + registerGauge(s"inputRate-${s.toString}", () => currentSourceInputRate(s)) + registerGauge(s"processingRate-${s.toString}", () => currentSourceProcessingRate(s)) + } + + // =========== Setter methods =========== + + def reportTriggerStarted(triggerId: Long): Unit = synchronized { + numInputRows.clear() + triggerDetails.clear() + sourceTriggerDetails.values.foreach(_.clear()) + + reportTriggerDetail(TRIGGER_ID, triggerId) + sources.foreach(s => reportSourceTriggerDetail(s, TRIGGER_ID, triggerId)) + reportTriggerDetail(IS_TRIGGER_ACTIVE, true) + currentTriggerStartTimestamp = triggerClock.getTimeMillis() + reportTriggerDetail(START_TIMESTAMP, currentTriggerStartTimestamp) + } + + def reportTriggerDetail[T](key: String, value: T): Unit = synchronized { + triggerDetails.put(key, value.toString) + } + + def reportSourceTriggerDetail[T](source: Source, key: String, value: T): Unit = synchronized { + sourceTriggerDetails(source).put(key, value.toString) + } + + def reportNumInputRows(inputRows: Map[Source, Long]): Unit = synchronized { + numInputRows ++= inputRows + } + + def reportTriggerFinished(): Unit = synchronized { + require(currentTriggerStartTimestamp >= 0) + val currentTriggerFinishTimestamp = triggerClock.getTimeMillis() + reportTriggerDetail(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) + triggerDetails.remove(STATUS_MESSAGE) + reportTriggerDetail(IS_TRIGGER_ACTIVE, false) + + // Report number of rows + val totalNumInputRows = numInputRows.values.sum + reportTriggerDetail(NUM_INPUT_ROWS, totalNumInputRows) + numInputRows.foreach { case (s, r) => + reportSourceTriggerDetail(s, NUM_SOURCE_INPUT_ROWS, r) + } + + val currentTriggerDuration = currentTriggerFinishTimestamp - currentTriggerStartTimestamp + val previousInputIntervalOption = if (previousTriggerStartTimestamp >= 0) { + Some(currentTriggerStartTimestamp - previousTriggerStartTimestamp) + } else None + + // Update input rate = num rows received by each source during the previous trigger interval + // Interval is measures as interval between start times of previous and current trigger. + // + // TODO: Instead of trigger start, we should use time when getOffset was called on each source + // as this may be different for each source if there are many sources in the query plan + // and getOffset is called serially on them. + if (previousInputIntervalOption.nonEmpty) { + sources.foreach { s => + inputRates(s).update(numInputRows.getOrElse(s, 0), previousInputIntervalOption.get) + } + } + + // Update processing rate = num rows processed for each source in current trigger duration + sources.foreach { s => + processingRates(s).update(numInputRows.getOrElse(s, 0), currentTriggerDuration) + } + + // Update latency = if data present, 0.5 * previous trigger interval + current trigger duration + if (previousInputIntervalOption.nonEmpty && totalNumInputRows > 0) { + latency = Some((previousInputIntervalOption.get.toDouble / 2) + currentTriggerDuration) + } else { + latency = None + } + + previousTriggerStartTimestamp = currentTriggerStartTimestamp + currentTriggerStartTimestamp = -1 + } + + // =========== Getter methods =========== + + def currentInputRate(): Double = synchronized { + // Since we are calculating source input rates using the same time interval for all sources + // it is fine to calculate total input rate as the sum of per source input rate. + inputRates.map(_._2.currentRate).sum + } + + def currentSourceInputRate(source: Source): Double = synchronized { + inputRates(source).currentRate + } + + def currentProcessingRate(): Double = synchronized { + // Since we are calculating source processing rates using the same time interval for all sources + // it is fine to calculate total processing rate as the sum of per source processing rate. + processingRates.map(_._2.currentRate).sum + } + + def currentSourceProcessingRate(source: Source): Double = synchronized { + processingRates(source).currentRate + } + + def currentLatency(): Option[Double] = synchronized { latency } + + def currentTriggerDetails(): Map[String, String] = synchronized { triggerDetails.toMap } + + def currentSourceTriggerDetails(source: Source): Map[String, String] = synchronized { + sourceTriggerDetails(source).toMap + } + + // =========== Other methods =========== + + private def registerGauge[T](name: String, f: () => T)(implicit num: Numeric[T]): Unit = { + synchronized { + metricRegistry.register(name, new Gauge[T] { + override def getValue: T = f() + }) + } + } + + def stop(): Unit = synchronized { + triggerDetails.clear() + inputRates.valuesIterator.foreach { _.stop() } + processingRates.valuesIterator.foreach { _.stop() } + latency = None + } +} + +object StreamMetrics extends Logging { + /** Simple utility class to calculate rate while avoiding DivideByZero */ + class RateCalculator { + @volatile private var rate: Option[Double] = None + + def update(numRows: Long, timeGapMs: Long): Unit = { + if (timeGapMs > 0) { + rate = Some(numRows.toDouble * 1000 / timeGapMs) + } else { + rate = None + logDebug(s"Rate updates cannot with zero or negative time gap $timeGapMs") + } + } + + def currentRate: Double = rate.getOrElse(0.0) + + def stop(): Unit = { rate = None } + } + + + val TRIGGER_ID = "triggerId" + val IS_TRIGGER_ACTIVE = "isTriggerActive" + val IS_DATA_PRESENT_IN_TRIGGER = "isDataPresentInTrigger" + val STATUS_MESSAGE = "statusMessage" + + val START_TIMESTAMP = "timestamp.triggerStart" + val GET_OFFSET_TIMESTAMP = "timestamp.afterGetOffset" + val GET_BATCH_TIMESTAMP = "timestamp.afterGetBatch" + val FINISH_TIMESTAMP = "timestamp.triggerFinish" + + val GET_OFFSET_LATENCY = "latency.getOffset.total" + val GET_BATCH_LATENCY = "latency.getBatch.total" + val OFFSET_WAL_WRITE_LATENCY = "latency.offsetLogWrite" + val OPTIMIZER_LATENCY = "latency.optimizer" + val TRIGGER_LATENCY = "latency.fullTrigger" + val SOURCE_GET_OFFSET_LATENCY = "latency.getOffset.source" + val SOURCE_GET_BATCH_LATENCY = "latency.getBatch.source" + + val NUM_INPUT_ROWS = "numRows.input.total" + val NUM_SOURCE_INPUT_ROWS = "numRows.input.source" + def NUM_TOTAL_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.total" + def NUM_UPDATED_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.updated" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 5052c4d50c5ed..788fcd0361bee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -112,6 +112,11 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } override def stop() {} + + def reset(): Unit = synchronized { + batches.clear() + currentOffset = new LongOffset(-1) + } } /** @@ -165,6 +170,8 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi logDebug(s"Skipping already committed batch: $batchId") } } + + override def toString(): String = "MemorySink" } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index bec966b15ed0f..7d71f5242c27d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -197,6 +197,8 @@ private[state] class HDFSBackedStateStoreProvider( allUpdates.values().asScala.toIterator } + override def numKeys(): Long = mapToUpdate.size() + /** * Whether all updates have been committed */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index a67fdceb3cee6..7132e284c28f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -77,6 +77,9 @@ trait StateStore { */ def updates(): Iterator[StoreUpdate] + /** Number of keys in the state store */ + def numKeys(): Long + /** * Whether all updates have been committed */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 192083e2ea5f5..e671604c39855 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -569,6 +569,12 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(10L) + val STREAMING_METRICS_ENABLED = + SQLConfigBuilder("spark.sql.streaming.metricsEnabled") + .doc("Whether Dropwizard/Codahale metrics will be reported for active streaming queries.") + .booleanConf + .createWithDefault(false) + val NDV_MAX_ERROR = SQLConfigBuilder("spark.sql.statistics.ndv.maxError") .internal() @@ -635,6 +641,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def streamingPollingDelay: Long = getConf(STREAMING_POLLING_DELAY) + def streamingMetricsEnabled: Boolean = getConf(STREAMING_METRICS_ENABLED) + def filesMaxPartitionBytes: Long = getConf(FILES_MAX_PARTITION_BYTES) def filesOpenCostInBytes: Long = getConf(FILES_OPEN_COST_IN_BYTES) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index de1efe961f8bd..c9911665f7d72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -18,17 +18,33 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.Sink +import org.apache.spark.sql.streaming.StreamingQueryStatus.indent /** * :: Experimental :: - * Status and metrics of a streaming [[Sink]]. + * Status and metrics of a streaming sink. * - * @param description Description of the source corresponding to this status - * @param offsetDesc Description of the current offset up to which data has been written by the sink + * @param description Description of the source corresponding to this status. + * @param offsetDesc Description of the current offsets up to which data has been written + * by the sink. * @since 2.0.0 */ @Experimental -class SinkStatus private[sql]( +class SinkStatus private( val description: String, - val offsetDesc: String) + val offsetDesc: String) { + + override def toString: String = + "SinkStatus:" + indent(prettyString) + + private[sql] def prettyString: String = { + s"""$description + |Committed offsets: $offsetDesc + |""".stripMargin + } +} + +/** Companion object, primarily for creating SinkStatus instances internally */ +private[sql] object SinkStatus { + def apply(desc: String, offsetDesc: String): SinkStatus = new SinkStatus(desc, offsetDesc) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index bd0c8485e4fdd..6ace4833be22f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -17,18 +17,60 @@ package org.apache.spark.sql.streaming +import java.{util => ju} + +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.Source +import org.apache.spark.sql.streaming.StreamingQueryStatus.indent /** * :: Experimental :: - * Status and metrics of a streaming [[Source]]. + * Status and metrics of a streaming Source. * - * @param description Description of the source corresponding to this status - * @param offsetDesc Description of the current [[Source]] offset if known + * @param description Description of the source corresponding to this status. + * @param offsetDesc Description of the current offset if known. + * @param inputRate Current rate (rows/sec) at which data is being generated by the source. + * @param processingRate Current rate (rows/sec) at which the query is processing data from + * the source. + * @param triggerDetails Low-level details of the currently active trigger (e.g. number of + * rows processed in trigger, latency of intermediate steps, etc.). + * If no trigger is active, then it will have details of the last completed + * trigger. * @since 2.0.0 */ @Experimental -class SourceStatus private[sql] ( +class SourceStatus private( val description: String, - val offsetDesc: Option[String]) + val offsetDesc: String, + val inputRate: Double, + val processingRate: Double, + val triggerDetails: ju.Map[String, String]) { + + override def toString: String = + "SourceStatus:" + indent(prettyString) + + private[sql] def prettyString: String = { + val triggerDetailsLines = + triggerDetails.asScala.map { case (k, v) => s"$k: $v" } + s"""$description + |Available offset: $offsetDesc + |Input rate: $inputRate rows/sec + |Processing rate: $processingRate rows/sec + |Trigger details: + |""".stripMargin + indent(triggerDetailsLines) + + } +} + +/** Companion object, primarily for creating SourceStatus instances internally */ +private[sql] object SourceStatus { + def apply( + desc: String, + offsetDesc: String, + inputRate: Double, + processingRate: Double, + triggerDetails: Map[String, String]): SourceStatus = { + new SourceStatus(desc, offsetDesc, inputRate, processingRate, triggerDetails.asJava) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index 91f0a1e3446a1..0a85414451981 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -62,13 +62,24 @@ trait StreamingQuery { */ def exception: Option[StreamingQueryException] + /** + * Returns the current status of the query. + * @since 2.0.2 + */ + def status: StreamingQueryStatus + /** * Returns current status of all the sources. * @since 2.0.0 */ + @deprecated("use status.sourceStatuses", "2.0.2") def sourceStatuses: Array[SourceStatus] - /** Returns current status of the sink. */ + /** + * Returns current status of the sink. + * @since 2.0.0 + */ + @deprecated("use status.sinkStatus", "2.0.2") def sinkStatus: SinkStatus /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala deleted file mode 100644 index 1af2668817eae..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.streaming - -import org.apache.spark.annotation.Experimental - -/** - * :: Experimental :: - * A class used to report information about the progress of a [[StreamingQuery]]. - * - * @param name The [[StreamingQuery]] name. This name is unique across all active queries. - * @param id The [[StreamingQuery]] id. This id is unique across - * all queries that have been started in the current process. - * @param sourceStatuses The current statuses of the [[StreamingQuery]]'s sources. - * @param sinkStatus The current status of the [[StreamingQuery]]'s sink. - */ -@Experimental -class StreamingQueryInfo private[sql]( - val name: String, - val id: Long, - val sourceStatuses: Seq[SourceStatus], - val sinkStatus: SinkStatus) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 8a8855d85a4c7..69790e33b2168 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -84,7 +84,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryStarted private[sql](val queryInfo: StreamingQueryInfo) extends Event + class QueryStarted private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -92,19 +92,19 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryProgress private[sql](val queryInfo: StreamingQueryInfo) extends Event + class QueryProgress private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: * Event representing that termination of a query * - * @param queryInfo Information about the status of the query. + * @param queryStatus Information about the status of the query. * @param exception The exception message of the [[StreamingQuery]] if the query was terminated * with an exception. Otherwise, it will be `None`. * @since 2.0.0 */ @Experimental class QueryTerminated private[sql]( - val queryInfo: StreamingQueryInfo, + val queryStatus: StreamingQueryStatus, val exception: Option[String]) extends Event } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala new file mode 100644 index 0000000000000..47689928730d0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -0,0 +1,139 @@ +/* + * 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.streaming + +import java.{util => ju} + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset} + +/** + * :: Experimental :: + * A class used to report information about the progress of a [[StreamingQuery]]. + * + * @param name Name of the query. This name is unique across all active queries. + * @param id Id of the query. This id is unique across + * all queries that have been started in the current process. + * @param timestamp Timestamp (ms) of when this query was generated. + * @param inputRate Current rate (rows/sec) at which data is being generated by all the sources. + * @param processingRate Current rate (rows/sec) at which the query is processing data from + * all the sources. + * @param latency Current average latency between the data being available in source and the sink + * writing the corresponding output. + * @param sourceStatuses Current statuses of the sources. + * @param sinkStatus Current status of the sink. + * @param triggerDetails Low-level details of the currently active trigger (e.g. number of + * rows processed in trigger, latency of intermediate steps, etc.). + * If no trigger is active, then it will have details of the last completed + * trigger. + * @since 2.0.0 + */ +@Experimental +class StreamingQueryStatus private( + val name: String, + val id: Long, + val timestamp: Long, + val inputRate: Double, + val processingRate: Double, + val latency: Option[Double], + val sourceStatuses: Array[SourceStatus], + val sinkStatus: SinkStatus, + val triggerDetails: ju.Map[String, String]) { + + import StreamingQueryStatus._ + + override def toString: String = { + val sourceStatusLines = sourceStatuses.zipWithIndex.map { case (s, i) => + s"Source ${i + 1}:" + indent(s.prettyString) + } + val sinkStatusLines = sinkStatus.prettyString + val triggerDetailsLines = triggerDetails.asScala.map { case (k, v) => s"$k: $v" }.toSeq.sorted + val numSources = sourceStatuses.length + val numSourcesString = s"$numSources source" + { if (numSources > 1) "s" else "" } + + val allLines = s""" + |Query name: $name + |Query id: $id + |Status timestamp: $timestamp + |Input rate: $inputRate rows/sec + |Processing rate $processingRate rows/sec + |Latency: ${latency.getOrElse("-")} ms + |Trigger details: + |${indent(triggerDetailsLines)} + |Source statuses [$numSourcesString]: + |${indent(sourceStatusLines)} + |Sink status: ${indent(sinkStatusLines)}""".stripMargin + + s"StreamingQueryStatus:${indent(allLines)}" + } +} + +/** Companion object, primarily for creating StreamingQueryInfo instances internally */ +private[sql] object StreamingQueryStatus { + def apply( + name: String, + id: Long, + timestamp: Long, + inputRate: Double, + processingRate: Double, + latency: Option[Double], + sourceStatuses: Array[SourceStatus], + sinkStatus: SinkStatus, + triggerDetails: Map[String, String]): StreamingQueryStatus = { + new StreamingQueryStatus(name, id, timestamp, inputRate, processingRate, + latency, sourceStatuses, sinkStatus, triggerDetails.asJava) + } + + def indent(strings: Iterable[String]): String = strings.map(indent).mkString("\n") + def indent(string: String): String = string.split("\n").map(" " + _).mkString("\n") + + /** Create an instance of status for python testing */ + def testStatus(): StreamingQueryStatus = { + import org.apache.spark.sql.execution.streaming.StreamMetrics._ + StreamingQueryStatus( + name = "query", + id = 1, + timestamp = 123, + inputRate = 15.5, + processingRate = 23.5, + latency = Some(345), + sourceStatuses = Array( + SourceStatus( + desc = "MySource1", + offsetDesc = LongOffset(0).toString, + inputRate = 15.5, + processingRate = 23.5, + triggerDetails = Map( + NUM_SOURCE_INPUT_ROWS -> "100", + SOURCE_GET_OFFSET_LATENCY -> "10", + SOURCE_GET_BATCH_LATENCY -> "20"))), + sinkStatus = SinkStatus( + desc = "MySink", + offsetDesc = CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), + triggerDetails = Map( + TRIGGER_ID -> "5", + IS_TRIGGER_ACTIVE -> "true", + IS_DATA_PRESENT_IN_TRIGGER -> "true", + GET_OFFSET_LATENCY -> "10", + GET_BATCH_LATENCY -> "20", + NUM_INPUT_ROWS -> "100" + )) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index bba40c6510cfb..229d8814e0143 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.metric import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ @@ -85,6 +86,22 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } + test("LocalTableScanExec computes metrics in collect and take") { + val df1 = spark.createDataset(Seq(1, 2, 3)) + val logical = df1.queryExecution.logical + require(logical.isInstanceOf[LocalRelation]) + df1.collect() + val metrics1 = df1.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics1.contains("numOutputRows")) + assert(metrics1("numOutputRows").value === 3) + + val df2 = spark.createDataset(Seq(1, 2, 3)).limit(2) + df2.collect() + val metrics2 = df2.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics2.contains("numOutputRows")) + assert(metrics2("numOutputRows").value === 2) + } + test("Filter metrics") { // Assume the execution plan is // PhysicalRDD(nodeId = 1) -> Filter(nodeId = 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala new file mode 100644 index 0000000000000..938423db64745 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import org.scalactic.TolerantNumerics + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.ManualClock + +class StreamMetricsSuite extends SparkFunSuite { + import StreamMetrics._ + + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + + test("rates, latencies, trigger details - basic life cycle") { + val sm = newStreamMetrics(source) + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails().isEmpty) + + // When trigger started, the rates should not change, but should return + // reported trigger details + sm.reportTriggerStarted(1) + sm.reportTriggerDetail("key", "value") + sm.reportSourceTriggerDetail(source, "key2", "value2") + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails() === + Map(TRIGGER_ID -> "1", IS_TRIGGER_ACTIVE -> "true", + START_TIMESTAMP -> "0", "key" -> "value")) + assert(sm.currentSourceTriggerDetails(source) === + Map(TRIGGER_ID -> "1", "key2" -> "value2")) + + // Finishing the trigger should calculate the rates, except input rate which needs + // to have another trigger interval + sm.reportNumInputRows(Map(source -> 100L)) // 100 input rows, 10 output rows + clock.advance(1000) + sm.reportTriggerFinished() + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 100.0) // 100 input rows processed in 1 sec + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 100.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails() === + Map(TRIGGER_ID -> "1", IS_TRIGGER_ACTIVE -> "false", + START_TIMESTAMP -> "0", FINISH_TIMESTAMP -> "1000", + NUM_INPUT_ROWS -> "100", "key" -> "value")) + assert(sm.currentSourceTriggerDetails(source) === + Map(TRIGGER_ID -> "1", NUM_SOURCE_INPUT_ROWS -> "100", "key2" -> "value2")) + + // After another trigger starts, the rates and latencies should not change until + // new rows are reported + clock.advance(1000) + sm.reportTriggerStarted(2) + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 100.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 100.0) + assert(sm.currentLatency() === None) + + // Reporting new rows should update the rates and latencies + sm.reportNumInputRows(Map(source -> 200L)) // 200 input rows + clock.advance(500) + sm.reportTriggerFinished() + assert(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts + assert(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec + assert(sm.currentSourceInputRate(source) === 100.0) + assert(sm.currentSourceProcessingRate(source) === 400.0) + assert(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms + + // Rates should be set to 0 after stop + sm.stop() + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails().isEmpty) + } + + test("rates and latencies - after trigger with no data") { + val sm = newStreamMetrics(source) + // Trigger 1 with data + sm.reportTriggerStarted(1) + sm.reportNumInputRows(Map(source -> 100L)) // 100 input rows + clock.advance(1000) + sm.reportTriggerFinished() + + // Trigger 2 with data + clock.advance(1000) + sm.reportTriggerStarted(2) + sm.reportNumInputRows(Map(source -> 200L)) // 200 input rows + clock.advance(500) + sm.reportTriggerFinished() + + // Make sure that all rates are set + require(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts + require(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec + require(sm.currentSourceInputRate(source) === 100.0) + require(sm.currentSourceProcessingRate(source) === 400.0) + require(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms + + // Trigger 3 with data + clock.advance(500) + sm.reportTriggerStarted(3) + clock.advance(500) + sm.reportTriggerFinished() + + // Rates are set to zero and latency is set to None + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + sm.stop() + } + + test("rates - after trigger with multiple sources, and one source having no info") { + val source1 = TestSource(1) + val source2 = TestSource(2) + val sm = newStreamMetrics(source1, source2) + // Trigger 1 with data + sm.reportTriggerStarted(1) + sm.reportNumInputRows(Map(source1 -> 100L, source2 -> 100L)) + clock.advance(1000) + sm.reportTriggerFinished() + + // Trigger 2 with data + clock.advance(1000) + sm.reportTriggerStarted(2) + sm.reportNumInputRows(Map(source1 -> 200L, source2 -> 200L)) + clock.advance(500) + sm.reportTriggerFinished() + + // Make sure that all rates are set + assert(sm.currentInputRate() === 200.0) // 200*2 input rows generated in 2 seconds b/w starts + assert(sm.currentProcessingRate() === 800.0) // 200*2 output rows processed in 0.5 sec + assert(sm.currentSourceInputRate(source1) === 100.0) + assert(sm.currentSourceInputRate(source2) === 100.0) + assert(sm.currentSourceProcessingRate(source1) === 400.0) + assert(sm.currentSourceProcessingRate(source2) === 400.0) + + // Trigger 3 with data + clock.advance(500) + sm.reportTriggerStarted(3) + clock.advance(500) + sm.reportNumInputRows(Map(source1 -> 200L)) + sm.reportTriggerFinished() + + // Rates are set to zero and latency is set to None + assert(sm.currentInputRate() === 200.0) + assert(sm.currentProcessingRate() === 400.0) + assert(sm.currentSourceInputRate(source1) === 200.0) + assert(sm.currentSourceInputRate(source2) === 0.0) + assert(sm.currentSourceProcessingRate(source1) === 400.0) + assert(sm.currentSourceProcessingRate(source2) === 0.0) + sm.stop() + } + + test("registered Codahale metrics") { + import scala.collection.JavaConverters._ + val sm = newStreamMetrics(source) + val gaugeNames = sm.metricRegistry.getGauges().keySet().asScala + + // so that all metrics are considered as a single metric group in Ganglia + assert(!gaugeNames.exists(_.contains("."))) + assert(gaugeNames === Set( + "inputRate-total", + "inputRate-source0", + "processingRate-total", + "processingRate-source0", + "latency")) + } + + private def newStreamMetrics(sources: Source*): StreamMetrics = { + new StreamMetrics(sources.toSet, clock, "test") + } + + private val clock = new ManualClock() + private val source = TestSource(0) + + case class TestSource(id: Int) extends Source { + override def schema: StructType = StructType(Array.empty[StructField]) + override def getOffset: Option[Offset] = Some(new LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { null } + override def stop() {} + override def toString(): String = s"source$id" + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala index 6b0ba7acb4804..5174a0415304c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala @@ -156,6 +156,30 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before } } + test("input row metrics") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString) + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val batch = source.getBatch(None, source.getOffset.get).as[String] + batch.collect() + val numRowsMetric = + batch.queryExecution.executedPlan.collectLeaves().head.metrics.get("numOutputRows") + assert(numRowsMetric.nonEmpty) + assert(numRowsMetric.get.value === 1) + source.stop() + source = null + } + } + private class ServerThread extends Thread with Logging { private val serverSocket = new ServerSocket(0) private val messageQueue = new LinkedBlockingQueue[String]() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 984b84fd13fbd..06f1bd6c3bcc7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -74,6 +74,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth // Verify state after updating put(store, "a", 1) + assert(store.numKeys() === 1) intercept[IllegalStateException] { store.iterator() } @@ -85,7 +86,9 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth // Make updates, commit and then verify state put(store, "b", 2) put(store, "aa", 3) + assert(store.numKeys() === 3) remove(store, _.startsWith("a")) + assert(store.numKeys() === 1) assert(store.commit() === 1) assert(store.hasCommitted) @@ -107,7 +110,9 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth val reloadedProvider = new HDFSBackedStateStoreProvider( store.id, keySchema, valueSchema, StateStoreConf.empty, new Configuration) val reloadedStore = reloadedProvider.getStore(1) + assert(reloadedStore.numKeys() === 1) put(reloadedStore, "c", 4) + assert(reloadedStore.numKeys() === 2) assert(reloadedStore.commit() === 2) assert(rowsToSet(reloadedStore.iterator()) === Set("b" -> 2, "c" -> 4)) assert(getDataFromFiles(provider) === Set("b" -> 2, "c" -> 4)) 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 7f9c981a4e9c9..aabdccaaf319d 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 @@ -998,6 +998,20 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } } + + test("input row metrics") { + withTempDirs { case (src, tmp) => + val input = spark.readStream.format("text").load(src.getCanonicalPath) + testStream(input)( + AddTextFileData("100", src, tmp), + CheckAnswer("100"), + AssertOnLastQueryStatus { status => + assert(status.triggerDetails.get("numRows.input.total") === "1") + assert(status.sourceStatuses(0).processingRate > 0.0) + } + ) + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index fa13d385cce75..3b9d3786349ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -28,6 +28,8 @@ import scala.util.control.NonFatal import org.scalatest.Assertions import org.scalatest.concurrent.{Eventually, Timeouts} +import org.scalatest.concurrent.AsyncAssertions.Waiter +import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.Span @@ -38,6 +40,7 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} @@ -198,6 +201,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } + case class AssertOnLastQueryStatus(condition: StreamingQueryStatus => Unit) + extends StreamAction + + /** * Executes the specified actions on the given streaming DataFrame and provides helpful * error messages in the case of failures or incorrect answers. @@ -299,9 +306,12 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val testThread = Thread.currentThread() val metadataRoot = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + val statusCollector = new QueryStatusCollector try { + spark.streams.addListener(statusCollector) startedTest.foreach { action => + logInfo(s"Processing test stream action: $action") action match { case StartStream(trigger, triggerClock) => verify(currentStream == null, "stream already running") @@ -399,6 +409,13 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val streamToAssert = Option(currentStream).getOrElse(lastStream) verify({ a.run(); true }, s"Assert failed: ${a.message}") + case a: AssertOnLastQueryStatus => + Eventually.eventually(timeout(streamingTimeout)) { + require(statusCollector.lastTriggerStatus.nonEmpty) + } + val status = statusCollector.lastTriggerStatus.get + verify({ a.condition(status); true }, "Assert on last query status failed") + case a: AddData => try { // Add data and get the source where it was added, and the expected offset of the @@ -473,6 +490,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { if (currentStream != null && currentStream.microBatchThread.isAlive) { currentStream.stop() } + spark.streams.removeListener(statusCollector) } } @@ -606,4 +624,58 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } } + + + class QueryStatusCollector extends StreamingQueryListener { + // to catch errors in the async listener events + @volatile private var asyncTestWaiter = new Waiter + + @volatile var startStatus: StreamingQueryStatus = null + @volatile var terminationStatus: StreamingQueryStatus = null + @volatile var terminationException: Option[String] = null + + private val progressStatuses = new mutable.ArrayBuffer[StreamingQueryStatus] + + /** Get the info of the last trigger that processed data */ + def lastTriggerStatus: Option[StreamingQueryStatus] = synchronized { + progressStatuses.filter { i => + i.triggerDetails.get("isTriggerActive").toBoolean == false && + i.triggerDetails.get("isDataPresentInTrigger").toBoolean == true + }.lastOption + } + + def reset(): Unit = { + startStatus = null + terminationStatus = null + progressStatuses.clear() + asyncTestWaiter = new Waiter + } + + def checkAsyncErrors(): Unit = { + asyncTestWaiter.await(timeout(10 seconds)) + } + + + override def onQueryStarted(queryStarted: QueryStarted): Unit = { + asyncTestWaiter { + startStatus = queryStarted.queryStatus + } + } + + override def onQueryProgress(queryProgress: QueryProgress): Unit = { + asyncTestWaiter { + assert(startStatus != null, "onQueryProgress called before onQueryStarted") + synchronized { progressStatuses += queryProgress.queryStatus } + } + } + + override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { + asyncTestWaiter { + assert(startStatus != null, "onQueryTerminated called before onQueryStarted") + terminationStatus = queryTerminated.queryStatus + terminationException = queryTerminated.exception + } + asyncTestWaiter.dismiss() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 8681199817fe6..e59b5491f90b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.InternalOutputModes._ +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scalalang.typed @@ -129,6 +130,59 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll { ) } + test("state metrics") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDS() + .flatMap(x => Seq(x, x + 1)) + .toDF("value") + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + implicit class RichStreamExecution(query: StreamExecution) { + def stateNodes: Seq[SparkPlan] = { + query.lastExecution.executedPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + } + } + + // Test with Update mode + testStream(aggregated, Update)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + + // Test with Complete mode + inputData.reset() + testStream(aggregated, Complete)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((1, 1), (2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 4 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + } + test("multiple keys") { val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 831543a47420a..6256385dfd0e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -17,92 +17,97 @@ package org.apache.spark.sql.streaming -import java.util.concurrent.ConcurrentLinkedQueue - +import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ -import org.scalatest.concurrent.AsyncAssertions.Waiter -import org.scalatest.concurrent.Eventually._ -import org.scalatest.concurrent.PatienceConfiguration.Timeout -import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.util.JsonProtocol +import org.apache.spark.sql.functions._ +import org.apache.spark.util.{JsonProtocol, ManualClock} class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ - import StreamingQueryListener._ + import StreamingQueryListenerSuite._ + + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) after { spark.streams.active.foreach(_.stop()) assert(spark.streams.active.isEmpty) assert(addedListeners.isEmpty) // Make sure we don't leak any events to the next test - spark.sparkContext.listenerBus.waitUntilEmpty(10000) } - test("single listener") { - val listener = new QueryStatusCollector - val input = MemoryStream[Int] - withListenerAdded(listener) { - testStream(input.toDS)( - StartStream(), - AssertOnQuery("Incorrect query status in onQueryStarted") { query => - val status = listener.startStatus - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).description.contains("Memory")) - - // The source and sink offsets must be None as this must be called before the - // batches have started - assert(status.sourceStatuses(0).offsetDesc === None) - assert(status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString) - - // No progress events or termination events - assert(listener.progressStatuses.isEmpty) - assert(listener.terminationStatus === null) - true - }, - AddDataMemory(input, Seq(1, 2, 3)), - CheckAnswer(1, 2, 3), - AssertOnQuery("Incorrect query status in onQueryProgress") { query => - eventually(Timeout(streamingTimeout)) { + test("single listener, check trigger statuses") { + import StreamingQueryListenerSuite._ + clock = new ManualClock() + + /** Custom MemoryStream that waits for manual clock to reach a time */ + val inputData = new MemoryStream[Int](0, sqlContext) { + // Wait for manual clock to be 100 first time there is data + override def getOffset: Option[Offset] = { + val offset = super.getOffset + if (offset.nonEmpty) { + clock.waitTillTime(100) + } + offset + } - // There should be only on progress event as batch has been processed - assert(listener.progressStatuses.size === 1) - val status = listener.progressStatuses.peek() - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) - assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) + // Wait for manual clock to be 300 first time there is data + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + clock.waitTillTime(300) + super.getBatch(start, end) + } + } - // No termination events - assert(listener.terminationStatus === null) - } - true - }, - StopStream, - AssertOnQuery("Incorrect query status in onQueryTerminated") { query => - eventually(Timeout(streamingTimeout)) { - val status = listener.terminationStatus - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) - assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) - assert(listener.terminationException === None) - } - listener.checkAsyncErrors() - true - } - ) + // This is to make sure thatquery waits for manual clock to be 600 first time there is data + val mapped = inputData.toDS().agg(count("*")).as[Long].coalesce(1).map { x => + clock.waitTillTime(600) + x } + + testStream(mapped, OutputMode.Complete)( + StartStream(triggerClock = clock), + AddData(inputData, 1, 2), + AdvanceManualClock(100), // unblock getOffset, will block on getBatch + AdvanceManualClock(200), // unblock getBatch, will block on computation + AdvanceManualClock(300), // unblock computation + AssertOnQuery { _ => clock.getTimeMillis() === 600 }, + AssertOnLastQueryStatus { status: StreamingQueryStatus => + // Check the correctness of the trigger info of the last completed batch reported by + // onQueryProgress + assert(status.triggerDetails.get("triggerId") == "0") + assert(status.triggerDetails.get("isTriggerActive") === "false") + assert(status.triggerDetails.get("isDataPresentInTrigger") === "true") + + assert(status.triggerDetails.get("timestamp.triggerStart") === "0") + assert(status.triggerDetails.get("timestamp.afterGetOffset") === "100") + assert(status.triggerDetails.get("timestamp.afterGetBatch") === "300") + assert(status.triggerDetails.get("timestamp.triggerFinish") === "600") + + assert(status.triggerDetails.get("latency.getOffset.total") === "100") + assert(status.triggerDetails.get("latency.getBatch.total") === "200") + assert(status.triggerDetails.get("latency.optimizer") === "0") + assert(status.triggerDetails.get("latency.offsetLogWrite") === "0") + assert(status.triggerDetails.get("latency.fullTrigger") === "600") + + assert(status.triggerDetails.get("numRows.input.total") === "2") + assert(status.triggerDetails.get("numRows.state.aggregation1.total") === "1") + assert(status.triggerDetails.get("numRows.state.aggregation1.updated") === "1") + + assert(status.sourceStatuses.length === 1) + assert(status.sourceStatuses(0).triggerDetails.get("triggerId") === "0") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getOffset.source") === "100") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getBatch.source") === "200") + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "2") + }, + CheckAnswer(2) + ) } test("adding and removing listener") { @@ -172,56 +177,37 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStartedInfo = new StreamingQueryInfo( - "name", - 1, - Seq(new SourceStatus("source1", None), new SourceStatus("source2", None)), - new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString)) - val queryStarted = new StreamingQueryListener.QueryStarted(queryStartedInfo) + val queryStarted = new StreamingQueryListener.QueryStarted(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryStarted] - assertStreamingQueryInfoEquals(queryStarted.queryInfo, newQueryStarted.queryInfo) + assertStreamingQueryInfoEquals(queryStarted.queryStatus, newQueryStarted.queryStatus) } test("QueryProgress serialization") { - val queryProcessInfo = new StreamingQueryInfo( - "name", - 1, - Seq( - new SourceStatus("source1", Some(LongOffset(0).toString)), - new SourceStatus("source2", Some(LongOffset(1).toString))), - new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) - val queryProcess = new StreamingQueryListener.QueryProgress(queryProcessInfo) + val queryProcess = new StreamingQueryListener.QueryProgress(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryProgress] - assertStreamingQueryInfoEquals(queryProcess.queryInfo, newQueryProcess.queryInfo) + assertStreamingQueryInfoEquals(queryProcess.queryStatus, newQueryProcess.queryStatus) } test("QueryTerminated serialization") { - val queryTerminatedInfo = new StreamingQueryInfo( - "name", - 1, - Seq( - new SourceStatus("source1", Some(LongOffset(0).toString)), - new SourceStatus("source2", Some(LongOffset(1).toString))), - new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) val exception = new RuntimeException("exception") val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( - queryTerminatedInfo, + StreamingQueryStatus.testStatus, Some(exception.getMessage)) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryTerminated] - assertStreamingQueryInfoEquals(queryQueryTerminated.queryInfo, newQueryTerminated.queryInfo) + assertStreamingQueryInfoEquals(queryQueryTerminated.queryStatus, newQueryTerminated.queryStatus) assert(queryQueryTerminated.exception === newQueryTerminated.exception) } private def assertStreamingQueryInfoEquals( - expected: StreamingQueryInfo, - actual: StreamingQueryInfo): Unit = { + expected: StreamingQueryStatus, + actual: StreamingQueryStatus): Unit = { assert(expected.name === actual.name) assert(expected.sourceStatuses.size === actual.sourceStatuses.size) expected.sourceStatuses.zip(actual.sourceStatuses).foreach { @@ -243,7 +229,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { private def withListenerAdded(listener: StreamingQueryListener)(body: => Unit): Unit = { try { - failAfter(1 minute) { + failAfter(streamingTimeout) { spark.streams.addListener(listener) body } @@ -258,49 +244,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val listenerBus = spark.streams invokePrivate listenerBusMethod() listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) } +} - class QueryStatusCollector extends StreamingQueryListener { - // to catch errors in the async listener events - @volatile private var asyncTestWaiter = new Waiter - - @volatile var startStatus: StreamingQueryInfo = null - @volatile var terminationStatus: StreamingQueryInfo = null - @volatile var terminationException: Option[String] = null - - val progressStatuses = new ConcurrentLinkedQueue[StreamingQueryInfo] - - def reset(): Unit = { - startStatus = null - terminationStatus = null - progressStatuses.clear() - asyncTestWaiter = new Waiter - } - - def checkAsyncErrors(): Unit = { - asyncTestWaiter.await(timeout(streamingTimeout)) - } - - - override def onQueryStarted(queryStarted: QueryStarted): Unit = { - asyncTestWaiter { - startStatus = queryStarted.queryInfo - } - } - - override def onQueryProgress(queryProgress: QueryProgress): Unit = { - asyncTestWaiter { - assert(startStatus != null, "onQueryProgress called before onQueryStarted") - progressStatuses.add(queryProgress.queryInfo) - } - } - - override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { - asyncTestWaiter { - assert(startStatus != null, "onQueryTerminated called before onQueryStarted") - terminationStatus = queryTerminated.queryInfo - terminationException = queryTerminated.exception - } - asyncTestWaiter.dismiss() - } - } +object StreamingQueryListenerSuite { + // Singleton reference to clock that does not get serialized in task closures + @volatile var clock: ManualClock = null } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 88f1f188ab2af..9f8e2db966367 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,18 +17,27 @@ package org.apache.spark.sql.streaming +import org.scalactic.TolerantNumerics +import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter +import org.apache.spark.internal.Logging +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.streaming.StreamingQueryListener._ +import org.apache.spark.sql.types.StructType import org.apache.spark.SparkException -import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset, MemoryStream, StreamExecution} +import org.apache.spark.sql.execution.streaming._ import org.apache.spark.util.Utils -class StreamingQuerySuite extends StreamTest with BeforeAndAfter { +class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { import AwaitTerminationTester._ import testImplicits._ + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + after { sqlContext.streams.active.foreach(_.stop()) } @@ -100,31 +109,145 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } - testQuietly("source and sink statuses") { + testQuietly("query statuses") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map(6 / _) - testStream(mapped)( - AssertOnQuery(_.sourceStatuses.length === 1), + AssertOnQuery(q => q.status.name === q.name), + AssertOnQuery(q => q.status.id === q.id), + AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === "-"), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === "-"), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.description.contains("Memory")), AssertOnQuery(_.sinkStatus.offsetDesc === new CompositeOffset(None :: Nil).toString), + AddData(inputData, 1, 2), CheckAnswer(6, 3), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.status.inputRate >= 0.0), + AssertOnQuery(_.status.processingRate >= 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(0).toString), + AssertOnQuery(_.status.sourceStatuses(0).inputRate >= 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate >= 0.0), + AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(0)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(0).toString), + AssertOnQuery(_.sourceStatuses(0).inputRate >= 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), + AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(1).toString), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(1).toString), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + + StopStream, + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(1).toString), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(1).toString), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.status.triggerDetails.isEmpty), + + StartStream(), AddData(inputData, 0), ExpectFailure[SparkException], - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(2).toString), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(2).toString), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString) ) } + test("codahale metrics") { + val inputData = MemoryStream[Int] + + /** Whether metrics of a query is registered for reporting */ + def isMetricsRegistered(query: StreamingQuery): Boolean = { + val sourceName = s"StructuredStreaming.${query.name}" + val sources = spark.sparkContext.env.metricsSystem.getSourcesByName(sourceName) + require(sources.size <= 1) + sources.nonEmpty + } + // Disabled by default + assert(spark.conf.get("spark.sql.streaming.metricsEnabled").toBoolean === false) + + withSQLConf("spark.sql.streaming.metricsEnabled" -> "false") { + testStream(inputData.toDF)( + AssertOnQuery { q => !isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + + // Registered when enabled + withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + testStream(inputData.toDF)( + AssertOnQuery { q => isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + } + + test("input row calculation with mixed batch and streaming sources") { + val streamingTriggerDF = spark.createDataset(1 to 10).toDF + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value") + val staticInputDF = spark.createDataFrame(Seq(1 -> "1", 2 -> "2")).toDF("value", "anotherValue") + + // Trigger input has 10 rows, static input has 2 rows, + // therefore after the first trigger, the calculated input rows should be 10 + val status = getFirstTriggerStatus(streamingInputDF.join(staticInputDF, "value")) + assert(status.triggerDetails.get("numRows.input.total") === "10") + assert(status.sourceStatuses.size === 1) + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "10") + } + + test("input row calculation with trigger DF having multiple leaves") { + val streamingTriggerDF = + spark.createDataset(1 to 5).toDF.union(spark.createDataset(6 to 10).toDF) + require(streamingTriggerDF.logicalPlan.collectLeaves().size > 1) + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF) + + // After the first trigger, the calculated input rows should be 10 + val status = getFirstTriggerStatus(streamingInputDF) + assert(status.triggerDetails.get("numRows.input.total") === "10") + assert(status.sourceStatuses.size === 1) + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "10") + } + testQuietly("StreamExecution metadata garbage collection") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map(6 / _) @@ -149,6 +272,45 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } + /** Create a streaming DF that only execute one batch in which it returns the given static DF */ + private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame = { + require(!triggerDF.isStreaming) + // A streaming Source that generate only on trigger and returns the given Dataframe as batch + val source = new Source() { + override def schema: StructType = triggerDF.schema + override def getOffset: Option[Offset] = Some(LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = triggerDF + override def stop(): Unit = {} + } + StreamingExecutionRelation(source) + } + + /** Returns the query status at the end of the first trigger of streaming DF */ + private def getFirstTriggerStatus(streamingDF: DataFrame): StreamingQueryStatus = { + // A StreamingQueryListener that gets the query status after the first completed trigger + val listener = new StreamingQueryListener { + @volatile var firstStatus: StreamingQueryStatus = null + override def onQueryStarted(queryStarted: QueryStarted): Unit = { } + override def onQueryProgress(queryProgress: QueryProgress): Unit = { + if (firstStatus == null) firstStatus = queryProgress.queryStatus + } + override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { } + } + + try { + spark.streams.addListener(listener) + val q = streamingDF.writeStream.format("memory").queryName("test").start() + q.processAllAvailable() + eventually(timeout(streamingTimeout)) { + assert(listener.firstStatus != null) + } + listener.firstStatus + } finally { + spark.streams.active.map(_.stop()) + spark.streams.removeListener(listener) + } + } + /** * A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`. * From adc112429d6fe671e6e8294824a0e41a2b1ec2e0 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 13 Oct 2016 14:16:39 -0700 Subject: [PATCH 080/177] [SPARK-17661][SQL] Consolidate various listLeafFiles implementations ## What changes were proposed in this pull request? There are 4 listLeafFiles-related functions in Spark: - ListingFileCatalog.listLeafFiles (which calls HadoopFsRelation.listLeafFilesInParallel if the number of paths passed in is greater than a threshold; if it is lower, then it has its own serial version implemented) - HadoopFsRelation.listLeafFiles (called only by HadoopFsRelation.listLeafFilesInParallel) - HadoopFsRelation.listLeafFilesInParallel (called only by ListingFileCatalog.listLeafFiles) It is actually very confusing and error prone because there are effectively two distinct implementations for the serial version of listing leaf files. As an example, SPARK-17599 updated only one of the code path and ignored the other one. This code can be improved by: - Move all file listing code into ListingFileCatalog, since it is the only class that needs this. - Keep only one function for listing files in serial. ## How was this patch tested? This change should be covered by existing unit and integration tests. I also moved a test case for HadoopFsRelation.shouldFilterOut from HadoopFsRelationSuite to ListingFileCatalogSuite. Author: petermaxlee Closes #15235 from petermaxlee/SPARK-17661. --- .../datasources/ListingFileCatalog.scala | 231 +++++++++++++----- .../datasources/fileSourceInterfaces.scala | 154 ------------ .../datasources/HadoopFsRelationSuite.scala | 11 - .../datasources/ListingFileCatalogSuite.scala | 34 +++ 4 files changed, 206 insertions(+), 224 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index 32532084236cf..a68ae523e0faa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -21,11 +21,14 @@ import java.io.FileNotFoundException import scala.collection.mutable -import org.apache.hadoop.fs.{FileStatus, LocatedFileStatus, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration /** @@ -82,73 +85,183 @@ class ListingFileCatalog( * This is publicly visible for testing. */ def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession) - } else { - // Right now, the number of paths is less than the value of - // parallelPartitionDiscoveryThreshold. So, we will list file statues at the driver. - // If there is any child that has more files than the threshold, we will use parallel - // listing. - - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - - val statuses: Seq[FileStatus] = paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - logTrace(s"Listing $path on driver") - - val childStatuses = { - try { - val stats = fs.listStatus(path) - if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats - } catch { - case _: FileNotFoundException => - logWarning(s"The directory $path was not found. Was it deleted very recently?") - Array.empty[FileStatus] - } - } + val files = + if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) + } else { + ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf) + } + + mutable.LinkedHashSet(files: _*) + } + + override def equals(other: Any): Boolean = other match { + case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet + case _ => false + } + + override def hashCode(): Int = paths.toSet.hashCode() +} + + +object ListingFileCatalog extends Logging { + + /** A serializable variant of HDFS's BlockLocation. */ + private case class SerializableBlockLocation( + names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + + /** A serializable variant of HDFS's FileStatus. */ + private case class SerializableFileStatus( + path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long, + blockLocations: Array[SerializableBlockLocation]) + + /** + * List a collection of path recursively. + */ + private def listLeafFilesInSerial( + paths: Seq[Path], + hadoopConf: Configuration): Seq[FileStatus] = { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val filter = FileInputFormat.getInputPathFilter(jobConf) + + paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + listLeafFiles0(fs, path, filter) + } + } - childStatuses.map { - case f: LocatedFileStatus => f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of - // paths exceeds threshold. - case f => - if (f.isDirectory ) { - // If f is a directory, we do not need to call getFileBlockLocations (SPARK-14959). - f - } else { - HadoopFsRelation.createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + /** + * List a collection of path recursively in parallel (using Spark executors). + * Each task launched will use [[listLeafFilesInSerial]] to list. + */ + private def listLeafFilesInParallel( + paths: Seq[Path], + hadoopConf: Configuration, + sparkSession: SparkSession): Seq[FileStatus] = { + assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) + logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") + + val sparkContext = sparkSession.sparkContext + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, 10000) + + val statuses = sparkContext + .parallelize(serializedPaths, numParallelism) + .mapPartitions { paths => + val hadoopConf = serializableConfiguration.value + listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator + }.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) } + + case _ => + Array.empty[SerializableBlockLocation] } - }.filterNot { status => - val name = status.getPath.getName - HadoopFsRelation.shouldFilterOut(name) - } - val (dirs, files) = statuses.partition(_.isDirectory) + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + }.collect() - // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) - if (dirs.isEmpty) { - mutable.LinkedHashSet(files: _*) - } else { - mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) + // Turn SerializableFileStatus back to Status + statuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), + blockLocations) } } - override def equals(other: Any): Boolean = other match { - case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet - case _ => false + /** + * List a single path, provided as a FileStatus, in serial. + */ + private def listLeafFiles0( + fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { + logTrace(s"Listing $path") + val name = path.getName.toLowerCase + if (shouldFilterOut(name)) { + Seq.empty[FileStatus] + } else { + // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses = try fs.listStatus(path) catch { + case _: FileNotFoundException => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + val allLeafStatuses = { + val (dirs, files) = statuses.partition(_.isDirectory) + val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) + if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats + } + + allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { + case f: LocatedFileStatus => + f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. + case f => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + val locations = fs.getFileBlockLocations(f, 0, f.getLen) + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs + } + } } - override def hashCode(): Int = paths.toSet.hashCode() + /** Checks if we should filter out this path name. */ + def shouldFilterOut(pathName: String): Boolean = { + // We filter everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && + !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 5cc5f32e6e809..69dd622ce4a54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -17,16 +17,12 @@ package org.apache.spark.sql.execution.datasources -import scala.collection.mutable - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} -import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.annotation.Experimental -import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -35,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.execution.FileRelation import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, Filter} import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration /** * ::Experimental:: @@ -352,152 +347,3 @@ trait FileCatalog { /** Refresh the file listing */ def refresh(): Unit } - - -/** - * Helper methods for gathering metadata from HDFS. - */ -object HadoopFsRelation extends Logging { - - /** Checks if we should filter out this path name. */ - def shouldFilterOut(pathName: String): Boolean = { - // We filter everything that starts with _ and ., except _common_metadata and _metadata - // because Parquet needs to find those metadata files from leaf files returned by this method. - // We should refactor this logic to not mix metadata files with data files. - ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && - !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") - } - - /** - * Create a LocatedFileStatus using FileStatus and block locations. - */ - def createLocatedFileStatus(f: FileStatus, locations: Array[BlockLocation]): LocatedFileStatus = { - // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), which is - // very slow on some file system (RawLocalFileSystem, which is launch a subprocess and parse the - // stdout). - val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, - f.getModificationTime, 0, null, null, null, null, f.getPath, locations) - if (f.isSymlink) { - lfs.setSymlink(f.getSymlink) - } - lfs - } - - // We don't filter files/directories whose name start with "_" except "_temporary" here, as - // specific data sources may take advantages over them (e.g. Parquet _metadata and - // _common_metadata files). "_temporary" directories are explicitly ignored since failed - // tasks/jobs may leave partial/corrupted data files there. Files and directories whose name - // start with "." are also ignored. - def listLeafFiles(fs: FileSystem, status: FileStatus, filter: PathFilter): Array[FileStatus] = { - logTrace(s"Listing ${status.getPath}") - val name = status.getPath.getName.toLowerCase - if (shouldFilterOut(name)) { - Array.empty[FileStatus] - } else { - val statuses = { - val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) - val stats = files ++ dirs.flatMap(dir => listLeafFiles(fs, dir, filter)) - if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats - } - // statuses do not have any dirs. - statuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { - case f: LocatedFileStatus => f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of - // paths exceeds threshold. - case f => createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) - } - } - } - - // `FileStatus` is Writable but not serializable. What make it worse, somehow it doesn't play - // well with `SerializableWritable`. So there seems to be no way to serialize a `FileStatus`. - // Here we use `FakeFileStatus` to extract key components of a `FileStatus` to serialize it from - // executor side and reconstruct it on driver side. - case class FakeBlockLocation( - names: Array[String], - hosts: Array[String], - offset: Long, - length: Long) - - case class FakeFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long, - blockLocations: Array[FakeBlockLocation]) - - def listLeafFilesInParallel( - paths: Seq[Path], - hadoopConf: Configuration, - sparkSession: SparkSession): mutable.LinkedHashSet[FileStatus] = { - assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) - logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") - - val sparkContext = sparkSession.sparkContext - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val serializedPaths = paths.map(_.toString) - - // Set the number of parallelism to prevent following file listing from generating many tasks - // in case of large #defaultParallelism. - val numParallelism = Math.min(paths.size, 10000) - - val fakeStatuses = sparkContext - .parallelize(serializedPaths, numParallelism) - .mapPartitions { paths => - // Dummy jobconf to get to the pathFilter defined in configuration - // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) - val jobConf = new JobConf(serializableConfiguration.value, this.getClass) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - paths.map(new Path(_)).flatMap { path => - val fs = path.getFileSystem(serializableConfiguration.value) - listLeafFiles(fs, fs.getFileStatus(path), pathFilter) - } - }.map { status => - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - FakeBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[FakeBlockLocation] - } - - FakeFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) - }.collect() - - val hadoopFakeStatuses = fakeStatuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) - } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), - blockLocations) - } - mutable.LinkedHashSet(hadoopFakeStatuses: _*) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala index 3c68dc8bb98d8..89d57653adcbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala @@ -39,15 +39,4 @@ class HadoopFsRelationSuite extends QueryTest with SharedSQLContext { assert(df.queryExecution.logical.statistics.sizeInBytes === BigInt(totalSize)) } } - - test("file filtering") { - assert(!HadoopFsRelation.shouldFilterOut("abcd")) - assert(HadoopFsRelation.shouldFilterOut(".ab")) - assert(HadoopFsRelation.shouldFilterOut("_cd")) - - assert(!HadoopFsRelation.shouldFilterOut("_metadata")) - assert(!HadoopFsRelation.shouldFilterOut("_common_metadata")) - assert(HadoopFsRelation.shouldFilterOut("_ab_metadata")) - assert(HadoopFsRelation.shouldFilterOut("_cd_common_metadata")) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala new file mode 100644 index 0000000000000..f15730aeb11f2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.SparkFunSuite + +class ListingFileCatalogSuite extends SparkFunSuite { + + test("file filtering") { + assert(!ListingFileCatalog.shouldFilterOut("abcd")) + assert(ListingFileCatalog.shouldFilterOut(".ab")) + assert(ListingFileCatalog.shouldFilterOut("_cd")) + + assert(!ListingFileCatalog.shouldFilterOut("_metadata")) + assert(!ListingFileCatalog.shouldFilterOut("_common_metadata")) + assert(ListingFileCatalog.shouldFilterOut("_ab_metadata")) + assert(ListingFileCatalog.shouldFilterOut("_cd_common_metadata")) + } +} From 9dc0ca060d5925cd666b34021e62f7b38bb3aabb Mon Sep 17 00:00:00 2001 From: Jakob Odersky Date: Thu, 13 Oct 2016 17:48:09 -0700 Subject: [PATCH 081/177] [SPARK-17368][SQL] Add support for value class serialization and deserialization ## What changes were proposed in this pull request? Value classes were unsupported because catalyst data types were obtained through reflection on erased types, which would resolve to a value class' wrapped type and hence lead to unavailable methods during code generation. E.g. the following class ```scala case class Foo(x: Int) extends AnyVal ``` would be seen as an `int` in catalyst and will cause instance cast failures when generated java code tries to treat it as a `Foo`. This patch simply removes the erasure step when getting data types for catalyst. ## How was this patch tested? Additional tests in `ExpressionEncoderSuite`. Author: Jakob Odersky Closes #15284 from jodersky/value-classes. --- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- .../catalyst/encoders/ExpressionEncoderSuite.scala | 14 ++++++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 7923cfce82100..31c6e5def143b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -628,7 +628,7 @@ object ScalaReflection extends ScalaReflection { /* * Retrieves the runtime class corresponding to the provided type. */ - def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.erasure.typeSymbol.asClass) + def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.typeSymbol.asClass) case class Schema(dataType: DataType, nullable: Boolean) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 4df9062018995..4d896c2e38f10 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -66,8 +66,6 @@ case class RepeatedData( mapFieldNull: scala.collection.Map[Int, java.lang.Long], structField: PrimitiveData) -case class SpecificCollection(l: List[Int]) - /** For testing Kryo serialization based encoder. */ class KryoSerializable(val value: Int) { override def hashCode(): Int = value @@ -107,6 +105,12 @@ class UDTForCaseClass extends UserDefinedType[UDTCaseClass] { } } +case class PrimitiveValueClass(wrapped: Int) extends AnyVal +case class ReferenceValueClass(wrapped: ReferenceValueClass.Container) extends AnyVal +object ReferenceValueClass { + case class Container(data: Int) +} + class ExpressionEncoderSuite extends PlanTest with AnalysisTest { OuterScopes.addOuterScope(this) @@ -290,6 +294,12 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { ExpressionEncoder.tuple(intEnc, ExpressionEncoder.tuple(intEnc, longEnc)) } + encodeDecodeTest( + PrimitiveValueClass(42), "primitive value class") + + encodeDecodeTest( + ReferenceValueClass(ReferenceValueClass.Container(1)), "reference value class") + productTest(("UDT", new ExamplePoint(0.1, 0.2))) test("nullable of encoder schema") { From 44cbb61b34a98e3e0d8e2543a4eb6e950e0019a5 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 13 Oct 2016 19:44:24 -0700 Subject: [PATCH 082/177] [SPARK-15957][FOLLOW-UP][ML][PYSPARK] Add Python API for RFormula forceIndexLabel. ## What changes were proposed in this pull request? Follow-up work of #13675, add Python API for ```RFormula forceIndexLabel```. ## How was this patch tested? Unit test. Author: Yanbo Liang Closes #15430 from yanboliang/spark-15957-python. --- python/pyspark/ml/feature.py | 31 +++++++++++++++++++++++++++---- python/pyspark/ml/tests.py | 16 ++++++++++++++++ 2 files changed, 43 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 64b21caa616ec..a33c3e79453e1 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2494,21 +2494,30 @@ class RFormula(JavaEstimator, HasFeaturesCol, HasLabelCol, JavaMLReadable, JavaM formula = Param(Params._dummy(), "formula", "R model formula", typeConverter=TypeConverters.toString) + forceIndexLabel = Param(Params._dummy(), "forceIndexLabel", + "Force to index label whether it is numeric or string", + typeConverter=TypeConverters.toBoolean) + @keyword_only - def __init__(self, formula=None, featuresCol="features", labelCol="label"): + def __init__(self, formula=None, featuresCol="features", labelCol="label", + forceIndexLabel=False): """ - __init__(self, formula=None, featuresCol="features", labelCol="label") + __init__(self, formula=None, featuresCol="features", labelCol="label", \ + forceIndexLabel=False) """ super(RFormula, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RFormula", self.uid) + self._setDefault(forceIndexLabel=False) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("1.5.0") - def setParams(self, formula=None, featuresCol="features", labelCol="label"): + def setParams(self, formula=None, featuresCol="features", labelCol="label", + forceIndexLabel=False): """ - setParams(self, formula=None, featuresCol="features", labelCol="label") + setParams(self, formula=None, featuresCol="features", labelCol="label", \ + forceIndexLabel=False) Sets params for RFormula. """ kwargs = self.setParams._input_kwargs @@ -2528,6 +2537,20 @@ def getFormula(self): """ return self.getOrDefault(self.formula) + @since("2.1.0") + def setForceIndexLabel(self, value): + """ + Sets the value of :py:attr:`forceIndexLabel`. + """ + return self._set(forceIndexLabel=value) + + @since("2.1.0") + def getForceIndexLabel(self): + """ + Gets the value of :py:attr:`forceIndexLabel`. + """ + return self.getOrDefault(self.forceIndexLabel) + def _create_model(self, java_model): return RFormulaModel(java_model) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index e233549850888..9d46cc3b4ae64 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -477,6 +477,22 @@ def test_count_vectorizer_with_binary(self): feature, expected = r self.assertEqual(feature, expected) + def test_rformula_force_index_label(self): + df = self.spark.createDataFrame([ + (1.0, 1.0, "a"), + (0.0, 2.0, "b"), + (1.0, 0.0, "a")], ["y", "x", "s"]) + # Does not index label by default since it's numeric type. + rf = RFormula(formula="y ~ x + s") + model = rf.fit(df) + transformedDF = model.transform(df) + self.assertEqual(transformedDF.head().label, 1.0) + # Force to index label. + rf2 = RFormula(formula="y ~ x + s").setForceIndexLabel(True) + model2 = rf2.fit(df) + transformedDF2 = model2.transform(df) + self.assertEqual(transformedDF2.head().label, 0.0) + class HasInducedError(Params): From 8543996c3f44098a521fc6b90ca0bb575f606e2a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 14 Oct 2016 12:35:59 +0800 Subject: [PATCH 083/177] [SPARK-17927][SQL] Remove dead code in WriterContainer. ## What changes were proposed in this pull request? speculationEnabled and DATASOURCE_OUTPUTPATH seem like just dead code. ## How was this patch tested? Tests should fail if they are not dead code. Author: Reynold Xin Closes #15477 from rxin/SPARK-17927. --- .../sql/execution/datasources/WriterContainer.scala | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 7880c7cfa16f8..253aa4405defa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -49,7 +49,6 @@ private[datasources] case class WriteRelation( object WriterContainer { val DATASOURCE_WRITEJOBUUID = "spark.sql.sources.writeJobUUID" - val DATASOURCE_OUTPUTPATH = "spark.sql.sources.output.path" } private[datasources] abstract class BaseWriterContainer( @@ -73,9 +72,6 @@ private[datasources] abstract class BaseWriterContainer( // This is only used on driver side. @transient private val jobContext: JobContext = job - private val speculationEnabled: Boolean = - relation.sparkSession.sparkContext.conf.getBoolean("spark.speculation", defaultValue = false) - // The following fields are initialized and used on both driver and executor side. @transient protected var outputCommitter: OutputCommitter = _ @transient private var jobId: JobID = _ @@ -247,8 +243,6 @@ private[datasources] class DefaultWriterContainer( def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { executorSideSetup(taskContext) - val configuration = taskAttemptContext.getConfiguration - configuration.set(WriterContainer.DATASOURCE_OUTPUTPATH, outputPath) var writer = newOutputWriter(getWorkPath) writer.initConverter(dataSchema) @@ -353,15 +347,10 @@ private[datasources] class DynamicPartitionWriterContainer( private def newOutputWriter( key: InternalRow, getPartitionString: UnsafeProjection): OutputWriter = { - val configuration = taskAttemptContext.getConfiguration val path = if (partitionColumns.nonEmpty) { val partitionPath = getPartitionString(key).getString(0) - configuration.set( - WriterContainer.DATASOURCE_OUTPUTPATH, - new Path(outputPath, partitionPath).toString) new Path(getWorkPath, partitionPath).toString } else { - configuration.set(WriterContainer.DATASOURCE_OUTPUTPATH, outputPath) getWorkPath } val bucketId = getBucketIdFromKey(key) From 6c29b3de763115d8676ed91f896e75c490e8c5b2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 14 Oct 2016 14:14:52 +0800 Subject: [PATCH 084/177] [SPARK-17925][SQL] Break fileSourceInterfaces.scala into multiple pieces ## What changes were proposed in this pull request? This patch does a few changes to the file structure of data sources: - Break fileSourceInterfaces.scala into multiple pieces (HadoopFsRelation, FileFormat, OutputWriter) - Move ParquetOutputWriter into its own file I created this as a separate patch so it'd be easier to review my future PRs that focus on refactoring this internal logic. This patch only moves code around, and has no logic changes. ## How was this patch tested? N/A - should be covered by existing tests. Author: Reynold Xin Closes #15473 from rxin/SPARK-17925. --- ...ourceInterfaces.scala => FileFormat.scala} | 143 +------------- .../datasources/HadoopFsRelation.scala | 77 ++++++++ .../execution/datasources/OutputWriter.scala | 101 ++++++++++ .../parquet/ParquetFileFormat.scala | 144 -------------- .../parquet/ParquetOutputWriter.scala | 178 ++++++++++++++++++ 5 files changed, 359 insertions(+), 284 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{fileSourceInterfaces.scala => FileFormat.scala} (59%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala similarity index 59% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 69dd622ce4a54..bde2d2b89d56f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -20,152 +20,15 @@ package org.apache.spark.sql.execution.datasources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} -import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} +import org.apache.hadoop.mapreduce.Job -import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.FileRelation -import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, Filter} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -/** - * ::Experimental:: - * A factory that produces [[OutputWriter]]s. A new [[OutputWriterFactory]] is created on driver - * side for each write job issued when writing to a [[HadoopFsRelation]], and then gets serialized - * to executor side to create actual [[OutputWriter]]s on the fly. - * - * @since 1.4.0 - */ -@Experimental -abstract class OutputWriterFactory extends Serializable { - /** - * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side - * to instantiate new [[OutputWriter]]s. - * - * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that - * this may not point to the final output file. For example, `FileOutputFormat` writes to - * temporary directories and then merge written files back to the final destination. In - * this case, `path` points to a temporary output file under the temporary directory. - * @param dataSchema Schema of the rows to be written. Partition columns are not included in the - * schema if the relation being written is partitioned. - * @param context The Hadoop MapReduce task context. - * @since 1.4.0 - */ - def newInstance( - path: String, - bucketId: Option[Int], // TODO: This doesn't belong here... - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter - - /** - * Returns a new instance of [[OutputWriter]] that will write data to the given path. - * This method gets called by each task on executor to write [[InternalRow]]s to - * format-specific files. Compared to the other `newInstance()`, this is a newer API that - * passes only the path that the writer must write to. The writer must write to the exact path - * and not modify it (do not add subdirectories, extensions, etc.). All other - * file-format-specific information needed to create the writer must be passed - * through the [[OutputWriterFactory]] implementation. - * @since 2.0.0 - */ - def newWriter(path: String): OutputWriter = { - throw new UnsupportedOperationException("newInstance with just path not supported") - } -} - -/** - * ::Experimental:: - * [[OutputWriter]] is used together with [[HadoopFsRelation]] for persisting rows to the - * underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor. - * An [[OutputWriter]] instance is created and initialized when a new output file is opened on - * executor side. This instance is used to persist rows to this single output file. - * - * @since 1.4.0 - */ -@Experimental -abstract class OutputWriter { - /** - * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned - * tables, dynamic partition columns are not included in rows to be written. - * - * @since 1.4.0 - */ - def write(row: Row): Unit - - /** - * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before - * the task output is committed. - * - * @since 1.4.0 - */ - def close(): Unit - - private var converter: InternalRow => Row = _ - - protected[sql] def initConverter(dataSchema: StructType) = { - converter = - CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row] - } - - protected[sql] def writeInternal(row: InternalRow): Unit = { - write(converter(row)) - } -} - -/** - * Acts as a container for all of the metadata required to read from a datasource. All discovery, - * resolution and merging logic for schemas and partitions has been removed. - * - * @param location A [[FileCatalog]] that can enumerate the locations of all the files that comprise - * this relation. - * @param partitionSchema The schema of the columns (if any) that are used to partition the relation - * @param dataSchema The schema of any remaining columns. Note that if any partition columns are - * present in the actual data files as well, they are preserved. - * @param bucketSpec Describes the bucketing (hash-partitioning of the files by some column values). - * @param fileFormat A file format that can be used to read and write the data in files. - * @param options Configuration used when reading / writing data. - */ -case class HadoopFsRelation( - location: FileCatalog, - partitionSchema: StructType, - dataSchema: StructType, - bucketSpec: Option[BucketSpec], - fileFormat: FileFormat, - options: Map[String, String])(val sparkSession: SparkSession) - extends BaseRelation with FileRelation { - - override def sqlContext: SQLContext = sparkSession.sqlContext - - val schema: StructType = { - val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet - StructType(dataSchema ++ partitionSchema.filterNot { column => - dataSchemaColumnNames.contains(column.name.toLowerCase) - }) - } - - def partitionSchemaOption: Option[StructType] = - if (partitionSchema.isEmpty) None else Some(partitionSchema) - def partitionSpec: PartitionSpec = location.partitionSpec() - - def refresh(): Unit = location.refresh() - - override def toString: String = { - fileFormat match { - case source: DataSourceRegister => source.shortName() - case _ => "HadoopFiles" - } - } - - /** Returns the list of files that will be read when scanning this relation. */ - override def inputFiles: Array[String] = - location.allFiles().map(_.getPath.toUri.toString).toArray - - override def sizeInBytes: Long = location.allFiles().map(_.getLen).sum -} - /** * Used to read and write data stored in files to/from the [[InternalRow]] format. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala new file mode 100644 index 0000000000000..c7ebe0b76a150 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.execution.FileRelation +import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister} +import org.apache.spark.sql.types.StructType + + +/** + * Acts as a container for all of the metadata required to read from a datasource. All discovery, + * resolution and merging logic for schemas and partitions has been removed. + * + * @param location A [[FileCatalog]] that can enumerate the locations of all the files that comprise + * this relation. + * @param partitionSchema The schema of the columns (if any) that are used to partition the relation + * @param dataSchema The schema of any remaining columns. Note that if any partition columns are + * present in the actual data files as well, they are preserved. + * @param bucketSpec Describes the bucketing (hash-partitioning of the files by some column values). + * @param fileFormat A file format that can be used to read and write the data in files. + * @param options Configuration used when reading / writing data. + */ +case class HadoopFsRelation( + location: FileCatalog, + partitionSchema: StructType, + dataSchema: StructType, + bucketSpec: Option[BucketSpec], + fileFormat: FileFormat, + options: Map[String, String])(val sparkSession: SparkSession) + extends BaseRelation with FileRelation { + + override def sqlContext: SQLContext = sparkSession.sqlContext + + val schema: StructType = { + val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet + StructType(dataSchema ++ partitionSchema.filterNot { column => + dataSchemaColumnNames.contains(column.name.toLowerCase) + }) + } + + def partitionSchemaOption: Option[StructType] = + if (partitionSchema.isEmpty) None else Some(partitionSchema) + + def partitionSpec: PartitionSpec = location.partitionSpec() + + def refresh(): Unit = location.refresh() + + override def toString: String = { + fileFormat match { + case source: DataSourceRegister => source.shortName() + case _ => "HadoopFiles" + } + } + + /** Returns the list of files that will be read when scanning this relation. */ + override def inputFiles: Array[String] = + location.allFiles().map(_.getPath.toUri.toString).toArray + + override def sizeInBytes: Long = location.allFiles().map(_.getLen).sum +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala new file mode 100644 index 0000000000000..d2eec7b1413f8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.hadoop.mapreduce.TaskAttemptContext + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.types.StructType + + +/** + * A factory that produces [[OutputWriter]]s. A new [[OutputWriterFactory]] is created on driver + * side for each write job issued when writing to a [[HadoopFsRelation]], and then gets serialized + * to executor side to create actual [[OutputWriter]]s on the fly. + */ +abstract class OutputWriterFactory extends Serializable { + /** + * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side + * to instantiate new [[OutputWriter]]s. + * + * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that + * this may not point to the final output file. For example, `FileOutputFormat` writes to + * temporary directories and then merge written files back to the final destination. In + * this case, `path` points to a temporary output file under the temporary directory. + * @param dataSchema Schema of the rows to be written. Partition columns are not included in the + * schema if the relation being written is partitioned. + * @param context The Hadoop MapReduce task context. + * @since 1.4.0 + */ + def newInstance( + path: String, + bucketId: Option[Int], // TODO: This doesn't belong here... + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter + + /** + * Returns a new instance of [[OutputWriter]] that will write data to the given path. + * This method gets called by each task on executor to write InternalRows to + * format-specific files. Compared to the other `newInstance()`, this is a newer API that + * passes only the path that the writer must write to. The writer must write to the exact path + * and not modify it (do not add subdirectories, extensions, etc.). All other + * file-format-specific information needed to create the writer must be passed + * through the [[OutputWriterFactory]] implementation. + * @since 2.0.0 + */ + def newWriter(path: String): OutputWriter = { + throw new UnsupportedOperationException("newInstance with just path not supported") + } +} + + +/** + * [[OutputWriter]] is used together with [[HadoopFsRelation]] for persisting rows to the + * underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor. + * An [[OutputWriter]] instance is created and initialized when a new output file is opened on + * executor side. This instance is used to persist rows to this single output file. + */ +abstract class OutputWriter { + /** + * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned + * tables, dynamic partition columns are not included in rows to be written. + * + * @since 1.4.0 + */ + def write(row: Row): Unit + + /** + * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before + * the task output is committed. + * + * @since 1.4.0 + */ + def close(): Unit + + private var converter: InternalRow => Row = _ + + protected[sql] def initConverter(dataSchema: StructType) = { + converter = + CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row] + } + + protected[sql] def writeInternal(row: InternalRow): Unit = { + write(converter(row)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 4a308ff1a32f8..6faafed1e6290 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -425,150 +425,6 @@ class ParquetFileFormat } } -/** - * A factory for generating OutputWriters for writing parquet files. This implemented is different - * from the [[ParquetOutputWriter]] as this does not use any [[OutputCommitter]]. It simply - * writes the data to the path used to generate the output writer. Callers of this factory - * has to ensure which files are to be considered as committed. - */ -private[parquet] class ParquetOutputWriterFactory( - sqlConf: SQLConf, - dataSchema: StructType, - hadoopConf: Configuration, - options: Map[String, String]) extends OutputWriterFactory { - - private val serializableConf: SerializableConfiguration = { - val job = Job.getInstance(hadoopConf) - val conf = ContextUtil.getConfiguration(job) - val parquetOptions = new ParquetOptions(options, sqlConf) - - // We're not really using `ParquetOutputFormat[Row]` for writing data here, because we override - // it in `ParquetOutputWriter` to support appending and dynamic partitioning. The reason why - // we set it here is to setup the output committer class to `ParquetOutputCommitter`, which is - // bundled with `ParquetOutputFormat[Row]`. - job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) - - ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) - - // We want to clear this temporary metadata from saving into Parquet file. - // This metadata is only useful for detecting optional columns when pushing down filters. - val dataSchemaToWrite = StructType.removeMetadata( - StructType.metadataKeyForOptionalField, - dataSchema).asInstanceOf[StructType] - ParquetWriteSupport.setSchema(dataSchemaToWrite, conf) - - // Sets flags for `CatalystSchemaConverter` (which converts Catalyst schema to Parquet schema) - // and `CatalystWriteSupport` (writing actual rows to Parquet files). - conf.set( - SQLConf.PARQUET_BINARY_AS_STRING.key, - sqlConf.isParquetBinaryAsString.toString) - - conf.set( - SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, - sqlConf.isParquetINT96AsTimestamp.toString) - - conf.set( - SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, - sqlConf.writeLegacyParquetFormat.toString) - - // Sets compression scheme - conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) - new SerializableConfiguration(conf) - } - - /** - * Returns a [[OutputWriter]] that writes data to the give path without using - * [[OutputCommitter]]. - */ - override def newWriter(path: String): OutputWriter = new OutputWriter { - - // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter - private val hadoopTaskAttemptId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) - private val hadoopAttemptContext = new TaskAttemptContextImpl( - serializableConf.value, hadoopTaskAttemptId) - - // Instance of ParquetRecordWriter that does not use OutputCommitter - private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) - - override def write(row: Row): Unit = { - throw new UnsupportedOperationException("call writeInternal") - } - - protected[sql] override def writeInternal(row: InternalRow): Unit = { - recordWriter.write(null, row) - } - - override def close(): Unit = recordWriter.close(hadoopAttemptContext) - } - - /** Create a [[ParquetRecordWriter]] that writes the given path without using OutputCommitter */ - private def createNoCommitterRecordWriter( - path: String, - hadoopAttemptContext: TaskAttemptContext): RecordWriter[Void, InternalRow] = { - // Custom ParquetOutputFormat that disable use of committer and writes to the given path - val outputFormat = new ParquetOutputFormat[InternalRow]() { - override def getOutputCommitter(c: TaskAttemptContext): OutputCommitter = { null } - override def getDefaultWorkFile(c: TaskAttemptContext, ext: String): Path = { new Path(path) } - } - outputFormat.getRecordWriter(hadoopAttemptContext) - } - - /** Disable the use of the older API. */ - def newInstance( - path: String, - bucketId: Option[Int], - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - throw new UnsupportedOperationException( - "this version of newInstance not supported for " + - "ParquetOutputWriterFactory") - } -} - - -// NOTE: This class is instantiated and used on executor side only, no need to be serializable. -private[parquet] class ParquetOutputWriter( - path: String, - bucketId: Option[Int], - context: TaskAttemptContext) - extends OutputWriter { - - private val recordWriter: RecordWriter[Void, InternalRow] = { - val outputFormat = { - new ParquetOutputFormat[InternalRow]() { - // Here we override `getDefaultWorkFile` for two reasons: - // - // 1. To allow appending. We need to generate unique output file names to avoid - // overwriting existing files (either exist before the write job, or are just written - // by other tasks within the same write job). - // - // 2. To allow dynamic partitioning. Default `getDefaultWorkFile` uses - // `FileOutputCommitter.getWorkPath()`, which points to the base directory of all - // partitions in the case of dynamic partitioning. - override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - // It has the `.parquet` extension at the end because (de)compression tools - // such as gunzip would not be able to decompress this as the compression - // is not applied on this whole file but on each "page" in Parquet format. - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") - } - } - } - - outputFormat.getRecordWriter(context) - } - - override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") - - override def writeInternal(row: InternalRow): Unit = recordWriter.write(null, row) - - override def close(): Unit = recordWriter.close(context) -} - object ParquetFileFormat extends Logging { private[parquet] def readSchema( footers: Seq[Footer], sparkSession: SparkSession): Option[StructType] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala new file mode 100644 index 0000000000000..f89ce05d82d90 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.parquet.hadoop.{ParquetOutputFormat, ParquetRecordWriter} +import org.apache.parquet.hadoop.util.ContextUtil + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{BucketingUtils, OutputWriter, OutputWriterFactory, WriterContainer} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + + +/** + * A factory for generating OutputWriters for writing parquet files. This implemented is different + * from the [[ParquetOutputWriter]] as this does not use any [[OutputCommitter]]. It simply + * writes the data to the path used to generate the output writer. Callers of this factory + * has to ensure which files are to be considered as committed. + */ +private[parquet] class ParquetOutputWriterFactory( + sqlConf: SQLConf, + dataSchema: StructType, + hadoopConf: Configuration, + options: Map[String, String]) + extends OutputWriterFactory { + + private val serializableConf: SerializableConfiguration = { + val job = Job.getInstance(hadoopConf) + val conf = ContextUtil.getConfiguration(job) + val parquetOptions = new ParquetOptions(options, sqlConf) + + // We're not really using `ParquetOutputFormat[Row]` for writing data here, because we override + // it in `ParquetOutputWriter` to support appending and dynamic partitioning. The reason why + // we set it here is to setup the output committer class to `ParquetOutputCommitter`, which is + // bundled with `ParquetOutputFormat[Row]`. + job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) + + ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) + + // We want to clear this temporary metadata from saving into Parquet file. + // This metadata is only useful for detecting optional columns when pushing down filters. + val dataSchemaToWrite = StructType.removeMetadata( + StructType.metadataKeyForOptionalField, + dataSchema).asInstanceOf[StructType] + ParquetWriteSupport.setSchema(dataSchemaToWrite, conf) + + // Sets flags for `CatalystSchemaConverter` (which converts Catalyst schema to Parquet schema) + // and `CatalystWriteSupport` (writing actual rows to Parquet files). + conf.set( + SQLConf.PARQUET_BINARY_AS_STRING.key, + sqlConf.isParquetBinaryAsString.toString) + + conf.set( + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, + sqlConf.isParquetINT96AsTimestamp.toString) + + conf.set( + SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, + sqlConf.writeLegacyParquetFormat.toString) + + // Sets compression scheme + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + new SerializableConfiguration(conf) + } + + /** + * Returns a [[OutputWriter]] that writes data to the give path without using + * [[OutputCommitter]]. + */ + override def newWriter(path: String): OutputWriter = new OutputWriter { + + // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter + private val hadoopTaskAttemptId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) + private val hadoopAttemptContext = new TaskAttemptContextImpl( + serializableConf.value, hadoopTaskAttemptId) + + // Instance of ParquetRecordWriter that does not use OutputCommitter + private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) + + override def write(row: Row): Unit = { + throw new UnsupportedOperationException("call writeInternal") + } + + protected[sql] override def writeInternal(row: InternalRow): Unit = { + recordWriter.write(null, row) + } + + override def close(): Unit = recordWriter.close(hadoopAttemptContext) + } + + /** Create a [[ParquetRecordWriter]] that writes the given path without using OutputCommitter */ + private def createNoCommitterRecordWriter( + path: String, + hadoopAttemptContext: TaskAttemptContext): RecordWriter[Void, InternalRow] = { + // Custom ParquetOutputFormat that disable use of committer and writes to the given path + val outputFormat = new ParquetOutputFormat[InternalRow]() { + override def getOutputCommitter(c: TaskAttemptContext): OutputCommitter = { null } + override def getDefaultWorkFile(c: TaskAttemptContext, ext: String): Path = { new Path(path) } + } + outputFormat.getRecordWriter(hadoopAttemptContext) + } + + /** Disable the use of the older API. */ + def newInstance( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + throw new UnsupportedOperationException( + "this version of newInstance not supported for " + + "ParquetOutputWriterFactory") + } +} + + +// NOTE: This class is instantiated and used on executor side only, no need to be serializable. +private[parquet] class ParquetOutputWriter( + path: String, + bucketId: Option[Int], + context: TaskAttemptContext) + extends OutputWriter { + + private val recordWriter: RecordWriter[Void, InternalRow] = { + val outputFormat = { + new ParquetOutputFormat[InternalRow]() { + // Here we override `getDefaultWorkFile` for two reasons: + // + // 1. To allow appending. We need to generate unique output file names to avoid + // overwriting existing files (either exist before the write job, or are just written + // by other tasks within the same write job). + // + // 2. To allow dynamic partitioning. Default `getDefaultWorkFile` uses + // `FileOutputCommitter.getWorkPath()`, which points to the base directory of all + // partitions in the case of dynamic partitioning. + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + val configuration = context.getConfiguration + val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) + val taskAttemptId = context.getTaskAttemptID + val split = taskAttemptId.getTaskID.getId + val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") + // It has the `.parquet` extension at the end because (de)compression tools + // such as gunzip would not be able to decompress this as the compression + // is not applied on this whole file but on each "page" in Parquet format. + new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") + } + } + } + + outputFormat.getRecordWriter(context) + } + + override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") + + override def writeInternal(row: InternalRow): Unit = recordWriter.write(null, row) + + override def close(): Unit = recordWriter.close(context) +} From 2fb12b0a33deeeadfac451095f64dea6c967caac Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 14 Oct 2016 15:53:50 +0800 Subject: [PATCH 085/177] [SPARK-17903][SQL] MetastoreRelation should talk to external catalog instead of hive client ## What changes were proposed in this pull request? `HiveExternalCatalog` should be the only interface to talk to the hive metastore. In `MetastoreRelation` we can just use `ExternalCatalog` instead of `HiveClient` to interact with hive metastore, and add missing API in `ExternalCatalog`. ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #15460 from cloud-fan/relation. --- .../catalyst/catalog/ExternalCatalog.scala | 13 +++++++++++++ .../catalyst/catalog/InMemoryCatalog.scala | 8 ++++++++ .../spark/sql/hive/HiveExternalCatalog.scala | 8 ++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 +++---- .../spark/sql/hive/MetastoreRelation.scala | 19 ++++++++++++------- .../apache/spark/sql/hive/TableReader.scala | 3 +-- .../spark/sql/hive/client/HiveClient.scala | 15 +++------------ .../sql/hive/client/HiveClientImpl.scala | 10 ++++++---- .../sql/hive/HiveExternalCatalogSuite.scala | 9 +++++++++ .../sql/hive/MetastoreRelationSuite.scala | 2 +- .../spark/sql/hive/client/VersionsSuite.scala | 4 ++-- 11 files changed, 66 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index dd93b467eeeb2..348d3d0be2152 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException} +import org.apache.spark.sql.catalyst.expressions.Expression /** @@ -196,6 +197,18 @@ abstract class ExternalCatalog { table: String, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] + /** + * List the metadata of selected partitions according to the given partition predicates. + * + * @param db database name + * @param table table name + * @param predicates partition predicated + */ + def listPartitionsByFilter( + db: String, + table: String, + predicates: Seq[Expression]): Seq[CatalogTablePartition] + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 3e31127118b44..49280f82e20be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -28,6 +28,7 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils /** @@ -477,6 +478,13 @@ class InMemoryCatalog( catalog(db).tables(table).partitions.values.toSeq } + override def listPartitionsByFilter( + db: String, + table: String, + predicates: Seq[Expression]): Seq[CatalogTablePartition] = { + throw new UnsupportedOperationException("listPartitionsByFilter is not implemented.") + } + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 237b829da882f..b5d93c3d7c804 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Statistics} import org.apache.spark.sql.execution.command.{ColumnStatStruct, DDLUtils} import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap @@ -646,6 +647,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat client.getPartitions(db, table, partialSpec) } + override def listPartitionsByFilter( + db: String, + table: String, + predicates: Seq[Expression]): Seq[CatalogTablePartition] = { + client.getPartitionsByFilter(db, table, predicates) + } + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 8410a2e4a47ca..c44f0adda44c0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -44,8 +44,6 @@ import org.apache.spark.sql.types._ */ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Logging { private val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState] - private val client = - sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client /** A fully qualified identifier for a table (i.e., database.tableName) */ case class QualifiedTableName(database: String, name: String) @@ -104,7 +102,8 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log def hiveDefaultTableFilePath(tableIdent: TableIdentifier): String = { // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent) - new Path(new Path(client.getDatabase(dbName).locationUri), tblName).toString + val dbLocation = sparkSession.sharedState.externalCatalog.getDatabase(dbName).locationUri + new Path(new Path(dbLocation), tblName).toString } def lookupRelation( @@ -129,7 +128,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } else { val qualifiedTable = MetastoreRelation( - qualifiedTableName.database, qualifiedTableName.name)(table, client, sparkSession) + qualifiedTableName.database, qualifiedTableName.name)(table, sparkSession) alias.map(a => SubqueryAlias(a, qualifiedTable, None)).getOrElse(qualifiedTable) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 33f0ecff63529..da809cf991de2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -43,7 +43,6 @@ private[hive] case class MetastoreRelation( databaseName: String, tableName: String) (val catalogTable: CatalogTable, - @transient private val client: HiveClient, @transient private val sparkSession: SparkSession) extends LeafNode with MultiInstanceRelation with FileRelation with CatalogRelation { @@ -59,7 +58,7 @@ private[hive] case class MetastoreRelation( Objects.hashCode(databaseName, tableName, output) } - override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: client :: sparkSession :: Nil + override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: sparkSession :: Nil private def toHiveColumn(c: StructField): FieldSchema = { new FieldSchema(c.name, c.dataType.catalogString, c.getComment.orNull) @@ -146,11 +145,18 @@ private[hive] case class MetastoreRelation( // When metastore partition pruning is turned off, we cache the list of all partitions to // mimic the behavior of Spark < 1.5 - private lazy val allPartitions: Seq[CatalogTablePartition] = client.getPartitions(catalogTable) + private lazy val allPartitions: Seq[CatalogTablePartition] = { + sparkSession.sharedState.externalCatalog.listPartitions( + catalogTable.database, + catalogTable.identifier.table) + } def getHiveQlPartitions(predicates: Seq[Expression] = Nil): Seq[Partition] = { val rawPartitions = if (sparkSession.sessionState.conf.metastorePartitionPruning) { - client.getPartitionsByFilter(catalogTable, predicates) + sparkSession.sharedState.externalCatalog.listPartitionsByFilter( + catalogTable.database, + catalogTable.identifier.table, + predicates) } else { allPartitions } @@ -234,8 +240,7 @@ private[hive] case class MetastoreRelation( val columnOrdinals = AttributeMap(attributes.zipWithIndex) override def inputFiles: Array[String] = { - val partLocations = client - .getPartitionsByFilter(catalogTable, Nil) + val partLocations = allPartitions .flatMap(_.storage.locationUri) .toArray if (partLocations.nonEmpty) { @@ -248,6 +253,6 @@ private[hive] case class MetastoreRelation( } override def newInstance(): MetastoreRelation = { - MetastoreRelation(databaseName, tableName)(catalogTable, client, sparkSession) + MetastoreRelation(databaseName, tableName)(catalogTable, sparkSession) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 2a54163a04e9b..aaf30f41f29c2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -149,8 +149,7 @@ class HadoopTableReader( * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, - Class[_ <: Deserializer]], + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[InternalRow] = { // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index 984d23bb09dbd..9ee3d629c9977 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -172,24 +172,15 @@ private[hive] trait HiveClient { * Returns the partitions for the given table that match the supplied partition spec. * If no partition spec is specified, all partitions are returned. */ - final def getPartitions( + def getPartitions( db: String, table: String, - partialSpec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = { - getPartitions(getTable(db, table), partialSpec) - } - - /** - * Returns the partitions for the given table that match the supplied partition spec. - * If no partition spec is specified, all partitions are returned. - */ - def getPartitions( - table: CatalogTable, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] /** Returns partitions filtered by predicates for the given table. */ def getPartitionsByFilter( - table: CatalogTable, + db: String, + table: String, predicates: Seq[Expression]): Seq[CatalogTablePartition] /** Loads a static partition into an existing table. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index dd33d750a4d45..5c8f7ff1af9fa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -525,9 +525,10 @@ private[hive] class HiveClientImpl( * If no partition spec is specified, all partitions are returned. */ override def getPartitions( - table: CatalogTable, + db: String, + table: String, spec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = withHiveState { - val hiveTable = toHiveTable(table) + val hiveTable = toHiveTable(getTable(db, table)) spec match { case None => shim.getAllPartitions(client, hiveTable).map(fromHivePartition) case Some(s) => client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition) @@ -535,9 +536,10 @@ private[hive] class HiveClientImpl( } override def getPartitionsByFilter( - table: CatalogTable, + db: String, + table: String, predicates: Seq[Expression]): Seq[CatalogTablePartition] = withHiveState { - val hiveTable = toHiveTable(table) + val hiveTable = toHiveTable(getTable(db, table)) shim.getPartitionsByFilter(client, hiveTable, predicates).map(fromHivePartition) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 26c2549820de6..efa0beb85030b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.dsl.expressions._ /** * Test suite for the [[HiveExternalCatalog]]. @@ -43,4 +44,12 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { externalCatalog.client.reset() } + import utils._ + + test("list partitions by filter") { + val catalog = newBasicCatalog() + val selectedPartitions = catalog.listPartitionsByFilter("db2", "tbl2", Seq('a.int === 1)) + assert(selectedPartitions.length == 1) + assert(selectedPartitions.head.spec == part1.spec) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala index 2f3055dcac4c5..c28e41a85c39d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala @@ -29,7 +29,7 @@ class MetastoreRelationSuite extends SparkFunSuite { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = StructType(StructField("a", IntegerType, true) :: Nil)) - val relation = MetastoreRelation("db", "test")(table, null, null) + val relation = MetastoreRelation("db", "test")(table, null) // No exception should be thrown relation.makeCopy(Array("db", "test")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 9a10957c8efa5..c158bf1ab09cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -295,12 +295,12 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: getPartitions(catalogTable)") { - assert(2 == client.getPartitions(client.getTable("default", "src_part")).size) + assert(2 == client.getPartitions("default", "src_part").size) } test(s"$version: getPartitionsByFilter") { // Only one partition [1, 1] for key2 == 1 - val result = client.getPartitionsByFilter(client.getTable("default", "src_part"), + val result = client.getPartitionsByFilter("default", "src_part", Seq(EqualTo(AttributeReference("key2", IntegerType)(), Literal(1)))) // Hive 0.12 doesn't support getPartitionsByFilter, it ignores the filter condition. From 1db8feab8c564053c05e8bdc1a7f5026fd637d4f Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 14 Oct 2016 04:17:03 -0700 Subject: [PATCH 086/177] [SPARK-15402][ML][PYSPARK] PySpark ml.evaluation should support save/load ## What changes were proposed in this pull request? Since ```ml.evaluation``` has supported save/load at Scala side, supporting it at Python side is very straightforward and easy. ## How was this patch tested? Add python doctest. Author: Yanbo Liang Closes #13194 from yanboliang/spark-15402. --- python/pyspark/ml/evaluation.py | 45 ++++++++++++++++++++++++++------- 1 file changed, 36 insertions(+), 9 deletions(-) diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 1fe8772da772a..7aa16fa5b90f2 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -22,6 +22,7 @@ from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasLabelCol, HasPredictionCol, HasRawPredictionCol from pyspark.ml.common import inherit_doc +from pyspark.ml.util import JavaMLReadable, JavaMLWritable __all__ = ['Evaluator', 'BinaryClassificationEvaluator', 'RegressionEvaluator', 'MulticlassClassificationEvaluator'] @@ -103,7 +104,8 @@ def isLargerBetter(self): @inherit_doc -class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPredictionCol): +class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPredictionCol, + JavaMLReadable, JavaMLWritable): """ .. note:: Experimental @@ -121,6 +123,11 @@ class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPrediction 0.70... >>> evaluator.evaluate(dataset, {evaluator.metricName: "areaUnderPR"}) 0.83... + >>> bce_path = temp_path + "/bce" + >>> evaluator.save(bce_path) + >>> evaluator2 = BinaryClassificationEvaluator.load(bce_path) + >>> str(evaluator2.getRawPredictionCol()) + 'raw' .. versionadded:: 1.4.0 """ @@ -172,7 +179,8 @@ def setParams(self, rawPredictionCol="rawPrediction", labelCol="label", @inherit_doc -class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): +class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol, + JavaMLReadable, JavaMLWritable): """ .. note:: Experimental @@ -190,6 +198,11 @@ class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): 0.993... >>> evaluator.evaluate(dataset, {evaluator.metricName: "mae"}) 2.649... + >>> re_path = temp_path + "/re" + >>> evaluator.save(re_path) + >>> evaluator2 = RegressionEvaluator.load(re_path) + >>> str(evaluator2.getPredictionCol()) + 'raw' .. versionadded:: 1.4.0 """ @@ -244,7 +257,8 @@ def setParams(self, predictionCol="prediction", labelCol="label", @inherit_doc -class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): +class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol, + JavaMLReadable, JavaMLWritable): """ .. note:: Experimental @@ -260,6 +274,11 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio 0.66... >>> evaluator.evaluate(dataset, {evaluator.metricName: "accuracy"}) 0.66... + >>> mce_path = temp_path + "/mce" + >>> evaluator.save(mce_path) + >>> evaluator2 = MulticlassClassificationEvaluator.load(mce_path) + >>> str(evaluator2.getPredictionCol()) + 'prediction' .. versionadded:: 1.5.0 """ @@ -311,19 +330,27 @@ def setParams(self, predictionCol="prediction", labelCol="label", if __name__ == "__main__": import doctest + import tempfile + import pyspark.ml.evaluation from pyspark.sql import SparkSession - globs = globals().copy() + globs = pyspark.ml.evaluation.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: spark = SparkSession.builder\ .master("local[2]")\ .appName("ml.evaluation tests")\ .getOrCreate() - sc = spark.sparkContext - globs['sc'] = sc globs['spark'] = spark - (failure_count, test_count) = doctest.testmod( - globs=globs, optionflags=doctest.ELLIPSIS) - spark.stop() + temp_path = tempfile.mkdtemp() + globs['temp_path'] = temp_path + try: + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + spark.stop() + finally: + from shutil import rmtree + try: + rmtree(temp_path) + except OSError: + pass if failure_count: exit(-1) From a1b136d05c6c458ae8211b0844bfc98d7693fa42 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 14 Oct 2016 04:25:14 -0700 Subject: [PATCH 087/177] [SPARK-14634][ML] Add BisectingKMeansSummary ## What changes were proposed in this pull request? Add BisectingKMeansSummary ## How was this patch tested? unit test Author: Zheng RuiFeng Closes #12394 from zhengruifeng/biKMSummary. --- .../spark/ml/clustering/BisectingKMeans.scala | 74 ++++++++++++++++++- .../ml/clustering/BisectingKMeansSuite.scala | 18 ++++- .../ml/clustering/GaussianMixtureSuite.scala | 2 +- .../spark/ml/clustering/KMeansSuite.scala | 2 +- 4 files changed, 91 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index a97bd0fb16fd7..add8ee2a4ff8e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.clustering import org.apache.hadoop.fs.Path +import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{Vector, VectorUDT} @@ -127,6 +128,29 @@ class BisectingKMeansModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new BisectingKMeansModel.BisectingKMeansModelWriter(this) + + private var trainingSummary: Option[BisectingKMeansSummary] = None + + private[clustering] def setSummary(summary: BisectingKMeansSummary): this.type = { + this.trainingSummary = Some(summary) + this + } + + /** + * Return true if there exists summary of model. + */ + @Since("2.1.0") + def hasSummary: Boolean = trainingSummary.nonEmpty + + /** + * Gets summary of model on training set. An exception is + * thrown if `trainingSummary == None`. + */ + @Since("2.1.0") + def summary: BisectingKMeansSummary = trainingSummary.getOrElse { + throw new SparkException( + s"No training summary available for the ${this.getClass.getSimpleName}") + } } object BisectingKMeansModel extends MLReadable[BisectingKMeansModel] { @@ -228,14 +252,22 @@ class BisectingKMeans @Since("2.0.0") ( case Row(point: Vector) => OldVectors.fromML(point) } + val instr = Instrumentation.create(this, rdd) + instr.logParams(featuresCol, predictionCol, k, maxIter, seed, minDivisibleClusterSize) + val bkm = new MLlibBisectingKMeans() .setK($(k)) .setMaxIterations($(maxIter)) .setMinDivisibleClusterSize($(minDivisibleClusterSize)) .setSeed($(seed)) val parentModel = bkm.run(rdd) - val model = new BisectingKMeansModel(uid, parentModel) - copyValues(model.setParent(this)) + val model = copyValues(new BisectingKMeansModel(uid, parentModel).setParent(this)) + val summary = new BisectingKMeansSummary( + model.transform(dataset), $(predictionCol), $(featuresCol), $(k)) + model.setSummary(summary) + val m = model.setSummary(summary) + instr.logSuccess(m) + m } @Since("2.0.0") @@ -251,3 +283,41 @@ object BisectingKMeans extends DefaultParamsReadable[BisectingKMeans] { @Since("2.0.0") override def load(path: String): BisectingKMeans = super.load(path) } + + +/** + * :: Experimental :: + * Summary of BisectingKMeans. + * + * @param predictions [[DataFrame]] produced by [[BisectingKMeansModel.transform()]] + * @param predictionCol Name for column of predicted clusters in `predictions` + * @param featuresCol Name for column of features in `predictions` + * @param k Number of clusters + */ +@Since("2.1.0") +@Experimental +class BisectingKMeansSummary private[clustering] ( + @Since("2.1.0") @transient val predictions: DataFrame, + @Since("2.1.0") val predictionCol: String, + @Since("2.1.0") val featuresCol: String, + @Since("2.1.0") val k: Int) extends Serializable { + + /** + * Cluster centers of the transformed data. + */ + @Since("2.1.0") + @transient lazy val cluster: DataFrame = predictions.select(predictionCol) + + /** + * Size of (number of data points in) each cluster. + */ + @Since("2.1.0") + lazy val clusterSizes: Array[Long] = { + val sizes = Array.fill[Long](k)(0) + cluster.groupBy(predictionCol).count().select(predictionCol, "count").collect().foreach { + case Row(cluster: Int, count: Long) => sizes(cluster) = count + } + sizes + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index 4f7d4418a8d09..f2368a9f8dad5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -68,7 +68,7 @@ class BisectingKMeansSuite } } - test("fit & transform") { + test("fit, transform and summary") { val predictionColName = "bisecting_kmeans_prediction" val bkm = new BisectingKMeans().setK(k).setPredictionCol(predictionColName).setSeed(1) val model = bkm.fit(dataset) @@ -85,6 +85,22 @@ class BisectingKMeansSuite assert(clusters === Set(0, 1, 2, 3, 4)) assert(model.computeCost(dataset) < 0.1) assert(model.hasParent) + + // Check validity of model summary + val numRows = dataset.count() + assert(model.hasSummary) + val summary: BisectingKMeansSummary = model.summary + assert(summary.predictionCol === predictionColName) + assert(summary.featuresCol === "features") + assert(summary.predictions.count() === numRows) + for (c <- Array(predictionColName, "features")) { + assert(summary.predictions.columns.contains(c)) + } + assert(summary.cluster.columns === Array(predictionColName)) + val clusterSizes = summary.clusterSizes + assert(clusterSizes.length === k) + assert(clusterSizes.sum === numRows) + assert(clusterSizes.forall(_ >= 0)) } test("read/write") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala index 04366f5250287..003fa6abf6597 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -70,7 +70,7 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext } } - test("fit, transform, and summary") { + test("fit, transform and summary") { val predictionColName = "gm_prediction" val probabilityColName = "gm_probability" val gm = new GaussianMixture().setK(k).setMaxIter(2).setPredictionCol(predictionColName) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index c9ba5a288aadf..ca392653557c4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -82,7 +82,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR } } - test("fit, transform, and summary") { + test("fit, transform and summary") { val predictionColName = "kmeans_prediction" val kmeans = new KMeans().setK(k).setPredictionCol(predictionColName).setSeed(1) val model = kmeans.fit(dataset) From c8b612decba28e51789891f7881b6d4ebc50e2bb Mon Sep 17 00:00:00 2001 From: Peng Date: Fri, 14 Oct 2016 12:48:57 +0100 Subject: [PATCH 088/177] [SPARK-17870][MLLIB][ML] Change statistic to pValue for SelectKBest and SelectPercentile because of DoF difference ## What changes were proposed in this pull request? For feature selection method ChiSquareSelector, it is based on the ChiSquareTestResult.statistic (ChiSqure value) to select the features. It select the features with the largest ChiSqure value. But the Degree of Freedom (df) of ChiSqure value is different in Statistics.chiSqTest(RDD), and for different df, you cannot base on ChiSqure value to select features. So we change statistic to pValue for SelectKBest and SelectPercentile ## How was this patch tested? change existing test Author: Peng Closes #15444 from mpjlu/chisqure-bug. --- .../org/apache/spark/mllib/feature/ChiSqSelector.scala | 4 ++-- .../org/apache/spark/ml/feature/ChiSqSelectorSuite.scala | 6 +++--- .../apache/spark/mllib/feature/ChiSqSelectorSuite.scala | 8 ++++---- python/pyspark/ml/feature.py | 4 ++-- python/pyspark/mllib/feature.py | 8 ++++---- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index c305b36278e87..f8276de4f23d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -234,11 +234,11 @@ class ChiSqSelector @Since("2.1.0") () extends Serializable { val features = selectorType match { case ChiSqSelector.KBest => chiSqTestResult - .sortBy { case (res, _) => -res.statistic } + .sortBy { case (res, _) => res.pValue } .take(numTopFeatures) case ChiSqSelector.Percentile => chiSqTestResult - .sortBy { case (res, _) => -res.statistic } + .sortBy { case (res, _) => res.pValue } .take((chiSqTestResult.length * percentile).toInt) case ChiSqSelector.FPR => chiSqTestResult diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index dfebfc87ea1d3..6af06d82d671a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -38,10 +38,10 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext ) val preFilteredData = Seq( - Vectors.dense(0.0), - Vectors.dense(6.0), Vectors.dense(8.0), - Vectors.dense(5.0) + Vectors.dense(0.0), + Vectors.dense(0.0), + Vectors.dense(8.0) ) val df = sc.parallelize(data.zip(preFilteredData)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala index ec23a4aa7364d..ac702b4b7c69e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala @@ -54,10 +54,10 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext { LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0))), LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0)))), 2) val preFilteredData = - Set(LabeledPoint(0.0, Vectors.dense(Array(0.0))), - LabeledPoint(1.0, Vectors.dense(Array(6.0))), - LabeledPoint(1.0, Vectors.dense(Array(8.0))), - LabeledPoint(2.0, Vectors.dense(Array(5.0)))) + Set(LabeledPoint(0.0, Vectors.dense(Array(8.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0)))) val model = new ChiSqSelector(1).fit(labeledDiscreteData) val filteredData = labeledDiscreteData.map { lp => LabeledPoint(lp.label, model.transform(lp.features)) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index a33c3e79453e1..7683360664ebd 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2592,9 +2592,9 @@ class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, Ja >>> selector = ChiSqSelector(numTopFeatures=1, outputCol="selectedFeatures") >>> model = selector.fit(df) >>> model.transform(df).head().selectedFeatures - DenseVector([1.0]) + DenseVector([18.0]) >>> model.selectedFeatures - [3] + [2] >>> chiSqSelectorPath = temp_path + "/chi-sq-selector" >>> selector.save(chiSqSelectorPath) >>> loadedSelector = ChiSqSelector.load(chiSqSelectorPath) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 4aea81840a162..50ef7c7901c2c 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -288,15 +288,15 @@ class ChiSqSelector(object): ... ] >>> model = ChiSqSelector().setNumTopFeatures(1).fit(sc.parallelize(data)) >>> model.transform(SparseVector(3, {1: 9.0, 2: 6.0})) - SparseVector(1, {0: 6.0}) + SparseVector(1, {}) >>> model.transform(DenseVector([8.0, 9.0, 5.0])) - DenseVector([5.0]) + DenseVector([8.0]) >>> model = ChiSqSelector().setSelectorType("percentile").setPercentile(0.34).fit( ... sc.parallelize(data)) >>> model.transform(SparseVector(3, {1: 9.0, 2: 6.0})) - SparseVector(1, {0: 6.0}) + SparseVector(1, {}) >>> model.transform(DenseVector([8.0, 9.0, 5.0])) - DenseVector([5.0]) + DenseVector([8.0]) >>> data = [ ... LabeledPoint(0.0, SparseVector(4, {0: 8.0, 1: 7.0})), ... LabeledPoint(1.0, SparseVector(4, {1: 9.0, 2: 6.0, 3: 4.0})), From 28b645b1e643ae0f6c56cbe5a92356623306717f Mon Sep 17 00:00:00 2001 From: invkrh Date: Fri, 14 Oct 2016 12:52:08 +0100 Subject: [PATCH 089/177] [SPARK-17855][CORE] Remove query string from jar url ## What changes were proposed in this pull request? Spark-submit support jar url with http protocol. However, if the url contains any query strings, `worker.DriverRunner.downloadUserJar()` method will throw "Did not see expected jar" exception. This is because this method checks the existance of a downloaded jar whose name contains query strings. This is a problem when your jar is located on some web service which requires some additional information to retrieve the file. This pr just removes query strings before checking jar existance on worker. ## How was this patch tested? For now, you can only test this patch by manual test. * Deploy a spark cluster locally * Make sure apache httpd service is on * Save an uber jar, e.g spark-job.jar under `/var/www/html/` * Use http://localhost/spark-job.jar?param=1 as jar url when running `spark-submit` * Job should be launched Author: invkrh Closes #15420 from invkrh/spark-17855. --- .../spark/deploy/worker/DriverRunner.scala | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 289b0b93b0e84..e878c10183f61 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -18,12 +18,12 @@ package org.apache.spark.deploy.worker import java.io._ +import java.net.URI import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ import com.google.common.io.Files -import org.apache.hadoop.fs.Path import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} @@ -147,30 +147,24 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { - val jarPath = new Path(driverDesc.jarUrl) - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) - val jarFileName = jarPath.getName + val jarFileName = new URI(driverDesc.jarUrl).getPath.split("/").last val localJarFile = new File(driverDir, jarFileName) - val localJarFilename = localJarFile.getAbsolutePath - if (!localJarFile.exists()) { // May already exist if running multiple workers on one node - logInfo(s"Copying user jar $jarPath to $destPath") + logInfo(s"Copying user jar ${driverDesc.jarUrl} to $localJarFile") Utils.fetchFile( driverDesc.jarUrl, driverDir, conf, securityManager, - hadoopConf, + SparkHadoopUtil.get.newConfiguration(conf), System.currentTimeMillis(), useCache = false) + if (!localJarFile.exists()) { // Verify copy succeeded + throw new IOException( + s"Can not find expected jar $jarFileName which should have been loaded in $driverDir") + } } - - if (!localJarFile.exists()) { // Verify copy succeeded - throw new Exception(s"Did not see expected jar $jarFileName in $driverDir") - } - - localJarFilename + localJarFile.getAbsolutePath } private[worker] def prepareAndRunDriver(): Int = { From 7486442fe0b70f2aea21d569604e71d7ddf19a77 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Fri, 14 Oct 2016 21:18:49 +0800 Subject: [PATCH 090/177] [SPARK-17073][SQL][FOLLOWUP] generate column-level statistics ## What changes were proposed in this pull request? This pr adds some test cases for statistics: case sensitive column names, non ascii column names, refresh table, and also improves some documentation. ## How was this patch tested? add test cases Author: wangzhenhua Closes #15360 from wzhfy/colStats2. --- .../command/AnalyzeColumnCommand.scala | 53 ++--- .../apache/spark/sql/internal/SQLConf.scala | 3 +- .../spark/sql/hive/StatisticsSuite.scala | 198 +++++++++++++++--- 3 files changed, 197 insertions(+), 57 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 7066378279971..488138709a12b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -59,10 +59,12 @@ case class AnalyzeColumnCommand( def updateStats(catalogTable: CatalogTable, newTotalSize: Long): Unit = { val (rowCount, columnStats) = computeColStats(sparkSession, relation) + // We also update table-level stats in order to keep them consistent with column-level stats. val statistics = Statistics( sizeInBytes = newTotalSize, rowCount = Some(rowCount), - colStats = columnStats ++ catalogTable.stats.map(_.colStats).getOrElse(Map())) + // Newly computed column stats should override the existing ones. + colStats = catalogTable.stats.map(_.colStats).getOrElse(Map()) ++ columnStats) sessionState.catalog.alterTable(catalogTable.copy(stats = Some(statistics))) // Refresh the cached data source table in the catalog. sessionState.catalog.refreshTable(tableIdentWithDB) @@ -90,8 +92,9 @@ case class AnalyzeColumnCommand( } } if (duplicatedColumns.nonEmpty) { - logWarning(s"Duplicated columns ${duplicatedColumns.mkString("(", ", ", ")")} detected " + - s"when analyzing columns ${columnNames.mkString("(", ", ", ")")}, ignoring them.") + logWarning("Duplicate column names were deduplicated in `ANALYZE TABLE` statement. " + + s"Input columns: ${columnNames.mkString("(", ", ", ")")}. " + + s"Duplicate columns: ${duplicatedColumns.mkString("(", ", ", ")")}.") } // Collect statistics per column. @@ -116,22 +119,24 @@ case class AnalyzeColumnCommand( } object ColumnStatStruct { - val zero = Literal(0, LongType) - val one = Literal(1, LongType) + private val zero = Literal(0, LongType) + private val one = Literal(1, LongType) - def numNulls(e: Expression): Expression = if (e.nullable) Sum(If(IsNull(e), one, zero)) else zero - def max(e: Expression): Expression = Max(e) - def min(e: Expression): Expression = Min(e) - def ndv(e: Expression, relativeSD: Double): Expression = { + private def numNulls(e: Expression): Expression = { + if (e.nullable) Sum(If(IsNull(e), one, zero)) else zero + } + private def max(e: Expression): Expression = Max(e) + private def min(e: Expression): Expression = Min(e) + private def ndv(e: Expression, relativeSD: Double): Expression = { // the approximate ndv should never be larger than the number of rows Least(Seq(HyperLogLogPlusPlus(e, relativeSD), Count(one))) } - def avgLength(e: Expression): Expression = Average(Length(e)) - def maxLength(e: Expression): Expression = Max(Length(e)) - def numTrues(e: Expression): Expression = Sum(If(e, one, zero)) - def numFalses(e: Expression): Expression = Sum(If(Not(e), one, zero)) + private def avgLength(e: Expression): Expression = Average(Length(e)) + private def maxLength(e: Expression): Expression = Max(Length(e)) + private def numTrues(e: Expression): Expression = Sum(If(e, one, zero)) + private def numFalses(e: Expression): Expression = Sum(If(Not(e), one, zero)) - def getStruct(exprs: Seq[Expression]): CreateStruct = { + private def getStruct(exprs: Seq[Expression]): CreateStruct = { CreateStruct(exprs.map { expr: Expression => expr.transformUp { case af: AggregateFunction => af.toAggregateExpression() @@ -139,19 +144,19 @@ object ColumnStatStruct { }) } - def numericColumnStat(e: Expression, relativeSD: Double): Seq[Expression] = { + private def numericColumnStat(e: Expression, relativeSD: Double): Seq[Expression] = { Seq(numNulls(e), max(e), min(e), ndv(e, relativeSD)) } - def stringColumnStat(e: Expression, relativeSD: Double): Seq[Expression] = { + private def stringColumnStat(e: Expression, relativeSD: Double): Seq[Expression] = { Seq(numNulls(e), avgLength(e), maxLength(e), ndv(e, relativeSD)) } - def binaryColumnStat(e: Expression): Seq[Expression] = { + private def binaryColumnStat(e: Expression): Seq[Expression] = { Seq(numNulls(e), avgLength(e), maxLength(e)) } - def booleanColumnStat(e: Expression): Seq[Expression] = { + private def booleanColumnStat(e: Expression): Seq[Expression] = { Seq(numNulls(e), numTrues(e), numFalses(e)) } @@ -162,14 +167,14 @@ object ColumnStatStruct { } } - def apply(e: Attribute, relativeSD: Double): CreateStruct = e.dataType match { + def apply(attr: Attribute, relativeSD: Double): CreateStruct = attr.dataType match { // Use aggregate functions to compute statistics we need. - case _: NumericType | TimestampType | DateType => getStruct(numericColumnStat(e, relativeSD)) - case StringType => getStruct(stringColumnStat(e, relativeSD)) - case BinaryType => getStruct(binaryColumnStat(e)) - case BooleanType => getStruct(booleanColumnStat(e)) + case _: NumericType | TimestampType | DateType => getStruct(numericColumnStat(attr, relativeSD)) + case StringType => getStruct(stringColumnStat(attr, relativeSD)) + case BinaryType => getStruct(binaryColumnStat(attr)) + case BooleanType => getStruct(booleanColumnStat(attr)) case otherType => throw new AnalysisException("Analyzing columns is not supported for column " + - s"${e.name} of data type: ${e.dataType}.") + s"${attr.name} of data type: ${attr.dataType}.") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e671604c39855..c8447651dd672 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -578,7 +578,8 @@ object SQLConf { val NDV_MAX_ERROR = SQLConfigBuilder("spark.sql.statistics.ndv.maxError") .internal() - .doc("The maximum estimation error allowed in HyperLogLog++ algorithm.") + .doc("The maximum estimation error allowed in HyperLogLog++ algorithm when generating " + + "column level statistics.") .doubleConf .createWithDefault(0.05) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 85228bb00123d..c351063a63ff8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -21,7 +21,7 @@ import java.io.{File, PrintWriter} import scala.reflect.ClassTag -import org.apache.spark.sql.{AnalysisException, QueryTest, Row, StatisticsTest} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Statistics} import org.apache.spark.sql.execution.command.{AnalyzeTableCommand, DDLUtils} @@ -358,53 +358,187 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils } } - test("generate column-level statistics and load them from hive metastore") { + private def getStatsBeforeAfterUpdate(isAnalyzeColumns: Boolean): (Statistics, Statistics) = { + val tableName = "tbl" + var statsBeforeUpdate: Statistics = null + var statsAfterUpdate: Statistics = null + withTable(tableName) { + val tableIndent = TableIdentifier(tableName, Some("default")) + val catalog = spark.sessionState.catalog.asInstanceOf[HiveSessionCatalog] + sql(s"CREATE TABLE $tableName (key int) USING PARQUET") + sql(s"INSERT INTO $tableName SELECT 1") + if (isAnalyzeColumns) { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS key") + } else { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") + } + // Table lookup will make the table cached. + catalog.lookupRelation(tableIndent) + statsBeforeUpdate = catalog.getCachedDataSourceTable(tableIndent) + .asInstanceOf[LogicalRelation].catalogTable.get.stats.get + + sql(s"INSERT INTO $tableName SELECT 2") + if (isAnalyzeColumns) { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS key") + } else { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") + } + catalog.lookupRelation(tableIndent) + statsAfterUpdate = catalog.getCachedDataSourceTable(tableIndent) + .asInstanceOf[LogicalRelation].catalogTable.get.stats.get + } + (statsBeforeUpdate, statsAfterUpdate) + } + + test("test refreshing table stats of cached data source table by `ANALYZE TABLE` statement") { + val (statsBeforeUpdate, statsAfterUpdate) = getStatsBeforeAfterUpdate(isAnalyzeColumns = false) + + assert(statsBeforeUpdate.sizeInBytes > 0) + assert(statsBeforeUpdate.rowCount == Some(1)) + + assert(statsAfterUpdate.sizeInBytes > statsBeforeUpdate.sizeInBytes) + assert(statsAfterUpdate.rowCount == Some(2)) + } + + test("test refreshing column stats of cached data source table by `ANALYZE TABLE` statement") { + val (statsBeforeUpdate, statsAfterUpdate) = getStatsBeforeAfterUpdate(isAnalyzeColumns = true) + + assert(statsBeforeUpdate.sizeInBytes > 0) + assert(statsBeforeUpdate.rowCount == Some(1)) + StatisticsTest.checkColStat( + dataType = IntegerType, + colStat = statsBeforeUpdate.colStats("key"), + expectedColStat = ColumnStat(InternalRow(0L, 1, 1, 1L)), + rsd = spark.sessionState.conf.ndvMaxError) + + assert(statsAfterUpdate.sizeInBytes > statsBeforeUpdate.sizeInBytes) + assert(statsAfterUpdate.rowCount == Some(2)) + StatisticsTest.checkColStat( + dataType = IntegerType, + colStat = statsAfterUpdate.colStats("key"), + expectedColStat = ColumnStat(InternalRow(0L, 2, 1, 2L)), + rsd = spark.sessionState.conf.ndvMaxError) + } + + private lazy val (testDataFrame, expectedColStatsSeq) = { import testImplicits._ val intSeq = Seq(1, 2) val stringSeq = Seq("a", "bb") + val binarySeq = Seq("a", "bb").map(_.getBytes) val booleanSeq = Seq(true, false) - val data = intSeq.indices.map { i => - (intSeq(i), stringSeq(i), booleanSeq(i)) + (intSeq(i), stringSeq(i), binarySeq(i), booleanSeq(i)) } - val tableName = "table" - withTable(tableName) { - val df = data.toDF("c1", "c2", "c3") - df.write.format("parquet").saveAsTable(tableName) - val expectedColStatsSeq = df.schema.map { f => - val colStat = f.dataType match { - case IntegerType => - ColumnStat(InternalRow(0L, intSeq.max, intSeq.min, intSeq.distinct.length.toLong)) - case StringType => - ColumnStat(InternalRow(0L, stringSeq.map(_.length).sum / stringSeq.length.toDouble, - stringSeq.map(_.length).max.toInt, stringSeq.distinct.length.toLong)) - case BooleanType => - ColumnStat(InternalRow(0L, booleanSeq.count(_.equals(true)).toLong, - booleanSeq.count(_.equals(false)).toLong)) - } - (f, colStat) + val df: DataFrame = data.toDF("c1", "c2", "c3", "c4") + val expectedColStatsSeq: Seq[(StructField, ColumnStat)] = df.schema.map { f => + val colStat = f.dataType match { + case IntegerType => + ColumnStat(InternalRow(0L, intSeq.max, intSeq.min, intSeq.distinct.length.toLong)) + case StringType => + ColumnStat(InternalRow(0L, stringSeq.map(_.length).sum / stringSeq.length.toDouble, + stringSeq.map(_.length).max.toInt, stringSeq.distinct.length.toLong)) + case BinaryType => + ColumnStat(InternalRow(0L, binarySeq.map(_.length).sum / binarySeq.length.toDouble, + binarySeq.map(_.length).max.toInt)) + case BooleanType => + ColumnStat(InternalRow(0L, booleanSeq.count(_.equals(true)).toLong, + booleanSeq.count(_.equals(false)).toLong)) } + (f, colStat) + } + (df, expectedColStatsSeq) + } + + private def checkColStats( + tableName: String, + isDataSourceTable: Boolean, + expectedColStatsSeq: Seq[(StructField, ColumnStat)]): Unit = { + val readback = spark.table(tableName) + val stats = readback.queryExecution.analyzed.collect { + case rel: MetastoreRelation => + assert(!isDataSourceTable, "Expected a Hive serde table, but got a data source table") + rel.catalogTable.stats.get + case rel: LogicalRelation => + assert(isDataSourceTable, "Expected a data source table, but got a Hive serde table") + rel.catalogTable.get.stats.get + } + assert(stats.length == 1) + val columnStats = stats.head.colStats + assert(columnStats.size == expectedColStatsSeq.length) + expectedColStatsSeq.foreach { case (field, expectedColStat) => + StatisticsTest.checkColStat( + dataType = field.dataType, + colStat = columnStats(field.name), + expectedColStat = expectedColStat, + rsd = spark.sessionState.conf.ndvMaxError) + } + } - sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS c1, c2, c3") - val readback = spark.table(tableName) - val relations = readback.queryExecution.analyzed.collect { case rel: LogicalRelation => - val columnStats = rel.catalogTable.get.stats.get.colStats - expectedColStatsSeq.foreach { case (field, expectedColStat) => - assert(columnStats.contains(field.name)) - val colStat = columnStats(field.name) + test("generate and load column-level stats for data source table") { + val dsTable = "dsTable" + withTable(dsTable) { + testDataFrame.write.format("parquet").saveAsTable(dsTable) + sql(s"ANALYZE TABLE $dsTable COMPUTE STATISTICS FOR COLUMNS c1, c2, c3, c4") + checkColStats(dsTable, isDataSourceTable = true, expectedColStatsSeq) + } + } + + test("generate and load column-level stats for hive serde table") { + val hTable = "hTable" + val tmp = "tmp" + withTable(hTable, tmp) { + testDataFrame.write.format("parquet").saveAsTable(tmp) + sql(s"CREATE TABLE $hTable (c1 int, c2 string, c3 binary, c4 boolean) STORED AS TEXTFILE") + sql(s"INSERT INTO $hTable SELECT * FROM $tmp") + sql(s"ANALYZE TABLE $hTable COMPUTE STATISTICS FOR COLUMNS c1, c2, c3, c4") + checkColStats(hTable, isDataSourceTable = false, expectedColStatsSeq) + } + } + + // When caseSensitive is on, for columns with only case difference, they are different columns + // and we should generate column stats for all of them. + private def checkCaseSensitiveColStats(columnName: String): Unit = { + val tableName = "tbl" + withTable(tableName) { + val column1 = columnName.toLowerCase + val column2 = columnName.toUpperCase + withSQLConf("spark.sql.caseSensitive" -> "true") { + sql(s"CREATE TABLE $tableName (`$column1` int, `$column2` double) USING PARQUET") + sql(s"INSERT INTO $tableName SELECT 1, 3.0") + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS `$column1`, `$column2`") + val readback = spark.table(tableName) + val relations = readback.queryExecution.analyzed.collect { case rel: LogicalRelation => + val columnStats = rel.catalogTable.get.stats.get.colStats + assert(columnStats.size == 2) + StatisticsTest.checkColStat( + dataType = IntegerType, + colStat = columnStats(column1), + expectedColStat = ColumnStat(InternalRow(0L, 1, 1, 1L)), + rsd = spark.sessionState.conf.ndvMaxError) StatisticsTest.checkColStat( - dataType = field.dataType, - colStat = colStat, - expectedColStat = expectedColStat, + dataType = DoubleType, + colStat = columnStats(column2), + expectedColStat = ColumnStat(InternalRow(0L, 3.0d, 3.0d, 1L)), rsd = spark.sessionState.conf.ndvMaxError) + rel } - rel + assert(relations.size == 1) } - assert(relations.size == 1) } } + test("check column statistics for case sensitive column names") { + checkCaseSensitiveColStats(columnName = "c1") + } + + test("check column statistics for case sensitive non-ascii column names") { + // scalastyle:off + // non ascii characters are not allowed in the source code, so we disable the scalastyle. + checkCaseSensitiveColStats(columnName = "列c") + // scalastyle:on + } + test("estimates the size of a test MetastoreRelation") { val df = sql("""SELECT * FROM src""") val sizes = df.queryExecution.analyzed.collect { case mr: MetastoreRelation => From a0ebcb3a30ec64e01608ed6fa7b7ffb7acbd3af2 Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Fri, 14 Oct 2016 17:45:27 +0100 Subject: [PATCH 091/177] [DOC] Fix typo in sql hive doc Change is too trivial to file a JIRA. Author: Dhruve Ashar Closes #15485 from dhruve/master. --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d0f43ab0a9cc9..dcc828cc69fed 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -998,7 +998,7 @@ The following options can be used to configure the version of Hive that is used
  • A classpath in the standard format for the JVM. This classpath must include all of Hive and its dependencies, including the correct version of Hadoop. These jars only need to be present on the driver, but if you are running in yarn cluster mode then you must ensure - they are packaged with you application.
  • + they are packaged with your application. From fa37877af02a956203e8a00811b20f34af0278f7 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Fri, 14 Oct 2016 18:13:19 +0100 Subject: [PATCH 092/177] Typo: form -> from ## What changes were proposed in this pull request? Minor typo fix ## How was this patch tested? Existing unit tests on Jenkins Author: Andrew Ash Closes #15486 from ash211/patch-8. --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index a716a916b7f7f..ac3358592202f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -363,7 +363,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * type. *
  • `quote` (default `"`): sets the single character used for escaping quoted values where * the separator can be part of the value. If you would like to turn off quotations, you need to - * set not `null` but an empty string. This behaviour is different form + * set not `null` but an empty string. This behaviour is different from * `com.databricks.spark.csv`.
  • *
  • `escape` (default `\`): sets the single character used for escaping quotes inside * an already quoted value.
  • From 05800b4b4e7873ebc445dfcd020b76d7539686e1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 14 Oct 2016 12:39:25 -0700 Subject: [PATCH 093/177] [TEST] Ignore flaky test in StreamingQueryListenerSuite ## What changes were proposed in this pull request? Ignoring the flaky test introduced in #15307 https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/1736/testReport/junit/org.apache.spark.sql.streaming/StreamingQueryListenerSuite/single_listener__check_trigger_statuses/ Author: Tathagata Das Closes #15491 from tdas/metrics-flaky-test. --- .../spark/sql/streaming/StreamingQueryListenerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 6256385dfd0e4..9e0eefbc58aa5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -43,7 +43,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Make sure we don't leak any events to the next test } - test("single listener, check trigger statuses") { + ignore("single listener, check trigger statuses") { import StreamingQueryListenerSuite._ clock = new ManualClock() From de1c1ca5c9d6064d3b7b3711e3bfb08fa018abe8 Mon Sep 17 00:00:00 2001 From: sethah Date: Fri, 14 Oct 2016 20:21:03 +0000 Subject: [PATCH 094/177] [SPARK-17941][ML][TEST] Logistic regression tests should use sample weights. ## What changes were proposed in this pull request? The sample weight testing for logistic regressions is not robust. Logistic regression suite already has many test cases comparing results to R glmnet. Since both libraries support sample weights, we should use sample weights in the test to increase coverage for sample weighting. This patch doesn't really add any code and makes the testing more complete. Also fixed some errors with the R code that was referenced in the test suit. Changed `standardization=T` to `standardize=T` since the former is invalid. ## How was this patch tested? Existing unit tests are modified. No non-test code is touched. Author: sethah Closes #15488 from sethah/logreg_weight_tests. --- .../LogisticRegressionSuite.scala | 1493 +++++++++-------- 1 file changed, 748 insertions(+), 745 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 42b56754e0835..bc631dc6d3149 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -25,14 +25,14 @@ import scala.util.control.Breaks._ import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.classification.LogisticRegressionSuite._ -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, SparseMatrix, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.functions.{col, lit, rand} import org.apache.spark.sql.types.LongType class LogisticRegressionSuite @@ -40,6 +40,7 @@ class LogisticRegressionSuite import testImplicits._ + private val seed = 42 @transient var smallBinaryDataset: Dataset[_] = _ @transient var smallMultinomialDataset: Dataset[_] = _ @transient var binaryDataset: Dataset[_] = _ @@ -49,7 +50,7 @@ class LogisticRegressionSuite override def beforeAll(): Unit = { super.beforeAll() - smallBinaryDataset = generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42).toDF() + smallBinaryDataset = generateLogisticInput(1.0, 1.0, nPoints = 100, seed = seed).toDF() smallMultinomialDataset = { val nPoints = 100 @@ -61,7 +62,7 @@ class LogisticRegressionSuite val xVariance = Array(0.6856, 0.1899) val testData = generateMultinomialLogisticInput( - coefficients, xMean, xVariance, addIntercept = true, nPoints, 42) + coefficients, xMean, xVariance, addIntercept = true, nPoints, seed) val df = sc.parallelize(testData, 4).toDF() df.cache() @@ -76,9 +77,9 @@ class LogisticRegressionSuite val testData = generateMultinomialLogisticInput(coefficients, xMean, xVariance, - addIntercept = true, nPoints, 42) + addIntercept = true, nPoints, seed) - sc.parallelize(testData, 4).toDF() + sc.parallelize(testData, 4).toDF().withColumn("weight", rand(seed)) } multinomialDataset = { @@ -91,9 +92,9 @@ class LogisticRegressionSuite val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) val testData = generateMultinomialLogisticInput( - coefficients, xMean, xVariance, addIntercept = true, nPoints, 42) + coefficients, xMean, xVariance, addIntercept = true, nPoints, seed) - val df = sc.parallelize(testData, 4).toDF() + val df = sc.parallelize(testData, 4).toDF().withColumn("weight", rand(seed)) df.cache() df } @@ -104,11 +105,11 @@ class LogisticRegressionSuite * so we can validate the training accuracy compared with R's glmnet package. */ ignore("export test data into CSV format") { - binaryDataset.rdd.map { case Row(label: Double, features: Vector) => - label + "," + features.toArray.mkString(",") + binaryDataset.rdd.map { case Row(label: Double, features: Vector, weight: Double) => + label + "," + weight + "," + features.toArray.mkString(",") }.repartition(1).saveAsTextFile("target/tmp/LogisticRegressionSuite/binaryDataset") - multinomialDataset.rdd.map { case Row(label: Double, features: Vector) => - label + "," + features.toArray.mkString(",") + multinomialDataset.rdd.map { case Row(label: Double, features: Vector, weight: Double) => + label + "," + weight + "," + features.toArray.mkString(",") }.repartition(1).saveAsTextFile("target/tmp/LogisticRegressionSuite/multinomialDataset") } @@ -519,31 +520,35 @@ class LogisticRegressionSuite test("binary logistic regression with intercept without regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true).setStandardization(true) + .setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true).setStandardization(false) + .setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 0)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 0)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 2.7355261 + data.V3 -0.5734389 + data.V4 0.8911736 + data.V5 -0.3878645 + data.V6 -0.8060570 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 2.8366423 - data.V2 -0.5895848 - data.V3 0.8931147 - data.V4 -0.3925051 - data.V5 -0.7996864 */ - val interceptR = 2.8366423 - val coefficientsR = Vectors.dense(-0.5895848, 0.8931147, -0.3925051, -0.7996864) + val coefficientsR = Vectors.dense(-0.5734389, 0.8911736, -0.3878645, -0.8060570) + val interceptR = 2.7355261 assert(model1.intercept ~== interceptR relTol 1E-3) assert(model1.coefficients ~= coefficientsR relTol 1E-3) @@ -555,413 +560,374 @@ class LogisticRegressionSuite test("binary logistic regression without intercept without regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false).setStandardization(true) + .setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false).setStandardization(false) + .setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. + Use the following R code to load the data and train the model using glmnet package. - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = - coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 0, intercept=FALSE)) - coefficients + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 0, intercept=FALSE)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 -0.3448461 + data.V4 1.2776453 + data.V5 -0.3539178 + data.V6 -0.7469384 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 -0.3534996 - data.V3 1.2964482 - data.V4 -0.3571741 - data.V5 -0.7407946 */ - val interceptR = 0.0 - val coefficientsR = Vectors.dense(-0.3534996, 1.2964482, -0.3571741, -0.7407946) + val coefficientsR = Vectors.dense(-0.3448461, 1.2776453, -0.3539178, -0.7469384) - assert(model1.intercept ~== interceptR relTol 1E-3) + assert(model1.intercept ~== 0.0 relTol 1E-3) assert(model1.coefficients ~= coefficientsR relTol 1E-2) // Without regularization, with or without standardization should converge to the same solution. - assert(model2.intercept ~== interceptR relTol 1E-3) + assert(model2.intercept ~== 0.0 relTol 1E-3) assert(model2.coefficients ~= coefficientsR relTol 1E-2) } test("binary logistic regression with intercept with L1 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. + Use the following R code to load the data and train the model using glmnet package. - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12)) - coefficients + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, standardize=T)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) -0.06775980 + data.V3 . + data.V4 . + data.V5 -0.03933146 + data.V6 -0.03047580 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) -0.05627428 - data.V2 . - data.V3 . - data.V4 -0.04325749 - data.V5 -0.02481551 */ - val interceptR1 = -0.05627428 - val coefficientsR1 = Vectors.dense(0.0, 0.0, -0.04325749, -0.02481551) + val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.03933146, -0.03047580) + val interceptRStd = -0.06775980 - assert(model1.intercept ~== interceptR1 relTol 1E-2) - assert(model1.coefficients ~= coefficientsR1 absTol 2E-2) + assert(model1.intercept ~== interceptRStd relTol 1E-2) + assert(model1.coefficients ~= coefficientsRStd absTol 2E-2) /* - Using the following R code to load the data and train the model using glmnet package. + Use the following R code to load the data and train the model using glmnet package. - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, - standardize=FALSE)) - coefficients + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, standardize=F)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.3544768 + data.V3 . + data.V4 . + data.V5 -0.1626191 + data.V6 . - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.3722152 - data.V2 . - data.V3 . - data.V4 -0.1665453 - data.V5 . */ - val interceptR2 = 0.3722152 - val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.1665453, 0.0) + val coefficientsR = Vectors.dense(0.0, 0.0, -0.1626191, 0.0) + val interceptR = 0.3544768 - assert(model2.intercept ~== interceptR2 relTol 1E-2) - assert(model2.coefficients ~== coefficientsR2 absTol 1E-3) + assert(model2.intercept ~== interceptR relTol 1E-2) + assert(model2.coefficients ~== coefficientsR absTol 1E-3) // TODO: move this to a standalone test of compression after SPARK-17471 assert(model2.coefficients.isInstanceOf[SparseVector]) } test("binary logistic regression without intercept with L1 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, - intercept=FALSE)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 . - data.V3 . - data.V4 -0.05189203 - data.V5 -0.03891782 - */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(0.0, 0.0, -0.05189203, -0.03891782) - - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 absTol 1E-3) + Use the following R code to load the data and train the model using glmnet package. - /* - Using the following R code to load the data and train the model using glmnet package. + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, intercept=F, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 . + data.V5 -0.04967635 + data.V6 -0.04757757 - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, - intercept=FALSE, standardize=FALSE)) - coefficients + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 . + data.V5 -0.08433195 + data.V6 . - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 . - data.V3 . - data.V4 -0.08420782 - data.V5 . */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.08420782, 0.0) + val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.04967635, -0.04757757) - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 absTol 1E-3) + val coefficientsR = Vectors.dense(0.0, 0.0, -0.08433195, 0.0) + + assert(model1.intercept ~== 0.0 absTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd absTol 1E-3) + assert(model2.intercept ~== 0.0 absTol 1E-3) + assert(model2.coefficients ~= coefficientsR absTol 1E-3) } test("binary logistic regression with intercept with L2 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.15021751 - data.V2 -0.07251837 - data.V3 0.10724191 - data.V4 -0.04865309 - data.V5 -0.10062872 - */ - val interceptR1 = 0.15021751 - val coefficientsR1 = Vectors.dense(-0.07251837, 0.10724191, -0.04865309, -0.10062872) - - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) + Use the following R code to load the data and train the model using glmnet package. - /* - Using the following R code to load the data and train the model using glmnet package. + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.12707703 + data.V3 -0.06980967 + data.V4 0.10803933 + data.V5 -0.04800404 + data.V6 -0.10165096 - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, - standardize=FALSE)) - coefficients + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.46613016 + data.V3 -0.04944529 + data.V4 0.02326772 + data.V5 -0.11362772 + data.V6 -0.06312848 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.48657516 - data.V2 -0.05155371 - data.V3 0.02301057 - data.V4 -0.11482896 - data.V5 -0.06266838 */ - val interceptR2 = 0.48657516 - val coefficientsR2 = Vectors.dense(-0.05155371, 0.02301057, -0.11482896, -0.06266838) + val coefficientsRStd = Vectors.dense(-0.06980967, 0.10803933, -0.04800404, -0.10165096) + val interceptRStd = 0.12707703 + val coefficientsR = Vectors.dense(-0.04944529, 0.02326772, -0.11362772, -0.06312848) + val interceptR = 0.46613016 - assert(model2.intercept ~== interceptR2 relTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) + assert(model1.intercept ~== interceptRStd relTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd relTol 1E-3) + assert(model2.intercept ~== interceptR relTol 1E-3) + assert(model2.coefficients ~= coefficientsR relTol 1E-3) } test("binary logistic regression without intercept with L2 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. + Use the following R code to load the data and train the model using glmnet package. - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, - intercept=FALSE)) - coefficients + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, intercept=F, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 -0.06000152 + data.V4 0.12598737 + data.V5 -0.04669009 + data.V6 -0.09941025 - 5 x 1 sparse Matrix of class "dgCMatrix" + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) . - data.V2 -0.06099165 - data.V3 0.12857058 - data.V4 -0.04708770 - data.V5 -0.09799775 - */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(-0.06099165, 0.12857058, -0.04708770, -0.09799775) - - assert(model1.intercept ~== interceptR1 absTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) - - /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, - intercept=FALSE, standardize=FALSE)) - coefficients + (Intercept) . + data.V3 -0.005482255 + data.V4 0.048106338 + data.V5 -0.093411640 + data.V6 -0.054149798 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 -0.005679651 - data.V3 0.048967094 - data.V4 -0.093714016 - data.V5 -0.053314311 */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(-0.005679651, 0.048967094, -0.093714016, -0.053314311) + val coefficientsRStd = Vectors.dense(-0.06000152, 0.12598737, -0.04669009, -0.09941025) + val coefficientsR = Vectors.dense(-0.005482255, 0.048106338, -0.093411640, -0.054149798) - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-2) + assert(model1.intercept ~== 0.0 absTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd relTol 1E-2) + assert(model2.intercept ~== 0.0 absTol 1E-3) + assert(model2.coefficients ~= coefficientsR relTol 1E-2) } test("binary logistic regression with intercept with ElasticNet regularization") { - val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true) + val trainer1 = (new LogisticRegression).setFitIntercept(true).setMaxIter(200) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.57734851 - data.V2 -0.05310287 - data.V3 . - data.V4 -0.08849250 - data.V5 -0.15458796 - */ - val interceptR1 = 0.57734851 - val coefficientsR1 = Vectors.dense(-0.05310287, 0.0, -0.08849250, -0.15458796) - - assert(model1.intercept ~== interceptR1 relTol 6E-3) - assert(model1.coefficients ~== coefficientsR1 absTol 5E-3) + Use the following R code to load the data and train the model using glmnet package. - /* - Using the following R code to load the data and train the model using glmnet package. + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.49991996 + data.V3 -0.04131110 + data.V4 . + data.V5 -0.08585233 + data.V6 -0.15875400 - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21, - standardize=FALSE)) - coefficients + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.5024256 + data.V3 . + data.V4 . + data.V5 -0.1846038 + data.V6 -0.0559614 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.51555993 - data.V2 . - data.V3 . - data.V4 -0.18807395 - data.V5 -0.05350074 */ - val interceptR2 = 0.51555993 - val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.18807395, -0.05350074) - - assert(model2.intercept ~== interceptR2 relTol 6E-3) - assert(model2.coefficients ~= coefficientsR2 absTol 1E-3) + val coefficientsRStd = Vectors.dense(-0.04131110, 0.0, -0.08585233, -0.15875400) + val interceptRStd = 0.49991996 + val coefficientsR = Vectors.dense(0.0, 0.0, -0.1846038, -0.0559614) + val interceptR = 0.5024256 + + assert(model1.intercept ~== interceptRStd relTol 6E-3) + assert(model1.coefficients ~== coefficientsRStd absTol 5E-3) + assert(model2.intercept ~== interceptR relTol 6E-3) + assert(model2.coefficients ~= coefficientsR absTol 1E-3) } test("binary logistic regression without intercept with ElasticNet regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21, - intercept=FALSE)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 -0.001005743 - data.V3 0.072577857 - data.V4 -0.081203769 - data.V5 -0.142534158 - */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(-0.001005743, 0.072577857, -0.081203769, -0.142534158) - - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 absTol 1E-2) + Use the following R code to load the data and train the model using glmnet package. - /* - Using the following R code to load the data and train the model using glmnet package. + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, intercept=FALSE, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, intercept=FALSE, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 0.06859390 + data.V5 -0.07900058 + data.V6 -0.14684320 - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21, - intercept=FALSE, standardize=FALSE)) - coefficients + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 0.03060637 + data.V5 -0.11126742 + data.V6 . - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 . - data.V3 0.03345223 - data.V4 -0.11304532 - data.V5 . */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(0.0, 0.03345223, -0.11304532, 0.0) + val coefficientsRStd = Vectors.dense(0.0, 0.06859390, -0.07900058, -0.14684320) + val coefficientsR = Vectors.dense(0.0, 0.03060637, -0.11126742, 0.0) - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 absTol 1E-3) + assert(model1.intercept ~== 0.0 relTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd absTol 1E-2) + assert(model2.intercept ~== 0.0 absTol 1E-3) + assert(model2.coefficients ~= coefficientsR absTol 1E-3) } test("binary logistic regression with intercept with strong L1 regularization") { - val trainer1 = (new LogisticRegression).setFitIntercept(true) + val trainer1 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(true) - val trainer2 = (new LogisticRegression).setFitIntercept(true) + val trainer2 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(false) val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) - val histogram = binaryDataset.rdd.map { case Row(label: Double, features: Vector) => label } + val histogram = binaryDataset.as[Instance].rdd.map { i => (i.label, i.weight)} .treeAggregate(new MultiClassSummarizer)( seqOp = (c, v) => (c, v) match { - case (classSummarizer: MultiClassSummarizer, label: Double) => classSummarizer.add(label) + case (classSummarizer: MultiClassSummarizer, (label: Double, weight: Double)) => + classSummarizer.add(label, weight) }, combOp = (c1, c2) => (c1, c2) match { case (classSummarizer1: MultiClassSummarizer, classSummarizer2: MultiClassSummarizer) => @@ -989,25 +955,26 @@ class LogisticRegressionSuite assert(model2.coefficients ~= coefficientsTheory absTol 1E-6) /* - TODO: why is this needed? The correctness of L1 regularization is already checked elsewhere Using the following R code to load the data and train the model using glmnet package. library("glmnet") data <- read.csv("path", header=FALSE) label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1.0, lambda = 6.0)) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1.0, + lambda = 6.0)) coefficients 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) -0.2480643 - data.V2 0.0000000 - data.V3 . - data.V4 . - data.V5 . + s0 + (Intercept) -0.2516986 + data.V3 0.0000000 + data.V4 . + data.V5 . + data.V6 . */ - val interceptR = -0.248065 + val interceptR = -0.2516986 val coefficientsR = Vectors.dense(0.0, 0.0, 0.0, 0.0) assert(model1.intercept ~== interceptR relTol 1E-5) @@ -1015,9 +982,9 @@ class LogisticRegressionSuite } test("multinomial logistic regression with intercept with strong L1 regularization") { - val trainer1 = (new LogisticRegression).setFitIntercept(true) + val trainer1 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(true) - val trainer2 = (new LogisticRegression).setFitIntercept(true) + val trainer2 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(false) val sqlContext = multinomialDataset.sqlContext @@ -1025,16 +992,17 @@ class LogisticRegressionSuite val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) - val histogram = multinomialDataset.as[LabeledPoint].rdd.map(_.label) + val histogram = multinomialDataset.as[Instance].rdd.map(i => (i.label, i.weight)) .treeAggregate(new MultiClassSummarizer)( seqOp = (c, v) => (c, v) match { - case (classSummarizer: MultiClassSummarizer, label: Double) => classSummarizer.add(label) + case (classSummarizer: MultiClassSummarizer, (label: Double, weight: Double)) => + classSummarizer.add(label, weight) }, combOp = (c1, c2) => (c1, c2) match { case (classSummarizer1: MultiClassSummarizer, classSummarizer2: MultiClassSummarizer) => classSummarizer1.merge(classSummarizer2) }).histogram - val numFeatures = multinomialDataset.as[LabeledPoint].first().features.size + val numFeatures = multinomialDataset.as[Instance].first().features.size val numClasses = histogram.length /* @@ -1068,52 +1036,58 @@ class LogisticRegressionSuite test("multinomial logistic regression with intercept without regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setMaxIter(100) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) /* - Using the following R code to load the data and train the model using glmnet package. - > library("glmnet") - > data <- read.csv("path", header=FALSE) - > label = as.factor(data$V1) - > features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - > coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, lambda = 0)) - > coefficients - $`0` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -2.24493379 - V2 0.25096771 - V3 -0.03915938 - V4 0.14766639 - V5 0.36810817 - $`1` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 0.3778931 - V2 -0.3327489 - V3 0.8893666 - V4 -0.2306948 - V5 -0.4442330 - $`2` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 1.86704066 - V2 0.08178121 - V3 -0.85020722 - V4 0.08302840 - V5 0.07612480 - */ + Use the following R code to load the data and train the model using glmnet package. + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", + alpha = 0, lambda = 0)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -2.10320093 + data.V3 0.24337896 + data.V4 -0.05916156 + data.V5 0.14446790 + data.V6 0.35976165 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.3394473 + data.V3 -0.3443375 + data.V4 0.9181331 + data.V5 -0.2283959 + data.V6 -0.4388066 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 1.76375361 + data.V3 0.10095851 + data.V4 -0.85897154 + data.V5 0.08392798 + data.V6 0.07904499 + + + */ val coefficientsR = new DenseMatrix(3, 4, Array( - 0.2509677, -0.0391594, 0.1476664, 0.3681082, - -0.3327489, 0.8893666, -0.2306948, -0.4442330, - 0.0817812, -0.8502072, 0.0830284, 0.0761248), isTransposed = true) - val interceptsR = Vectors.dense(-2.2449338, 0.3778931, 1.8670407) + 0.24337896, -0.05916156, 0.14446790, 0.35976165, + -0.3443375, 0.9181331, -0.2283959, -0.4388066, + 0.10095851, -0.85897154, 0.08392798, 0.07904499), isTransposed = true) + val interceptsR = Vectors.dense(-2.10320093, 0.3394473, 1.76375361) assert(model1.coefficientMatrix ~== coefficientsR relTol 0.05) assert(model1.coefficientMatrix.toArray.sum ~== 0.0 absTol eps) @@ -1128,52 +1102,57 @@ class LogisticRegressionSuite test("multinomial logistic regression without intercept without regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) /* - Using the following R code to load the data and train the model using glmnet package. - library("glmnet") - data <- read.csv("path", header=FALSE) - label = as.factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, lambda = 0, - intercept=F)) - > coefficients - $`0` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 0.06992464 - V3 -0.36562784 - V4 0.12142680 - V5 0.32052211 - $`1` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 -0.3036269 - V3 0.9449630 - V4 -0.2271038 - V5 -0.4364839 - $`2` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 0.2337022 - V3 -0.5793351 - V4 0.1056770 - V5 0.1159618 - */ + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0, intercept=F)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.07276291 + data.V4 -0.36325496 + data.V5 0.12015088 + data.V6 0.31397340 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 -0.3180040 + data.V4 0.9679074 + data.V5 -0.2252219 + data.V6 -0.4319914 + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.2452411 + data.V4 -0.6046524 + data.V5 0.1050710 + data.V6 0.1180180 + + + */ val coefficientsR = new DenseMatrix(3, 4, Array( - 0.0699246, -0.3656278, 0.1214268, 0.3205221, - -0.3036269, 0.9449630, -0.2271038, -0.4364839, - 0.2337022, -0.5793351, 0.1056770, 0.1159618), isTransposed = true) + 0.07276291, -0.36325496, 0.12015088, 0.31397340, + -0.3180040, 0.9679074, -0.2252219, -0.4319914, + 0.2452411, -0.6046524, 0.1050710, 0.1180180), isTransposed = true) assert(model1.coefficientMatrix ~== coefficientsR relTol 0.05) assert(model1.coefficientMatrix.toArray.sum ~== 0.0 absTol eps) @@ -1190,92 +1169,95 @@ class LogisticRegressionSuite // use tighter constraints because OWL-QN solver takes longer to converge val trainer1 = (new LogisticRegression).setFitIntercept(true) .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true) - .setMaxIter(300).setTol(1e-10) + .setMaxIter(300).setTol(1e-10).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false) - .setMaxIter(300).setTol(1e-10) + .setMaxIter(300).setTol(1e-10).setWeightCol("weight") val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) /* - Use the following R code to load the data and train the model using glmnet package. - library("glmnet") - data <- read.csv("path", header=FALSE) - label = as.factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 1, - lambda = 0.05, standardization=T)) - coefficients = coef(glmnet(features, label, family="multinomial", alpha = 1, lambda = 0.05, - standardization=F)) - > coefficientsStd - $`0` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.68988825 - V2 . - V3 . - V4 . - V5 0.09404023 - - $`1` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.2303499 - V2 -0.1232443 - V3 0.3258380 - V4 -0.1564688 - V5 -0.2053965 - - $`2` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 0.9202381 - V2 . - V3 -0.4803856 - V4 . - V5 . - - > coefficients - $`0` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.44893320 - V2 . - V3 . - V4 0.01933812 - V5 0.03666044 - - $`1` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 0.7376760 - V2 -0.0577182 - V3 . - V4 -0.2081718 - V5 -0.1304592 - - $`2` - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.2887428 - V2 . - V3 . - V4 . - V5 . - */ + Use the following R code to load the data and train the model using glmnet package. - val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0, 0.09404023, - -0.1232443, 0.3258380, -0.1564688, -0.2053965, - 0.0, -0.4803856, 0.0, 0.0), isTransposed = true) - val interceptsRStd = Vectors.dense(-0.68988825, -0.2303499, 0.9202381) + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", + alpha = 1, lambda = 0.05, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 1, + lambda = 0.05, standardize=F)) + coefficientsStd + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.62244703 + data.V3 . + data.V4 . + data.V5 . + data.V6 0.08419825 + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.2804845 + data.V3 -0.1336960 + data.V4 0.3717091 + data.V5 -0.1530363 + data.V6 -0.2035286 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.9029315 + data.V3 . + data.V4 -0.4629737 + data.V5 . + data.V6 . + + + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.44215290 + data.V3 . + data.V4 . + data.V5 0.01767089 + data.V6 0.02542866 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.76308326 + data.V3 -0.06818576 + data.V4 . + data.V5 -0.20446351 + data.V6 -0.13017924 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.3209304 + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.08419825, + -0.1336960, 0.3717091, -0.1530363, -0.2035286, + 0.0, -0.4629737, 0.0, 0.0), isTransposed = true) + val interceptsRStd = Vectors.dense(-0.62244703, -0.2804845, 0.9029315) val coefficientsR = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.01933812, 0.03666044, - -0.0577182, 0.0, -0.2081718, -0.1304592, + 0.0, 0.0, 0.01767089, 0.02542866, + -0.06818576, 0.0, -0.20446351, -0.13017924, 0.0, 0.0, 0.0, 0.0), isTransposed = true) - val interceptsR = Vectors.dense(-0.44893320, 0.7376760, -0.2887428) + val interceptsR = Vectors.dense(-0.44215290, 0.76308326, -0.3209304) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.02) assert(model1.interceptVector ~== interceptsRStd relTol 0.1) @@ -1287,87 +1269,91 @@ class LogisticRegressionSuite test("multinomial logistic regression without intercept with L1 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true) + .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false) + .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) /* Use the following R code to load the data and train the model using glmnet package. + library("glmnet") data <- read.csv("path", header=FALSE) label = as.factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 1, - lambda = 0.05, intercept=F, standardization=T)) - coefficients = coef(glmnet(features, label, family="multinomial", alpha = 1, lambda = 0.05, - intercept=F, standardization=F)) - > coefficientsStd + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 1, + lambda = 0.05, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 1, + lambda = 0.05, intercept=F, standardize=F)) + coefficientsStd $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 . - V4 . - V5 0.01525105 + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 0.01144225 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 -0.1502410 - V3 0.5134658 - V4 -0.1601146 - V5 -0.2500232 + s0 + . + data.V3 -0.1678787 + data.V4 0.5385351 + data.V5 -0.1573039 + data.V6 -0.2471624 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 0.003301875 - V3 . - V4 . - V5 . - - > coefficients + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + + coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 . - V4 . - V5 . + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 0.1943624 - V4 -0.1902577 - V5 -0.1028789 + s0 + . + data.V3 . + data.V4 0.1929409 + data.V5 -0.1889121 + data.V6 -0.1010413 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 . - V4 . - V5 . - */ + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0, 0.01525105, - -0.1502410, 0.5134658, -0.1601146, -0.2500232, - 0.003301875, 0.0, 0.0, 0.0), isTransposed = true) + 0.0, 0.0, 0.0, 0.01144225, + -0.1678787, 0.5385351, -0.1573039, -0.2471624, + 0.0, 0.0, 0.0, 0.0), isTransposed = true) val coefficientsR = new DenseMatrix(3, 4, Array( 0.0, 0.0, 0.0, 0.0, - 0.0, 0.1943624, -0.1902577, -0.1028789, + 0.0, 0.1929409, -0.1889121, -0.1010413, 0.0, 0.0, 0.0, 0.0), isTransposed = true) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) @@ -1380,92 +1366,95 @@ class LogisticRegressionSuite test("multinomial logistic regression with intercept with L2 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) /* Use the following R code to load the data and train the model using glmnet package. + library("glmnet") data <- read.csv("path", header=FALSE) label = as.factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0, - lambda = 0.1, intercept=T, standardization=T)) - coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, - lambda = 0.1, intercept=T, standardization=F)) - > coefficientsStd + w = data$V2 + features = as.matrix(data.frame( data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", + alpha = 0, lambda = 0.1, intercept=T, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0.1, intercept=T, standardize=F)) + coefficientsStd $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -1.70040424 - V2 0.17576070 - V3 0.01527894 - V4 0.10216108 - V5 0.26099531 + s0 + -1.5898288335 + data.V3 0.1691226336 + data.V4 0.0002983651 + data.V5 0.1001732896 + data.V6 0.2554575585 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 0.2438590 - V2 -0.2238875 - V3 0.5967610 - V4 -0.1555496 - V5 -0.3010479 + s0 + 0.2125746 + data.V3 -0.2304586 + data.V4 0.6153492 + data.V5 -0.1537017 + data.V6 -0.2975443 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 1.45654525 - V2 0.04812679 - V3 -0.61203992 - V4 0.05338850 - V5 0.04005258 - - > coefficients + s0 + 1.37725427 + data.V3 0.06133600 + data.V4 -0.61564761 + data.V5 0.05352840 + data.V6 0.04208671 + + + coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -1.65488543 - V2 0.15715048 - V3 0.01992903 - V4 0.12428858 - V5 0.22130317 + s0 + -1.5681088 + data.V3 0.1508182 + data.V4 0.0121955 + data.V5 0.1217930 + data.V6 0.2162850 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 1.1297533 - V2 -0.1974768 - V3 0.2776373 - V4 -0.1869445 - V5 -0.2510320 + s0 + 1.1217130 + data.V3 -0.2028984 + data.V4 0.2862431 + data.V5 -0.1843559 + data.V6 -0.2481218 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 0.52513212 - V2 0.04032627 - V3 -0.29756637 - V4 0.06265594 - V5 0.02972883 - */ + s0 + 0.44639579 + data.V3 0.05208012 + data.V4 -0.29843864 + data.V5 0.06256289 + data.V6 0.03183676 - val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.17576070, 0.01527894, 0.10216108, 0.26099531, - -0.2238875, 0.5967610, -0.1555496, -0.3010479, - 0.04812679, -0.61203992, 0.05338850, 0.04005258), isTransposed = true) - val interceptsRStd = Vectors.dense(-1.70040424, 0.2438590, 1.45654525) + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.1691226336, 0.0002983651, 0.1001732896, 0.2554575585, + -0.2304586, 0.6153492, -0.1537017, -0.2975443, + 0.06133600, -0.61564761, 0.05352840, 0.04208671), isTransposed = true) + val interceptsRStd = Vectors.dense(-1.5898288335, 0.2125746, 1.37725427) val coefficientsR = new DenseMatrix(3, 4, Array( - 0.15715048, 0.01992903, 0.12428858, 0.22130317, - -0.1974768, 0.2776373, -0.1869445, -0.2510320, - 0.04032627, -0.29756637, 0.06265594, 0.02972883), isTransposed = true) - val interceptsR = Vectors.dense(-1.65488543, 1.1297533, 0.52513212) + 0.1508182, 0.0121955, 0.1217930, 0.2162850, + -0.2028984, 0.2862431, -0.1843559, -0.2481218, + 0.05208012, -0.29843864, 0.06256289, 0.03183676), isTransposed = true) + val interceptsR = Vectors.dense(-1.5681088, 1.1217130, 0.44639579) - assert(model1.coefficientMatrix ~== coefficientsRStd relTol 0.05) + assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.001) assert(model1.interceptVector ~== interceptsRStd relTol 0.05) assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) assert(model2.coefficientMatrix ~== coefficientsR relTol 0.05) @@ -1475,86 +1464,92 @@ class LogisticRegressionSuite test("multinomial logistic regression without intercept with L2 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) /* Use the following R code to load the data and train the model using glmnet package. + library("glmnet") data <- read.csv("path", header=FALSE) label = as.factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0, - lambda = 0.1, intercept=F, standardization=T)) - coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, - lambda = 0.1, intercept=F, standardization=F)) - > coefficientsStd + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0.1, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0.1, intercept=F, standardize=F)) + coefficientsStd $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 0.03904171 - V3 -0.23354322 - V4 0.08288096 - V5 0.22706393 + s0 + . + data.V3 0.04048126 + data.V4 -0.23075758 + data.V5 0.08228864 + data.V6 0.22277648 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 -0.2061848 - V3 0.6341398 - V4 -0.1530059 - V5 -0.2958455 + s0 + . + data.V3 -0.2149745 + data.V4 0.6478666 + data.V5 -0.1515158 + data.V6 -0.2930498 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 0.16714312 - V3 -0.40059658 - V4 0.07012496 - V5 0.06878158 - > coefficients + s0 + . + data.V3 0.17449321 + data.V4 -0.41710901 + data.V5 0.06922716 + data.V6 0.07027332 + + + coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 -0.005704542 - V3 -0.144466409 - V4 0.092080736 - V5 0.182927657 + s0 + . + data.V3 -0.003949652 + data.V4 -0.142982415 + data.V5 0.091439598 + data.V6 0.179286241 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 -0.08469036 - V3 0.38996748 - V4 -0.16468436 - V5 -0.22522976 + s0 + . + data.V3 -0.09071124 + data.V4 0.39752531 + data.V5 -0.16233832 + data.V6 -0.22206059 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 0.09039490 - V3 -0.24550107 - V4 0.07260362 - V5 0.04230210 + s0 + . + data.V3 0.09466090 + data.V4 -0.25454290 + data.V5 0.07089872 + data.V6 0.04277435 + + */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.03904171, -0.23354322, 0.08288096, 0.2270639, - -0.2061848, 0.6341398, -0.1530059, -0.2958455, - 0.16714312, -0.40059658, 0.07012496, 0.06878158), isTransposed = true) + 0.04048126, -0.23075758, 0.08228864, 0.22277648, + -0.2149745, 0.6478666, -0.1515158, -0.2930498, + 0.17449321, -0.41710901, 0.06922716, 0.07027332), isTransposed = true) val coefficientsR = new DenseMatrix(3, 4, Array( - -0.005704542, -0.144466409, 0.092080736, 0.182927657, - -0.08469036, 0.38996748, -0.16468436, -0.22522976, - 0.0903949, -0.24550107, 0.07260362, 0.0423021), isTransposed = true) + -0.003949652, -0.142982415, 0.091439598, 0.179286241, + -0.09071124, 0.39752531, -0.16233832, -0.22206059, + 0.09466090, -0.25454290, 0.07089872, 0.04277435), isTransposed = true) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) @@ -1565,10 +1560,10 @@ class LogisticRegressionSuite } test("multinomial logistic regression with intercept with elasticnet regularization") { - val trainer1 = (new LogisticRegression).setFitIntercept(true) + val trainer1 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true) .setMaxIter(300).setTol(1e-10) - val trainer2 = (new LogisticRegression).setFitIntercept(true) + val trainer2 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false) .setMaxIter(300).setTol(1e-10) @@ -1576,82 +1571,85 @@ class LogisticRegressionSuite val model2 = trainer2.fit(multinomialDataset) /* Use the following R code to load the data and train the model using glmnet package. + library("glmnet") data <- read.csv("path", header=FALSE) label = as.factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0.5, - lambda = 0.1, intercept=T, standardization=T)) - coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0.5, - lambda = 0.1, intercept=T, standardization=F)) - > coefficientsStd + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=T, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=T, standardize=F)) + coefficientsStd $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.5521819483 - V2 0.0003092611 - V3 . - V4 . - V5 0.0913818490 + s0 + -0.50133383 + data.V3 . + data.V4 . + data.V5 . + data.V6 0.08351653 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.27531989 - V2 -0.09790029 - V3 0.28502034 - V4 -0.12416487 - V5 -0.16513373 + s0 + -0.3151913 + data.V3 -0.1058702 + data.V4 0.3183251 + data.V5 -0.1212969 + data.V6 -0.1629778 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 0.8275018 - V2 . - V3 -0.4044859 - V4 . - V5 . - - > coefficients + s0 + 0.8165252 + data.V3 . + data.V4 -0.3943069 + data.V5 . + data.V6 . + + + coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.39876213 - V2 . - V3 . - V4 0.02547520 - V5 0.03893991 + s0 + -0.38857157 + data.V3 . + data.V4 . + data.V5 0.02384198 + data.V6 0.03127749 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - 0.61089869 - V2 -0.04224269 - V3 . - V4 -0.18923970 - V5 -0.09104249 + s0 + 0.62492165 + data.V3 -0.04949061 + data.V4 . + data.V5 -0.18584462 + data.V6 -0.08952455 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.2121366 - V2 . - V3 . - V4 . - V5 . - */ + s0 + -0.2363501 + data.V3 . + data.V4 . + data.V5 . + data.V6 . - val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0003092611, 0.0, 0.0, 0.091381849, - -0.09790029, 0.28502034, -0.12416487, -0.16513373, - 0.0, -0.4044859, 0.0, 0.0), isTransposed = true) - val interceptsRStd = Vectors.dense(-0.5521819483, -0.27531989, 0.8275018) + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.08351653, + -0.1058702, 0.3183251, -0.1212969, -0.1629778, + 0.0, -0.3943069, 0.0, 0.0), isTransposed = true) + val interceptsRStd = Vectors.dense(-0.50133383, -0.3151913, 0.8165252) val coefficientsR = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0254752, 0.03893991, - -0.04224269, 0.0, -0.1892397, -0.09104249, + 0.0, 0.0, 0.02384198, 0.03127749, + -0.04949061, 0.0, -0.18584462, -0.08952455, 0.0, 0.0, 0.0, 0.0), isTransposed = true) - val interceptsR = Vectors.dense(-0.39876213, 0.61089869, -0.2121366) + val interceptsR = Vectors.dense(-0.38857157, 0.62492165, -0.2363501) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) assert(model1.interceptVector ~== interceptsRStd absTol 0.01) @@ -1662,10 +1660,10 @@ class LogisticRegressionSuite } test("multinomial logistic regression without intercept with elasticnet regularization") { - val trainer1 = (new LogisticRegression).setFitIntercept(false) + val trainer1 = (new LogisticRegression).setFitIntercept(false).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true) .setMaxIter(300).setTol(1e-10) - val trainer2 = (new LogisticRegression).setFitIntercept(false) + val trainer2 = (new LogisticRegression).setFitIntercept(false).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false) .setMaxIter(300).setTol(1e-10) @@ -1673,78 +1671,83 @@ class LogisticRegressionSuite val model2 = trainer2.fit(multinomialDataset) /* Use the following R code to load the data and train the model using glmnet package. + library("glmnet") data <- read.csv("path", header=FALSE) label = as.factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0.5, - lambda = 0.1, intercept=F, standardization=T)) - coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0.5, - lambda = 0.1, intercept=F, standardization=F)) - > coefficientsStd + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=F, standardize=F)) + coefficientsStd $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 . - V4 . - V5 0.03543706 + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 0.03238285 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 -0.1187387 - V3 0.4025482 - V4 -0.1270969 - V5 -0.1918386 + s0 + . + data.V3 -0.1328284 + data.V4 0.4219321 + data.V5 -0.1247544 + data.V6 -0.1893318 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 0.00774365 - V3 . - V4 . - V5 . - - > coefficients + s0 + . + data.V3 0.004572312 + data.V4 . + data.V5 . + data.V6 . + + + coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 . - V4 . - V5 . + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 0.14666497 - V4 -0.16570638 - V5 -0.05982875 + s0 + . + data.V3 . + data.V4 0.14571623 + data.V5 -0.16456351 + data.V6 -0.05866264 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - V2 . - V3 . - V4 . - V5 . + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0, 0.03543706, - -0.1187387, 0.4025482, -0.1270969, -0.1918386, - 0.0, 0.0, 0.0, 0.00774365), isTransposed = true) + 0.0, 0.0, 0.0, 0.03238285, + -0.1328284, 0.4219321, -0.1247544, -0.1893318, + 0.004572312, 0.0, 0.0, 0.0), isTransposed = true) val coefficientsR = new DenseMatrix(3, 4, Array( 0.0, 0.0, 0.0, 0.0, - 0.0, 0.14666497, -0.16570638, -0.05982875, + 0.0, 0.14571623, -0.16456351, -0.05866264, 0.0, 0.0, 0.0, 0.0), isTransposed = true) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) From 7ab86244e30ca81eb4fa40ea77b4c2b8881cbab2 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Fri, 14 Oct 2016 13:22:59 -0700 Subject: [PATCH 095/177] [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables ## What changes were proposed in this pull request? Make sure the hive.default.fileformat is used to when creating the storage format metadata. Output ``` SQL scala> spark.sql("SET hive.default.fileformat=orc") res1: org.apache.spark.sql.DataFrame = [key: string, value: string] scala> spark.sql("CREATE TABLE tmp_default(id INT)") res2: org.apache.spark.sql.DataFrame = [] ``` Before ```SQL scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println) .. [# Storage Information,,] [SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,] [InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,] [OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,] [Compressed:,No,] [Storage Desc Parameters:,,] [ serialization.format,1,] ``` After ```SQL scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println) .. [# Storage Information,,] [SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,] [InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,] [OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,] [Compressed:,No,] [Storage Desc Parameters:,,] [ serialization.format,1,] ``` ## How was this patch tested? Added new tests to HiveDDLCommandSuite Author: Dilip Biswal Closes #15190 from dilipbiswal/orc. --- .../spark/sql/execution/SparkSqlParser.scala | 4 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 26 ++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 39 +++++++++++++++++-- 3 files changed, 60 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index be2eddbb0e423..8c68d1e3a2379 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1010,9 +1010,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { .orElse(Some("org.apache.hadoop.mapred.TextInputFormat")), outputFormat = defaultHiveSerde.flatMap(_.outputFormat) .orElse(Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")), - // Note: Keep this unspecified because we use the presence of the serde to decide - // whether to convert a table created by CTAS to a datasource table. - serde = None, + serde = defaultHiveSerde.flatMap(_.serde), compressed = false, properties = Map()) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 9ce3338647398..81337493c7f28 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -30,10 +30,12 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Generate, ScriptTransformation} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.CreateTable -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType -class HiveDDLCommandSuite extends PlanTest { +class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingleton { val parser = TestHive.sessionState.sqlParser private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { @@ -556,4 +558,24 @@ class HiveDDLCommandSuite extends PlanTest { assert(partition2.get.apply("c") == "1" && partition2.get.apply("d") == "2") } + test("Test the default fileformat for Hive-serde tables") { + withSQLConf("hive.default.fileformat" -> "orc") { + val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + assert(exists) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) + assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) + assert(desc.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } + + withSQLConf("hive.default.fileformat" -> "parquet") { + val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + assert(exists) + val input = desc.storage.inputFormat + val output = desc.storage.outputFormat + val serde = desc.storage.serde + assert(input == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(output == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + assert(serde == Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 6f2a16662bf10..5798f47228216 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -492,7 +492,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { def checkRelation( tableName: String, - isDataSourceParquet: Boolean, + isDataSourceTable: Boolean, format: String, userSpecifiedLocation: Option[String] = None): Unit = { val relation = EliminateSubqueryAliases( @@ -501,7 +501,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) relation match { case LogicalRelation(r: HadoopFsRelation, _, _) => - if (!isDataSourceParquet) { + if (!isDataSourceTable) { fail( s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + s"${HadoopFsRelation.getClass.getCanonicalName}.") @@ -514,7 +514,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(catalogTable.provider.get === format) case r: MetastoreRelation => - if (isDataSourceParquet) { + if (isDataSourceTable) { fail( s"${HadoopFsRelation.getClass.getCanonicalName} is expected, but found " + s"${classOf[MetastoreRelation].getCanonicalName}.") @@ -524,8 +524,15 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(r.catalogTable.storage.locationUri.get === location) case None => // OK. } - // Also make sure that the format is the desired format. + // Also make sure that the format and serde are as desired. assert(catalogTable.storage.inputFormat.get.toLowerCase.contains(format)) + assert(catalogTable.storage.outputFormat.get.toLowerCase.contains(format)) + val serde = catalogTable.storage.serde.get + format match { + case "sequence" | "text" => assert(serde.contains("LazySimpleSerDe")) + case "rcfile" => assert(serde.contains("LazyBinaryColumnarSerDe")) + case _ => assert(serde.toLowerCase.contains(format)) + } } // When a user-specified location is defined, the table type needs to be EXTERNAL. @@ -587,6 +594,30 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("CTAS with default fileformat") { + val table = "ctas1" + val ctas = s"CREATE TABLE IF NOT EXISTS $table SELECT key k, value FROM src" + withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { + withSQLConf("hive.default.fileformat" -> "textfile") { + withTable(table) { + sql(ctas) + // We should use parquet here as that is the default datasource fileformat. The default + // datasource file format is controlled by `spark.sql.sources.default` configuration. + // This testcase verifies that setting `hive.default.fileformat` has no impact on + // the target table's fileformat in case of CTAS. + assert(sessionState.conf.defaultDataSourceName === "parquet") + checkRelation(table, isDataSourceTable = true, "parquet") + } + } + withSQLConf("spark.sql.sources.default" -> "orc") { + withTable(table) { + sql(ctas) + checkRelation(table, isDataSourceTable = true, "orc") + } + } + } + } + test("CTAS without serde with location") { withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { withTempDir { dir => From 522dd0d0e5af83e45a3c3526c191aa4b8bcaeeeb Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 14 Oct 2016 14:09:35 -0700 Subject: [PATCH 096/177] Revert "[SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables" This reverts commit 7ab86244e30ca81eb4fa40ea77b4c2b8881cbab2. --- .../spark/sql/execution/SparkSqlParser.scala | 4 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 26 +------------ .../sql/hive/execution/SQLQuerySuite.scala | 39 ++----------------- 3 files changed, 9 insertions(+), 60 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8c68d1e3a2379..be2eddbb0e423 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1010,7 +1010,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { .orElse(Some("org.apache.hadoop.mapred.TextInputFormat")), outputFormat = defaultHiveSerde.flatMap(_.outputFormat) .orElse(Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")), - serde = defaultHiveSerde.flatMap(_.serde), + // Note: Keep this unspecified because we use the presence of the serde to decide + // whether to convert a table created by CTAS to a datasource table. + serde = None, compressed = false, properties = Map()) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 81337493c7f28..9ce3338647398 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -30,12 +30,10 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Generate, ScriptTransformation} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.CreateTable -import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.types.StructType -class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingleton { +class HiveDDLCommandSuite extends PlanTest { val parser = TestHive.sessionState.sqlParser private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { @@ -558,24 +556,4 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(partition2.get.apply("c") == "1" && partition2.get.apply("d") == "2") } - test("Test the default fileformat for Hive-serde tables") { - withSQLConf("hive.default.fileformat" -> "orc") { - val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") - assert(exists) - assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) - assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) - assert(desc.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) - } - - withSQLConf("hive.default.fileformat" -> "parquet") { - val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") - assert(exists) - val input = desc.storage.inputFormat - val output = desc.storage.outputFormat - val serde = desc.storage.serde - assert(input == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) - assert(output == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) - assert(serde == Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) - } - } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 5798f47228216..6f2a16662bf10 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -492,7 +492,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { def checkRelation( tableName: String, - isDataSourceTable: Boolean, + isDataSourceParquet: Boolean, format: String, userSpecifiedLocation: Option[String] = None): Unit = { val relation = EliminateSubqueryAliases( @@ -501,7 +501,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) relation match { case LogicalRelation(r: HadoopFsRelation, _, _) => - if (!isDataSourceTable) { + if (!isDataSourceParquet) { fail( s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + s"${HadoopFsRelation.getClass.getCanonicalName}.") @@ -514,7 +514,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(catalogTable.provider.get === format) case r: MetastoreRelation => - if (isDataSourceTable) { + if (isDataSourceParquet) { fail( s"${HadoopFsRelation.getClass.getCanonicalName} is expected, but found " + s"${classOf[MetastoreRelation].getCanonicalName}.") @@ -524,15 +524,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(r.catalogTable.storage.locationUri.get === location) case None => // OK. } - // Also make sure that the format and serde are as desired. + // Also make sure that the format is the desired format. assert(catalogTable.storage.inputFormat.get.toLowerCase.contains(format)) - assert(catalogTable.storage.outputFormat.get.toLowerCase.contains(format)) - val serde = catalogTable.storage.serde.get - format match { - case "sequence" | "text" => assert(serde.contains("LazySimpleSerDe")) - case "rcfile" => assert(serde.contains("LazyBinaryColumnarSerDe")) - case _ => assert(serde.toLowerCase.contains(format)) - } } // When a user-specified location is defined, the table type needs to be EXTERNAL. @@ -594,30 +587,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("CTAS with default fileformat") { - val table = "ctas1" - val ctas = s"CREATE TABLE IF NOT EXISTS $table SELECT key k, value FROM src" - withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { - withSQLConf("hive.default.fileformat" -> "textfile") { - withTable(table) { - sql(ctas) - // We should use parquet here as that is the default datasource fileformat. The default - // datasource file format is controlled by `spark.sql.sources.default` configuration. - // This testcase verifies that setting `hive.default.fileformat` has no impact on - // the target table's fileformat in case of CTAS. - assert(sessionState.conf.defaultDataSourceName === "parquet") - checkRelation(table, isDataSourceTable = true, "parquet") - } - } - withSQLConf("spark.sql.sources.default" -> "orc") { - withTable(table) { - sql(ctas) - checkRelation(table, isDataSourceTable = true, "orc") - } - } - } - } - test("CTAS without serde with location") { withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { withTempDir { dir => From da9aeb0fde589f7c21c2f4a32036a68c0353965d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 14 Oct 2016 14:45:20 -0700 Subject: [PATCH 097/177] [SPARK-17863][SQL] should not add column into Distinct ## What changes were proposed in this pull request? We are trying to resolve the attribute in sort by pulling up some column for grandchild into child, but that's wrong when the child is Distinct, because the added column will change the behavior of Distinct, we should not do that. ## How was this patch tested? Added regression test. Author: Davies Liu Closes #15489 from davies/order_distinct. --- .../sql/catalyst/analysis/Analyzer.scala | 2 ++ .../org/apache/spark/sql/SQLQuerySuite.scala | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 536d38777f89d..f8f4799322b3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -838,6 +838,8 @@ class Analyzer( // attributes that its child might have or could have. val missing = missingAttrs -- g.child.outputSet g.copy(join = true, child = addMissingAttr(g.child, missing)) + case d: Distinct => + throw new AnalysisException(s"Can't add $missingAttrs to $d") case u: UnaryNode => u.withNewChildren(addMissingAttr(u.child, missingAttrs) :: Nil) case other => 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 0ee8c959eeb4d..60978efddd7f8 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 @@ -1106,6 +1106,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } + test("SPARK-17863: SELECT distinct does not work correctly if order by missing attribute") { + checkAnswer( + sql("""select distinct struct.a, struct.b + |from ( + | select named_struct('a', 1, 'b', 2, 'c', 3) as struct + | union all + | select named_struct('a', 1, 'b', 2, 'c', 4) as struct) tmp + |order by a, b + |""".stripMargin), + Row(1, 2) :: Nil) + + val error = intercept[AnalysisException] { + sql("""select distinct struct.a, struct.b + |from ( + | select named_struct('a', 1, 'b', 2, 'c', 3) as struct + | union all + | select named_struct('a', 1, 'b', 2, 'c', 4) as struct) tmp + |order by struct.a, struct.b + |""".stripMargin) + } + assert(error.message contains "cannot resolve '`struct.a`' given input columns: [a, b]") + + } + test("cast boolean to string") { // TODO Ensure true/false string letter casing is consistent with Hive in all cases. checkAnswer( From 5aeb7384c7aa5f487f031f9ae07d3f1653399d14 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Fri, 14 Oct 2016 15:07:32 -0700 Subject: [PATCH 098/177] [SPARK-16063][SQL] Add storageLevel to Dataset [SPARK-11905](https://issues.apache.org/jira/browse/SPARK-11905) added support for `persist`/`cache` for `Dataset`. However, there is no user-facing API to check if a `Dataset` is cached and if so what the storage level is. This PR adds `getStorageLevel` to `Dataset`, analogous to `RDD.getStorageLevel`. Updated `DatasetCacheSuite`. Author: Nick Pentreath Closes #13780 from MLnick/ds-storagelevel. Signed-off-by: Michael Armbrust --- python/pyspark/sql/dataframe.py | 36 +++++++++++++++---- .../scala/org/apache/spark/sql/Dataset.scala | 12 +++++++ .../apache/spark/sql/DatasetCacheSuite.scala | 36 +++++++++++++------ 3 files changed, 68 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ce277eb204d13..7606ac08bae67 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -407,24 +407,48 @@ def foreachPartition(self, f): @since(1.3) def cache(self): - """ Persists with the default storage level (C{MEMORY_ONLY}). + """Persists the :class:`DataFrame` with the default storage level (C{MEMORY_AND_DISK}). + + .. note:: the default storage level has changed to C{MEMORY_AND_DISK} to match Scala in 2.0. """ self.is_cached = True self._jdf.cache() return self @since(1.3) - def persist(self, storageLevel=StorageLevel.MEMORY_ONLY): - """Sets the storage level to persist its values across operations - after the first time it is computed. This can only be used to assign - a new storage level if the RDD does not have a storage level set yet. - If no storage level is specified defaults to (C{MEMORY_ONLY}). + def persist(self, storageLevel=StorageLevel.MEMORY_AND_DISK): + """Sets the storage level to persist the contents of the :class:`DataFrame` across + operations after the first time it is computed. This can only be used to assign + a new storage level if the :class:`DataFrame` does not have a storage level set yet. + If no storage level is specified defaults to (C{MEMORY_AND_DISK}). + + .. note:: the default storage level has changed to C{MEMORY_AND_DISK} to match Scala in 2.0. """ self.is_cached = True javaStorageLevel = self._sc._getJavaStorageLevel(storageLevel) self._jdf.persist(javaStorageLevel) return self + @property + @since(2.1) + def storageLevel(self): + """Get the :class:`DataFrame`'s current storage level. + + >>> df.storageLevel + StorageLevel(False, False, False, False, 1) + >>> df.cache().storageLevel + StorageLevel(True, True, False, True, 1) + >>> df2.persist(StorageLevel.DISK_ONLY_2).storageLevel + StorageLevel(True, False, False, False, 2) + """ + java_storage_level = self._jdf.storageLevel() + storage_level = StorageLevel(java_storage_level.useDisk(), + java_storage_level.useMemory(), + java_storage_level.useOffHeap(), + java_storage_level.deserialized(), + java_storage_level.replication()) + return storage_level + @since(1.3) def unpersist(self, blocking=False): """Marks the :class:`DataFrame` as non-persistent, and remove all blocks for it from diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index e59a483075c94..70c9cf5ae2440 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2401,6 +2401,18 @@ class Dataset[T] private[sql]( this } + /** + * Get the Dataset's current storage level, or StorageLevel.NONE if not persisted. + * + * @group basic + * @since 2.1.0 + */ + def storageLevel: StorageLevel = { + sparkSession.sharedState.cacheManager.lookupCachedData(this).map { cachedData => + cachedData.cachedRepresentation.storageLevel + }.getOrElse(StorageLevel.NONE) + } + /** * Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 8d5e9645df894..e0561ee2797a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -19,11 +19,32 @@ package org.apache.spark.sql import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.storage.StorageLevel class DatasetCacheSuite extends QueryTest with SharedSQLContext { import testImplicits._ + test("get storage level") { + val ds1 = Seq("1", "2").toDS().as("a") + val ds2 = Seq(2, 3).toDS().as("b") + + // default storage level + ds1.persist() + ds2.cache() + assert(ds1.storageLevel == StorageLevel.MEMORY_AND_DISK) + assert(ds2.storageLevel == StorageLevel.MEMORY_AND_DISK) + // unpersist + ds1.unpersist() + assert(ds1.storageLevel == StorageLevel.NONE) + // non-default storage level + ds1.persist(StorageLevel.MEMORY_ONLY_2) + assert(ds1.storageLevel == StorageLevel.MEMORY_ONLY_2) + // joined Dataset should not be persisted + val joined = ds1.joinWith(ds2, $"a.value" === $"b.value") + assert(joined.storageLevel == StorageLevel.NONE) + } + test("persist and unpersist") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) val cached = ds.cache() @@ -37,8 +58,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { 2, 3, 4) // Drop the cache. cached.unpersist() - assert(spark.sharedState.cacheManager.lookupCachedData(cached).isEmpty, - "The Dataset should not be cached.") + assert(cached.storageLevel == StorageLevel.NONE, "The Dataset should not be cached.") } test("persist and then rebind right encoder when join 2 datasets") { @@ -55,11 +75,9 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { assertCached(joined, 2) ds1.unpersist() - assert(spark.sharedState.cacheManager.lookupCachedData(ds1).isEmpty, - "The Dataset ds1 should not be cached.") + assert(ds1.storageLevel == StorageLevel.NONE, "The Dataset ds1 should not be cached.") ds2.unpersist() - assert(spark.sharedState.cacheManager.lookupCachedData(ds2).isEmpty, - "The Dataset ds2 should not be cached.") + assert(ds2.storageLevel == StorageLevel.NONE, "The Dataset ds2 should not be cached.") } test("persist and then groupBy columns asKey, map") { @@ -74,10 +92,8 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { assertCached(agged.filter(_._1 == "b")) ds.unpersist() - assert(spark.sharedState.cacheManager.lookupCachedData(ds).isEmpty, - "The Dataset ds should not be cached.") + assert(ds.storageLevel == StorageLevel.NONE, "The Dataset ds should not be cached.") agged.unpersist() - assert(spark.sharedState.cacheManager.lookupCachedData(agged).isEmpty, - "The Dataset agged should not be cached.") + assert(agged.storageLevel == StorageLevel.NONE, "The Dataset agged should not be cached.") } } From f00df40cfefef0f3fc73f16ada1006e4dcfa5a39 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Fri, 14 Oct 2016 15:50:35 -0700 Subject: [PATCH 099/177] [SPARK-11775][PYSPARK][SQL] Allow PySpark to register Java UDF Currently pyspark can only call the builtin java UDF, but can not call custom java UDF. It would be better to allow that. 2 benefits: * Leverage the power of rich third party java library * Improve the performance. Because if we use python UDF, python daemons will be started on worker which will affect the performance. Author: Jeff Zhang Closes #9766 from zjffdu/SPARK-11775. --- python/pyspark/sql/context.py | 28 ++++++- .../sql/catalyst/JavaTypeInference.scala | 2 +- .../apache/spark/sql/UDFRegistration.scala | 75 ++++++++++++++++++- .../apache/spark/sql/JavaStringLength.java | 30 ++++++++ .../org/apache/spark/sql/JavaUDFSuite.java | 21 ++++++ 5 files changed, 152 insertions(+), 4 deletions(-) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 8264dcf8a97d2..de4c335ad2752 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -28,7 +28,7 @@ from pyspark.sql.dataframe import DataFrame from pyspark.sql.readwriter import DataFrameReader from pyspark.sql.streaming import DataStreamReader -from pyspark.sql.types import Row, StringType +from pyspark.sql.types import IntegerType, Row, StringType from pyspark.sql.utils import install_exception_handler __all__ = ["SQLContext", "HiveContext", "UDFRegistration"] @@ -202,6 +202,32 @@ def registerFunction(self, name, f, returnType=StringType()): """ self.sparkSession.catalog.registerFunction(name, f, returnType) + @ignore_unicode_prefix + @since(2.1) + def registerJavaFunction(self, name, javaClassName, returnType=None): + """Register a java UDF so it can be used in SQL statements. + + In addition to a name and the function itself, the return type can be optionally specified. + When the return type is not specified we would infer it via reflection. + :param name: name of the UDF + :param javaClassName: fully qualified name of java class + :param returnType: a :class:`pyspark.sql.types.DataType` object + + >>> sqlContext.registerJavaFunction("javaStringLength", + ... "test.org.apache.spark.sql.JavaStringLength", IntegerType()) + >>> sqlContext.sql("SELECT javaStringLength('test')").collect() + [Row(UDF(test)=4)] + >>> sqlContext.registerJavaFunction("javaStringLength2", + ... "test.org.apache.spark.sql.JavaStringLength") + >>> sqlContext.sql("SELECT javaStringLength2('test')").collect() + [Row(UDF(test)=4)] + + """ + jdt = None + if returnType is not None: + jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) + self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) + # TODO(andrew): delete this once we refactor things to take in SparkSession def _inferSchema(self, rdd, samplingRatio=None): """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index e6f61b00ebd70..04f0cfce883f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -59,7 +59,7 @@ object JavaTypeInference { * @param typeToken Java type * @return (SQL data type, nullable) */ - private def inferDataType(typeToken: TypeToken[_]): (DataType, Boolean) = { + private[sql] def inferDataType(typeToken: TypeToken[_]): (DataType, Boolean) = { typeToken.getRawType match { case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 617a14793697b..0444ad10d34fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -17,19 +17,25 @@ package org.apache.spark.sql +import java.io.IOException +import java.lang.reflect.{ParameterizedType, Type} + import scala.reflect.runtime.universe.TypeTag import scala.util.Try +import com.google.common.reflect.TypeToken + import org.apache.spark.annotation.InterfaceStability import org.apache.spark.internal.Logging import org.apache.spark.sql.api.java._ +import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUDF} import org.apache.spark.sql.execution.aggregate.ScalaUDAF import org.apache.spark.sql.execution.python.UserDefinedPythonFunction import org.apache.spark.sql.expressions.{UserDefinedAggregateFunction, UserDefinedFunction} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.{DataType, DataTypes} +import org.apache.spark.util.Utils /** * Functions for registering user-defined functions. Use [[SQLContext.udf]] to access this. @@ -413,6 +419,71 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * Register a Java UDF class using reflection, for use from pyspark + * + * @param name udf name + * @param className fully qualified class name of udf + * @param returnDataType return type of udf. If it is null, spark would try to infer + * via reflection. + */ + private[sql] def registerJava(name: String, className: String, returnDataType: DataType): Unit = { + + try { + val clazz = Utils.classForName(className) + val udfInterfaces = clazz.getGenericInterfaces + .filter(_.isInstanceOf[ParameterizedType]) + .map(_.asInstanceOf[ParameterizedType]) + .filter(e => e.getRawType.isInstanceOf[Class[_]] && e.getRawType.asInstanceOf[Class[_]].getCanonicalName.startsWith("org.apache.spark.sql.api.java.UDF")) + if (udfInterfaces.length == 0) { + throw new IOException(s"UDF class ${className} doesn't implement any UDF interface") + } else if (udfInterfaces.length > 1) { + throw new IOException(s"It is invalid to implement multiple UDF interfaces, UDF class ${className}") + } else { + try { + val udf = clazz.newInstance() + val udfReturnType = udfInterfaces(0).getActualTypeArguments.last + var returnType = returnDataType + if (returnType == null) { + returnType = JavaTypeInference.inferDataType(TypeToken.of(udfReturnType))._1 + } + + udfInterfaces(0).getActualTypeArguments.length match { + case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType) + case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType) + case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType) + case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType) + case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType) + case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType) + case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType) + case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType) + case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType) + case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType) + case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case n => logError(s"UDF class with ${n} type arguments is not supported ") + } + } catch { + case e @ (_: InstantiationException | _: IllegalArgumentException) => + logError(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") + } + } + } catch { + case e: ClassNotFoundException => logError(s"Can not load class ${className}, please make sure it is on the classpath") + } + + } + /** * Register a user-defined function with 1 arguments. * @since 1.3.0 diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java new file mode 100644 index 0000000000000..b90224f2ae397 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java @@ -0,0 +1,30 @@ +/* + * 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 test.org.apache.spark.sql; + +import org.apache.spark.sql.api.java.UDF1; + +/** + * It is used for register Java UDF from PySpark + */ +public class JavaStringLength implements UDF1 { + @Override + public Integer call(String str) throws Exception { + return new Integer(str.length()); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index 2274912521a56..8bf3278c43880 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -87,4 +87,25 @@ public Integer call(String str1, String str2) { Row result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); Assert.assertEquals(9, result.getInt(0)); } + + public static class StringLengthTest implements UDF2 { + @Override + public Integer call(String str1, String str2) throws Exception { + return new Integer(str1.length() + str2.length()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void udf3Test() { + spark.udf().registerJava("stringLengthTest", StringLengthTest.class.getName(), + DataTypes.IntegerType); + Row result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); + Assert.assertEquals(9, result.getInt(0)); + + // returnType is not provided + spark.udf().registerJava("stringLengthTest2", StringLengthTest.class.getName(), null); + result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); + Assert.assertEquals(9, result.getInt(0)); + } } From 72adfbf94ab6a6ce2a5f3111140274476150f201 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 14 Oct 2016 16:13:42 -0700 Subject: [PATCH 100/177] [SPARK-17900][SQL] Graduate a list of Spark SQL APIs to stable ## What changes were proposed in this pull request? This patch graduates a list of Spark SQL APIs and mark them stable. The following are marked stable: Dataset/DataFrame - functions, since 1.3 - ColumnName, since 1.3 - DataFrameNaFunctions, since 1.3.1 - DataFrameStatFunctions, since 1.4 - UserDefinedFunction, since 1.3 - UserDefinedAggregateFunction, since 1.5 - Window and WindowSpec, since 1.4 Data sources: - DataSourceRegister, since 1.5 - RelationProvider, since 1.3 - SchemaRelationProvider, since 1.3 - CreatableRelationProvider, since 1.3 - BaseRelation, since 1.3 - TableScan, since 1.3 - PrunedScan, since 1.3 - PrunedFilteredScan, since 1.3 - InsertableRelation, since 1.3 The following are kept experimental / evolving: Data sources: - CatalystScan (tied to internal logical plans so it is not stable by definition) Structured streaming: - all classes (introduced new in 2.0 and will likely change) Dataset typed operations (introduced in 1.6 and 2.0 and might change, although probability is low) - all typed methods on Dataset - KeyValueGroupedDataset - o.a.s.sql.expressions.javalang.typed - o.a.s.sql.expressions.scalalang.typed - methods that return typed Dataset in SparkSession We should discuss more whether we want to mark Dataset typed operations stable in 2.1. ## How was this patch tested? N/A - just annotation changes. Author: Reynold Xin Closes #15469 from rxin/SPARK-17900. --- .../scala/org/apache/spark/sql/Column.scala | 6 ++-- .../spark/sql/DataFrameNaFunctions.scala | 6 ++-- .../spark/sql/DataFrameStatFunctions.scala | 6 ++-- .../sql/expressions/UserDefinedFunction.scala | 10 ++++-- .../apache/spark/sql/expressions/Window.scala | 10 ++---- .../spark/sql/expressions/WindowSpec.scala | 6 ++-- .../apache/spark/sql/expressions/udaf.scala | 30 ++++++++++++---- .../org/apache/spark/sql/functions.scala | 4 +-- .../apache/spark/sql/sources/interfaces.scala | 35 +++++-------------- 9 files changed, 51 insertions(+), 62 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 d22bb17934ce7..05e867bf5be96 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.language.implicitConversions -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} @@ -1181,13 +1181,11 @@ class Column(protected[sql] val expr: Expression) extends Logging { /** - * :: Experimental :: * A convenient class used for constructing schema. * * @since 1.3.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable class ColumnName(name: String) extends Column(name) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 65a9c008f9650..0d43f09bc54cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -21,20 +21,18 @@ import java.{lang => jl} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** - * :: Experimental :: * Functionality for working with missing data in [[DataFrame]]s. * * @since 1.3.1 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable final class DataFrameNaFunctions private[sql](df: DataFrame) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index a212bb6205328..b5bbcee37150f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -21,20 +21,18 @@ import java.{lang => jl, util => ju} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.stat._ import org.apache.spark.sql.types._ import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch} /** - * :: Experimental :: * Statistic functions for [[DataFrame]]s. * * @since 1.4.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable final class DataFrameStatFunctions private[sql](df: DataFrame) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 2e0e937e4aff7..28598af781653 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.ScalaUDF import org.apache.spark.sql.Column import org.apache.spark.sql.functions @@ -39,13 +39,17 @@ import org.apache.spark.sql.types.DataType * * @since 1.3.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable case class UserDefinedFunction protected[sql] ( f: AnyRef, dataType: DataType, inputTypes: Option[Seq[DataType]]) { + /** + * Returns an expression that invokes the UDF, using the given arguments. + * + * @since 1.3.0 + */ def apply(exprs: Column*): Column = { Column(ScalaUDF(f, dataType, exprs.map(_.expr), inputTypes.getOrElse(Nil))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index 07ef60183f6fb..0b26d863cac5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions._ /** - * :: Experimental :: * Utility functions for defining window in DataFrames. * * {{{ @@ -36,8 +35,7 @@ import org.apache.spark.sql.catalyst.expressions._ * * @since 1.4.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable object Window { /** @@ -164,7 +162,6 @@ object Window { } /** - * :: Experimental :: * Utility functions for defining window in DataFrames. * * {{{ @@ -177,6 +174,5 @@ object Window { * * @since 1.4.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable class Window private() // So we can see Window in JavaDoc. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index 18778c8d1c294..1e85b6e7881ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -17,20 +17,18 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions._ /** - * :: Experimental :: * A window specification that defines the partitioning, ordering, and frame boundaries. * * Use the static methods in [[Window]] to create a [[WindowSpec]]. * * @since 1.4.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable class WindowSpec private[sql]( partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index ef7c09c72b82d..bc9788d81fe6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -17,20 +17,18 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.{Column, Row} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.ScalaUDAF import org.apache.spark.sql.types._ /** - * :: Experimental :: * The base class for implementing user-defined aggregate functions (UDAF). * * @since 1.5.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable abstract class UserDefinedAggregateFunction extends Serializable { /** @@ -46,6 +44,8 @@ abstract class UserDefinedAggregateFunction extends Serializable { * * The name of a field of this [[StructType]] is only used to identify the corresponding * input argument. Users can choose names to identify the input arguments. + * + * @since 1.5.0 */ def inputSchema: StructType @@ -63,17 +63,23 @@ abstract class UserDefinedAggregateFunction extends Serializable { * * The name of a field of this [[StructType]] is only used to identify the corresponding * buffer value. Users can choose names to identify the input arguments. + * + * @since 1.5.0 */ def bufferSchema: StructType /** * The [[DataType]] of the returned value of this [[UserDefinedAggregateFunction]]. + * + * @since 1.5.0 */ def dataType: DataType /** * Returns true iff this function is deterministic, i.e. given the same input, * always return the same output. + * + * @since 1.5.0 */ def deterministic: Boolean @@ -83,6 +89,8 @@ abstract class UserDefinedAggregateFunction extends Serializable { * The contract should be that applying the merge function on two initial buffers should just * return the initial buffer itself, i.e. * `merge(initialBuffer, initialBuffer)` should equal `initialBuffer`. + * + * @since 1.5.0 */ def initialize(buffer: MutableAggregationBuffer): Unit @@ -90,6 +98,8 @@ abstract class UserDefinedAggregateFunction extends Serializable { * Updates the given aggregation buffer `buffer` with new input data from `input`. * * This is called once per input row. + * + * @since 1.5.0 */ def update(buffer: MutableAggregationBuffer, input: Row): Unit @@ -97,17 +107,23 @@ abstract class UserDefinedAggregateFunction extends Serializable { * Merges two aggregation buffers and stores the updated buffer values back to `buffer1`. * * This is called when we merge two partially aggregated data together. + * + * @since 1.5.0 */ def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit /** * Calculates the final result of this [[UserDefinedAggregateFunction]] based on the given * aggregation buffer. + * + * @since 1.5.0 */ def evaluate(buffer: Row): Any /** * Creates a [[Column]] for this UDAF using given [[Column]]s as input arguments. + * + * @since 1.5.0 */ @scala.annotation.varargs def apply(exprs: Column*): Column = { @@ -122,6 +138,8 @@ abstract class UserDefinedAggregateFunction extends Serializable { /** * Creates a [[Column]] for this UDAF using the distinct values of the given * [[Column]]s as input arguments. + * + * @since 1.5.0 */ @scala.annotation.varargs def distinct(exprs: Column*): Column = { @@ -135,15 +153,13 @@ abstract class UserDefinedAggregateFunction extends Serializable { } /** - * :: Experimental :: * A [[Row]] representing a mutable aggregation buffer. * * This is not meant to be extended outside of Spark. * * @since 1.5.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable abstract class MutableAggregationBuffer extends Row { /** Update the ith value of this buffer. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index de4943152720c..5f1efd22d8204 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -37,7 +37,6 @@ import org.apache.spark.util.Utils /** - * :: Experimental :: * Functions available for DataFrame operations. * * @groupname udf_funcs UDF functions @@ -53,8 +52,7 @@ import org.apache.spark.util.Utils * @groupname Ungrouped Support functions for DataFrames * @since 1.3.0 */ -@Experimental -@InterfaceStability.Evolving +@InterfaceStability.Stable // scalastyle:off object functions { // scalastyle:on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 3172d5ded9504..15a48072525b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType /** - * ::DeveloperApi:: * Data sources should implement this trait so that they can register an alias to their data source. * This allows users to give the data source alias as the format type over the fully qualified * class name. @@ -36,8 +35,7 @@ import org.apache.spark.sql.types.StructType * * @since 1.5.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait DataSourceRegister { /** @@ -54,7 +52,6 @@ trait DataSourceRegister { } /** - * ::DeveloperApi:: * Implemented by objects that produce relations for a specific kind of data source. When * Spark SQL is given a DDL operation with a USING clause specified (to specify the implemented * RelationProvider), this interface is used to pass in the parameters specified by a user. @@ -68,8 +65,7 @@ trait DataSourceRegister { * * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait RelationProvider { /** * Returns a new base relation with the given parameters. @@ -80,7 +76,6 @@ trait RelationProvider { } /** - * ::DeveloperApi:: * Implemented by objects that produce relations for a specific kind of data source * with a given schema. When Spark SQL is given a DDL operation with a USING clause specified ( * to specify the implemented SchemaRelationProvider) and a user defined schema, this interface @@ -100,8 +95,7 @@ trait RelationProvider { * * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait SchemaRelationProvider { /** * Returns a new base relation with the given parameters and user defined schema. @@ -164,8 +158,7 @@ trait StreamSinkProvider { /** * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait CreatableRelationProvider { /** * Save the DataFrame to the destination and return a relation with the given parameters based on @@ -189,7 +182,6 @@ trait CreatableRelationProvider { } /** - * ::DeveloperApi:: * Represents a collection of tuples with a known schema. Classes that extend BaseRelation must * be able to produce the schema of their data in the form of a [[StructType]]. Concrete * implementation should inherit from one of the descendant `Scan` classes, which define various @@ -201,8 +193,7 @@ trait CreatableRelationProvider { * * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable abstract class BaseRelation { def sqlContext: SQLContext def schema: StructType @@ -248,32 +239,27 @@ abstract class BaseRelation { } /** - * ::DeveloperApi:: * A BaseRelation that can produce all of its tuples as an RDD of Row objects. * * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait TableScan { def buildScan(): RDD[Row] } /** - * ::DeveloperApi:: * A BaseRelation that can eliminate unneeded columns before producing an RDD * containing all of its tuples as Row objects. * * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait PrunedScan { def buildScan(requiredColumns: Array[String]): RDD[Row] } /** - * ::DeveloperApi:: * A BaseRelation that can eliminate unneeded columns and filter using selected * predicates before producing an RDD containing all matching tuples as Row objects. * @@ -286,14 +272,12 @@ trait PrunedScan { * * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait PrunedFilteredScan { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } /** - * ::DeveloperApi:: * A BaseRelation that can be used to insert data into it through the insert method. * If overwrite in insert method is true, the old data in the relation should be overwritten with * the new data. If overwrite in insert method is false, the new data should be appended. @@ -310,8 +294,7 @@ trait PrunedFilteredScan { * * @since 1.3.0 */ -@DeveloperApi -@InterfaceStability.Evolving +@InterfaceStability.Stable trait InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit } From 2d96d35dc0fed6df249606d9ce9272c0f0109fa2 Mon Sep 17 00:00:00 2001 From: Srinath Shankar Date: Fri, 14 Oct 2016 18:24:47 -0700 Subject: [PATCH 101/177] [SPARK-17946][PYSPARK] Python crossJoin API similar to Scala ## What changes were proposed in this pull request? Add a crossJoin function to the DataFrame API similar to that in Scala. Joins with no condition (cartesian products) must be specified with the crossJoin API ## How was this patch tested? Added python tests to ensure that an AnalysisException if a cartesian product is specified without crossJoin(), and that cartesian products can execute if specified via crossJoin() (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. Author: Srinath Shankar Closes #15493 from srinathshankar/crosspython. --- python/pyspark/sql/dataframe.py | 26 +++++++++++++++---- python/pyspark/sql/tests.py | 15 ++++++++++- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- 3 files changed, 36 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 7606ac08bae67..29710acf54c4f 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -650,6 +650,25 @@ def alias(self, alias): assert isinstance(alias, basestring), "alias should be a string" return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx) + @ignore_unicode_prefix + @since(2.1) + def crossJoin(self, other): + """Returns the cartesian product with another :class:`DataFrame`. + + :param other: Right side of the cartesian product. + + >>> df.select("age", "name").collect() + [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + >>> df2.select("name", "height").collect() + [Row(name=u'Tom', height=80), Row(name=u'Bob', height=85)] + >>> df.crossJoin(df2.select("height")).select("age", "name", "height").collect() + [Row(age=2, name=u'Alice', height=80), Row(age=2, name=u'Alice', height=85), + Row(age=5, name=u'Bob', height=80), Row(age=5, name=u'Bob', height=85)] + """ + + jdf = self._jdf.crossJoin(other._jdf) + return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix @since(1.3) def join(self, other, on=None, how=None): @@ -690,14 +709,11 @@ def join(self, other, on=None, how=None): on = self._jseq(on) else: assert isinstance(on[0], Column), "on should be Column or list of Column" - if len(on) > 1: - on = reduce(lambda x, y: x.__and__(y), on) - else: - on = on[0] + on = reduce(lambda x, y: x.__and__(y), on) on = on._jc if on is None and how is None: - jdf = self._jdf.crossJoin(other._jdf) + jdf = self._jdf.join(other._jdf) else: if how is None: how = "inner" diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 51d5e7ab0568e..3d46b852c52e1 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1466,7 +1466,7 @@ def test_functions_broadcast(self): self.assertEqual(1, plan1.toString().count("BroadcastHashJoin")) # no join key -- should not be a broadcast join - plan2 = df1.join(broadcast(df2))._jdf.queryExecution().executedPlan() + plan2 = df1.crossJoin(broadcast(df2))._jdf.queryExecution().executedPlan() self.assertEqual(0, plan2.toString().count("BroadcastHashJoin")) # planner should not crash without a join @@ -1514,6 +1514,19 @@ def test_invalid_join_method(self): df2 = self.spark.createDataFrame([("Alice", 80), ("Bob", 90)], ["name", "height"]) self.assertRaises(IllegalArgumentException, lambda: df1.join(df2, how="invalid-join-type")) + # Cartesian products require cross join syntax + def test_require_cross(self): + from pyspark.sql.functions import broadcast + + df1 = self.spark.createDataFrame([(1, "1")], ("key", "value")) + df2 = self.spark.createDataFrame([(1, "1")], ("key", "value")) + + # joins without conditions require cross join syntax + self.assertRaises(AnalysisException, lambda: df1.join(df2).collect()) + + # works with crossJoin + self.assertEqual(1, df1.crossJoin(df2).count()) + def test_conf(self): spark = self.spark spark.conf.set("bogo", "sipeo") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 70c9cf5ae2440..7ae3275245c5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -774,7 +774,7 @@ class Dataset[T] private[sql]( * @param right Right side of the join operation. * * @group untypedrel - * @since 2.0.0 + * @since 2.1.0 */ def crossJoin(right: Dataset[_]): DataFrame = withPlan { Join(logicalPlan, right.logicalPlan, joinType = Cross, None) From 6ce1b675ee9fc9a6034439c3ca00441f9f172f84 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Fri, 14 Oct 2016 18:26:18 -0700 Subject: [PATCH 102/177] [SPARK-16980][SQL] Load only catalog table partition metadata required to answer a query (This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.) ## What changes were proposed in this pull request? In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference. If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild. In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space. This PR proposes an alternative approach. Basically, it makes four changes: 1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates. 1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates. 1. It removes partition loading and caching from `HiveMetastoreCatalog`. 1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog. The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters. As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted. ## Open Issues 1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR. 1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue. 1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`. 1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly. ## How was this patch tested? The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded. Author: Michael Allman Author: Eric Liang Author: Eric Liang Closes #14690 from mallman/spark-16980-lazy_partition_fetching. --- .../spark/metrics/source/StaticSources.scala | 34 ++- .../catalyst/catalog/ExternalCatalog.scala | 5 +- .../catalyst/catalog/InMemoryCatalog.scala | 4 +- .../sql/catalyst/catalog/interface.scala | 15 +- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../spark/sql/execution/CacheManager.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 28 ++- .../spark/sql/execution/SparkOptimizer.scala | 2 + .../command/createDataSourceTables.scala | 2 +- .../execution/datasources/DataSource.scala | 4 +- .../datasources/DataSourceStrategy.scala | 8 +- .../execution/datasources/FileFormat.scala | 46 +++- .../datasources/HadoopFsRelation.scala | 16 +- .../datasources/ListingFileCatalog.scala | 197 +-------------- .../datasources/LogicalRelation.scala | 2 +- .../PartitioningAwareFileCatalog.scala | 24 +- .../PruneFileSourcePartitions.scala | 72 ++++++ .../datasources/SessionFileCatalog.scala | 225 ++++++++++++++++++ .../datasources/TableFileCatalog.scala | 113 +++++++++ .../parquet/ParquetReadSupport.scala | 6 +- .../streaming/MetadataLogFileCatalog.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 9 + .../datasources/FileCatalogSuite.scala | 5 +- .../datasources/FileSourceStrategySuite.scala | 2 +- ...te.scala => SessionFileCatalogSuite.scala} | 16 +- .../ParquetPartitionDiscoverySuite.scala | 6 +- .../parquet/ParquetSchemaSuite.scala | 28 +++ .../spark/sql/hive/HiveExternalCatalog.scala | 37 ++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 126 ++++------ .../spark/sql/hive/client/HiveClient.scala | 15 +- .../sql/hive/client/HiveClientImpl.scala | 19 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 12 +- .../spark/sql/hive/HiveDataFrameSuite.scala | 109 ++++++++- .../sql/hive/HiveMetadataCacheSuite.scala | 41 ++++ .../spark/sql/hive/client/VersionsSuite.scala | 4 +- .../spark/sql/hive/orc/OrcQuerySuite.scala | 22 ++ .../apache/spark/sql/hive/parquetSuites.scala | 20 +- 37 files changed, 914 insertions(+), 368 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala rename sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/{ListingFileCatalogSuite.scala => SessionFileCatalogSuite.scala} (66%) diff --git a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala index 6bba259acc391..cf92a10deabd5 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala @@ -26,7 +26,7 @@ private[spark] object StaticSources { * The set of all static sources. These sources may be reported to from any class, including * static classes, without requiring reference to a SparkEnv. */ - val allSources = Seq(CodegenMetrics) + val allSources = Seq(CodegenMetrics, HiveCatalogMetrics) } /** @@ -60,3 +60,35 @@ object CodegenMetrics extends Source { val METRIC_GENERATED_METHOD_BYTECODE_SIZE = metricRegistry.histogram(MetricRegistry.name("generatedMethodSize")) } + +/** + * :: Experimental :: + * Metrics for access to the hive external catalog. + */ +@Experimental +object HiveCatalogMetrics extends Source { + override val sourceName: String = "HiveExternalCatalog" + override val metricRegistry: MetricRegistry = new MetricRegistry() + + /** + * Tracks the total number of partition metadata entries fetched via the client api. + */ + val METRIC_PARTITIONS_FETCHED = metricRegistry.counter(MetricRegistry.name("partitionsFetched")) + + /** + * Tracks the total number of files discovered off of the filesystem by ListingFileCatalog. + */ + val METRIC_FILES_DISCOVERED = metricRegistry.counter(MetricRegistry.name("filesDiscovered")) + + /** + * Resets the values of all metrics to zero. This is useful in tests. + */ + def reset(): Unit = { + METRIC_PARTITIONS_FETCHED.dec(METRIC_PARTITIONS_FETCHED.getCount()) + METRIC_FILES_DISCOVERED.dec(METRIC_FILES_DISCOVERED.getCount()) + } + + // clients can use these to avoid classloader issues with the codahale classes + def incrementFetchedPartitions(n: Int): Unit = METRIC_PARTITIONS_FETCHED.inc(n) + def incrementFilesDiscovered(n: Int): Unit = METRIC_FILES_DISCOVERED.inc(n) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 348d3d0be2152..a5e02523d2889 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -198,11 +198,12 @@ abstract class ExternalCatalog { partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] /** - * List the metadata of selected partitions according to the given partition predicates. + * List the metadata of partitions that belong to the specified table, assuming it exists, that + * satisfy the given partition-pruning predicate expressions. * * @param db database name * @param table table name - * @param predicates partition predicated + * @param predicates partition-pruning predicates */ def listPartitionsByFilter( db: String, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 49280f82e20be..f95c9f8cfa2d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -482,7 +482,9 @@ class InMemoryCatalog( db: String, table: String, predicates: Seq[Expression]): Seq[CatalogTablePartition] = { - throw new UnsupportedOperationException("listPartitionsByFilter is not implemented.") + // TODO: Provide an implementation + throw new UnsupportedOperationException( + "listPartitionsByFilter is not implemented for InMemoryCatalog") } // -------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 51326ca25e9cc..1a57a7707caa1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.catalyst.catalog import java.util.Date import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.quoteIdentifier -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StructField, StructType} /** @@ -97,6 +97,15 @@ case class CatalogTablePartition( output.filter(_.nonEmpty).mkString("CatalogPartition(\n\t", "\n\t", ")") } + + /** + * Given the partition schema, returns a row with that schema holding the partition values. + */ + def toRow(partitionSchema: StructType): InternalRow = { + InternalRow.fromSeq(partitionSchema.map { case StructField(name, dataType, _, _) => + Cast(Literal(spec(name)), dataType).eval() + }) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 7ae3275245c5d..7dccbbd3f0a5b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.streaming.{DataStreamWriter, StreamingQuery} @@ -2614,7 +2614,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ def inputFiles: Array[String] = { - val files: Seq[String] = logicalPlan.collect { + val files: Seq[String] = queryExecution.optimizedPlan.collect { case LogicalRelation(fsBasedRelation: FileRelation, _, _) => fsBasedRelation.inputFiles case fr: FileRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 83b7c779ab818..92fd366e101fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -185,7 +185,7 @@ class CacheManager extends Logging { plan match { case lr: LogicalRelation => lr.relation match { case hr: HadoopFsRelation => - val invalidate = hr.location.paths + val invalidate = hr.location.rootPaths .map(_.makeQualified(fs.getUri, fs.getWorkingDirectory)) .contains(qualifiedPath) if (invalidate) hr.location.refresh() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 6cdba406937de..623d2be55dcec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -225,13 +225,27 @@ case class FileSourceScanExec( } // These metadata values make scan plans uniquely identifiable for equality checking. - override val metadata: Map[String, String] = Map( - "Format" -> relation.fileFormat.toString, - "ReadSchema" -> outputSchema.catalogString, - "Batched" -> supportsBatch.toString, - "PartitionFilters" -> partitionFilters.mkString("[", ", ", "]"), - "PushedFilters" -> dataFilters.mkString("[", ", ", "]"), - "InputPaths" -> relation.location.paths.mkString(", ")) + override val metadata: Map[String, String] = { + def seqToString(seq: Seq[Any]) = seq.mkString("[", ", ", "]") + val location = relation.location + val locationDesc = + location.getClass.getSimpleName + seqToString(location.rootPaths) + val metadata = + Map( + "Format" -> relation.fileFormat.toString, + "ReadSchema" -> outputSchema.catalogString, + "Batched" -> supportsBatch.toString, + "PartitionFilters" -> seqToString(partitionFilters), + "PushedFilters" -> seqToString(dataFilters), + "Location" -> locationDesc) + val withOptPartitionCount = + relation.partitionSchemaOption.map { _ => + metadata + ("PartitionCount" -> selectedPartitions.size.toString) + } getOrElse { + metadata + } + withOptPartitionCount + } private lazy val inputRDD: RDD[InternalRow] = { val readFile: (PartitionedFile) => Iterator[InternalRow] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 8b762b5d6c5f2..981728331d361 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate import org.apache.spark.sql.internal.SQLConf @@ -32,5 +33,6 @@ class SparkOptimizer( override def batches: Seq[Batch] = super.batches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog, conf)) :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ + Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index a04a13e698c43..a8c75a7f29cef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -67,7 +67,7 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo dataSource match { case fs: HadoopFsRelation => - if (table.tableType == CatalogTableType.EXTERNAL && fs.location.paths.isEmpty) { + if (table.tableType == CatalogTableType.EXTERNAL && fs.location.rootPaths.isEmpty) { throw new AnalysisException( "Cannot create a file-based external data source table without path") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index e75e7d2770b4e..92b1fff7d8127 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -471,9 +471,7 @@ case class DataSource( val existingPartitionColumns = Try { resolveRelation() .asInstanceOf[HadoopFsRelation] - .location - .partitionSpec() - .partitionColumns + .partitionSchema .fieldNames .toSeq }.getOrElse(Seq.empty[String]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 6f9ed50a02b09..7d0abe86a44df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -163,14 +163,14 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { if query.resolved && t.schema.asNullable == query.schema.asNullable => // Sanity checks - if (t.location.paths.size != 1) { + if (t.location.rootPaths.size != 1) { throw new AnalysisException( "Can only write data to relations with a single path.") } - val outputPath = t.location.paths.head + val outputPath = t.location.rootPaths.head val inputPaths = query.collect { - case LogicalRelation(r: HadoopFsRelation, _, _) => r.location.paths + case LogicalRelation(r: HadoopFsRelation, _, _) => r.location.rootPaths }.flatten val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append @@ -184,7 +184,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { query.resolve(t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver), t.bucketSpec, t.fileFormat, - () => t.refresh(), + () => t.location.refresh(), t.options, query, mode) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index bde2d2b89d56f..e7239ef91b326 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType + /** * Used to read and write data stored in files to/from the [[InternalRow]] format. */ @@ -182,16 +183,17 @@ abstract class TextBasedFileFormat extends FileFormat { case class Partition(values: InternalRow, files: Seq[FileStatus]) /** - * An interface for objects capable of enumerating the files that comprise a relation as well - * as the partitioning characteristics of those files. + * An interface for objects capable of enumerating the root paths of a relation as well as the + * partitions of a relation subject to some pruning expressions. */ -trait FileCatalog { - - /** Returns the list of input paths from which the catalog will get files. */ - def paths: Seq[Path] +trait BasicFileCatalog { - /** Returns the specification of the partitions inferred from the data. */ - def partitionSpec(): PartitionSpec + /** + * Returns the list of root input paths from which the catalog will get files. There may be a + * single root path from which partitions are discovered, or individual partitions may be + * specified by each path. + */ + def rootPaths: Seq[Path] /** * Returns all valid files grouped into partitions when the data is partitioned. If the data is @@ -204,9 +206,33 @@ trait FileCatalog { */ def listFiles(filters: Seq[Expression]): Seq[Partition] + /** Returns the list of files that will be read when scanning this relation. */ + def inputFiles: Array[String] + + /** Refresh any cached file listings */ + def refresh(): Unit + + /** Sum of table file sizes, in bytes */ + def sizeInBytes: Long +} + +/** + * A [[BasicFileCatalog]] which can enumerate all of the files comprising a relation and, from + * those, infer the relation's partition specification. + */ +// TODO: Consider a more descriptive, appropriate name which suggests this is a file catalog for +// which it is safe to list all of its files? +trait FileCatalog extends BasicFileCatalog { + + /** Returns the specification of the partitions inferred from the data. */ + def partitionSpec(): PartitionSpec + /** Returns all the valid files. */ def allFiles(): Seq[FileStatus] - /** Refresh the file listing */ - def refresh(): Unit + /** Returns the list of files that will be read when scanning this relation. */ + override def inputFiles: Array[String] = + allFiles().map(_.getPath.toUri.toString).toArray + + override def sizeInBytes: Long = allFiles().map(_.getLen).sum } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index c7ebe0b76a150..db889edf032d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -28,8 +28,8 @@ import org.apache.spark.sql.types.StructType * Acts as a container for all of the metadata required to read from a datasource. All discovery, * resolution and merging logic for schemas and partitions has been removed. * - * @param location A [[FileCatalog]] that can enumerate the locations of all the files that comprise - * this relation. + * @param location A [[BasicFileCatalog]] that can enumerate the locations of all the files that + * comprise this relation. * @param partitionSchema The schema of the columns (if any) that are used to partition the relation * @param dataSchema The schema of any remaining columns. Note that if any partition columns are * present in the actual data files as well, they are preserved. @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType * @param options Configuration used when reading / writing data. */ case class HadoopFsRelation( - location: FileCatalog, + location: BasicFileCatalog, partitionSchema: StructType, dataSchema: StructType, bucketSpec: Option[BucketSpec], @@ -58,10 +58,6 @@ case class HadoopFsRelation( def partitionSchemaOption: Option[StructType] = if (partitionSchema.isEmpty) None else Some(partitionSchema) - def partitionSpec: PartitionSpec = location.partitionSpec() - - def refresh(): Unit = location.refresh() - override def toString: String = { fileFormat match { case source: DataSourceRegister => source.shortName() @@ -69,9 +65,7 @@ case class HadoopFsRelation( } } - /** Returns the list of files that will be read when scanning this relation. */ - override def inputFiles: Array[String] = - location.allFiles().map(_.getPath.toUri.toString).toArray + override def sizeInBytes: Long = location.sizeInBytes - override def sizeInBytes: Long = location.allFiles().map(_.getLen).sum + override def inputFiles: Array[String] = location.inputFiles } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index a68ae523e0faa..6d10501b7265d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -17,32 +17,26 @@ package org.apache.spark.sql.execution.datasources -import java.io.FileNotFoundException - import scala.collection.mutable -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.mapred.{FileInputFormat, JobConf} -import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration /** * A [[FileCatalog]] that generates the list of files to process by recursively listing all the * files present in `paths`. * + * @param rootPaths the list of root table paths to scan * @param parameters as set of options to control discovery - * @param paths a list of paths to scan * @param partitionSchema an optional partition schema that will be use to provide types for the * discovered partitions */ class ListingFileCatalog( sparkSession: SparkSession, - override val paths: Seq[Path], + override val rootPaths: Seq[Path], parameters: Map[String, String], partitionSchema: Option[StructType]) extends PartitioningAwareFileCatalog(sparkSession, parameters, partitionSchema) { @@ -70,198 +64,17 @@ class ListingFileCatalog( } override def refresh(): Unit = { - val files = listLeafFiles(paths) + val files = listLeafFiles(rootPaths) cachedLeafFiles = new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f) cachedLeafDirToChildrenFiles = files.toArray.groupBy(_.getPath.getParent) cachedPartitionSpec = null } - /** - * List leaf files of given paths. This method will submit a Spark job to do parallel - * listing whenever there is a path having more files than the parallel partition discovery - * discovery threshold. - * - * This is publicly visible for testing. - */ - def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - val files = - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) - } else { - ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf) - } - - mutable.LinkedHashSet(files: _*) - } - override def equals(other: Any): Boolean = other match { - case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet + case hdfs: ListingFileCatalog => rootPaths.toSet == hdfs.rootPaths.toSet case _ => false } - override def hashCode(): Int = paths.toSet.hashCode() -} - - -object ListingFileCatalog extends Logging { - - /** A serializable variant of HDFS's BlockLocation. */ - private case class SerializableBlockLocation( - names: Array[String], - hosts: Array[String], - offset: Long, - length: Long) - - /** A serializable variant of HDFS's FileStatus. */ - private case class SerializableFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long, - blockLocations: Array[SerializableBlockLocation]) - - /** - * List a collection of path recursively. - */ - private def listLeafFilesInSerial( - paths: Seq[Path], - hadoopConf: Configuration): Seq[FileStatus] = { - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val filter = FileInputFormat.getInputPathFilter(jobConf) - - paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - listLeafFiles0(fs, path, filter) - } - } - - /** - * List a collection of path recursively in parallel (using Spark executors). - * Each task launched will use [[listLeafFilesInSerial]] to list. - */ - private def listLeafFilesInParallel( - paths: Seq[Path], - hadoopConf: Configuration, - sparkSession: SparkSession): Seq[FileStatus] = { - assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) - logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") - - val sparkContext = sparkSession.sparkContext - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val serializedPaths = paths.map(_.toString) - - // Set the number of parallelism to prevent following file listing from generating many tasks - // in case of large #defaultParallelism. - val numParallelism = Math.min(paths.size, 10000) - - val statuses = sparkContext - .parallelize(serializedPaths, numParallelism) - .mapPartitions { paths => - val hadoopConf = serializableConfiguration.value - listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator - }.map { status => - // Turn FileStatus into SerializableFileStatus so we can send it back to the driver - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - SerializableBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[SerializableBlockLocation] - } - - SerializableFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) - }.collect() - - // Turn SerializableFileStatus back to Status - statuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) - } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), - blockLocations) - } - } - - /** - * List a single path, provided as a FileStatus, in serial. - */ - private def listLeafFiles0( - fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { - logTrace(s"Listing $path") - val name = path.getName.toLowerCase - if (shouldFilterOut(name)) { - Seq.empty[FileStatus] - } else { - // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist - // Note that statuses only include FileStatus for the files and dirs directly under path, - // and does not include anything else recursively. - val statuses = try fs.listStatus(path) catch { - case _: FileNotFoundException => - logWarning(s"The directory $path was not found. Was it deleted very recently?") - Array.empty[FileStatus] - } - - val allLeafStatuses = { - val (dirs, files) = statuses.partition(_.isDirectory) - val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) - if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats - } - - allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { - case f: LocatedFileStatus => - f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of - // paths exceeds threshold. - case f => - // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), - // which is very slow on some file system (RawLocalFileSystem, which is launch a - // subprocess and parse the stdout). - val locations = fs.getFileBlockLocations(f, 0, f.getLen) - val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, - f.getModificationTime, 0, null, null, null, null, f.getPath, locations) - if (f.isSymlink) { - lfs.setSymlink(f.getSymlink) - } - lfs - } - } - } - - /** Checks if we should filter out this path name. */ - def shouldFilterOut(pathName: String): Boolean = { - // We filter everything that starts with _ and ., except _common_metadata and _metadata - // because Parquet needs to find those metadata files from leaf files returned by this method. - // We should refactor this logic to not mix metadata files with data files. - ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && - !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") - } + override def hashCode(): Int = rootPaths.toSet.hashCode() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index d9562fd32e87d..7c28d48f26416 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -94,7 +94,7 @@ case class LogicalRelation( } override def refresh(): Unit = relation match { - case fs: HadoopFsRelation => fs.refresh() + case fs: HadoopFsRelation => fs.location.refresh() case _ => // Do nothing. } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index 702ba97222e34..b2508115c282f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -21,7 +21,6 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ @@ -40,9 +39,10 @@ abstract class PartitioningAwareFileCatalog( sparkSession: SparkSession, parameters: Map[String, String], partitionSchema: Option[StructType]) - extends FileCatalog with Logging { + extends SessionFileCatalog(sparkSession) with FileCatalog { + import PartitioningAwareFileCatalog.BASE_PATH_PARAM - protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) + override protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] @@ -72,8 +72,8 @@ abstract class PartitioningAwareFileCatalog( override def allFiles(): Seq[FileStatus] = { if (partitionSpec().partitionColumns.isEmpty) { - // For each of the input paths, get the list of files inside them - paths.flatMap { path => + // For each of the root input paths, get the list of files inside them + rootPaths.flatMap { path => // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). val fs = path.getFileSystem(hadoopConf) val qualifiedPathPre = fs.makeQualified(path) @@ -105,8 +105,6 @@ abstract class PartitioningAwareFileCatalog( protected def inferPartitioning(): PartitionSpec = { // We use leaf dirs containing data files to discover the schema. val leafDirs = leafDirToChildrenFiles.filter { case (_, files) => - // SPARK-15895: Metadata files (e.g. Parquet summary files) and temporary files should not be - // counted as data files, so that they shouldn't participate partition discovery. files.exists(f => isDataPath(f.getPath)) }.keys.toSeq partitionSchema match { @@ -194,24 +192,30 @@ abstract class PartitioningAwareFileCatalog( * and the returned DataFrame will have the column of `something`. */ private def basePaths: Set[Path] = { - parameters.get("basePath").map(new Path(_)) match { + parameters.get(BASE_PATH_PARAM).map(new Path(_)) match { case Some(userDefinedBasePath) => val fs = userDefinedBasePath.getFileSystem(hadoopConf) if (!fs.isDirectory(userDefinedBasePath)) { - throw new IllegalArgumentException("Option 'basePath' must be a directory") + throw new IllegalArgumentException(s"Option '$BASE_PATH_PARAM' must be a directory") } Set(fs.makeQualified(userDefinedBasePath)) case None => - paths.map { path => + rootPaths.map { path => // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). val qualifiedPath = path.getFileSystem(hadoopConf).makeQualified(path) if (leafFiles.contains(qualifiedPath)) qualifiedPath.getParent else qualifiedPath }.toSet } } + // SPARK-15895: Metadata files (e.g. Parquet summary files) and temporary files should not be + // counted as data files, so that they shouldn't participate partition discovery. private def isDataPath(path: Path): Boolean = { val name = path.getName !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } } + +object PartitioningAwareFileCatalog { + val BASE_PATH_PARAM = "basePath" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala new file mode 100644 index 0000000000000..29121a47d92d1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule + +private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case op @ PhysicalOperation(projects, filters, + logicalRelation @ + LogicalRelation(fsRelation @ + HadoopFsRelation( + tableFileCatalog: TableFileCatalog, + partitionSchema, + _, + _, + _, + _), + _, + _)) + if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => + // The attribute name of predicate could be different than the one in schema in case of + // case insensitive, we should change them to match the one in schema, so we donot need to + // worry about case sensitivity anymore. + val normalizedFilters = filters.map { e => + e transform { + case a: AttributeReference => + a.withName(logicalRelation.output.find(_.semanticEquals(a)).get.name) + } + } + + val sparkSession = fsRelation.sparkSession + val partitionColumns = + logicalRelation.resolve( + partitionSchema, sparkSession.sessionState.analyzer.resolver) + val partitionSet = AttributeSet(partitionColumns) + val partitionKeyFilters = + ExpressionSet(normalizedFilters.filter(_.references.subsetOf(partitionSet))) + + if (partitionKeyFilters.nonEmpty) { + val prunedFileCatalog = tableFileCatalog.filterPartitions(partitionKeyFilters.toSeq) + val prunedFsRelation = + fsRelation.copy(location = prunedFileCatalog)(sparkSession) + val prunedLogicalRelation = logicalRelation.copy(relation = prunedFsRelation) + + // Keep partition-pruning predicates so that they are visible in physical planning + val filterExpression = filters.reduceLeft(And) + val filter = Filter(filterExpression, prunedLogicalRelation) + Project(projects, filter) + } else { + op + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala new file mode 100644 index 0000000000000..4807a92c2e6b8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.SerializableConfiguration + + +/** + * A base class for [[BasicFileCatalog]]s that need a [[SparkSession]] and the ability to find leaf + * files in a list of HDFS paths. + * + * @param sparkSession a [[SparkSession]] + * @param ignoreFileNotFound (see [[ListingFileCatalog]]) + */ +abstract class SessionFileCatalog(sparkSession: SparkSession) + extends BasicFileCatalog with Logging { + protected val hadoopConf: Configuration + + /** + * List leaf files of given paths. This method will submit a Spark job to do parallel + * listing whenever there is a path having more files than the parallel partition discovery + * discovery threshold. + * + * This is publicly visible for testing. + */ + def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + val files = + if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + SessionFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) + } else { + SessionFileCatalog.listLeafFilesInSerial(paths, hadoopConf) + } + + HiveCatalogMetrics.incrementFilesDiscovered(files.size) + mutable.LinkedHashSet(files: _*) + } +} + +object SessionFileCatalog extends Logging { + + /** A serializable variant of HDFS's BlockLocation. */ + private case class SerializableBlockLocation( + names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + + /** A serializable variant of HDFS's FileStatus. */ + private case class SerializableFileStatus( + path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long, + blockLocations: Array[SerializableBlockLocation]) + + /** + * List a collection of path recursively. + */ + private def listLeafFilesInSerial( + paths: Seq[Path], + hadoopConf: Configuration): Seq[FileStatus] = { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val filter = FileInputFormat.getInputPathFilter(jobConf) + + paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + listLeafFiles0(fs, path, filter) + } + } + + /** + * List a collection of path recursively in parallel (using Spark executors). + * Each task launched will use [[listLeafFilesInSerial]] to list. + */ + private def listLeafFilesInParallel( + paths: Seq[Path], + hadoopConf: Configuration, + sparkSession: SparkSession): Seq[FileStatus] = { + assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) + logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") + + val sparkContext = sparkSession.sparkContext + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, 10000) + + val statuses = sparkContext + .parallelize(serializedPaths, numParallelism) + .mapPartitions { paths => + val hadoopConf = serializableConfiguration.value + listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator + }.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + }.collect() + + // Turn SerializableFileStatus back to Status + statuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), + blockLocations) + } + } + + /** + * List a single path, provided as a FileStatus, in serial. + */ + private def listLeafFiles0( + fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { + logTrace(s"Listing $path") + val name = path.getName.toLowerCase + if (shouldFilterOut(name)) { + Seq.empty[FileStatus] + } else { + // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses = try fs.listStatus(path) catch { + case _: FileNotFoundException => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + val allLeafStatuses = { + val (dirs, files) = statuses.partition(_.isDirectory) + val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) + if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats + } + + allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { + case f: LocatedFileStatus => + f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. + case f => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + val locations = fs.getFileBlockLocations(f, 0, f.getLen) + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs + } + } + } + + /** Checks if we should filter out this path name. */ + def shouldFilterOut(pathName: String): Boolean = { + // We filter everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && + !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala new file mode 100644 index 0000000000000..a5c41b244589b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.StructType + + +/** + * A [[BasicFileCatalog]] for a metastore catalog table. + * + * @param sparkSession a [[SparkSession]] + * @param db the table's database name + * @param table the table's (unqualified) name + * @param partitionSchema the schema of a partitioned table's partition columns + * @param sizeInBytes the table's data size in bytes + */ +class TableFileCatalog( + sparkSession: SparkSession, + db: String, + table: String, + partitionSchema: Option[StructType], + override val sizeInBytes: Long) + extends SessionFileCatalog(sparkSession) { + + override protected val hadoopConf = sparkSession.sessionState.newHadoopConf + + private val externalCatalog = sparkSession.sharedState.externalCatalog + + private val catalogTable = externalCatalog.getTable(db, table) + + private val baseLocation = catalogTable.storage.locationUri + + override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq + + override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + filterPartitions(filters).listFiles(Nil) + } + + override def refresh(): Unit = {} + + /** + * Returns a [[ListingFileCatalog]] for this table restricted to the subset of partitions + * specified by the given partition-pruning filters. + * + * @param filters partition-pruning filters + */ + def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = { + if (filters.isEmpty) { + cachedAllPartitions + } else { + filterPartitions0(filters) + } + } + + private def filterPartitions0(filters: Seq[Expression]): ListingFileCatalog = { + val parameters = baseLocation + .map(loc => Map(PartitioningAwareFileCatalog.BASE_PATH_PARAM -> loc)) + .getOrElse(Map.empty) + partitionSchema match { + case Some(schema) => + val selectedPartitions = externalCatalog.listPartitionsByFilter(db, table, filters) + val partitions = selectedPartitions.map { p => + PartitionDirectory(p.toRow(schema), p.storage.locationUri.get) + } + val partitionSpec = PartitionSpec(schema, partitions) + new PrunedTableFileCatalog( + sparkSession, new Path(baseLocation.get), partitionSpec) + case None => + new ListingFileCatalog(sparkSession, rootPaths, parameters, None) + } + } + + // Not used in the hot path of queries when metastore partition pruning is enabled + lazy val cachedAllPartitions: ListingFileCatalog = filterPartitions0(Nil) + + override def inputFiles: Array[String] = cachedAllPartitions.inputFiles +} + +/** + * An override of the standard HDFS listing based catalog, that overrides the partition spec with + * the information from the metastore. + * + * @param tableBasePath The default base path of the Hive metastore table + * @param partitionSpec The partition specifications from Hive metastore + */ +private class PrunedTableFileCatalog( + sparkSession: SparkSession, + tableBasePath: Path, + override val partitionSpec: PartitionSpec) + extends ListingFileCatalog( + sparkSession, + partitionSpec.partitions.map(_.path), + Map.empty, + Some(partitionSpec.partitionColumns)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index f1a35dd8a6200..4dea8cf29ec58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -269,11 +269,15 @@ private[parquet] object ParquetReadSupport { */ private def clipParquetGroupFields( parquetRecord: GroupType, structType: StructType): Seq[Type] = { - val parquetFieldMap = parquetRecord.getFields.asScala.map(f => f.getName -> f).toMap + val parquetFieldMap = parquetRecord.getFields.asScala + .map(f => f.getName -> f).toMap + val caseInsensitiveParquetFieldMap = parquetRecord.getFields.asScala + .map(f => f.getName.toLowerCase -> f).toMap val toParquet = new ParquetSchemaConverter(writeLegacyParquetFormat = false) structType.map { f => parquetFieldMap .get(f.name) + .orElse(caseInsensitiveParquetFieldMap.get(f.name.toLowerCase)) .map(clipParquetType(_, f.dataType)) .getOrElse(toParquet.convertField(f)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala index a32c4671e3475..82b67cb1ca6ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala @@ -47,7 +47,7 @@ class MetadataLogFileCatalog(sparkSession: SparkSession, path: Path) allFilesFromLog.toArray.groupBy(_.getPath.getParent) } - override def paths: Seq[Path] = path :: Nil + override def rootPaths: Seq[Path] = path :: Nil override def refresh(): Unit = { } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c8447651dd672..e73d0187b584b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -269,6 +269,13 @@ object SQLConf { .booleanConf .createWithDefault(false) + val HIVE_FILESOURCE_PARTITION_PRUNING = + SQLConfigBuilder("spark.sql.hive.filesourcePartitionPruning") + .doc("When true, enable metastore partition pruning for file source tables as well. " + + "This is currently implemented for converted Hive tables only.") + .booleanConf + .createWithDefault(true) + val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + @@ -676,6 +683,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def metastorePartitionPruning: Boolean = getConf(HIVE_METASTORE_PARTITION_PRUNING) + def filesourcePartitionPruning: Boolean = getConf(HIVE_FILESOURCE_PARTITION_PRUNING) + def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT) def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index fa3abd0098f5b..2695974b84b00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -77,13 +77,14 @@ class FileCatalogSuite extends SharedSQLContext { val catalog1 = new ListingFileCatalog( spark, Seq(new Path(deletedFolder.getCanonicalPath)), Map.empty, None) // doesn't throw an exception - assert(catalog1.listLeafFiles(catalog1.paths).isEmpty) + assert(catalog1.listLeafFiles(catalog1.rootPaths).isEmpty) } } test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { class MockCatalog( - override val paths: Seq[Path]) extends PartitioningAwareFileCatalog(spark, Map.empty, None) { + override val rootPaths: Seq[Path]) + extends PartitioningAwareFileCatalog(spark, Map.empty, None) { override def refresh(): Unit = {} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index c5deb31fec183..c32254d9dfde2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -395,7 +395,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi val fileCatalog = new ListingFileCatalog( sparkSession = spark, - paths = Seq(new Path(tempDir)), + rootPaths = Seq(new Path(tempDir)), parameters = Map.empty[String, String], partitionSchema = None) // This should not fail. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala similarity index 66% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala index f15730aeb11f2..df509583377ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala @@ -19,16 +19,16 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.SparkFunSuite -class ListingFileCatalogSuite extends SparkFunSuite { +class SessionFileCatalogSuite extends SparkFunSuite { test("file filtering") { - assert(!ListingFileCatalog.shouldFilterOut("abcd")) - assert(ListingFileCatalog.shouldFilterOut(".ab")) - assert(ListingFileCatalog.shouldFilterOut("_cd")) + assert(!SessionFileCatalog.shouldFilterOut("abcd")) + assert(SessionFileCatalog.shouldFilterOut(".ab")) + assert(SessionFileCatalog.shouldFilterOut("_cd")) - assert(!ListingFileCatalog.shouldFilterOut("_metadata")) - assert(!ListingFileCatalog.shouldFilterOut("_common_metadata")) - assert(ListingFileCatalog.shouldFilterOut("_ab_metadata")) - assert(ListingFileCatalog.shouldFilterOut("_cd_common_metadata")) + assert(!SessionFileCatalog.shouldFilterOut("_metadata")) + assert(!SessionFileCatalog.shouldFilterOut("_common_metadata")) + assert(SessionFileCatalog.shouldFilterOut("_ab_metadata")) + assert(SessionFileCatalog.shouldFilterOut("_cd_common_metadata")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 8d18be9300f7e..43357c97c395a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -30,7 +30,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -626,8 +626,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: HadoopFsRelation, _, _) => - assert(relation.partitionSpec === PartitionSpec.emptySpec) + case LogicalRelation(HadoopFsRelation(location: FileCatalog, _, _, _, _, _), _, _) => + assert(location.partitionSpec === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 8a980a7eb538f..c3d202ced24c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -1080,6 +1080,34 @@ class ParquetSchemaSuite extends ParquetSchemaTest { } } + testSchemaClipping( + "falls back to case insensitive resolution", + + parquetSchema = + """message root { + | required group A { + | optional int32 B; + | } + | optional int32 c; + |} + """.stripMargin, + + catalystSchema = { + val nestedType = new StructType().add("b", IntegerType, nullable = true) + new StructType() + .add("a", nestedType, nullable = true) + .add("c", IntegerType, nullable = true) + }, + + expectedSchema = + """message root { + | required group A { + | optional int32 B; + | } + | optional int32 c; + |} + """.stripMargin) + testSchemaClipping( "simple nested struct", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index b5d93c3d7c804..ff59b54f53909 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -29,17 +29,17 @@ import org.apache.thrift.TException import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Statistics} import org.apache.spark.sql.execution.command.{ColumnStatStruct, DDLUtils} import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.{DataType, StructField, StructType} /** @@ -650,8 +650,35 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat override def listPartitionsByFilter( db: String, table: String, - predicates: Seq[Expression]): Seq[CatalogTablePartition] = { - client.getPartitionsByFilter(db, table, predicates) + predicates: Seq[Expression]): Seq[CatalogTablePartition] = withClient { + val catalogTable = client.getTable(db, table) + val partitionColumnNames = catalogTable.partitionColumnNames.toSet + val nonPartitionPruningPredicates = predicates.filterNot { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + + if (nonPartitionPruningPredicates.nonEmpty) { + sys.error("Expected only partition pruning predicates: " + + predicates.reduceLeft(And)) + } + + val partitionSchema = catalogTable.partitionSchema + + if (predicates.nonEmpty) { + val clientPrunedPartitions = + client.getPartitionsByFilter(catalogTable, predicates) + val boundPredicate = + InterpretedPredicate.create(predicates.reduce(And).transform { + case att: AttributeReference => + val index = partitionSchema.indexWhere(_.name == att.name) + BoundReference(index, partitionSchema(index).dataType, nullable = true) + }) + clientPrunedPartitions.filter { case p: CatalogTablePartition => + boundPredicate(p.toRow(partitionSchema)) + } + } else { + client.getPartitions(catalogTable) + } } // -------------------------------------------------------------------------- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index c44f0adda44c0..4a2aaa7d4f6ca 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -135,12 +135,12 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log private def getCached( tableIdentifier: QualifiedTableName, - pathsInMetastore: Seq[String], + pathsInMetastore: Seq[Path], metastoreRelation: MetastoreRelation, schemaInMetastore: StructType, expectedFileFormat: Class[_ <: FileFormat], expectedBucketSpec: Option[BucketSpec], - partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { + partitionSchema: Option[StructType]): Option[LogicalRelation] = { cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => None // Cache miss @@ -152,12 +152,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // If we have the same paths, same schema, and same partition spec, // we will use the cached relation. val useCached = - relation.location.paths.map(_.toString).toSet == pathsInMetastore.toSet && + relation.location.rootPaths.toSet == pathsInMetastore.toSet && logical.schema.sameType(schemaInMetastore) && relation.bucketSpec == expectedBucketSpec && - relation.partitionSpec == partitionSpecInMetastore.getOrElse { - PartitionSpec(StructType(Nil), Array.empty[PartitionDirectory]) - } + relation.partitionSchema == partitionSchema.getOrElse(StructType(Nil)) if (useCached) { Some(logical) @@ -196,61 +194,59 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log QualifiedTableName(metastoreRelation.databaseName, metastoreRelation.tableName) val bucketSpec = None // We don't support hive bucketed tables, only ones we write out. + val lazyPruningEnabled = sparkSession.sqlContext.conf.filesourcePartitionPruning val result = if (metastoreRelation.hiveQlTable.isPartitioned) { val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys) - val partitionColumnDataTypes = partitionSchema.map(_.dataType) - // We're converting the entire table into HadoopFsRelation, so predicates to Hive metastore - // are empty. - val partitions = metastoreRelation.getHiveQlPartitions().map { p => - val location = p.getLocation - val values = InternalRow.fromSeq(p.getValues.asScala.zip(partitionColumnDataTypes).map { - case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) - }) - PartitionDirectory(values, location) - } - val partitionSpec = PartitionSpec(partitionSchema, partitions) - val partitionPaths = partitions.map(_.path.toString) - - // By convention (for example, see MetaStorePartitionedTableFileCatalog), the definition of a - // partitioned table's paths depends on whether that table has any actual partitions. - // Partitioned tables without partitions use the location of the table's base path. - // Partitioned tables with partitions use the locations of those partitions' data locations, - // _omitting_ the table's base path. - val paths = if (partitionPaths.isEmpty) { - Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) + + val rootPaths: Seq[Path] = if (lazyPruningEnabled) { + Seq(metastoreRelation.hiveQlTable.getDataLocation) } else { - partitionPaths + // By convention (for example, see TableFileCatalog), the definition of a + // partitioned table's paths depends on whether that table has any actual partitions. + // Partitioned tables without partitions use the location of the table's base path. + // Partitioned tables with partitions use the locations of those partitions' data + // locations,_omitting_ the table's base path. + val paths = metastoreRelation.getHiveQlPartitions().map { p => + new Path(p.getLocation) + } + if (paths.isEmpty) { + Seq(metastoreRelation.hiveQlTable.getDataLocation) + } else { + paths + } } val cached = getCached( tableIdentifier, - paths, + rootPaths, metastoreRelation, metastoreSchema, fileFormatClass, bucketSpec, - Some(partitionSpec)) - - val hadoopFsRelation = cached.getOrElse { - val fileCatalog = new MetaStorePartitionedTableFileCatalog( - sparkSession, - new Path(metastoreRelation.catalogTable.storage.locationUri.get), - partitionSpec) - - val inferredSchema = if (fileType.equals("parquet")) { - val inferredSchema = - defaultSource.inferSchema(sparkSession, options, fileCatalog.allFiles()) - inferredSchema.map { inferred => - ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, inferred) - }.getOrElse(metastoreSchema) - } else { - defaultSource.inferSchema(sparkSession, options, fileCatalog.allFiles()).get + Some(partitionSchema)) + + val logicalRelation = cached.getOrElse { + val db = metastoreRelation.databaseName + val table = metastoreRelation.tableName + val sizeInBytes = metastoreRelation.statistics.sizeInBytes.toLong + val fileCatalog = { + val catalog = new TableFileCatalog( + sparkSession, db, table, Some(partitionSchema), sizeInBytes) + if (lazyPruningEnabled) { + catalog + } else { + catalog.cachedAllPartitions + } } + val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet + val dataSchema = + StructType(metastoreSchema + .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase))) val relation = HadoopFsRelation( location = fileCatalog, partitionSchema = partitionSchema, - dataSchema = inferredSchema, + dataSchema = dataSchema, bucketSpec = bucketSpec, fileFormat = defaultSource, options = options)(sparkSession = sparkSession) @@ -260,12 +256,12 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log created } - hadoopFsRelation + logicalRelation } else { - val paths = Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) + val rootPath = metastoreRelation.hiveQlTable.getDataLocation val cached = getCached(tableIdentifier, - paths, + Seq(rootPath), metastoreRelation, metastoreSchema, fileFormatClass, @@ -276,14 +272,13 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log LogicalRelation( DataSource( sparkSession = sparkSession, - paths = paths, + paths = rootPath.toString :: Nil, userSpecifiedSchema = Some(metastoreRelation.schema), bucketSpec = bucketSpec, options = options, className = fileType).resolveRelation(), catalogTable = Some(metastoreRelation.catalogTable)) - cachedDataSourceTables.put(tableIdentifier, created) created } @@ -371,34 +366,3 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } } } - -/** - * An override of the standard HDFS listing based catalog, that overrides the partition spec with - * the information from the metastore. - * - * @param tableBasePath The default base path of the Hive metastore table - * @param partitionSpec The partition specifications from Hive metastore - */ -private[hive] class MetaStorePartitionedTableFileCatalog( - sparkSession: SparkSession, - tableBasePath: Path, - override val partitionSpec: PartitionSpec) - extends ListingFileCatalog( - sparkSession, - MetaStorePartitionedTableFileCatalog.getPaths(tableBasePath, partitionSpec), - Map.empty, - Some(partitionSpec.partitionColumns)) { -} - -private[hive] object MetaStorePartitionedTableFileCatalog { - /** Get the list of paths to list files in the for a metastore table */ - def getPaths(tableBasePath: Path, partitionSpec: PartitionSpec): Seq[Path] = { - // If there are no partitions currently specified then use base path, - // otherwise use the paths corresponding to the partitions. - if (partitionSpec.partitions.isEmpty) { - Seq(tableBasePath) - } else { - partitionSpec.partitions.map(_.path) - } - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index 9ee3d629c9977..569a9c11398ea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -172,15 +172,24 @@ private[hive] trait HiveClient { * Returns the partitions for the given table that match the supplied partition spec. * If no partition spec is specified, all partitions are returned. */ - def getPartitions( + final def getPartitions( db: String, table: String, + partialSpec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = { + getPartitions(getTable(db, table), partialSpec) + } + + /** + * Returns the partitions for the given table that match the supplied partition spec. + * If no partition spec is specified, all partitions are returned. + */ + def getPartitions( + catalogTable: CatalogTable, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] /** Returns partitions filtered by predicates for the given table. */ def getPartitionsByFilter( - db: String, - table: String, + catalogTable: CatalogTable, predicates: Seq[Expression]): Seq[CatalogTablePartition] /** Loads a static partition into an existing table. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 5c8f7ff1af9fa..e745a8c5b3589 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -37,6 +37,7 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} @@ -525,22 +526,24 @@ private[hive] class HiveClientImpl( * If no partition spec is specified, all partitions are returned. */ override def getPartitions( - db: String, - table: String, + table: CatalogTable, spec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = withHiveState { - val hiveTable = toHiveTable(getTable(db, table)) - spec match { + val hiveTable = toHiveTable(table) + val parts = spec match { case None => shim.getAllPartitions(client, hiveTable).map(fromHivePartition) case Some(s) => client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition) } + HiveCatalogMetrics.incrementFetchedPartitions(parts.length) + parts } override def getPartitionsByFilter( - db: String, - table: String, + table: CatalogTable, predicates: Seq[Expression]): Seq[CatalogTablePartition] = withHiveState { - val hiveTable = toHiveTable(getTable(db, table)) - shim.getPartitionsByFilter(client, hiveTable, predicates).map(fromHivePartition) + val hiveTable = toHiveTable(table) + val parts = shim.getPartitionsByFilter(client, hiveTable, predicates).map(fromHivePartition) + HiveCatalogMetrics.incrementFetchedPartitions(parts.length) + parts } override def listTables(dbName: String): Seq[String] = withHiveState { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index e94f49ea81177..1af3280e18a89 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -313,7 +313,17 @@ private[orc] object OrcRelation extends HiveInspectors { def setRequiredColumns( conf: Configuration, physicalSchema: StructType, requestedSchema: StructType): Unit = { - val ids = requestedSchema.map(a => physicalSchema.fieldIndex(a.name): Integer) + val caseInsensitiveFieldMap: Map[String, Int] = physicalSchema.fieldNames + .zipWithIndex + .map(f => (f._1.toLowerCase, f._2)) + .toMap + val ids = requestedSchema.map { a => + val exactMatch: Option[Int] = physicalSchema.getFieldIndex(a.name) + val res = exactMatch.getOrElse( + caseInsensitiveFieldMap.getOrElse(a.name, + throw new IllegalArgumentException(s"""Field "$a.name" does not exist."""))) + res: Integer + } val (sortedIDs, sortedNames) = ids.zip(requestedSchema.fieldNames).sorted.unzip HiveShim.appendReadColumns(conf, sortedIDs, sortedNames) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala index 96e9054cd4876..f65e74de87a57 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala @@ -17,10 +17,14 @@ package org.apache.spark.sql.hive +import java.io.File + +import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.QueryTest -class HiveDataFrameSuite extends QueryTest with TestHiveSingleton { +class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("table name with schema") { // regression test for SPARK-11778 spark.sql("create schema usrdb") @@ -34,4 +38,107 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton { val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client assert(hiveClient.getConf("hive.in.test", "") == "true") } + + private def setupPartitionedTable(tableName: String, dir: File): Unit = { + spark.range(5).selectExpr("id", "id as partCol1", "id as partCol2").write + .partitionBy("partCol1", "partCol2") + .mode("overwrite") + .parquet(dir.getAbsolutePath) + + spark.sql(s""" + |create external table $tableName (id long) + |partitioned by (partCol1 int, partCol2 int) + |stored as parquet + |location "${dir.getAbsolutePath}"""".stripMargin) + spark.sql(s"msck repair table $tableName") + } + + test("partitioned pruned table reports only selected files") { + assert(spark.sqlContext.getConf(HiveUtils.CONVERT_METASTORE_PARQUET.key) == "true") + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + val df = spark.sql("select * from test") + assert(df.count() == 5) + assert(df.inputFiles.length == 5) // unpruned + + val df2 = spark.sql("select * from test where partCol1 = 3 or partCol2 = 4") + assert(df2.count() == 2) + assert(df2.inputFiles.length == 2) // pruned, so we have less files + + val df3 = spark.sql("select * from test where PARTCOL1 = 3 or partcol2 = 4") + assert(df3.count() == 2) + assert(df3.inputFiles.length == 2) + + val df4 = spark.sql("select * from test where partCol1 = 999") + assert(df4.count() == 0) + assert(df4.inputFiles.length == 0) + } + } + } + + test("lazy partition pruning reads only necessary partition data") { + withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> "true") { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 = 999").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 < 2").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 < 3").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 3) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 3) + + // should read all + HiveCatalogMetrics.reset() + spark.sql("select * from test").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + // read all should be cached + HiveCatalogMetrics.reset() + spark.sql("select * from test").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + } + } + } + } + + test("all partitions read and cached when filesource partition pruning is off") { + withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> "false") { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + + // We actually query the partitions from hive each time the table is resolved in this + // mode. This is kind of terrible, but is needed to preserve the legacy behavior + // of doing plan cache validation based on the entire partition set. + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 = 999").count() + // 5 from table resolution, another 5 from ListingFileCatalog + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 10) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 < 2").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + + HiveCatalogMetrics.reset() + spark.sql("select * from test").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala index 3414f5e0409a1..7af81a3a90504 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -59,4 +59,45 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi } } } + + def testCaching(pruningEnabled: Boolean): Unit = { + test(s"partitioned table is cached when partition pruning is $pruningEnabled") { + withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> pruningEnabled.toString) { + withTable("test") { + withTempDir { dir => + spark.range(5).selectExpr("id", "id as f1", "id as f2").write + .partitionBy("f1", "f2") + .mode("overwrite") + .parquet(dir.getAbsolutePath) + + spark.sql(s""" + |create external table test (id long) + |partitioned by (f1 int, f2 int) + |stored as parquet + |location "${dir.getAbsolutePath}"""".stripMargin) + spark.sql("msck repair table test") + + val df = spark.sql("select * from test") + assert(sql("select * from test").count() == 5) + + // Delete a file, then assert that we tried to read it. This means the table was cached. + val p = new Path(spark.table("test").inputFiles.head) + assert(p.getFileSystem(hiveContext.sessionState.newHadoopConf()).delete(p, true)) + val e = intercept[SparkException] { + sql("select * from test").count() + } + assert(e.getMessage.contains("FileNotFoundException")) + + // Test refreshing the cache. + spark.catalog.refreshTable("test") + assert(sql("select * from test").count() == 4) + } + } + } + } + } + + for (pruningEnabled <- Seq(true, false)) { + testCaching(pruningEnabled) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index c158bf1ab09cb..9a10957c8efa5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -295,12 +295,12 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: getPartitions(catalogTable)") { - assert(2 == client.getPartitions("default", "src_part").size) + assert(2 == client.getPartitions(client.getTable("default", "src_part")).size) } test(s"$version: getPartitionsByFilter") { // Only one partition [1, 1] for key2 == 1 - val result = client.getPartitionsByFilter("default", "src_part", + val result = client.getPartitionsByFilter(client.getTable("default", "src_part"), Seq(EqualTo(AttributeReference("key2", IntegerType)(), Literal(1)))) // Hive 0.12 doesn't support getPartitionsByFilter, it ignores the filter condition. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index b2ee49c441ef2..ecb5972984523 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -474,6 +474,28 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } } + test("converted ORC table supports resolving mixed case field") { + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { + withTable("dummy_orc") { + withTempPath { dir => + val df = spark.range(5).selectExpr("id", "id as valueField", "id as partitionValue") + df.write + .partitionBy("partitionValue") + .mode("overwrite") + .orc(dir.getAbsolutePath) + + spark.sql(s""" + |create external table dummy_orc (id long, valueField long) + |partitioned by (partitionValue int) + |stored as orc + |location "${dir.getAbsolutePath}"""".stripMargin) + spark.sql(s"msck repair table dummy_orc") + checkAnswer(spark.sql("select * from dummy_orc"), df) + } + } + } + } + test("SPARK-14962 Produce correct results on array type with isnotnull") { withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { val data = (0 until 10).map(i => Tuple1(Array(i))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 2f6d9fb96b825..9fc62a389db4d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -175,7 +175,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a") .createOrReplaceTempView("jt_array") - setConf(HiveUtils.CONVERT_METASTORE_PARQUET, true) + assert(spark.sqlContext.getConf(HiveUtils.CONVERT_METASTORE_PARQUET.key) == "true") } override def afterAll(): Unit = { @@ -187,7 +187,6 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { "jt", "jt_array", "test_parquet") - setConf(HiveUtils.CONVERT_METASTORE_PARQUET, false) } test(s"conversion is working") { @@ -586,6 +585,23 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { checkAnswer( sql("SELECT * FROM test_added_partitions"), Seq(("foo", 0), ("bar", 0), ("baz", 1)).toDF("a", "b")) + + // Check it with pruning predicates + checkAnswer( + sql("SELECT * FROM test_added_partitions where b = 0"), + Seq(("foo", 0), ("bar", 0)).toDF("a", "b")) + checkAnswer( + sql("SELECT * FROM test_added_partitions where b = 1"), + Seq(("baz", 1)).toDF("a", "b")) + checkAnswer( + sql("SELECT * FROM test_added_partitions where b = 2"), + Seq[(String, Int)]().toDF("a", "b")) + + // Also verify the inputFiles implementation + assert(sql("select * from test_added_partitions").inputFiles.length == 2) + assert(sql("select * from test_added_partitions where b = 0").inputFiles.length == 1) + assert(sql("select * from test_added_partitions where b = 1").inputFiles.length == 1) + assert(sql("select * from test_added_partitions where b = 2").inputFiles.length == 0) } } } From 36d81c2c68ef4114592b069287743eb5cb078318 Mon Sep 17 00:00:00 2001 From: Jun Kim Date: Sat, 15 Oct 2016 00:36:55 -0700 Subject: [PATCH 103/177] [SPARK-17953][DOCUMENTATION] Fix typo in SparkSession scaladoc ## What changes were proposed in this pull request? ### Before: ```scala SparkSession.builder() .master("local") .appName("Word Count") .config("spark.some.config.option", "some-value"). .getOrCreate() ``` ### After: ```scala SparkSession.builder() .master("local") .appName("Word Count") .config("spark.some.config.option", "some-value") .getOrCreate() ``` There was one unexpected dot! Author: Jun Kim Closes #15498 from tae-jun/SPARK-17953. --- sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 137c426b4b88d..baae55013787d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -64,7 +64,7 @@ import org.apache.spark.util.Utils * SparkSession.builder() * .master("local") * .appName("Word Count") - * .config("spark.some.config.option", "some-value"). + * .config("spark.some.config.option", "some-value") * .getOrCreate() * }}} */ From ed1463341455830b8867b721a1b34f291139baf3 Mon Sep 17 00:00:00 2001 From: Zhan Zhang Date: Sat, 15 Oct 2016 18:45:04 -0700 Subject: [PATCH 104/177] [SPARK-17637][SCHEDULER] Packed scheduling for Spark tasks across executors ## What changes were proposed in this pull request? Restructure the code and implement two new task assigner. PackedAssigner: try to allocate tasks to the executors with least available cores, so that spark can release reserved executors when dynamic allocation is enabled. BalancedAssigner: try to allocate tasks to the executors with more available cores in order to balance the workload across all executors. By default, the original round robin assigner is used. We test a pipeline, and new PackedAssigner save around 45% regarding the reserved cpu and memory with dynamic allocation enabled. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Both unit test in TaskSchedulerImplSuite and manual tests in production pipeline. Author: Zhan Zhang Closes #15218 from zhzhan/packed-scheduler. --- .../apache/spark/scheduler/TaskAssigner.scala | 154 ++++++++++++++++++ .../spark/scheduler/TaskSchedulerImpl.scala | 53 +++--- .../scheduler/TaskSchedulerImplSuite.scala | 67 ++++++++ docs/configuration.md | 11 ++ 4 files changed, 266 insertions(+), 19 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala new file mode 100644 index 0000000000000..62df9657a6ac6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.PriorityQueue +import scala.util.Random + +import org.apache.spark.SparkConf + +case class OfferState(workOffer: WorkerOffer, var cores: Int) { + // Build a list of tasks to assign to each worker. + val tasks = new ArrayBuffer[TaskDescription](cores) +} + +abstract class TaskAssigner(conf: SparkConf) { + var offer: Seq[OfferState] = _ + val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) + + // The final assigned offer returned to TaskScheduler. + def tasks(): Seq[ArrayBuffer[TaskDescription]] = offer.map(_.tasks) + + // construct the assigner by the workoffer. + def construct(workOffer: Seq[WorkerOffer]): Unit = { + offer = workOffer.map(o => OfferState(o, o.cores)) + } + + // Invoked in each round of Taskset assignment to initialize the internal structure. + def init(): Unit + + // Indicating whether there is offer available to be used by one round of Taskset assignment. + def hasNext(): Boolean + + // Next available offer returned to one round of Taskset assignment. + def getNext(): OfferState + + // Called by the TaskScheduler to indicate whether the current offer is accepted + // In order to decide whether the current is valid for the next offering. + def taskAssigned(assigned: Boolean): Unit + + // Release internally maintained resources. Subclass is responsible to + // release its own private resources. + def reset: Unit = { + offer = null + } +} + +class RoundRobinAssigner(conf: SparkConf) extends TaskAssigner(conf) { + var i = 0 + override def construct(workOffer: Seq[WorkerOffer]): Unit = { + offer = Random.shuffle(workOffer.map(o => OfferState(o, o.cores))) + } + override def init(): Unit = { + i = 0 + } + override def hasNext: Boolean = { + i < offer.size + } + override def getNext(): OfferState = { + offer(i) + } + override def taskAssigned(assigned: Boolean): Unit = { + i += 1 + } + override def reset: Unit = { + super.reset + i = 0 + } +} + +class BalancedAssigner(conf: SparkConf) extends TaskAssigner(conf) { + var maxHeap: PriorityQueue[OfferState] = _ + var current: OfferState = _ + + override def construct(workOffer: Seq[WorkerOffer]): Unit = { + offer = Random.shuffle(workOffer.map(o => OfferState(o, o.cores))) + } + implicit val ord: Ordering[OfferState] = new Ordering[OfferState] { + def compare(x: OfferState, y: OfferState): Int = { + return Ordering[Int].compare(x.cores, y.cores) + } + } + def init(): Unit = { + maxHeap = new PriorityQueue[OfferState]() + offer.filter(_.cores >= CPUS_PER_TASK).foreach(maxHeap.enqueue(_)) + } + override def hasNext: Boolean = { + maxHeap.size > 0 + } + override def getNext(): OfferState = { + current = maxHeap.dequeue() + current + } + + override def taskAssigned(assigned: Boolean): Unit = { + if (current.cores >= CPUS_PER_TASK && assigned) { + maxHeap.enqueue(current) + } + } + override def reset: Unit = { + super.reset + maxHeap = null + current = null + } +} + +class PackedAssigner(conf: SparkConf) extends TaskAssigner(conf) { + + var sorted: Seq[OfferState] = _ + var i = 0 + var current: OfferState = _ + + override def init(): Unit = { + i = 0 + sorted = offer.filter(_.cores >= CPUS_PER_TASK).sortBy(_.cores) + } + + override def hasNext: Boolean = { + i < sorted.size + } + + override def getNext(): OfferState = { + current = sorted(i) + current + } + + def taskAssigned(assigned: Boolean): Unit = { + if (current.cores < CPUS_PER_TASK || !assigned) { + i += 1 + } + } + + override def reset: Unit = { + super.reset + sorted = null + current = null + i = 0 + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 3e3f1ad031e66..fb732ea8e5a3b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -22,9 +22,7 @@ import java.util.{Timer, TimerTask} import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong -import scala.collection.Set import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState @@ -61,6 +59,21 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf + val DEFAULT_TASK_ASSIGNER = classOf[RoundRobinAssigner].getName + lazy val taskAssigner: TaskAssigner = { + val className = conf.get("spark.task.assigner", DEFAULT_TASK_ASSIGNER) + try { + logInfo(s"""constructing assigner as $className""") + val ctor = Utils.classForName(className).getConstructor(classOf[SparkConf]) + ctor.newInstance(conf).asInstanceOf[TaskAssigner] + } catch { + case _: Throwable => + logWarning( + s"""$className cannot be constructed fallback to default + | $DEFAULT_TASK_ASSIGNER""".stripMargin) + new RoundRobinAssigner(conf) + } + } // How often to check for speculative tasks val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") @@ -250,24 +263,26 @@ private[spark] class TaskSchedulerImpl( private def resourceOfferSingleTaskSet( taskSet: TaskSetManager, maxLocality: TaskLocality, - shuffledOffers: Seq[WorkerOffer], - availableCpus: Array[Int], - tasks: IndexedSeq[ArrayBuffer[TaskDescription]]) : Boolean = { + taskAssigner: TaskAssigner) : Boolean = { var launchedTask = false - for (i <- 0 until shuffledOffers.size) { - val execId = shuffledOffers(i).executorId - val host = shuffledOffers(i).host - if (availableCpus(i) >= CPUS_PER_TASK) { + taskAssigner.init() + while(taskAssigner.hasNext()) { + var assigned = false + val current = taskAssigner.getNext() + val execId = current.workOffer.executorId + val host = current.workOffer.host + if (current.cores >= CPUS_PER_TASK) { try { for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { - tasks(i) += task + current.tasks += task val tid = task.taskId taskIdToTaskSetManager(tid) = taskSet taskIdToExecutorId(tid) = execId executorIdToTaskCount(execId) += 1 - availableCpus(i) -= CPUS_PER_TASK - assert(availableCpus(i) >= 0) + current.cores = current.cores - CPUS_PER_TASK + assert(current.cores >= 0) launchedTask = true + assigned = true } } catch { case e: TaskNotSerializableException => @@ -277,8 +292,10 @@ private[spark] class TaskSchedulerImpl( return launchedTask } } + taskAssigner.taskAssigned(assigned) } return launchedTask + } /** @@ -305,12 +322,8 @@ private[spark] class TaskSchedulerImpl( hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += o.host } } + taskAssigner.construct(offers) - // Randomly shuffle offers to avoid always placing tasks on the same set of workers. - val shuffledOffers = Random.shuffle(offers) - // Build a list of tasks to assign to each worker. - val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - val availableCpus = shuffledOffers.map(o => o.cores).toArray val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( @@ -329,7 +342,7 @@ private[spark] class TaskSchedulerImpl( for (currentMaxLocality <- taskSet.myLocalityLevels) { do { launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet( - taskSet, currentMaxLocality, shuffledOffers, availableCpus, tasks) + taskSet, currentMaxLocality, taskAssigner) launchedAnyTask |= launchedTaskAtCurrentMaxLocality } while (launchedTaskAtCurrentMaxLocality) } @@ -337,10 +350,12 @@ private[spark] class TaskSchedulerImpl( taskSet.abortIfCompletelyBlacklisted(hostToExecutors) } } - + val tasks = taskAssigner.tasks + taskAssigner.reset if (tasks.size > 0) { hasLaunchedTask = true } + return tasks } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index f5f1947661d9a..2584f85bc553a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -109,6 +109,72 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!failedTaskSet) } + test("Scheduler balance the assignment to the worker with more free cores") { + val taskScheduler = setupScheduler(("spark.task.assigner", classOf[BalancedAssigner].getName)) + val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2), + new WorkerOffer("executor1", "host1", 4)) + val selectedExecutorIds = { + val taskSet = FakeTask.createTaskSet(2) + taskScheduler.submitTasks(taskSet) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(2 === taskDescriptions.length) + taskDescriptions.map(_.executorId) + } + val count = selectedExecutorIds.count(_ == workerOffers(1).executorId) + assert(count == 2) + assert(!failedTaskSet) + } + + test("Scheduler balance the assignment across workers with same free cores") { + val taskScheduler = setupScheduler(("spark.task.assigner", classOf[BalancedAssigner].getName)) + val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2), + new WorkerOffer("executor1", "host1", 2)) + val selectedExecutorIds = { + val taskSet = FakeTask.createTaskSet(2) + taskScheduler.submitTasks(taskSet) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(2 === taskDescriptions.length) + taskDescriptions.map(_.executorId) + } + val count = selectedExecutorIds.count(_ == workerOffers(1).executorId) + assert(count == 1) + assert(!failedTaskSet) + } + + test("Scheduler packs the assignment to workers with less free cores") { + val taskScheduler = setupScheduler(("spark.task.assigner", classOf[PackedAssigner].getName)) + val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2), + new WorkerOffer("executor1", "host1", 4)) + val selectedExecutorIds = { + val taskSet = FakeTask.createTaskSet(2) + taskScheduler.submitTasks(taskSet) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(2 === taskDescriptions.length) + taskDescriptions.map(_.executorId) + } + val count = selectedExecutorIds.count(_ == workerOffers(0).executorId) + assert(count == 2) + assert(!failedTaskSet) + } + + test("Scheduler keeps packing the assignment to the same worker") { + val taskScheduler = setupScheduler(("spark.task.assigner", classOf[PackedAssigner].getName)) + val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 4), + new WorkerOffer("executor1", "host1", 4)) + val selectedExecutorIds = { + val taskSet = FakeTask.createTaskSet(4) + taskScheduler.submitTasks(taskSet) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 === taskDescriptions.length) + taskDescriptions.map(_.executorId) + } + + val count = selectedExecutorIds.count(_ == workerOffers(0).executorId) + assert(count == 4) + assert(!failedTaskSet) + } + + test("Scheduler correctly accounts for multiple CPUs per task") { val taskCpus = 2 val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) @@ -408,4 +474,5 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(thirdTaskDescs.size === 0) assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1", "executor3"))) } + } diff --git a/docs/configuration.md b/docs/configuration.md index 373e22d71a872..6f3fbeb76cc24 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1334,6 +1334,17 @@ Apart from these, the following properties are also available, and may be useful Should be greater than or equal to 1. Number of allowed retries = this value - 1. + + spark.task.assigner + org.apache.spark.scheduler.RoundRobinAssigner + + The strategy of how to allocate tasks among workers with free cores. + By default, round robin with randomness is used. + org.apache.spark.scheduler.BalancedAssigner tries to balance the task across all workers (allocating tasks to + workers with most free cores). org.apache.spark.scheduler.PackedAssigner tries to allocate tasks to workers + with the least free cores, which may help releasing the resources when dynamic allocation is enabled. + + #### Dynamic Allocation From 72a6e7a57a63aba69f26c84bf68a5fb213d2a521 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 15 Oct 2016 22:31:37 -0700 Subject: [PATCH 105/177] Revert "[SPARK-17637][SCHEDULER] Packed scheduling for Spark tasks across executors" This reverts commit ed1463341455830b8867b721a1b34f291139baf3. The patch merged had obvious quality and documentation issue. The idea is useful, and we should work towards improving its quality and merging it in again. --- .../apache/spark/scheduler/TaskAssigner.scala | 154 ------------------ .../spark/scheduler/TaskSchedulerImpl.scala | 53 +++--- .../scheduler/TaskSchedulerImplSuite.scala | 67 -------- docs/configuration.md | 11 -- 4 files changed, 19 insertions(+), 266 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala deleted file mode 100644 index 62df9657a6ac6..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.PriorityQueue -import scala.util.Random - -import org.apache.spark.SparkConf - -case class OfferState(workOffer: WorkerOffer, var cores: Int) { - // Build a list of tasks to assign to each worker. - val tasks = new ArrayBuffer[TaskDescription](cores) -} - -abstract class TaskAssigner(conf: SparkConf) { - var offer: Seq[OfferState] = _ - val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) - - // The final assigned offer returned to TaskScheduler. - def tasks(): Seq[ArrayBuffer[TaskDescription]] = offer.map(_.tasks) - - // construct the assigner by the workoffer. - def construct(workOffer: Seq[WorkerOffer]): Unit = { - offer = workOffer.map(o => OfferState(o, o.cores)) - } - - // Invoked in each round of Taskset assignment to initialize the internal structure. - def init(): Unit - - // Indicating whether there is offer available to be used by one round of Taskset assignment. - def hasNext(): Boolean - - // Next available offer returned to one round of Taskset assignment. - def getNext(): OfferState - - // Called by the TaskScheduler to indicate whether the current offer is accepted - // In order to decide whether the current is valid for the next offering. - def taskAssigned(assigned: Boolean): Unit - - // Release internally maintained resources. Subclass is responsible to - // release its own private resources. - def reset: Unit = { - offer = null - } -} - -class RoundRobinAssigner(conf: SparkConf) extends TaskAssigner(conf) { - var i = 0 - override def construct(workOffer: Seq[WorkerOffer]): Unit = { - offer = Random.shuffle(workOffer.map(o => OfferState(o, o.cores))) - } - override def init(): Unit = { - i = 0 - } - override def hasNext: Boolean = { - i < offer.size - } - override def getNext(): OfferState = { - offer(i) - } - override def taskAssigned(assigned: Boolean): Unit = { - i += 1 - } - override def reset: Unit = { - super.reset - i = 0 - } -} - -class BalancedAssigner(conf: SparkConf) extends TaskAssigner(conf) { - var maxHeap: PriorityQueue[OfferState] = _ - var current: OfferState = _ - - override def construct(workOffer: Seq[WorkerOffer]): Unit = { - offer = Random.shuffle(workOffer.map(o => OfferState(o, o.cores))) - } - implicit val ord: Ordering[OfferState] = new Ordering[OfferState] { - def compare(x: OfferState, y: OfferState): Int = { - return Ordering[Int].compare(x.cores, y.cores) - } - } - def init(): Unit = { - maxHeap = new PriorityQueue[OfferState]() - offer.filter(_.cores >= CPUS_PER_TASK).foreach(maxHeap.enqueue(_)) - } - override def hasNext: Boolean = { - maxHeap.size > 0 - } - override def getNext(): OfferState = { - current = maxHeap.dequeue() - current - } - - override def taskAssigned(assigned: Boolean): Unit = { - if (current.cores >= CPUS_PER_TASK && assigned) { - maxHeap.enqueue(current) - } - } - override def reset: Unit = { - super.reset - maxHeap = null - current = null - } -} - -class PackedAssigner(conf: SparkConf) extends TaskAssigner(conf) { - - var sorted: Seq[OfferState] = _ - var i = 0 - var current: OfferState = _ - - override def init(): Unit = { - i = 0 - sorted = offer.filter(_.cores >= CPUS_PER_TASK).sortBy(_.cores) - } - - override def hasNext: Boolean = { - i < sorted.size - } - - override def getNext(): OfferState = { - current = sorted(i) - current - } - - def taskAssigned(assigned: Boolean): Unit = { - if (current.cores < CPUS_PER_TASK || !assigned) { - i += 1 - } - } - - override def reset: Unit = { - super.reset - sorted = null - current = null - i = 0 - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index fb732ea8e5a3b..3e3f1ad031e66 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -22,7 +22,9 @@ import java.util.{Timer, TimerTask} import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong +import scala.collection.Set import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState @@ -59,21 +61,6 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf - val DEFAULT_TASK_ASSIGNER = classOf[RoundRobinAssigner].getName - lazy val taskAssigner: TaskAssigner = { - val className = conf.get("spark.task.assigner", DEFAULT_TASK_ASSIGNER) - try { - logInfo(s"""constructing assigner as $className""") - val ctor = Utils.classForName(className).getConstructor(classOf[SparkConf]) - ctor.newInstance(conf).asInstanceOf[TaskAssigner] - } catch { - case _: Throwable => - logWarning( - s"""$className cannot be constructed fallback to default - | $DEFAULT_TASK_ASSIGNER""".stripMargin) - new RoundRobinAssigner(conf) - } - } // How often to check for speculative tasks val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") @@ -263,26 +250,24 @@ private[spark] class TaskSchedulerImpl( private def resourceOfferSingleTaskSet( taskSet: TaskSetManager, maxLocality: TaskLocality, - taskAssigner: TaskAssigner) : Boolean = { + shuffledOffers: Seq[WorkerOffer], + availableCpus: Array[Int], + tasks: IndexedSeq[ArrayBuffer[TaskDescription]]) : Boolean = { var launchedTask = false - taskAssigner.init() - while(taskAssigner.hasNext()) { - var assigned = false - val current = taskAssigner.getNext() - val execId = current.workOffer.executorId - val host = current.workOffer.host - if (current.cores >= CPUS_PER_TASK) { + for (i <- 0 until shuffledOffers.size) { + val execId = shuffledOffers(i).executorId + val host = shuffledOffers(i).host + if (availableCpus(i) >= CPUS_PER_TASK) { try { for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { - current.tasks += task + tasks(i) += task val tid = task.taskId taskIdToTaskSetManager(tid) = taskSet taskIdToExecutorId(tid) = execId executorIdToTaskCount(execId) += 1 - current.cores = current.cores - CPUS_PER_TASK - assert(current.cores >= 0) + availableCpus(i) -= CPUS_PER_TASK + assert(availableCpus(i) >= 0) launchedTask = true - assigned = true } } catch { case e: TaskNotSerializableException => @@ -292,10 +277,8 @@ private[spark] class TaskSchedulerImpl( return launchedTask } } - taskAssigner.taskAssigned(assigned) } return launchedTask - } /** @@ -322,8 +305,12 @@ private[spark] class TaskSchedulerImpl( hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += o.host } } - taskAssigner.construct(offers) + // Randomly shuffle offers to avoid always placing tasks on the same set of workers. + val shuffledOffers = Random.shuffle(offers) + // Build a list of tasks to assign to each worker. + val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + val availableCpus = shuffledOffers.map(o => o.cores).toArray val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( @@ -342,7 +329,7 @@ private[spark] class TaskSchedulerImpl( for (currentMaxLocality <- taskSet.myLocalityLevels) { do { launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet( - taskSet, currentMaxLocality, taskAssigner) + taskSet, currentMaxLocality, shuffledOffers, availableCpus, tasks) launchedAnyTask |= launchedTaskAtCurrentMaxLocality } while (launchedTaskAtCurrentMaxLocality) } @@ -350,12 +337,10 @@ private[spark] class TaskSchedulerImpl( taskSet.abortIfCompletelyBlacklisted(hostToExecutors) } } - val tasks = taskAssigner.tasks - taskAssigner.reset + if (tasks.size > 0) { hasLaunchedTask = true } - return tasks } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 2584f85bc553a..f5f1947661d9a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -109,72 +109,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!failedTaskSet) } - test("Scheduler balance the assignment to the worker with more free cores") { - val taskScheduler = setupScheduler(("spark.task.assigner", classOf[BalancedAssigner].getName)) - val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2), - new WorkerOffer("executor1", "host1", 4)) - val selectedExecutorIds = { - val taskSet = FakeTask.createTaskSet(2) - taskScheduler.submitTasks(taskSet) - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(2 === taskDescriptions.length) - taskDescriptions.map(_.executorId) - } - val count = selectedExecutorIds.count(_ == workerOffers(1).executorId) - assert(count == 2) - assert(!failedTaskSet) - } - - test("Scheduler balance the assignment across workers with same free cores") { - val taskScheduler = setupScheduler(("spark.task.assigner", classOf[BalancedAssigner].getName)) - val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2), - new WorkerOffer("executor1", "host1", 2)) - val selectedExecutorIds = { - val taskSet = FakeTask.createTaskSet(2) - taskScheduler.submitTasks(taskSet) - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(2 === taskDescriptions.length) - taskDescriptions.map(_.executorId) - } - val count = selectedExecutorIds.count(_ == workerOffers(1).executorId) - assert(count == 1) - assert(!failedTaskSet) - } - - test("Scheduler packs the assignment to workers with less free cores") { - val taskScheduler = setupScheduler(("spark.task.assigner", classOf[PackedAssigner].getName)) - val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2), - new WorkerOffer("executor1", "host1", 4)) - val selectedExecutorIds = { - val taskSet = FakeTask.createTaskSet(2) - taskScheduler.submitTasks(taskSet) - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(2 === taskDescriptions.length) - taskDescriptions.map(_.executorId) - } - val count = selectedExecutorIds.count(_ == workerOffers(0).executorId) - assert(count == 2) - assert(!failedTaskSet) - } - - test("Scheduler keeps packing the assignment to the same worker") { - val taskScheduler = setupScheduler(("spark.task.assigner", classOf[PackedAssigner].getName)) - val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 4), - new WorkerOffer("executor1", "host1", 4)) - val selectedExecutorIds = { - val taskSet = FakeTask.createTaskSet(4) - taskScheduler.submitTasks(taskSet) - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(4 === taskDescriptions.length) - taskDescriptions.map(_.executorId) - } - - val count = selectedExecutorIds.count(_ == workerOffers(0).executorId) - assert(count == 4) - assert(!failedTaskSet) - } - - test("Scheduler correctly accounts for multiple CPUs per task") { val taskCpus = 2 val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) @@ -474,5 +408,4 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(thirdTaskDescs.size === 0) assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1", "executor3"))) } - } diff --git a/docs/configuration.md b/docs/configuration.md index 6f3fbeb76cc24..373e22d71a872 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1334,17 +1334,6 @@ Apart from these, the following properties are also available, and may be useful Should be greater than or equal to 1. Number of allowed retries = this value - 1. - - spark.task.assigner - org.apache.spark.scheduler.RoundRobinAssigner - - The strategy of how to allocate tasks among workers with free cores. - By default, round robin with randomness is used. - org.apache.spark.scheduler.BalancedAssigner tries to balance the task across all workers (allocating tasks to - workers with most free cores). org.apache.spark.scheduler.PackedAssigner tries to allocate tasks to workers - with the least free cores, which may help releasing the resources when dynamic allocation is enabled. - - #### Dynamic Allocation From 59e3eb5af8d0969bbb785af77b66343bda7acc38 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 16 Oct 2016 20:15:32 -0700 Subject: [PATCH 106/177] [SPARK-17819][SQL] Support default database in connection URIs for Spark Thrift Server ## What changes were proposed in this pull request? Currently, Spark Thrift Server ignores the default database in URI. This PR supports that like the following. ```sql $ bin/beeline -u jdbc:hive2://localhost:10000 -e "create database testdb" $ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "create table t(a int)" $ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "show tables" ... +------------+--------------+--+ | tableName | isTemporary | +------------+--------------+--+ | t | false | +------------+--------------+--+ 1 row selected (0.347 seconds) $ bin/beeline -u jdbc:hive2://localhost:10000 -e "show tables" ... +------------+--------------+--+ | tableName | isTemporary | +------------+--------------+--+ +------------+--------------+--+ No rows selected (0.098 seconds) ``` ## How was this patch tested? Manual. Note: I tried to add a test case for this, but I cannot found a suitable testsuite for this. I'll add the testcase if some advice is given. Author: Dongjoon Hyun Closes #15399 from dongjoon-hyun/SPARK-17819. --- .../thriftserver/SparkSQLSessionManager.scala | 3 + .../thriftserver/JdbcConnectionUriSuite.scala | 70 +++++++++++++++++++ 2 files changed, 73 insertions(+) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 6a5117aea492d..226b7e175a9d9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -79,6 +79,9 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: sqlContext.newSession() } ctx.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) + if (sessionConf != null && sessionConf.containsKey("use:database")) { + ctx.sql(s"use ${sessionConf.get("use:database")}") + } sparkSqlOperationManager.sessionToContexts.put(sessionHandle, ctx) sessionHandle } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala new file mode 100644 index 0000000000000..fb8a7e273ae44 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.sql.DriverManager + +import org.apache.hive.jdbc.HiveDriver + +import org.apache.spark.util.Utils + +class JdbcConnectionUriSuite extends HiveThriftServer2Test { + Utils.classForName(classOf[HiveDriver].getCanonicalName) + + override def mode: ServerMode.Value = ServerMode.binary + + val JDBC_TEST_DATABASE = "jdbc_test_database" + val USER = System.getProperty("user.name") + val PASSWORD = "" + + override protected def beforeAll(): Unit = { + super.beforeAll() + + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + statement.execute(s"CREATE DATABASE $JDBC_TEST_DATABASE") + connection.close() + } + + override protected def afterAll(): Unit = { + try { + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + statement.execute(s"DROP DATABASE $JDBC_TEST_DATABASE") + connection.close() + } finally { + super.afterAll() + } + } + + test("SPARK-17819 Support default database in connection URIs") { + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/$JDBC_TEST_DATABASE" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + try { + val resultSet = statement.executeQuery("select current_database()") + resultSet.next() + assert(resultSet.getString(1) === JDBC_TEST_DATABASE) + } finally { + statement.close() + connection.close() + } + } +} From e18d02c5a8f8af2e42079ab414f5d84b3e1a279e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 17 Oct 2016 12:08:25 +0800 Subject: [PATCH 107/177] [SPARK-17947][SQL] Add Doc and Comment about spark.sql.debug ### What changes were proposed in this pull request? Just document the impact of `spark.sql.debug`: When enabling the debug, Spark SQL internal table properties are not filtered out; however, some related DDL commands (e.g., Analyze Table and CREATE TABLE LIKE) might not work properly. ### How was this patch tested? N/A Author: gatorsmile Closes #15494 from gatorsmile/addDocForSQLDebug. --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e73d0187b584b..a055e0135c136 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -934,8 +934,11 @@ object StaticSQLConf { .intConf .createWithDefault(4000) + // When enabling the debug, Spark SQL internal table properties are not filtered out; however, + // some related DDL commands (e.g., ANALYZE TABLE and CREATE TABLE LIKE) might not work properly. val DEBUG_MODE = buildConf("spark.sql.debug") .internal() + .doc("Only used for internal debugging. Not all functions are supported when it is enabled.") .booleanConf .createWithDefault(false) } From 56b0f5f4d1d7826737b81ebc4ec5dad83b6463e3 Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Sun, 16 Oct 2016 22:38:30 -0700 Subject: [PATCH 108/177] [MINOR][SQL] Add prettyName for current_database function ## What changes were proposed in this pull request? Added a `prettyname` for current_database function. ## How was this patch tested? Manually. Before: ``` scala> sql("select current_database()").show +-----------------+ |currentdatabase()| +-----------------+ | default| +-----------------+ ``` After: ``` scala> sql("select current_database()").show +------------------+ |current_database()| +------------------+ | default| +------------------+ ``` Author: Weiqing Yang Closes #15506 from weiqingy/prettyName. --- .../scala/org/apache/spark/sql/catalyst/expressions/misc.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 138ef2a1dcc01..5ead16908732f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -618,6 +618,7 @@ case class CurrentDatabase() extends LeafExpression with Unevaluable { override def dataType: DataType = StringType override def foldable: Boolean = true override def nullable: Boolean = false + override def prettyName: String = "current_database" } /** From e3bf37fa3ada43624b2e77bef90ad3d3dbcd8ce1 Mon Sep 17 00:00:00 2001 From: Maxime Rihouey Date: Mon, 17 Oct 2016 10:56:22 +0100 Subject: [PATCH 109/177] Fix example of tf_idf with minDocFreq ## What changes were proposed in this pull request? The python example for tf_idf with the parameter "minDocFreq" is not properly set up because the same variable is used to transform the document for both with and without the "minDocFreq" parameter. The IDF(minDocFreq=2) is stored in the variable "idfIgnore" but then it is the original variable "idf" used to transform the "tf" instead of the "idfIgnore". ## How was this patch tested? Before the results for "tfidf" and "tfidfIgnore" were the same: tfidf: (1048576,[1046921],[3.75828890549]) (1048576,[1046920],[3.75828890549]) (1048576,[1046923],[3.75828890549]) (1048576,[892732],[3.75828890549]) (1048576,[892733],[3.75828890549]) (1048576,[892734],[3.75828890549]) tfidfIgnore: (1048576,[1046921],[3.75828890549]) (1048576,[1046920],[3.75828890549]) (1048576,[1046923],[3.75828890549]) (1048576,[892732],[3.75828890549]) (1048576,[892733],[3.75828890549]) (1048576,[892734],[3.75828890549]) After the fix those are how they should be: tfidf: (1048576,[1046921],[3.75828890549]) (1048576,[1046920],[3.75828890549]) (1048576,[1046923],[3.75828890549]) (1048576,[892732],[3.75828890549]) (1048576,[892733],[3.75828890549]) (1048576,[892734],[3.75828890549]) tfidfIgnore: (1048576,[1046921],[0.0]) (1048576,[1046920],[0.0]) (1048576,[1046923],[0.0]) (1048576,[892732],[0.0]) (1048576,[892733],[0.0]) (1048576,[892734],[0.0]) Author: Maxime Rihouey Closes #15503 from maximerihouey/patch-1. --- examples/src/main/python/mllib/tf_idf_example.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/mllib/tf_idf_example.py b/examples/src/main/python/mllib/tf_idf_example.py index c4d53333a95a9..b66412b2334e7 100644 --- a/examples/src/main/python/mllib/tf_idf_example.py +++ b/examples/src/main/python/mllib/tf_idf_example.py @@ -43,7 +43,7 @@ # In such cases, the IDF for these terms is set to 0. # This feature can be used by passing the minDocFreq value to the IDF constructor. idfIgnore = IDF(minDocFreq=2).fit(tf) - tfidfIgnore = idf.transform(tf) + tfidfIgnore = idfIgnore.transform(tf) # $example off$ print("tfidf:") From c7ac027d5fd7a80d3122a9269b2bb9c28c6a57db Mon Sep 17 00:00:00 2001 From: Sital Kedia Date: Mon, 17 Oct 2016 11:03:04 -0700 Subject: [PATCH 110/177] [SPARK-17839][CORE] Use Nio's directbuffer instead of BufferedInputStream in order to avoid additional copy from os buffer cache to user buffer ## What changes were proposed in this pull request? Currently we use BufferedInputStream to read the shuffle file which copies the file content from os buffer cache to the user buffer. This adds additional latency in reading the spill files. We made a change to use java nio's direct buffer to read the spill files and for certain pipelines spilling significant amount of data, we see up to 7% speedup for the entire pipeline. ## How was this patch tested? Tested by running the job in the cluster and observed up to 7% speedup. Author: Sital Kedia Closes #15408 from sitalkedia/skedia/nio_spill_read. --- .../spark/io/NioBufferedFileInputStream.java | 137 ++++++++++++++++++ .../unsafe/sort/UnsafeSorterSpillReader.java | 5 +- .../shuffle/IndexShuffleBlockResolver.scala | 3 +- .../io/NioBufferedFileInputStreamSuite.java | 135 +++++++++++++++++ .../spark/sql/execution/python/RowQueue.scala | 3 +- 5 files changed, 279 insertions(+), 4 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java create mode 100644 core/src/test/java/org/apache/spark/io/NioBufferedFileInputStreamSuite.java diff --git a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java new file mode 100644 index 0000000000000..f6d1288cb263d --- /dev/null +++ b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java @@ -0,0 +1,137 @@ +/* + * Licensed 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.io; + +import org.apache.spark.storage.StorageUtils; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; + +/** + * {@link InputStream} implementation which uses direct buffer + * to read a file to avoid extra copy of data between Java and + * native memory which happens when using {@link java.io.BufferedInputStream}. + * Unfortunately, this is not something already available in JDK, + * {@link sun.nio.ch.ChannelInputStream} supports reading a file using nio, + * but does not support buffering. + */ +public final class NioBufferedFileInputStream extends InputStream { + + private static final int DEFAULT_BUFFER_SIZE_BYTES = 8192; + + private final ByteBuffer byteBuffer; + + private final FileChannel fileChannel; + + public NioBufferedFileInputStream(File file, int bufferSizeInBytes) throws IOException { + byteBuffer = ByteBuffer.allocateDirect(bufferSizeInBytes); + fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.READ); + byteBuffer.flip(); + } + + public NioBufferedFileInputStream(File file) throws IOException { + this(file, DEFAULT_BUFFER_SIZE_BYTES); + } + + /** + * Checks weather data is left to be read from the input stream. + * @return true if data is left, false otherwise + * @throws IOException + */ + private boolean refill() throws IOException { + if (!byteBuffer.hasRemaining()) { + byteBuffer.clear(); + int nRead = 0; + while (nRead == 0) { + nRead = fileChannel.read(byteBuffer); + } + if (nRead < 0) { + return false; + } + byteBuffer.flip(); + } + return true; + } + + @Override + public synchronized int read() throws IOException { + if (!refill()) { + return -1; + } + return byteBuffer.get() & 0xFF; + } + + @Override + public synchronized int read(byte[] b, int offset, int len) throws IOException { + if (offset < 0 || len < 0 || offset + len < 0 || offset + len > b.length) { + throw new IndexOutOfBoundsException(); + } + if (!refill()) { + return -1; + } + len = Math.min(len, byteBuffer.remaining()); + byteBuffer.get(b, offset, len); + return len; + } + + @Override + public synchronized int available() throws IOException { + return byteBuffer.remaining(); + } + + @Override + public synchronized long skip(long n) throws IOException { + if (n <= 0L) { + return 0L; + } + if (byteBuffer.remaining() >= n) { + // The buffered content is enough to skip + byteBuffer.position(byteBuffer.position() + (int) n); + return n; + } + long skippedFromBuffer = byteBuffer.remaining(); + long toSkipFromFileChannel = n - skippedFromBuffer; + // Discard everything we have read in the buffer. + byteBuffer.position(0); + byteBuffer.flip(); + return skippedFromBuffer + skipFromFileChannel(toSkipFromFileChannel); + } + + private long skipFromFileChannel(long n) throws IOException { + long currentFilePosition = fileChannel.position(); + long size = fileChannel.size(); + if (n > size - currentFilePosition) { + fileChannel.position(size); + return size - currentFilePosition; + } else { + fileChannel.position(currentFilePosition + n); + return n; + } + } + + @Override + public synchronized void close() throws IOException { + fileChannel.close(); + StorageUtils.dispose(byteBuffer); + } + + @Override + protected void finalize() throws IOException { + close(); + } +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java index e6d9766c31574..a658e5eb47b78 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -23,6 +23,7 @@ import com.google.common.io.Closeables; import org.apache.spark.SparkEnv; +import org.apache.spark.io.NioBufferedFileInputStream; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.BlockId; import org.apache.spark.unsafe.Platform; @@ -69,8 +70,8 @@ public UnsafeSorterSpillReader( bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES; } - final BufferedInputStream bs = - new BufferedInputStream(new FileInputStream(file), (int) bufferSizeBytes); + final InputStream bs = + new NioBufferedFileInputStream(file, (int) bufferSizeBytes); try { this.in = serializerManager.wrapStream(blockId, bs); this.din = new DataInputStream(this.in); diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 8d6396bededa9..91858f0912b65 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -23,6 +23,7 @@ import com.google.common.io.ByteStreams import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging +import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID @@ -89,7 +90,7 @@ private[spark] class IndexShuffleBlockResolver( val lengths = new Array[Long](blocks) // Read the lengths of blocks val in = try { - new DataInputStream(new BufferedInputStream(new FileInputStream(index))) + new DataInputStream(new NioBufferedFileInputStream(index)) } catch { case e: IOException => return null diff --git a/core/src/test/java/org/apache/spark/io/NioBufferedFileInputStreamSuite.java b/core/src/test/java/org/apache/spark/io/NioBufferedFileInputStreamSuite.java new file mode 100644 index 0000000000000..2c1a34a607592 --- /dev/null +++ b/core/src/test/java/org/apache/spark/io/NioBufferedFileInputStreamSuite.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.io; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + +import static org.junit.Assert.assertEquals; + +/** + * Tests functionality of {@link NioBufferedFileInputStream} + */ +public class NioBufferedFileInputStreamSuite { + + private byte[] randomBytes; + + private File inputFile; + + @Before + public void setUp() throws IOException { + // Create a byte array of size 2 MB with random bytes + randomBytes = RandomUtils.nextBytes(2 * 1024 * 1024); + inputFile = File.createTempFile("temp-file", ".tmp"); + FileUtils.writeByteArrayToFile(inputFile, randomBytes); + } + + @After + public void tearDown() { + inputFile.delete(); + } + + @Test + public void testReadOneByte() throws IOException { + InputStream inputStream = new NioBufferedFileInputStream(inputFile); + for (int i = 0; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testReadMultipleBytes() throws IOException { + InputStream inputStream = new NioBufferedFileInputStream(inputFile); + byte[] readBytes = new byte[8 * 1024]; + int i = 0; + while (i < randomBytes.length) { + int read = inputStream.read(readBytes, 0, 8 * 1024); + for (int j = 0; j < read; j++) { + assertEquals(randomBytes[i], readBytes[j]); + i++; + } + } + } + + @Test + public void testBytesSkipped() throws IOException { + InputStream inputStream = new NioBufferedFileInputStream(inputFile); + assertEquals(1024, inputStream.skip(1024)); + for (int i = 1024; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testBytesSkippedAfterRead() throws IOException { + InputStream inputStream = new NioBufferedFileInputStream(inputFile); + for (int i = 0; i < 1024; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + assertEquals(1024, inputStream.skip(1024)); + for (int i = 2048; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testNegativeBytesSkippedAfterRead() throws IOException { + InputStream inputStream = new NioBufferedFileInputStream(inputFile); + for (int i = 0; i < 1024; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + // Skipping negative bytes should essential be a no-op + assertEquals(0, inputStream.skip(-1)); + assertEquals(0, inputStream.skip(-1024)); + assertEquals(0, inputStream.skip(Long.MIN_VALUE)); + assertEquals(1024, inputStream.skip(1024)); + for (int i = 2048; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testSkipFromFileChannel() throws IOException { + InputStream inputStream = new NioBufferedFileInputStream(inputFile, 10); + // Since the buffer is smaller than the skipped bytes, this will guarantee + // we skip from underlying file channel. + assertEquals(1024, inputStream.skip(1024)); + for (int i = 1024; i < 2048; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + assertEquals(256, inputStream.skip(256)); + assertEquals(256, inputStream.skip(256)); + assertEquals(512, inputStream.skip(512)); + for (int i = 3072; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testBytesSkippedAfterEOF() throws IOException { + InputStream inputStream = new NioBufferedFileInputStream(inputFile); + assertEquals(randomBytes.length, inputStream.skip(randomBytes.length + 1)); + assertEquals(-1, inputStream.read()); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala index 422a3f862d96f..cd1e77f524afd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -22,6 +22,7 @@ import java.io._ import com.google.common.io.Closeables import org.apache.spark.SparkException +import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.Platform @@ -130,7 +131,7 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu if (out != null) { out.close() out = null - in = new DataInputStream(new BufferedInputStream(new FileInputStream(file.toString))) + in = new DataInputStream(new NioBufferedFileInputStream(file)) } if (unreadBytes > 0) { From d88a1bae6a9c975c39549ec2326d839ea93949b2 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 17 Oct 2016 11:33:06 -0700 Subject: [PATCH 111/177] [SPARK-17751][SQL] Remove spark.sql.eagerAnalysis and Output the Plan if Existed in AnalysisException ### What changes were proposed in this pull request? Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it. This PR also outputs the plan. Without the fix, the analysis error is like ``` cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12 ``` After the fix, the analysis error becomes: ``` org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12; 'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6] +- SubqueryAlias t +- Project [_1#2 AS k#5, _2#3 AS v#6] +- LocalRelation [_1#2, _2#3] ``` ### How was this patch tested? N/A Author: gatorsmile Closes #15316 from gatorsmile/eagerAnalysis. --- .../scala/org/apache/spark/sql/AnalysisException.scala | 7 +++++++ .../org/apache/spark/sql/execution/debug/package.scala | 9 --------- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 10 ---------- .../scala/org/apache/spark/sql/SQLQueryTestSuite.scala | 3 +++ 4 files changed, 10 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index f3003306acc6d..7defb9df862c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -42,6 +42,13 @@ class AnalysisException protected[sql] ( } override def getMessage: String = { + val planAnnotation = plan.map(p => s";\n$p").getOrElse("") + getSimpleMessage + planAnnotation + } + + // Outputs an exception without the logical plan. + // For testing only + def getSimpleMessage: String = { val lineAnnotation = line.map(l => s" line $l").getOrElse("") val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("") s"$message;$lineAnnotation$positionAnnotation" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index d321f4cd76877..dd9d83767e221 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -69,15 +69,6 @@ package object debug { output } - /** - * Augments [[SparkSession]] with debug methods. - */ - implicit class DebugSQLContext(sparkSession: SparkSession) { - def debug(): Unit = { - sparkSession.conf.set(SQLConf.DATAFRAME_EAGER_ANALYSIS.key, false) - } - } - /** * Augments [[Dataset]]s with debug methods. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a055e0135c136..8afd39d657865 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -388,14 +388,6 @@ object SQLConf { .intConf .createWithDefault(32) - // Whether to perform eager analysis when constructing a dataframe. - // Set to false when debugging requires the ability to look at invalid query plans. - val DATAFRAME_EAGER_ANALYSIS = SQLConfigBuilder("spark.sql.eagerAnalysis") - .internal() - .doc("When true, eagerly applies query analysis on DataFrame operations.") - .booleanConf - .createWithDefault(true) - // Whether to automatically resolve ambiguity in join conditions for self-joins. // See SPARK-6231. val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = @@ -748,8 +740,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def bucketingEnabled: Boolean = getConf(SQLConf.BUCKETING_ENABLED) - def dataFrameEagerAnalysis: Boolean = getConf(DATAFRAME_EAGER_ANALYSIS) - def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 55d5a56f1040a..02841d7bb03ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -220,6 +220,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) } catch { + case a: AnalysisException if a.plan.nonEmpty => + // Do not output the logical plan tree which contains expression IDs. + (StructType(Seq.empty), Seq(a.getClass.getName, a.getSimpleMessage)) case NonFatal(e) => // If there is an exception, put the exception class followed by the message. (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage)) From 813ab5e02539d17a66a6740d965b9f847d38c258 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 17 Oct 2016 20:46:30 -0700 Subject: [PATCH 112/177] [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables ## What changes were proposed in this pull request? Reopens the closed PR https://github.com/apache/spark/pull/15190 (Please refer to the above link for review comments on the PR) Make sure the hive.default.fileformat is used to when creating the storage format metadata. Output ``` SQL scala> spark.sql("SET hive.default.fileformat=orc") res1: org.apache.spark.sql.DataFrame = [key: string, value: string] scala> spark.sql("CREATE TABLE tmp_default(id INT)") res2: org.apache.spark.sql.DataFrame = [] ``` Before ```SQL scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println) .. [# Storage Information,,] [SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,] [InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,] [OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,] [Compressed:,No,] [Storage Desc Parameters:,,] [ serialization.format,1,] ``` After ```SQL scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println) .. [# Storage Information,,] [SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,] [InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,] [OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,] [Compressed:,No,] [Storage Desc Parameters:,,] [ serialization.format,1,] ``` ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Added new tests to HiveDDLCommandSuite, SQLQuerySuite Author: Dilip Biswal Closes #15495 from dilipbiswal/orc2. --- .../spark/sql/execution/SparkSqlParser.scala | 4 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 26 ++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 39 +++++++++++++++++-- 3 files changed, 60 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index be2eddbb0e423..8c68d1e3a2379 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1010,9 +1010,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { .orElse(Some("org.apache.hadoop.mapred.TextInputFormat")), outputFormat = defaultHiveSerde.flatMap(_.outputFormat) .orElse(Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")), - // Note: Keep this unspecified because we use the presence of the serde to decide - // whether to convert a table created by CTAS to a datasource table. - serde = None, + serde = defaultHiveSerde.flatMap(_.serde), compressed = false, properties = Map()) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 9ce3338647398..81337493c7f28 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -30,10 +30,12 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Generate, ScriptTransformation} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.CreateTable -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType -class HiveDDLCommandSuite extends PlanTest { +class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingleton { val parser = TestHive.sessionState.sqlParser private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { @@ -556,4 +558,24 @@ class HiveDDLCommandSuite extends PlanTest { assert(partition2.get.apply("c") == "1" && partition2.get.apply("d") == "2") } + test("Test the default fileformat for Hive-serde tables") { + withSQLConf("hive.default.fileformat" -> "orc") { + val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + assert(exists) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) + assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) + assert(desc.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } + + withSQLConf("hive.default.fileformat" -> "parquet") { + val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + assert(exists) + val input = desc.storage.inputFormat + val output = desc.storage.outputFormat + val serde = desc.storage.serde + assert(input == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(output == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + assert(serde == Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 6f2a16662bf10..e26b6b57ef569 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -492,7 +492,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { def checkRelation( tableName: String, - isDataSourceParquet: Boolean, + isDataSourceTable: Boolean, format: String, userSpecifiedLocation: Option[String] = None): Unit = { val relation = EliminateSubqueryAliases( @@ -501,7 +501,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) relation match { case LogicalRelation(r: HadoopFsRelation, _, _) => - if (!isDataSourceParquet) { + if (!isDataSourceTable) { fail( s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + s"${HadoopFsRelation.getClass.getCanonicalName}.") @@ -514,7 +514,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(catalogTable.provider.get === format) case r: MetastoreRelation => - if (isDataSourceParquet) { + if (isDataSourceTable) { fail( s"${HadoopFsRelation.getClass.getCanonicalName} is expected, but found " + s"${classOf[MetastoreRelation].getCanonicalName}.") @@ -524,8 +524,15 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(r.catalogTable.storage.locationUri.get === location) case None => // OK. } - // Also make sure that the format is the desired format. + // Also make sure that the format and serde are as desired. assert(catalogTable.storage.inputFormat.get.toLowerCase.contains(format)) + assert(catalogTable.storage.outputFormat.get.toLowerCase.contains(format)) + val serde = catalogTable.storage.serde.get + format match { + case "sequence" | "text" => assert(serde.contains("LazySimpleSerDe")) + case "rcfile" => assert(serde.contains("LazyBinaryColumnarSerDe")) + case _ => assert(serde.toLowerCase.contains(format)) + } } // When a user-specified location is defined, the table type needs to be EXTERNAL. @@ -587,6 +594,30 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("CTAS with default fileformat") { + val table = "ctas1" + val ctas = s"CREATE TABLE IF NOT EXISTS $table SELECT key k, value FROM src" + withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { + withSQLConf("hive.default.fileformat" -> "textfile") { + withTable(table) { + sql(ctas) + // We should use parquet here as that is the default datasource fileformat. The default + // datasource file format is controlled by `spark.sql.sources.default` configuration. + // This testcase verifies that setting `hive.default.fileformat` has no impact on + // the target table's fileformat in case of CTAS. + assert(sessionState.conf.defaultDataSourceName === "parquet") + checkRelation(tableName = table, isDataSourceTable = true, format = "parquet") + } + } + withSQLConf("spark.sql.sources.default" -> "orc") { + withTable(table) { + sql(ctas) + checkRelation(tableName = table, isDataSourceTable = true, format = "orc") + } + } + } + } + test("CTAS without serde with location") { withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { withTempDir { dir => From 8daa1a29b65a9b5337518458e9ece1619e8a01e3 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 17 Oct 2016 21:01:22 -0700 Subject: [PATCH 113/177] [SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree ## What changes were proposed in this pull request? This renames `BasicFileCatalog => FileCatalog`, combines `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait. In summary, ``` MetadataLogFileCatalog extends PartitioningAwareFileCatalog ListingFileCatalog extends PartitioningAwareFileCatalog PartitioningAwareFileCatalog extends FileCatalog TableFileCatalog extends FileCatalog ``` cc cloud-fan mallman ## How was this patch tested? Existing tests Author: Eric Liang Closes #15518 from ericl/refactor-session-file-catalog. --- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 4 +- .../execution/datasources/FileCatalog.scala | 66 +++++ .../execution/datasources/FileFormat.scala | 61 ----- .../datasources/HadoopFsRelation.scala | 4 +- .../PartitioningAwareFileCatalog.scala | 217 ++++++++++++++++- .../datasources/PartitioningUtils.scala | 12 +- .../datasources/SessionFileCatalog.scala | 225 ------------------ .../datasources/TableFileCatalog.scala | 11 +- .../datasources/FileCatalogSuite.scala | 10 + .../datasources/SessionFileCatalogSuite.scala | 34 --- .../ParquetPartitionDiscoverySuite.scala | 9 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 13 files changed, 303 insertions(+), 354 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 7dccbbd3f0a5b..073d2b1512b95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} -import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.streaming.{DataStreamWriter, StreamingQuery} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 623d2be55dcec..fdd1fa3648251 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -431,7 +431,7 @@ case class FileSourceScanExec( private def createBucketedReadRDD( bucketSpec: BucketSpec, readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[Partition], + selectedPartitions: Seq[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { logInfo(s"Planning with ${bucketSpec.numBuckets} buckets") val bucketed = @@ -463,7 +463,7 @@ case class FileSourceScanExec( */ private def createNonBucketedReadRDD( readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[Partition], + selectedPartitions: Seq[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { val defaultMaxSplitBytes = fsRelation.sparkSession.sessionState.conf.filesMaxPartitionBytes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala new file mode 100644 index 0000000000000..2bc66ceeebdb4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.hadoop.fs._ + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ + +/** + * A collection of data files from a partitioned relation, along with the partition values in the + * form of an [[InternalRow]]. + */ +case class PartitionDirectory(values: InternalRow, files: Seq[FileStatus]) + +/** + * An interface for objects capable of enumerating the root paths of a relation as well as the + * partitions of a relation subject to some pruning expressions. + */ +trait FileCatalog { + + /** + * Returns the list of root input paths from which the catalog will get files. There may be a + * single root path from which partitions are discovered, or individual partitions may be + * specified by each path. + */ + def rootPaths: Seq[Path] + + /** + * Returns all valid files grouped into partitions when the data is partitioned. If the data is + * unpartitioned, this will return a single partition with no partition values. + * + * @param filters The filters used to prune which partitions are returned. These filters must + * only refer to partition columns and this method will only return files + * where these predicates are guaranteed to evaluate to `true`. Thus, these + * filters will not need to be evaluated again on the returned data. + */ + def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] + + /** + * Returns the list of files that will be read when scanning this relation. This call may be + * very expensive for large tables. + */ + def inputFiles: Array[String] + + /** Refresh any cached file listings */ + def refresh(): Unit + + /** Sum of table file sizes, in bytes */ + def sizeInBytes: Long +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index e7239ef91b326..9d153cec731a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -175,64 +175,3 @@ abstract class TextBasedFileFormat extends FileFormat { codec == null || codec.isInstanceOf[SplittableCompressionCodec] } } - -/** - * A collection of data files from a partitioned relation, along with the partition values in the - * form of an [[InternalRow]]. - */ -case class Partition(values: InternalRow, files: Seq[FileStatus]) - -/** - * An interface for objects capable of enumerating the root paths of a relation as well as the - * partitions of a relation subject to some pruning expressions. - */ -trait BasicFileCatalog { - - /** - * Returns the list of root input paths from which the catalog will get files. There may be a - * single root path from which partitions are discovered, or individual partitions may be - * specified by each path. - */ - def rootPaths: Seq[Path] - - /** - * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with no partition values. - * - * @param filters The filters used to prune which partitions are returned. These filters must - * only refer to partition columns and this method will only return files - * where these predicates are guaranteed to evaluate to `true`. Thus, these - * filters will not need to be evaluated again on the returned data. - */ - def listFiles(filters: Seq[Expression]): Seq[Partition] - - /** Returns the list of files that will be read when scanning this relation. */ - def inputFiles: Array[String] - - /** Refresh any cached file listings */ - def refresh(): Unit - - /** Sum of table file sizes, in bytes */ - def sizeInBytes: Long -} - -/** - * A [[BasicFileCatalog]] which can enumerate all of the files comprising a relation and, from - * those, infer the relation's partition specification. - */ -// TODO: Consider a more descriptive, appropriate name which suggests this is a file catalog for -// which it is safe to list all of its files? -trait FileCatalog extends BasicFileCatalog { - - /** Returns the specification of the partitions inferred from the data. */ - def partitionSpec(): PartitionSpec - - /** Returns all the valid files. */ - def allFiles(): Seq[FileStatus] - - /** Returns the list of files that will be read when scanning this relation. */ - override def inputFiles: Array[String] = - allFiles().map(_.getPath.toUri.toString).toArray - - override def sizeInBytes: Long = allFiles().map(_.getLen).sum -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index db889edf032d6..afad8898089bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.StructType * Acts as a container for all of the metadata required to read from a datasource. All discovery, * resolution and merging logic for schemas and partitions has been removed. * - * @param location A [[BasicFileCatalog]] that can enumerate the locations of all the files that + * @param location A [[FileCatalog]] that can enumerate the locations of all the files that * comprise this relation. * @param partitionSchema The schema of the columns (if any) that are used to partition the relation * @param dataSchema The schema of any remaining columns. Note that if any partition columns are @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType * @param options Configuration used when reading / writing data. */ case class HadoopFsRelation( - location: BasicFileCatalog, + location: FileCatalog, partitionSchema: StructType, dataSchema: StructType, bucketSpec: Option[BucketSpec], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index b2508115c282f..5c8eff7ec46b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -17,14 +17,21 @@ package org.apache.spark.sql.execution.datasources +import java.io.FileNotFoundException + import scala.collection.mutable -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.SerializableConfiguration /** @@ -38,22 +45,24 @@ import org.apache.spark.sql.types.{StringType, StructType} abstract class PartitioningAwareFileCatalog( sparkSession: SparkSession, parameters: Map[String, String], - partitionSchema: Option[StructType]) - extends SessionFileCatalog(sparkSession) with FileCatalog { + partitionSchema: Option[StructType]) extends FileCatalog with Logging { import PartitioningAwareFileCatalog.BASE_PATH_PARAM - override protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) + /** Returns the specification of the partitions inferred from the data. */ + def partitionSpec(): PartitionSpec + + protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] - override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) { - Partition(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil + PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil } else { prunePartitions(filters, partitionSpec()).map { - case PartitionDirectory(values, path) => + case PartitionPath(values, path) => val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match { case Some(existingDir) => // Directory has children files in it, return them @@ -63,14 +72,20 @@ abstract class PartitioningAwareFileCatalog( // Directory does not exist, or has no children files Nil } - Partition(values, files) + PartitionDirectory(values, files) } } logTrace("Selected files after partition pruning:\n\t" + selectedPartitions.mkString("\n\t")) selectedPartitions } - override def allFiles(): Seq[FileStatus] = { + /** Returns the list of files that will be read when scanning this relation. */ + override def inputFiles: Array[String] = + allFiles().map(_.getPath.toUri.toString).toArray + + override def sizeInBytes: Long = allFiles().map(_.getLen).sum + + def allFiles(): Seq[FileStatus] = { if (partitionSpec().partitionColumns.isEmpty) { // For each of the root input paths, get the list of files inside them rootPaths.flatMap { path => @@ -139,7 +154,7 @@ abstract class PartitioningAwareFileCatalog( private def prunePartitions( predicates: Seq[Expression], - partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { + partitionSpec: PartitionSpec): Seq[PartitionPath] = { val PartitionSpec(partitionColumns, partitions) = partitionSpec val partitionColumnNames = partitionColumns.map(_.name).toSet val partitionPruningPredicates = predicates.filter { @@ -156,7 +171,7 @@ abstract class PartitioningAwareFileCatalog( }) val selected = partitions.filter { - case PartitionDirectory(values, _) => boundPredicate(values) + case PartitionPath(values, _) => boundPredicate(values) } logInfo { val total = partitions.length @@ -214,8 +229,186 @@ abstract class PartitioningAwareFileCatalog( val name = path.getName !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } + + /** + * List leaf files of given paths. This method will submit a Spark job to do parallel + * listing whenever there is a path having more files than the parallel partition discovery + * discovery threshold. + * + * This is publicly visible for testing. + */ + def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + val files = + if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + PartitioningAwareFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) + } else { + PartitioningAwareFileCatalog.listLeafFilesInSerial(paths, hadoopConf) + } + + HiveCatalogMetrics.incrementFilesDiscovered(files.size) + mutable.LinkedHashSet(files: _*) + } } -object PartitioningAwareFileCatalog { +object PartitioningAwareFileCatalog extends Logging { val BASE_PATH_PARAM = "basePath" + + /** A serializable variant of HDFS's BlockLocation. */ + private case class SerializableBlockLocation( + names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + + /** A serializable variant of HDFS's FileStatus. */ + private case class SerializableFileStatus( + path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long, + blockLocations: Array[SerializableBlockLocation]) + + /** + * List a collection of path recursively. + */ + private def listLeafFilesInSerial( + paths: Seq[Path], + hadoopConf: Configuration): Seq[FileStatus] = { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val filter = FileInputFormat.getInputPathFilter(jobConf) + + paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + listLeafFiles0(fs, path, filter) + } + } + + /** + * List a collection of path recursively in parallel (using Spark executors). + * Each task launched will use [[listLeafFilesInSerial]] to list. + */ + private def listLeafFilesInParallel( + paths: Seq[Path], + hadoopConf: Configuration, + sparkSession: SparkSession): Seq[FileStatus] = { + assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) + logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") + + val sparkContext = sparkSession.sparkContext + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, 10000) + + val statuses = sparkContext + .parallelize(serializedPaths, numParallelism) + .mapPartitions { paths => + val hadoopConf = serializableConfiguration.value + listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator + }.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + }.collect() + + // Turn SerializableFileStatus back to Status + statuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), + blockLocations) + } + } + + /** + * List a single path, provided as a FileStatus, in serial. + */ + private def listLeafFiles0( + fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { + logTrace(s"Listing $path") + val name = path.getName.toLowerCase + if (shouldFilterOut(name)) { + Seq.empty[FileStatus] + } else { + // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses = try fs.listStatus(path) catch { + case _: FileNotFoundException => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + val allLeafStatuses = { + val (dirs, files) = statuses.partition(_.isDirectory) + val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) + if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats + } + + allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { + case f: LocatedFileStatus => + f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. + case f => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + val locations = fs.getFileBlockLocations(f, 0, f.getLen) + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs + } + } + } + + /** Checks if we should filter out this path name. */ + def shouldFilterOut(pathName: String): Boolean = { + // We filter everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && + !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 504464216e5a4..ac6795b9a2e7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -33,8 +33,8 @@ import org.apache.spark.sql.types._ // TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. -object PartitionDirectory { - def apply(values: InternalRow, path: String): PartitionDirectory = +object PartitionPath { + def apply(values: InternalRow, path: String): PartitionPath = apply(values, new Path(path)) } @@ -42,14 +42,14 @@ object PartitionDirectory { * Holds a directory in a partitioned collection of files as well as as the partition values * in the form of a Row. Before scanning, the files at `path` need to be enumerated. */ -case class PartitionDirectory(values: InternalRow, path: Path) +case class PartitionPath(values: InternalRow, path: Path) case class PartitionSpec( partitionColumns: StructType, - partitions: Seq[PartitionDirectory]) + partitions: Seq[PartitionPath]) object PartitionSpec { - val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionDirectory]) + val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionPath]) } object PartitioningUtils { @@ -141,7 +141,7 @@ object PartitioningUtils { // Finally, we create `Partition`s based on paths and resolved partition values. val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map { case (PartitionValues(_, literals), (path, _)) => - PartitionDirectory(InternalRow.fromSeq(literals.map(_.value)), path) + PartitionPath(InternalRow.fromSeq(literals.map(_.value)), path) } PartitionSpec(StructType(fields), partitions) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala deleted file mode 100644 index 4807a92c2e6b8..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala +++ /dev/null @@ -1,225 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import java.io.FileNotFoundException - -import scala.collection.mutable - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs._ -import org.apache.hadoop.mapred.{FileInputFormat, JobConf} - -import org.apache.spark.internal.Logging -import org.apache.spark.metrics.source.HiveCatalogMetrics -import org.apache.spark.sql.SparkSession -import org.apache.spark.util.SerializableConfiguration - - -/** - * A base class for [[BasicFileCatalog]]s that need a [[SparkSession]] and the ability to find leaf - * files in a list of HDFS paths. - * - * @param sparkSession a [[SparkSession]] - * @param ignoreFileNotFound (see [[ListingFileCatalog]]) - */ -abstract class SessionFileCatalog(sparkSession: SparkSession) - extends BasicFileCatalog with Logging { - protected val hadoopConf: Configuration - - /** - * List leaf files of given paths. This method will submit a Spark job to do parallel - * listing whenever there is a path having more files than the parallel partition discovery - * discovery threshold. - * - * This is publicly visible for testing. - */ - def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - val files = - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - SessionFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) - } else { - SessionFileCatalog.listLeafFilesInSerial(paths, hadoopConf) - } - - HiveCatalogMetrics.incrementFilesDiscovered(files.size) - mutable.LinkedHashSet(files: _*) - } -} - -object SessionFileCatalog extends Logging { - - /** A serializable variant of HDFS's BlockLocation. */ - private case class SerializableBlockLocation( - names: Array[String], - hosts: Array[String], - offset: Long, - length: Long) - - /** A serializable variant of HDFS's FileStatus. */ - private case class SerializableFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long, - blockLocations: Array[SerializableBlockLocation]) - - /** - * List a collection of path recursively. - */ - private def listLeafFilesInSerial( - paths: Seq[Path], - hadoopConf: Configuration): Seq[FileStatus] = { - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val filter = FileInputFormat.getInputPathFilter(jobConf) - - paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - listLeafFiles0(fs, path, filter) - } - } - - /** - * List a collection of path recursively in parallel (using Spark executors). - * Each task launched will use [[listLeafFilesInSerial]] to list. - */ - private def listLeafFilesInParallel( - paths: Seq[Path], - hadoopConf: Configuration, - sparkSession: SparkSession): Seq[FileStatus] = { - assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) - logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") - - val sparkContext = sparkSession.sparkContext - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val serializedPaths = paths.map(_.toString) - - // Set the number of parallelism to prevent following file listing from generating many tasks - // in case of large #defaultParallelism. - val numParallelism = Math.min(paths.size, 10000) - - val statuses = sparkContext - .parallelize(serializedPaths, numParallelism) - .mapPartitions { paths => - val hadoopConf = serializableConfiguration.value - listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator - }.map { status => - // Turn FileStatus into SerializableFileStatus so we can send it back to the driver - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - SerializableBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[SerializableBlockLocation] - } - - SerializableFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) - }.collect() - - // Turn SerializableFileStatus back to Status - statuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) - } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), - blockLocations) - } - } - - /** - * List a single path, provided as a FileStatus, in serial. - */ - private def listLeafFiles0( - fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { - logTrace(s"Listing $path") - val name = path.getName.toLowerCase - if (shouldFilterOut(name)) { - Seq.empty[FileStatus] - } else { - // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist - // Note that statuses only include FileStatus for the files and dirs directly under path, - // and does not include anything else recursively. - val statuses = try fs.listStatus(path) catch { - case _: FileNotFoundException => - logWarning(s"The directory $path was not found. Was it deleted very recently?") - Array.empty[FileStatus] - } - - val allLeafStatuses = { - val (dirs, files) = statuses.partition(_.isDirectory) - val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) - if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats - } - - allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { - case f: LocatedFileStatus => - f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of - // paths exceeds threshold. - case f => - // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), - // which is very slow on some file system (RawLocalFileSystem, which is launch a - // subprocess and parse the stdout). - val locations = fs.getFileBlockLocations(f, 0, f.getLen) - val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, - f.getModificationTime, 0, null, null, null, null, f.getPath, locations) - if (f.isSymlink) { - lfs.setSymlink(f.getSymlink) - } - lfs - } - } - } - - /** Checks if we should filter out this path name. */ - def shouldFilterOut(pathName: String): Boolean = { - // We filter everything that starts with _ and ., except _common_metadata and _metadata - // because Parquet needs to find those metadata files from leaf files returned by this method. - // We should refactor this logic to not mix metadata files with data files. - ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && - !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala index a5c41b244589b..5648ab480a98a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types.StructType /** - * A [[BasicFileCatalog]] for a metastore catalog table. + * A [[FileCatalog]] for a metastore catalog table. * * @param sparkSession a [[SparkSession]] * @param db the table's database name @@ -38,10 +38,9 @@ class TableFileCatalog( db: String, table: String, partitionSchema: Option[StructType], - override val sizeInBytes: Long) - extends SessionFileCatalog(sparkSession) { + override val sizeInBytes: Long) extends FileCatalog { - override protected val hadoopConf = sparkSession.sessionState.newHadoopConf + protected val hadoopConf = sparkSession.sessionState.newHadoopConf private val externalCatalog = sparkSession.sharedState.externalCatalog @@ -51,7 +50,7 @@ class TableFileCatalog( override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq - override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { filterPartitions(filters).listFiles(Nil) } @@ -79,7 +78,7 @@ class TableFileCatalog( case Some(schema) => val selectedPartitions = externalCatalog.listPartitionsByFilter(db, table, filters) val partitions = selectedPartitions.map { p => - PartitionDirectory(p.toRow(schema), p.storage.locationUri.get) + PartitionPath(p.toRow(schema), p.storage.locationUri.get) } val partitionSpec = PartitionSpec(schema, partitions) new PrunedTableFileCatalog( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 2695974b84b00..9c43169cbf898 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -81,6 +81,16 @@ class FileCatalogSuite extends SharedSQLContext { } } + test("PartitioningAwareFileCatalog - file filtering") { + assert(!PartitioningAwareFileCatalog.shouldFilterOut("abcd")) + assert(PartitioningAwareFileCatalog.shouldFilterOut(".ab")) + assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd")) + assert(!PartitioningAwareFileCatalog.shouldFilterOut("_metadata")) + assert(!PartitioningAwareFileCatalog.shouldFilterOut("_common_metadata")) + assert(PartitioningAwareFileCatalog.shouldFilterOut("_ab_metadata")) + assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd_common_metadata")) + } + test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { class MockCatalog( override val rootPaths: Seq[Path]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala deleted file mode 100644 index df509583377ae..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import org.apache.spark.SparkFunSuite - -class SessionFileCatalogSuite extends SparkFunSuite { - - test("file filtering") { - assert(!SessionFileCatalog.shouldFilterOut("abcd")) - assert(SessionFileCatalog.shouldFilterOut(".ab")) - assert(SessionFileCatalog.shouldFilterOut("_cd")) - - assert(!SessionFileCatalog.shouldFilterOut("_metadata")) - assert(!SessionFileCatalog.shouldFilterOut("_common_metadata")) - assert(SessionFileCatalog.shouldFilterOut("_ab_metadata")) - assert(SessionFileCatalog.shouldFilterOut("_cd_common_metadata")) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 43357c97c395a..36d4df0015ffd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -30,7 +30,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitionPath => Partition, PartitioningAwareFileCatalog, PartitioningUtils, PartitionSpec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -626,10 +626,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(HadoopFsRelation(location: FileCatalog, _, _, _, _, _), _, _) => - assert(location.partitionSpec === PartitionSpec.emptySpec) + case LogicalRelation( + HadoopFsRelation(location: PartitioningAwareFileCatalog, _, _, _, _, _), _, _) => + assert(location.partitionSpec() === PartitionSpec.emptySpec) }.getOrElse { - fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") + fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 4a2aaa7d4f6ca..16e1e37b2fb02 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.{Partition => _, _} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ From 1c5a7d7f64993540baa5558be80130ee6911ba3c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 17 Oct 2016 21:26:28 -0700 Subject: [PATCH 114/177] Revert "[SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree" This reverts commit 8daa1a29b65a9b5337518458e9ece1619e8a01e3. --- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 4 +- .../execution/datasources/FileCatalog.scala | 66 ----- .../execution/datasources/FileFormat.scala | 61 +++++ .../datasources/HadoopFsRelation.scala | 4 +- .../PartitioningAwareFileCatalog.scala | 217 +---------------- .../datasources/PartitioningUtils.scala | 12 +- .../datasources/SessionFileCatalog.scala | 225 ++++++++++++++++++ .../datasources/TableFileCatalog.scala | 11 +- .../datasources/FileCatalogSuite.scala | 10 - .../datasources/SessionFileCatalogSuite.scala | 34 +++ .../ParquetPartitionDiscoverySuite.scala | 9 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 13 files changed, 354 insertions(+), 303 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 073d2b1512b95..7dccbbd3f0a5b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.streaming.{DataStreamWriter, StreamingQuery} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index fdd1fa3648251..623d2be55dcec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -431,7 +431,7 @@ case class FileSourceScanExec( private def createBucketedReadRDD( bucketSpec: BucketSpec, readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[PartitionDirectory], + selectedPartitions: Seq[Partition], fsRelation: HadoopFsRelation): RDD[InternalRow] = { logInfo(s"Planning with ${bucketSpec.numBuckets} buckets") val bucketed = @@ -463,7 +463,7 @@ case class FileSourceScanExec( */ private def createNonBucketedReadRDD( readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[PartitionDirectory], + selectedPartitions: Seq[Partition], fsRelation: HadoopFsRelation): RDD[InternalRow] = { val defaultMaxSplitBytes = fsRelation.sparkSession.sessionState.conf.filesMaxPartitionBytes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala deleted file mode 100644 index 2bc66ceeebdb4..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import org.apache.hadoop.fs._ - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ - -/** - * A collection of data files from a partitioned relation, along with the partition values in the - * form of an [[InternalRow]]. - */ -case class PartitionDirectory(values: InternalRow, files: Seq[FileStatus]) - -/** - * An interface for objects capable of enumerating the root paths of a relation as well as the - * partitions of a relation subject to some pruning expressions. - */ -trait FileCatalog { - - /** - * Returns the list of root input paths from which the catalog will get files. There may be a - * single root path from which partitions are discovered, or individual partitions may be - * specified by each path. - */ - def rootPaths: Seq[Path] - - /** - * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with no partition values. - * - * @param filters The filters used to prune which partitions are returned. These filters must - * only refer to partition columns and this method will only return files - * where these predicates are guaranteed to evaluate to `true`. Thus, these - * filters will not need to be evaluated again on the returned data. - */ - def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] - - /** - * Returns the list of files that will be read when scanning this relation. This call may be - * very expensive for large tables. - */ - def inputFiles: Array[String] - - /** Refresh any cached file listings */ - def refresh(): Unit - - /** Sum of table file sizes, in bytes */ - def sizeInBytes: Long -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 9d153cec731a8..e7239ef91b326 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -175,3 +175,64 @@ abstract class TextBasedFileFormat extends FileFormat { codec == null || codec.isInstanceOf[SplittableCompressionCodec] } } + +/** + * A collection of data files from a partitioned relation, along with the partition values in the + * form of an [[InternalRow]]. + */ +case class Partition(values: InternalRow, files: Seq[FileStatus]) + +/** + * An interface for objects capable of enumerating the root paths of a relation as well as the + * partitions of a relation subject to some pruning expressions. + */ +trait BasicFileCatalog { + + /** + * Returns the list of root input paths from which the catalog will get files. There may be a + * single root path from which partitions are discovered, or individual partitions may be + * specified by each path. + */ + def rootPaths: Seq[Path] + + /** + * Returns all valid files grouped into partitions when the data is partitioned. If the data is + * unpartitioned, this will return a single partition with no partition values. + * + * @param filters The filters used to prune which partitions are returned. These filters must + * only refer to partition columns and this method will only return files + * where these predicates are guaranteed to evaluate to `true`. Thus, these + * filters will not need to be evaluated again on the returned data. + */ + def listFiles(filters: Seq[Expression]): Seq[Partition] + + /** Returns the list of files that will be read when scanning this relation. */ + def inputFiles: Array[String] + + /** Refresh any cached file listings */ + def refresh(): Unit + + /** Sum of table file sizes, in bytes */ + def sizeInBytes: Long +} + +/** + * A [[BasicFileCatalog]] which can enumerate all of the files comprising a relation and, from + * those, infer the relation's partition specification. + */ +// TODO: Consider a more descriptive, appropriate name which suggests this is a file catalog for +// which it is safe to list all of its files? +trait FileCatalog extends BasicFileCatalog { + + /** Returns the specification of the partitions inferred from the data. */ + def partitionSpec(): PartitionSpec + + /** Returns all the valid files. */ + def allFiles(): Seq[FileStatus] + + /** Returns the list of files that will be read when scanning this relation. */ + override def inputFiles: Array[String] = + allFiles().map(_.getPath.toUri.toString).toArray + + override def sizeInBytes: Long = allFiles().map(_.getLen).sum +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index afad8898089bd..db889edf032d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.StructType * Acts as a container for all of the metadata required to read from a datasource. All discovery, * resolution and merging logic for schemas and partitions has been removed. * - * @param location A [[FileCatalog]] that can enumerate the locations of all the files that + * @param location A [[BasicFileCatalog]] that can enumerate the locations of all the files that * comprise this relation. * @param partitionSchema The schema of the columns (if any) that are used to partition the relation * @param dataSchema The schema of any remaining columns. Note that if any partition columns are @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType * @param options Configuration used when reading / writing data. */ case class HadoopFsRelation( - location: FileCatalog, + location: BasicFileCatalog, partitionSchema: StructType, dataSchema: StructType, bucketSpec: Option[BucketSpec], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index 5c8eff7ec46b4..b2508115c282f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -17,21 +17,14 @@ package org.apache.spark.sql.execution.datasources -import java.io.FileNotFoundException - import scala.collection.mutable -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs._ -import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.spark.internal.Logging -import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StringType, StructType} -import org.apache.spark.util.SerializableConfiguration /** @@ -45,24 +38,22 @@ import org.apache.spark.util.SerializableConfiguration abstract class PartitioningAwareFileCatalog( sparkSession: SparkSession, parameters: Map[String, String], - partitionSchema: Option[StructType]) extends FileCatalog with Logging { + partitionSchema: Option[StructType]) + extends SessionFileCatalog(sparkSession) with FileCatalog { import PartitioningAwareFileCatalog.BASE_PATH_PARAM - /** Returns the specification of the partitions inferred from the data. */ - def partitionSpec(): PartitionSpec - - protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) + override protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] - override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { + override def listFiles(filters: Seq[Expression]): Seq[Partition] = { val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) { - PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil + Partition(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil } else { prunePartitions(filters, partitionSpec()).map { - case PartitionPath(values, path) => + case PartitionDirectory(values, path) => val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match { case Some(existingDir) => // Directory has children files in it, return them @@ -72,20 +63,14 @@ abstract class PartitioningAwareFileCatalog( // Directory does not exist, or has no children files Nil } - PartitionDirectory(values, files) + Partition(values, files) } } logTrace("Selected files after partition pruning:\n\t" + selectedPartitions.mkString("\n\t")) selectedPartitions } - /** Returns the list of files that will be read when scanning this relation. */ - override def inputFiles: Array[String] = - allFiles().map(_.getPath.toUri.toString).toArray - - override def sizeInBytes: Long = allFiles().map(_.getLen).sum - - def allFiles(): Seq[FileStatus] = { + override def allFiles(): Seq[FileStatus] = { if (partitionSpec().partitionColumns.isEmpty) { // For each of the root input paths, get the list of files inside them rootPaths.flatMap { path => @@ -154,7 +139,7 @@ abstract class PartitioningAwareFileCatalog( private def prunePartitions( predicates: Seq[Expression], - partitionSpec: PartitionSpec): Seq[PartitionPath] = { + partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { val PartitionSpec(partitionColumns, partitions) = partitionSpec val partitionColumnNames = partitionColumns.map(_.name).toSet val partitionPruningPredicates = predicates.filter { @@ -171,7 +156,7 @@ abstract class PartitioningAwareFileCatalog( }) val selected = partitions.filter { - case PartitionPath(values, _) => boundPredicate(values) + case PartitionDirectory(values, _) => boundPredicate(values) } logInfo { val total = partitions.length @@ -229,186 +214,8 @@ abstract class PartitioningAwareFileCatalog( val name = path.getName !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } - - /** - * List leaf files of given paths. This method will submit a Spark job to do parallel - * listing whenever there is a path having more files than the parallel partition discovery - * discovery threshold. - * - * This is publicly visible for testing. - */ - def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - val files = - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - PartitioningAwareFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) - } else { - PartitioningAwareFileCatalog.listLeafFilesInSerial(paths, hadoopConf) - } - - HiveCatalogMetrics.incrementFilesDiscovered(files.size) - mutable.LinkedHashSet(files: _*) - } } -object PartitioningAwareFileCatalog extends Logging { +object PartitioningAwareFileCatalog { val BASE_PATH_PARAM = "basePath" - - /** A serializable variant of HDFS's BlockLocation. */ - private case class SerializableBlockLocation( - names: Array[String], - hosts: Array[String], - offset: Long, - length: Long) - - /** A serializable variant of HDFS's FileStatus. */ - private case class SerializableFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long, - blockLocations: Array[SerializableBlockLocation]) - - /** - * List a collection of path recursively. - */ - private def listLeafFilesInSerial( - paths: Seq[Path], - hadoopConf: Configuration): Seq[FileStatus] = { - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val filter = FileInputFormat.getInputPathFilter(jobConf) - - paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - listLeafFiles0(fs, path, filter) - } - } - - /** - * List a collection of path recursively in parallel (using Spark executors). - * Each task launched will use [[listLeafFilesInSerial]] to list. - */ - private def listLeafFilesInParallel( - paths: Seq[Path], - hadoopConf: Configuration, - sparkSession: SparkSession): Seq[FileStatus] = { - assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) - logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") - - val sparkContext = sparkSession.sparkContext - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val serializedPaths = paths.map(_.toString) - - // Set the number of parallelism to prevent following file listing from generating many tasks - // in case of large #defaultParallelism. - val numParallelism = Math.min(paths.size, 10000) - - val statuses = sparkContext - .parallelize(serializedPaths, numParallelism) - .mapPartitions { paths => - val hadoopConf = serializableConfiguration.value - listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator - }.map { status => - // Turn FileStatus into SerializableFileStatus so we can send it back to the driver - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - SerializableBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[SerializableBlockLocation] - } - - SerializableFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) - }.collect() - - // Turn SerializableFileStatus back to Status - statuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) - } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), - blockLocations) - } - } - - /** - * List a single path, provided as a FileStatus, in serial. - */ - private def listLeafFiles0( - fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { - logTrace(s"Listing $path") - val name = path.getName.toLowerCase - if (shouldFilterOut(name)) { - Seq.empty[FileStatus] - } else { - // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist - // Note that statuses only include FileStatus for the files and dirs directly under path, - // and does not include anything else recursively. - val statuses = try fs.listStatus(path) catch { - case _: FileNotFoundException => - logWarning(s"The directory $path was not found. Was it deleted very recently?") - Array.empty[FileStatus] - } - - val allLeafStatuses = { - val (dirs, files) = statuses.partition(_.isDirectory) - val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) - if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats - } - - allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { - case f: LocatedFileStatus => - f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of - // paths exceeds threshold. - case f => - // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), - // which is very slow on some file system (RawLocalFileSystem, which is launch a - // subprocess and parse the stdout). - val locations = fs.getFileBlockLocations(f, 0, f.getLen) - val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, - f.getModificationTime, 0, null, null, null, null, f.getPath, locations) - if (f.isSymlink) { - lfs.setSymlink(f.getSymlink) - } - lfs - } - } - } - - /** Checks if we should filter out this path name. */ - def shouldFilterOut(pathName: String): Boolean = { - // We filter everything that starts with _ and ., except _common_metadata and _metadata - // because Parquet needs to find those metadata files from leaf files returned by this method. - // We should refactor this logic to not mix metadata files with data files. - ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && - !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index ac6795b9a2e7b..504464216e5a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -33,8 +33,8 @@ import org.apache.spark.sql.types._ // TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. -object PartitionPath { - def apply(values: InternalRow, path: String): PartitionPath = +object PartitionDirectory { + def apply(values: InternalRow, path: String): PartitionDirectory = apply(values, new Path(path)) } @@ -42,14 +42,14 @@ object PartitionPath { * Holds a directory in a partitioned collection of files as well as as the partition values * in the form of a Row. Before scanning, the files at `path` need to be enumerated. */ -case class PartitionPath(values: InternalRow, path: Path) +case class PartitionDirectory(values: InternalRow, path: Path) case class PartitionSpec( partitionColumns: StructType, - partitions: Seq[PartitionPath]) + partitions: Seq[PartitionDirectory]) object PartitionSpec { - val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionPath]) + val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionDirectory]) } object PartitioningUtils { @@ -141,7 +141,7 @@ object PartitioningUtils { // Finally, we create `Partition`s based on paths and resolved partition values. val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map { case (PartitionValues(_, literals), (path, _)) => - PartitionPath(InternalRow.fromSeq(literals.map(_.value)), path) + PartitionDirectory(InternalRow.fromSeq(literals.map(_.value)), path) } PartitionSpec(StructType(fields), partitions) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala new file mode 100644 index 0000000000000..4807a92c2e6b8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.SerializableConfiguration + + +/** + * A base class for [[BasicFileCatalog]]s that need a [[SparkSession]] and the ability to find leaf + * files in a list of HDFS paths. + * + * @param sparkSession a [[SparkSession]] + * @param ignoreFileNotFound (see [[ListingFileCatalog]]) + */ +abstract class SessionFileCatalog(sparkSession: SparkSession) + extends BasicFileCatalog with Logging { + protected val hadoopConf: Configuration + + /** + * List leaf files of given paths. This method will submit a Spark job to do parallel + * listing whenever there is a path having more files than the parallel partition discovery + * discovery threshold. + * + * This is publicly visible for testing. + */ + def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + val files = + if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + SessionFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) + } else { + SessionFileCatalog.listLeafFilesInSerial(paths, hadoopConf) + } + + HiveCatalogMetrics.incrementFilesDiscovered(files.size) + mutable.LinkedHashSet(files: _*) + } +} + +object SessionFileCatalog extends Logging { + + /** A serializable variant of HDFS's BlockLocation. */ + private case class SerializableBlockLocation( + names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + + /** A serializable variant of HDFS's FileStatus. */ + private case class SerializableFileStatus( + path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long, + blockLocations: Array[SerializableBlockLocation]) + + /** + * List a collection of path recursively. + */ + private def listLeafFilesInSerial( + paths: Seq[Path], + hadoopConf: Configuration): Seq[FileStatus] = { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val filter = FileInputFormat.getInputPathFilter(jobConf) + + paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + listLeafFiles0(fs, path, filter) + } + } + + /** + * List a collection of path recursively in parallel (using Spark executors). + * Each task launched will use [[listLeafFilesInSerial]] to list. + */ + private def listLeafFilesInParallel( + paths: Seq[Path], + hadoopConf: Configuration, + sparkSession: SparkSession): Seq[FileStatus] = { + assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) + logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") + + val sparkContext = sparkSession.sparkContext + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, 10000) + + val statuses = sparkContext + .parallelize(serializedPaths, numParallelism) + .mapPartitions { paths => + val hadoopConf = serializableConfiguration.value + listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator + }.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + }.collect() + + // Turn SerializableFileStatus back to Status + statuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), + blockLocations) + } + } + + /** + * List a single path, provided as a FileStatus, in serial. + */ + private def listLeafFiles0( + fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { + logTrace(s"Listing $path") + val name = path.getName.toLowerCase + if (shouldFilterOut(name)) { + Seq.empty[FileStatus] + } else { + // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses = try fs.listStatus(path) catch { + case _: FileNotFoundException => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + val allLeafStatuses = { + val (dirs, files) = statuses.partition(_.isDirectory) + val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) + if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats + } + + allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { + case f: LocatedFileStatus => + f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. + case f => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + val locations = fs.getFileBlockLocations(f, 0, f.getLen) + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs + } + } + } + + /** Checks if we should filter out this path name. */ + def shouldFilterOut(pathName: String): Boolean = { + // We filter everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && + !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala index 5648ab480a98a..a5c41b244589b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types.StructType /** - * A [[FileCatalog]] for a metastore catalog table. + * A [[BasicFileCatalog]] for a metastore catalog table. * * @param sparkSession a [[SparkSession]] * @param db the table's database name @@ -38,9 +38,10 @@ class TableFileCatalog( db: String, table: String, partitionSchema: Option[StructType], - override val sizeInBytes: Long) extends FileCatalog { + override val sizeInBytes: Long) + extends SessionFileCatalog(sparkSession) { - protected val hadoopConf = sparkSession.sessionState.newHadoopConf + override protected val hadoopConf = sparkSession.sessionState.newHadoopConf private val externalCatalog = sparkSession.sharedState.externalCatalog @@ -50,7 +51,7 @@ class TableFileCatalog( override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq - override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { + override def listFiles(filters: Seq[Expression]): Seq[Partition] = { filterPartitions(filters).listFiles(Nil) } @@ -78,7 +79,7 @@ class TableFileCatalog( case Some(schema) => val selectedPartitions = externalCatalog.listPartitionsByFilter(db, table, filters) val partitions = selectedPartitions.map { p => - PartitionPath(p.toRow(schema), p.storage.locationUri.get) + PartitionDirectory(p.toRow(schema), p.storage.locationUri.get) } val partitionSpec = PartitionSpec(schema, partitions) new PrunedTableFileCatalog( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 9c43169cbf898..2695974b84b00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -81,16 +81,6 @@ class FileCatalogSuite extends SharedSQLContext { } } - test("PartitioningAwareFileCatalog - file filtering") { - assert(!PartitioningAwareFileCatalog.shouldFilterOut("abcd")) - assert(PartitioningAwareFileCatalog.shouldFilterOut(".ab")) - assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd")) - assert(!PartitioningAwareFileCatalog.shouldFilterOut("_metadata")) - assert(!PartitioningAwareFileCatalog.shouldFilterOut("_common_metadata")) - assert(PartitioningAwareFileCatalog.shouldFilterOut("_ab_metadata")) - assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd_common_metadata")) - } - test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { class MockCatalog( override val rootPaths: Seq[Path]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala new file mode 100644 index 0000000000000..df509583377ae --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.SparkFunSuite + +class SessionFileCatalogSuite extends SparkFunSuite { + + test("file filtering") { + assert(!SessionFileCatalog.shouldFilterOut("abcd")) + assert(SessionFileCatalog.shouldFilterOut(".ab")) + assert(SessionFileCatalog.shouldFilterOut("_cd")) + + assert(!SessionFileCatalog.shouldFilterOut("_metadata")) + assert(!SessionFileCatalog.shouldFilterOut("_common_metadata")) + assert(SessionFileCatalog.shouldFilterOut("_ab_metadata")) + assert(SessionFileCatalog.shouldFilterOut("_cd_common_metadata")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 36d4df0015ffd..43357c97c395a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -30,7 +30,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitionPath => Partition, PartitioningAwareFileCatalog, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -626,11 +626,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileCatalog, _, _, _, _, _), _, _) => - assert(location.partitionSpec() === PartitionSpec.emptySpec) + case LogicalRelation(HadoopFsRelation(location: FileCatalog, _, _, _, _, _), _, _) => + assert(location.partitionSpec === PartitionSpec.emptySpec) }.getOrElse { - fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") + fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 16e1e37b2fb02..4a2aaa7d4f6ca 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.{Partition => _, _} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ From 7d878cf2da04800bc4147b05610170865b148c64 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Tue, 18 Oct 2016 00:49:57 -0700 Subject: [PATCH 115/177] [SQL][STREAMING][TEST] Fix flaky tests in StreamingQueryListenerSuite This work has largely been done by lw-lin in his PR #15497. This is a slight refactoring of it. ## What changes were proposed in this pull request? There were two sources of flakiness in StreamingQueryListener test. - When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock. ``` +-----------------------------------+--------------------------------+ | StreamExecution thread | testing thread | +-----------------------------------+--------------------------------+ | ManualClock.waitTillTime(100) { | | | _isWaiting = true | | | wait(10) | | | still in wait(10) | if (_isWaiting) advance(100) | | still in wait(10) | if (_isWaiting) advance(200) | <- this should be disallowed ! | still in wait(10) | if (_isWaiting) advance(300) | <- this should be disallowed ! | wake up from wait(10) | | | current time is 600 | | | _isWaiting = false | | | } | | +-----------------------------------+--------------------------------+ ``` - Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger. My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(200)` (see above) will wait for stream execution thread to complete the wait that started at time 0, and start a new wait at time 200 (i.e. time stamp after the previous `advance(100)`). In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest. ## How was this patch tested? Ran existing unit test MANY TIME in Jenkins Author: Tathagata Das Author: Liwei Lin Closes #15519 from tdas/metrics-flaky-test-fix. --- .../org/apache/spark/util/ManualClock.scala | 18 ++------- .../spark/sql/streaming/StreamSuite.scala | 4 +- .../spark/sql/streaming/StreamTest.scala | 38 ++++++++++++++++--- .../StreamingQueryListenerSuite.scala | 8 ++-- 4 files changed, 41 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala index 91a95871014f0..e7a65d74a440e 100644 --- a/core/src/main/scala/org/apache/spark/util/ManualClock.scala +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -26,8 +26,6 @@ package org.apache.spark.util */ private[spark] class ManualClock(private var time: Long) extends Clock { - private var _isWaiting = false - /** * @return `ManualClock` with initial time 0 */ @@ -59,19 +57,9 @@ private[spark] class ManualClock(private var time: Long) extends Clock { * @return current time reported by the clock when waiting finishes */ def waitTillTime(targetTime: Long): Long = synchronized { - _isWaiting = true - try { - while (time < targetTime) { - wait(10) - } - getTimeMillis() - } finally { - _isWaiting = false + while (time < targetTime) { + wait(10) } + getTimeMillis() } - - /** - * Returns whether there is any thread being blocked in `waitTillTime`. - */ - def isWaiting: Boolean = synchronized { _isWaiting } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index cdbad901dba8e..6bdf47901ae68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -161,7 +161,7 @@ class StreamSuite extends StreamTest { val inputData = MemoryStream[Int] testStream(inputData.toDS())( - StartStream(ProcessingTime("10 seconds"), new ManualClock), + StartStream(ProcessingTime("10 seconds"), new StreamManualClock), /* -- batch 0 ----------------------- */ // Add some data in batch 0 @@ -199,7 +199,7 @@ class StreamSuite extends StreamTest { /* Stop then restart the Stream */ StopStream, - StartStream(ProcessingTime("10 seconds"), new ManualClock), + StartStream(ProcessingTime("10 seconds"), new StreamManualClock(60 * 1000)), /* -- batch 1 rerun ----------------- */ // this batch 1 would re-run because the latest batch id logged in offset log is 1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 3b9d3786349ad..254f823bf54f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -204,6 +204,21 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { case class AssertOnLastQueryStatus(condition: StreamingQueryStatus => Unit) extends StreamAction + class StreamManualClock(time: Long = 0L) extends ManualClock(time) { + private var waitStartTime: Option[Long] = None + + override def waitTillTime(targetTime: Long): Long = synchronized { + try { + waitStartTime = Some(getTimeMillis()) + super.waitTillTime(targetTime) + } finally { + waitStartTime = None + } + } + + def isStreamWaitingAt(time: Long): Boolean = synchronized { waitStartTime.contains(time) } + } + /** * Executes the specified actions on the given streaming DataFrame and provides helpful @@ -307,7 +322,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val testThread = Thread.currentThread() val metadataRoot = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath val statusCollector = new QueryStatusCollector - + var manualClockExpectedTime = -1L try { spark.streams.addListener(statusCollector) startedTest.foreach { action => @@ -315,6 +330,12 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { action match { case StartStream(trigger, triggerClock) => verify(currentStream == null, "stream already running") + verify(triggerClock.isInstanceOf[SystemClock] + || triggerClock.isInstanceOf[StreamManualClock], + "Use either SystemClock or StreamManualClock to start the stream") + if (triggerClock.isInstanceOf[StreamManualClock]) { + manualClockExpectedTime = triggerClock.asInstanceOf[StreamManualClock].getTimeMillis() + } lastStream = currentStream currentStream = spark @@ -338,14 +359,19 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { case AdvanceManualClock(timeToAdd) => verify(currentStream != null, "can not advance manual clock when a stream is not running") - verify(currentStream.triggerClock.isInstanceOf[ManualClock], + verify(currentStream.triggerClock.isInstanceOf[StreamManualClock], s"can not advance clock of type ${currentStream.triggerClock.getClass}") - val clock = currentStream.triggerClock.asInstanceOf[ManualClock] + val clock = currentStream.triggerClock.asInstanceOf[StreamManualClock] + assert(manualClockExpectedTime >= 0) // Make sure we don't advance ManualClock too early. See SPARK-16002. - eventually("ManualClock has not yet entered the waiting state") { - assert(clock.isWaiting) + eventually("StreamManualClock has not yet entered the waiting state") { + assert(clock.isStreamWaitingAt(manualClockExpectedTime)) } - currentStream.triggerClock.asInstanceOf[ManualClock].advance(timeToAdd) + clock.advance(timeToAdd) + manualClockExpectedTime += timeToAdd + verify(clock.getTimeMillis() === manualClockExpectedTime, + s"Unexpected clock time after updating: " + + s"expecting $manualClockExpectedTime, current ${clock.getTimeMillis()}") case StopStream => verify(currentStream != null, "can not stop a stream that is not running") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 9e0eefbc58aa5..623f66a778eac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -43,9 +43,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Make sure we don't leak any events to the next test } - ignore("single listener, check trigger statuses") { + test("single listener, check trigger statuses") { import StreamingQueryListenerSuite._ - clock = new ManualClock() + clock = new StreamManualClock /** Custom MemoryStream that waits for manual clock to reach a time */ val inputData = new MemoryStream[Int](0, sqlContext) { @@ -81,7 +81,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { AssertOnLastQueryStatus { status: StreamingQueryStatus => // Check the correctness of the trigger info of the last completed batch reported by // onQueryProgress - assert(status.triggerDetails.get("triggerId") == "0") + assert(status.triggerDetails.containsKey("triggerId")) assert(status.triggerDetails.get("isTriggerActive") === "false") assert(status.triggerDetails.get("isDataPresentInTrigger") === "true") @@ -101,7 +101,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.triggerDetails.get("numRows.state.aggregation1.updated") === "1") assert(status.sourceStatuses.length === 1) - assert(status.sourceStatuses(0).triggerDetails.get("triggerId") === "0") + assert(status.sourceStatuses(0).triggerDetails.containsKey("triggerId")) assert(status.sourceStatuses(0).triggerDetails.get("latency.getOffset.source") === "100") assert(status.sourceStatuses(0).triggerDetails.get("latency.getBatch.source") === "200") assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "2") From a9e79a41ee19258e5eb8da74bef4b8af9a2ccb95 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 18 Oct 2016 02:29:55 -0700 Subject: [PATCH 116/177] [SQL][STREAMING][TEST] Follow up to remove Option.contains for Scala 2.10 compatibility ## What changes were proposed in this pull request? Scala 2.10 does not have Option.contains, which broke Scala 2.10 build. ## How was this patch tested? Locally compiled and ran sql/core unit tests in 2.10 Author: Tathagata Das Closes #15531 from tdas/metrics-flaky-test-fix-1. --- .../scala/org/apache/spark/sql/streaming/StreamTest.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 254f823bf54f7..8dfeb8da4b826 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -216,7 +216,9 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } - def isStreamWaitingAt(time: Long): Boolean = synchronized { waitStartTime.contains(time) } + def isStreamWaitingAt(time: Long): Boolean = synchronized { + waitStartTime == Some(time) + } } From e59df62e62ec4c5f8bd02a13f05fa3ec6f0fc694 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 18 Oct 2016 11:03:10 -0700 Subject: [PATCH 117/177] [SPARK-17899][SQL][FOLLOW-UP] debug mode should work for corrupted table ## What changes were proposed in this pull request? Debug mode should work for corrupted table, so that we can really debug ## How was this patch tested? new test in `MetastoreDataSourcesSuite` Author: Wenchen Fan Closes #15528 from cloud-fan/debug. --- .../spark/sql/hive/HiveExternalCatalog.scala | 9 ++------- .../sql/hive/MetastoreDataSourcesSuite.scala | 18 +++++++++++++++--- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index ff59b54f53909..2003ff42d4f0c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -448,7 +448,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat * properties, and filter out these special entries from table properties. */ private def restoreTableMetadata(table: CatalogTable): CatalogTable = { - val catalogTable = if (table.tableType == VIEW) { + val catalogTable = if (table.tableType == VIEW || conf.get(DEBUG_MODE)) { table } else { getProviderFromTableProperties(table).map { provider => @@ -467,18 +467,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } else { table.storage } - val tableProps = if (conf.get(DEBUG_MODE)) { - table.properties - } else { - getOriginalTableProperties(table) - } table.copy( storage = storage, schema = getSchemaFromTableProperties(table), provider = Some(provider), partitionColumnNames = getPartitionColumnsFromTableProperties(table), bucketSpec = getBucketSpecFromTableProperties(table), - properties = tableProps) + properties = getOriginalTableProperties(table)) } getOrElse { table.copy(provider = Some("hive")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 7cc6179d44977..eaa67d370db37 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -1321,20 +1321,32 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sharedState.externalCatalog.getTable("default", "t") }.getMessage assert(e.contains(s"Could not read schema from the hive metastore because it is corrupted")) + + withDebugMode { + val tableMeta = sharedState.externalCatalog.getTable("default", "t") + assert(tableMeta.identifier == TableIdentifier("t", Some("default"))) + assert(tableMeta.properties(DATASOURCE_PROVIDER) == "json") + } } finally { hiveClient.dropTable("default", "t", ignoreIfNotExists = true, purge = true) } } test("should keep data source entries in table properties when debug mode is on") { - val previousValue = sparkSession.sparkContext.conf.get(DEBUG_MODE) - try { - sparkSession.sparkContext.conf.set(DEBUG_MODE, true) + withDebugMode { val newSession = sparkSession.newSession() newSession.sql("CREATE TABLE abc(i int) USING json") val tableMeta = newSession.sessionState.catalog.getTableMetadata(TableIdentifier("abc")) assert(tableMeta.properties(DATASOURCE_SCHEMA_NUMPARTS).toInt == 1) assert(tableMeta.properties(DATASOURCE_PROVIDER) == "json") + } + } + + private def withDebugMode(f: => Unit): Unit = { + val previousValue = sparkSession.sparkContext.conf.get(DEBUG_MODE) + try { + sparkSession.sparkContext.conf.set(DEBUG_MODE, true) + f } finally { sparkSession.sparkContext.conf.set(DEBUG_MODE, previousValue) } From 37686539f546ac7a3657dbfc59b7ac982b4b9bce Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 18 Oct 2016 13:20:42 -0700 Subject: [PATCH 118/177] [SPARK-17388] [SQL] Support for inferring type date/timestamp/decimal for partition column ## What changes were proposed in this pull request? Currently, Spark only supports to infer `IntegerType`, `LongType`, `DoubleType` and `StringType`. `DecimalType` is being tried but it seems it never infers type as `DecimalType` as `DoubleType` is being tried first. Also, it seems `DateType` and `TimestampType` could be inferred. As far as I know, it is pretty common to use both for a partition column. This PR fixes the incorrect `DecimalType` try and also adds the support for both `DateType` and `TimestampType` for inferring partition column type. ## How was this patch tested? Unit tests in `ParquetPartitionDiscoverySuite`. Author: hyukjinkwon Closes #14947 from HyukjinKwon/SPARK-17388. --- .../datasources/PartitioningUtils.scala | 21 ++++++++-- .../ParquetPartitionDiscoverySuite.scala | 42 ++++++++++++++++++- 2 files changed, 59 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 504464216e5a4..381261cf65ca0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources import java.lang.{Double => JDouble, Long => JLong} import java.math.{BigDecimal => JBigDecimal} +import java.sql.{Date => JDate, Timestamp => JTimestamp} import scala.collection.mutable.ArrayBuffer import scala.util.Try @@ -307,20 +308,34 @@ object PartitioningUtils { /** * Converts a string to a [[Literal]] with automatic type inference. Currently only supports - * [[IntegerType]], [[LongType]], [[DoubleType]], [[DecimalType.SYSTEM_DEFAULT]], and - * [[StringType]]. + * [[IntegerType]], [[LongType]], [[DoubleType]], [[DecimalType]], [[DateType]] + * [[TimestampType]], and [[StringType]]. */ private[datasources] def inferPartitionColumnValue( raw: String, defaultPartitionName: String, typeInference: Boolean): Literal = { + val decimalTry = Try { + // `BigDecimal` conversion can fail when the `field` is not a form of number. + val bigDecimal = new JBigDecimal(raw) + // It reduces the cases for decimals by disallowing values having scale (eg. `1.1`). + require(bigDecimal.scale <= 0) + // `DecimalType` conversion can fail when + // 1. The precision is bigger than 38. + // 2. scale is bigger than precision. + Literal(bigDecimal) + } + if (typeInference) { // First tries integral types Try(Literal.create(Integer.parseInt(raw), IntegerType)) .orElse(Try(Literal.create(JLong.parseLong(raw), LongType))) + .orElse(decimalTry) // Then falls back to fractional types .orElse(Try(Literal.create(JDouble.parseDouble(raw), DoubleType))) - .orElse(Try(Literal(new JBigDecimal(raw)))) + // Then falls back to date/timestamp types + .orElse(Try(Literal(JDate.valueOf(raw)))) + .orElse(Try(Literal(JTimestamp.valueOf(unescapePathName(raw))))) // Then falls back to string .getOrElse { if (raw == defaultPartitionName) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 43357c97c395a..2ef66baee1eac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File import java.math.BigInteger -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.collection.mutable.ArrayBuffer @@ -56,8 +56,14 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha check("10", Literal.create(10, IntegerType)) check("1000000000000000", Literal.create(1000000000000000L, LongType)) + val decimal = Decimal("1" * 20) + check("1" * 20, + Literal.create(decimal, DecimalType(decimal.precision, decimal.scale))) check("1.5", Literal.create(1.5, DoubleType)) check("hello", Literal.create("hello", StringType)) + check("1990-02-24", Literal.create(Date.valueOf("1990-02-24"), DateType)) + check("1990-02-24 12:00:30", + Literal.create(Timestamp.valueOf("1990-02-24 12:00:30"), TimestampType)) check(defaultPartitionName, Literal.create(null, NullType)) } @@ -687,6 +693,40 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } + test("Various inferred partition value types") { + val row = + Row( + Long.MaxValue, + 4.5, + new java.math.BigDecimal(new BigInteger("1" * 20)), + java.sql.Date.valueOf("2015-05-23"), + java.sql.Timestamp.valueOf("1990-02-24 12:00:30"), + "This is a string, /[]?=:", + "This is not a partition column") + + val partitionColumnTypes = + Seq( + LongType, + DoubleType, + DecimalType(20, 0), + DateType, + TimestampType, + StringType) + + val partitionColumns = partitionColumnTypes.zipWithIndex.map { + case (t, index) => StructField(s"p_$index", t) + } + + val schema = StructType(partitionColumns :+ StructField(s"i", StringType)) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { dir => + df.write.format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString) + val fields = schema.map(f => Column(f.name)) + checkAnswer(spark.read.load(dir.toString).select(fields: _*), row) + } + } + test("SPARK-8037: Ignores files whose name starts with dot") { withTempPath { dir => val df = (1 to 3).map(i => (i, i, i, i)).toDF("a", "b", "c", "d") From 231f39e3f6641953a90bc4c40444ede63f363b23 Mon Sep 17 00:00:00 2001 From: Yu Peng Date: Tue, 18 Oct 2016 13:23:31 -0700 Subject: [PATCH 119/177] [SPARK-17711] Compress rolled executor log ## What changes were proposed in this pull request? This PR adds support for executor log compression. ## How was this patch tested? Unit tests cc: yhuai tdas mengxr Author: Yu Peng Closes #15285 from loneknightpy/compress-executor-log. --- .../spark/deploy/worker/ui/LogPage.scala | 7 +- .../scala/org/apache/spark/util/Utils.scala | 80 ++++++++++++++-- .../util/logging/RollingFileAppender.scala | 45 +++++++-- .../spark/deploy/worker/ui/LogPageSuite.scala | 6 +- .../apache/spark/util/FileAppenderSuite.scala | 60 +++++++++++- .../org/apache/spark/util/UtilsSuite.scala | 92 ++++++++++++++----- docs/configuration.md | 8 ++ docs/spark-standalone.md | 9 ++ 8 files changed, 263 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 3473c41b935fd..465c214362b25 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -22,6 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Unparsed} +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} + import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils @@ -138,7 +140,8 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with val files = RollingFileAppender.getSortedRolledOverFiles(logDirectory, logType) logDebug(s"Sorted log files of type $logType in $logDirectory:\n${files.mkString("\n")}") - val totalLength = files.map { _.length }.sum + val fileLengths: Seq[Long] = files.map(Utils.getFileLength(_, worker.conf)) + val totalLength = fileLengths.sum val offset = offsetOption.getOrElse(totalLength - byteLength) val startIndex = { if (offset < 0) { @@ -151,7 +154,7 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with } val endIndex = math.min(startIndex + byteLength, totalLength) logDebug(s"Getting log from $startIndex to $endIndex") - val logText = Utils.offsetBytes(files, startIndex, endIndex) + val logText = Utils.offsetBytes(files, fileLengths, startIndex, endIndex) logDebug(s"Got log of length ${logText.length} bytes") (logText, startIndex, endIndex, totalLength) } catch { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index ef832756ce3b7..a4da138e71992 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -27,6 +27,7 @@ import java.nio.file.{Files, Paths} import java.util.{Locale, Properties, Random, UUID} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean +import java.util.zip.GZIPInputStream import javax.net.ssl.HttpsURLConnection import scala.annotation.tailrec @@ -38,8 +39,10 @@ import scala.reflect.ClassTag import scala.util.Try import scala.util.control.{ControlThrowable, NonFatal} +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.google.common.io.{ByteStreams, Files => GFiles} import com.google.common.net.InetAddresses +import org.apache.commons.io.IOUtils import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} @@ -55,6 +58,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DYN_ALLOCATION_INITIAL_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS, EXECUTOR_INSTANCES} import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +import org.apache.spark.util.logging.RollingFileAppender /** CallSite represents a place in user code. It can have a short and a long form. */ private[spark] case class CallSite(shortForm: String, longForm: String) @@ -1440,14 +1444,72 @@ private[spark] object Utils extends Logging { CallSite(shortForm, longForm) } + private val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = + "spark.worker.ui.compressedLogFileLengthCacheSize" + private val DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE = 100 + private var compressedLogFileLengthCache: LoadingCache[String, java.lang.Long] = null + private def getCompressedLogFileLengthCache( + sparkConf: SparkConf): LoadingCache[String, java.lang.Long] = this.synchronized { + if (compressedLogFileLengthCache == null) { + val compressedLogFileLengthCacheSize = sparkConf.getInt( + UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF, + DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE) + compressedLogFileLengthCache = CacheBuilder.newBuilder() + .maximumSize(compressedLogFileLengthCacheSize) + .build[String, java.lang.Long](new CacheLoader[String, java.lang.Long]() { + override def load(path: String): java.lang.Long = { + Utils.getCompressedFileLength(new File(path)) + } + }) + } + compressedLogFileLengthCache + } + + /** + * Return the file length, if the file is compressed it returns the uncompressed file length. + * It also caches the uncompressed file size to avoid repeated decompression. The cache size is + * read from workerConf. + */ + def getFileLength(file: File, workConf: SparkConf): Long = { + if (file.getName.endsWith(".gz")) { + getCompressedLogFileLengthCache(workConf).get(file.getAbsolutePath) + } else { + file.length + } + } + + /** Return uncompressed file length of a compressed file. */ + private def getCompressedFileLength(file: File): Long = { + try { + // Uncompress .gz file to determine file size. + var fileSize = 0L + val gzInputStream = new GZIPInputStream(new FileInputStream(file)) + val bufSize = 1024 + val buf = new Array[Byte](bufSize) + var numBytes = IOUtils.read(gzInputStream, buf) + while (numBytes > 0) { + fileSize += numBytes + numBytes = IOUtils.read(gzInputStream, buf) + } + fileSize + } catch { + case e: Throwable => + logError(s"Cannot get file length of ${file}", e) + throw e + } + } + /** Return a string containing part of a file from byte 'start' to 'end'. */ - def offsetBytes(path: String, start: Long, end: Long): String = { + def offsetBytes(path: String, length: Long, start: Long, end: Long): String = { val file = new File(path) - val length = file.length() val effectiveEnd = math.min(length, end) val effectiveStart = math.max(0, start) val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) - val stream = new FileInputStream(file) + val stream = if (path.endsWith(".gz")) { + new GZIPInputStream(new FileInputStream(file)) + } else { + new FileInputStream(file) + } try { ByteStreams.skipFully(stream, effectiveStart) @@ -1463,8 +1525,8 @@ private[spark] object Utils extends Logging { * and `endIndex` is based on the cumulative size of all the files take in * the given order. See figure below for more details. */ - def offsetBytes(files: Seq[File], start: Long, end: Long): String = { - val fileLengths = files.map { _.length } + def offsetBytes(files: Seq[File], fileLengths: Seq[Long], start: Long, end: Long): String = { + assert(files.length == fileLengths.length) val startIndex = math.max(start, 0) val endIndex = math.min(end, fileLengths.sum) val fileToLength = files.zip(fileLengths).toMap @@ -1472,7 +1534,7 @@ private[spark] object Utils extends Logging { val stringBuffer = new StringBuffer((endIndex - startIndex).toInt) var sum = 0L - for (file <- files) { + files.zip(fileLengths).foreach { case (file, fileLength) => val startIndexOfFile = sum val endIndexOfFile = sum + fileToLength(file) logDebug(s"Processing file $file, " + @@ -1491,19 +1553,19 @@ private[spark] object Utils extends Logging { if (startIndex <= startIndexOfFile && endIndex >= endIndexOfFile) { // Case C: read the whole file - stringBuffer.append(offsetBytes(file.getAbsolutePath, 0, fileToLength(file))) + stringBuffer.append(offsetBytes(file.getAbsolutePath, fileLength, 0, fileToLength(file))) } else if (startIndex > startIndexOfFile && startIndex < endIndexOfFile) { // Case A and B: read from [start of required range] to [end of file / end of range] val effectiveStartIndex = startIndex - startIndexOfFile val effectiveEndIndex = math.min(endIndex - startIndexOfFile, fileToLength(file)) stringBuffer.append(Utils.offsetBytes( - file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) } else if (endIndex > startIndexOfFile && endIndex < endIndexOfFile) { // Case D: read from [start of file] to [end of require range] val effectiveStartIndex = math.max(startIndex - startIndexOfFile, 0) val effectiveEndIndex = endIndex - startIndexOfFile stringBuffer.append(Utils.offsetBytes( - file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) } sum += fileToLength(file) logDebug(s"After processing file $file, string built is ${stringBuffer.toString}") diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index a0eb05c7c0e82..5d8cec8447b53 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -17,9 +17,11 @@ package org.apache.spark.util.logging -import java.io.{File, FileFilter, InputStream} +import java.io._ +import java.util.zip.GZIPOutputStream import com.google.common.io.Files +import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf @@ -45,6 +47,7 @@ private[spark] class RollingFileAppender( import RollingFileAppender._ private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) + private val enableCompression = conf.getBoolean(ENABLE_COMPRESSION, false) /** Stop the appender */ override def stop() { @@ -76,6 +79,33 @@ private[spark] class RollingFileAppender( } } + // Roll the log file and compress if enableCompression is true. + private def rotateFile(activeFile: File, rolloverFile: File): Unit = { + if (enableCompression) { + val gzFile = new File(rolloverFile.getAbsolutePath + GZIP_LOG_SUFFIX) + var gzOutputStream: GZIPOutputStream = null + var inputStream: InputStream = null + try { + inputStream = new FileInputStream(activeFile) + gzOutputStream = new GZIPOutputStream(new FileOutputStream(gzFile)) + IOUtils.copy(inputStream, gzOutputStream) + inputStream.close() + gzOutputStream.close() + activeFile.delete() + } finally { + IOUtils.closeQuietly(inputStream) + IOUtils.closeQuietly(gzOutputStream) + } + } else { + Files.move(activeFile, rolloverFile) + } + } + + // Check if the rollover file already exists. + private def rolloverFileExist(file: File): Boolean = { + file.exists || new File(file.getAbsolutePath + GZIP_LOG_SUFFIX).exists + } + /** Move the active log file to a new rollover file */ private def moveFile() { val rolloverSuffix = rollingPolicy.generateRolledOverFileSuffix() @@ -83,8 +113,8 @@ private[spark] class RollingFileAppender( activeFile.getParentFile, activeFile.getName + rolloverSuffix).getAbsoluteFile logDebug(s"Attempting to rollover file $activeFile to file $rolloverFile") if (activeFile.exists) { - if (!rolloverFile.exists) { - Files.move(activeFile, rolloverFile) + if (!rolloverFileExist(rolloverFile)) { + rotateFile(activeFile, rolloverFile) logInfo(s"Rolled over $activeFile to $rolloverFile") } else { // In case the rollover file name clashes, make a unique file name. @@ -97,11 +127,11 @@ private[spark] class RollingFileAppender( altRolloverFile = new File(activeFile.getParent, s"${activeFile.getName}$rolloverSuffix--$i").getAbsoluteFile i += 1 - } while (i < 10000 && altRolloverFile.exists) + } while (i < 10000 && rolloverFileExist(altRolloverFile)) logWarning(s"Rollover file $rolloverFile already exists, " + s"rolled over $activeFile to file $altRolloverFile") - Files.move(activeFile, altRolloverFile) + rotateFile(activeFile, altRolloverFile) } } else { logWarning(s"File $activeFile does not exist") @@ -142,6 +172,9 @@ private[spark] object RollingFileAppender { val SIZE_DEFAULT = (1024 * 1024).toString val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" val DEFAULT_BUFFER_SIZE = 8192 + val ENABLE_COMPRESSION = "spark.executor.logs.rolling.enableCompression" + + val GZIP_LOG_SUFFIX = ".gz" /** * Get the sorted list of rolled over files. This assumes that the all the rolled @@ -158,6 +191,6 @@ private[spark] object RollingFileAppender { val file = new File(directory, activeFileName).getAbsoluteFile if (file.exists) Some(file) else None } - rolledOverFiles ++ activeFile + rolledOverFiles.sortBy(_.getName.stripSuffix(GZIP_LOG_SUFFIX)) ++ activeFile } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala index 72eaffb416981..4c3e96777940d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala @@ -22,16 +22,20 @@ import java.io.{File, FileWriter} import org.mockito.Mockito.{mock, when} import org.scalatest.PrivateMethodTester -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.worker.Worker class LogPageSuite extends SparkFunSuite with PrivateMethodTester { test("get logs simple") { val webui = mock(classOf[WorkerWebUI]) + val worker = mock(classOf[Worker]) val tmpDir = new File(sys.props("java.io.tmpdir")) val workDir = new File(tmpDir, "work-dir") workDir.mkdir() when(webui.workDir).thenReturn(workDir) + when(webui.worker).thenReturn(worker) + when(worker.conf).thenReturn(new SparkConf()) val logPage = new LogPage(webui) // Prepare some fake log files to read later diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 4fa9f9a8f590f..7e2da8e141532 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -20,11 +20,13 @@ package org.apache.spark.util import java.io._ import java.nio.charset.StandardCharsets import java.util.concurrent.CountDownLatch +import java.util.zip.GZIPInputStream import scala.collection.mutable.HashSet import scala.reflect._ import com.google.common.io.Files +import org.apache.commons.io.IOUtils import org.apache.log4j.{Appender, Level, Logger} import org.apache.log4j.spi.LoggingEvent import org.mockito.ArgumentCaptor @@ -72,6 +74,25 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { testRolling(appender, testOutputStream, textToAppend, rolloverIntervalMillis) } + test("rolling file appender - time-based rolling (compressed)") { + // setup input stream and appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) + val rolloverIntervalMillis = 100 + val durationMillis = 1000 + val numRollovers = durationMillis / rolloverIntervalMillis + val textToAppend = (1 to numRollovers).map( _.toString * 10 ) + + val sparkConf = new SparkConf() + sparkConf.set("spark.executor.logs.rolling.enableCompression", "true") + val appender = new RollingFileAppender(testInputStream, testFile, + new TimeBasedRollingPolicy(rolloverIntervalMillis, s"--HH-mm-ss-SSSS", false), + sparkConf, 10) + + testRolling( + appender, testOutputStream, textToAppend, rolloverIntervalMillis, isCompressed = true) + } + test("rolling file appender - size-based rolling") { // setup input stream and appender val testOutputStream = new PipedOutputStream() @@ -89,6 +110,25 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { } } + test("rolling file appender - size-based rolling (compressed)") { + // setup input stream and appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) + val rolloverSize = 1000 + val textToAppend = (1 to 3).map( _.toString * 1000 ) + + val sparkConf = new SparkConf() + sparkConf.set("spark.executor.logs.rolling.enableCompression", "true") + val appender = new RollingFileAppender(testInputStream, testFile, + new SizeBasedRollingPolicy(rolloverSize, false), sparkConf, 99) + + val files = testRolling(appender, testOutputStream, textToAppend, 0, isCompressed = true) + files.foreach { file => + logInfo(file.toString + ": " + file.length + " bytes") + assert(file.length < rolloverSize) + } + } + test("rolling file appender - cleaning") { // setup input stream and appender val testOutputStream = new PipedOutputStream() @@ -273,7 +313,8 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { appender: FileAppender, outputStream: OutputStream, textToAppend: Seq[String], - sleepTimeBetweenTexts: Long + sleepTimeBetweenTexts: Long, + isCompressed: Boolean = false ): Seq[File] = { // send data to appender through the input stream, and wait for the data to be written val expectedText = textToAppend.mkString("") @@ -290,10 +331,23 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { // verify whether all the data written to rolled over files is same as expected val generatedFiles = RollingFileAppender.getSortedRolledOverFiles( testFile.getParentFile.toString, testFile.getName) - logInfo("Filtered files: \n" + generatedFiles.mkString("\n")) + logInfo("Generate files: \n" + generatedFiles.mkString("\n")) assert(generatedFiles.size > 1) + if (isCompressed) { + assert( + generatedFiles.filter(_.getName.endsWith(RollingFileAppender.GZIP_LOG_SUFFIX)).size > 0) + } val allText = generatedFiles.map { file => - Files.toString(file, StandardCharsets.UTF_8) + if (file.getName.endsWith(RollingFileAppender.GZIP_LOG_SUFFIX)) { + val inputStream = new GZIPInputStream(new FileInputStream(file)) + try { + IOUtils.toString(inputStream, StandardCharsets.UTF_8) + } finally { + IOUtils.closeQuietly(inputStream) + } + } else { + Files.toString(file, StandardCharsets.UTF_8) + } }.mkString("") assert(allText === expectedText) generatedFiles diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index bc28b2d9cb831..b427f7fb50158 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -25,11 +25,13 @@ import java.nio.charset.StandardCharsets import java.text.DecimalFormatSymbols import java.util.Locale import java.util.concurrent.TimeUnit +import java.util.zip.GZIPOutputStream import scala.collection.mutable.ListBuffer import scala.util.Random import com.google.common.io.Files +import org.apache.commons.io.IOUtils import org.apache.commons.lang3.SystemUtils import org.apache.commons.math3.stat.inference.ChiSquareTest import org.apache.hadoop.conf.Configuration @@ -274,65 +276,109 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11" + sep + "00 h") } - test("reading offset bytes of a file") { + def getSuffix(isCompressed: Boolean): String = { + if (isCompressed) { + ".gz" + } else { + "" + } + } + + def writeLogFile(path: String, content: Array[Byte]): Unit = { + val outputStream = if (path.endsWith(".gz")) { + new GZIPOutputStream(new FileOutputStream(path)) + } else { + new FileOutputStream(path) + } + IOUtils.write(content, outputStream) + outputStream.close() + content.size + } + + private val workerConf = new SparkConf() + + def testOffsetBytes(isCompressed: Boolean): Unit = { val tmpDir2 = Utils.createTempDir() - val f1Path = tmpDir2 + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(StandardCharsets.UTF_8)) - f1.close() + val suffix = getSuffix(isCompressed) + val f1Path = tmpDir2 + "/f1" + suffix + writeLogFile(f1Path, "1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(StandardCharsets.UTF_8)) + val f1Length = Utils.getFileLength(new File(f1Path), workerConf) // Read first few bytes - assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + assert(Utils.offsetBytes(f1Path, f1Length, 0, 5) === "1\n2\n3") // Read some middle bytes - assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + assert(Utils.offsetBytes(f1Path, f1Length, 4, 11) === "3\n4\n5\n6") // Read last few bytes - assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + assert(Utils.offsetBytes(f1Path, f1Length, 12, 18) === "7\n8\n9\n") // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + assert(Utils.offsetBytes(f1Path, f1Length, -5, 5) === "1\n2\n3") // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + assert(Utils.offsetBytes(f1Path, f1Length, 12, 22) === "7\n8\n9\n") // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + assert(Utils.offsetBytes(f1Path, f1Length, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") Utils.deleteRecursively(tmpDir2) } - test("reading offset bytes across multiple files") { + test("reading offset bytes of a file") { + testOffsetBytes(isCompressed = false) + } + + test("reading offset bytes of a file (compressed)") { + testOffsetBytes(isCompressed = true) + } + + def testOffsetBytesMultipleFiles(isCompressed: Boolean): Unit = { val tmpDir = Utils.createTempDir() - val files = (1 to 3).map(i => new File(tmpDir, i.toString)) - Files.write("0123456789", files(0), StandardCharsets.UTF_8) - Files.write("abcdefghij", files(1), StandardCharsets.UTF_8) - Files.write("ABCDEFGHIJ", files(2), StandardCharsets.UTF_8) + val suffix = getSuffix(isCompressed) + val files = (1 to 3).map(i => new File(tmpDir, i.toString + suffix)) :+ new File(tmpDir, "4") + writeLogFile(files(0).getAbsolutePath, "0123456789".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(1).getAbsolutePath, "abcdefghij".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(2).getAbsolutePath, "ABCDEFGHIJ".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(3).getAbsolutePath, "9876543210".getBytes(StandardCharsets.UTF_8)) + val fileLengths = files.map(Utils.getFileLength(_, workerConf)) // Read first few bytes in the 1st file - assert(Utils.offsetBytes(files, 0, 5) === "01234") + assert(Utils.offsetBytes(files, fileLengths, 0, 5) === "01234") // Read bytes within the 1st file - assert(Utils.offsetBytes(files, 5, 8) === "567") + assert(Utils.offsetBytes(files, fileLengths, 5, 8) === "567") // Read bytes across 1st and 2nd file - assert(Utils.offsetBytes(files, 8, 18) === "89abcdefgh") + assert(Utils.offsetBytes(files, fileLengths, 8, 18) === "89abcdefgh") // Read bytes across 1st, 2nd and 3rd file - assert(Utils.offsetBytes(files, 5, 24) === "56789abcdefghijABCD") + assert(Utils.offsetBytes(files, fileLengths, 5, 24) === "56789abcdefghijABCD") + + // Read bytes across 3rd and 4th file + assert(Utils.offsetBytes(files, fileLengths, 25, 35) === "FGHIJ98765") // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(files, -5, 18) === "0123456789abcdefgh") + assert(Utils.offsetBytes(files, fileLengths, -5, 18) === "0123456789abcdefgh") // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(files, 18, 35) === "ijABCDEFGHIJ") + assert(Utils.offsetBytes(files, fileLengths, 18, 45) === "ijABCDEFGHIJ9876543210") // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(files, -5, 35) === "0123456789abcdefghijABCDEFGHIJ") + assert(Utils.offsetBytes(files, fileLengths, -5, 45) === + "0123456789abcdefghijABCDEFGHIJ9876543210") Utils.deleteRecursively(tmpDir) } + test("reading offset bytes across multiple files") { + testOffsetBytesMultipleFiles(isCompressed = false) + } + + test("reading offset bytes across multiple files (compressed)") { + testOffsetBytesMultipleFiles(isCompressed = true) + } + test("deserialize long value") { val testval : Long = 9730889947L val bbuf = ByteBuffer.allocate(8) diff --git a/docs/configuration.md b/docs/configuration.md index 373e22d71a872..a4a99d6fa4630 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -293,6 +293,14 @@ Apart from these, the following properties are also available, and may be useful Older log files will be deleted. Disabled by default. + + spark.executor.logs.rolling.enableCompression + false + + Enable executor log compression. If it is enabled, the rolled executor logs will be compressed. + Disabled by default. + + spark.executor.logs.rolling.maxSize (none) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7b82b957d5299..1c0b60f7b9346 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -250,6 +250,15 @@ SPARK_WORKER_OPTS supports the following system properties: especially if you run jobs very frequently. + + spark.worker.ui.compressedLogFileLengthCacheSize + 100 + + For compressed log files, the uncompressed file can only be computed by uncompressing the files. + Spark caches the uncompressed file size of compressed log files. This property controls the cache + size. + + # Connecting an Application to the Cluster From 4ef39c2f4436fa22d0b957fe7ad477e4c4a16452 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 18 Oct 2016 13:33:46 -0700 Subject: [PATCH 120/177] [SPARK-17974] try 2) Refactor FileCatalog classes to simplify the inheritance tree ## What changes were proposed in this pull request? This renames `BasicFileCatalog => FileCatalog`, combines `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait. In summary, ``` MetadataLogFileCatalog extends PartitioningAwareFileCatalog ListingFileCatalog extends PartitioningAwareFileCatalog PartitioningAwareFileCatalog extends FileCatalog TableFileCatalog extends FileCatalog ``` (note that this is a re-submission of https://github.com/apache/spark/pull/15518 which got reverted) ## How was this patch tested? Existing tests Author: Eric Liang Closes #15533 from ericl/fix-scalastyle-revert. --- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 4 +- .../execution/datasources/FileCatalog.scala | 66 +++++ .../execution/datasources/FileFormat.scala | 61 ----- .../datasources/HadoopFsRelation.scala | 4 +- .../PartitioningAwareFileCatalog.scala | 217 ++++++++++++++++- .../datasources/PartitioningUtils.scala | 12 +- .../datasources/SessionFileCatalog.scala | 225 ------------------ .../datasources/TableFileCatalog.scala | 11 +- .../datasources/FileCatalogSuite.scala | 10 + .../datasources/SessionFileCatalogSuite.scala | 34 --- .../ParquetPartitionDiscoverySuite.scala | 10 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 13 files changed, 304 insertions(+), 354 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 7dccbbd3f0a5b..073d2b1512b95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} -import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.streaming.{DataStreamWriter, StreamingQuery} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 623d2be55dcec..fdd1fa3648251 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -431,7 +431,7 @@ case class FileSourceScanExec( private def createBucketedReadRDD( bucketSpec: BucketSpec, readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[Partition], + selectedPartitions: Seq[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { logInfo(s"Planning with ${bucketSpec.numBuckets} buckets") val bucketed = @@ -463,7 +463,7 @@ case class FileSourceScanExec( */ private def createNonBucketedReadRDD( readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[Partition], + selectedPartitions: Seq[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { val defaultMaxSplitBytes = fsRelation.sparkSession.sessionState.conf.filesMaxPartitionBytes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala new file mode 100644 index 0000000000000..2bc66ceeebdb4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.hadoop.fs._ + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ + +/** + * A collection of data files from a partitioned relation, along with the partition values in the + * form of an [[InternalRow]]. + */ +case class PartitionDirectory(values: InternalRow, files: Seq[FileStatus]) + +/** + * An interface for objects capable of enumerating the root paths of a relation as well as the + * partitions of a relation subject to some pruning expressions. + */ +trait FileCatalog { + + /** + * Returns the list of root input paths from which the catalog will get files. There may be a + * single root path from which partitions are discovered, or individual partitions may be + * specified by each path. + */ + def rootPaths: Seq[Path] + + /** + * Returns all valid files grouped into partitions when the data is partitioned. If the data is + * unpartitioned, this will return a single partition with no partition values. + * + * @param filters The filters used to prune which partitions are returned. These filters must + * only refer to partition columns and this method will only return files + * where these predicates are guaranteed to evaluate to `true`. Thus, these + * filters will not need to be evaluated again on the returned data. + */ + def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] + + /** + * Returns the list of files that will be read when scanning this relation. This call may be + * very expensive for large tables. + */ + def inputFiles: Array[String] + + /** Refresh any cached file listings */ + def refresh(): Unit + + /** Sum of table file sizes, in bytes */ + def sizeInBytes: Long +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index e7239ef91b326..9d153cec731a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -175,64 +175,3 @@ abstract class TextBasedFileFormat extends FileFormat { codec == null || codec.isInstanceOf[SplittableCompressionCodec] } } - -/** - * A collection of data files from a partitioned relation, along with the partition values in the - * form of an [[InternalRow]]. - */ -case class Partition(values: InternalRow, files: Seq[FileStatus]) - -/** - * An interface for objects capable of enumerating the root paths of a relation as well as the - * partitions of a relation subject to some pruning expressions. - */ -trait BasicFileCatalog { - - /** - * Returns the list of root input paths from which the catalog will get files. There may be a - * single root path from which partitions are discovered, or individual partitions may be - * specified by each path. - */ - def rootPaths: Seq[Path] - - /** - * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with no partition values. - * - * @param filters The filters used to prune which partitions are returned. These filters must - * only refer to partition columns and this method will only return files - * where these predicates are guaranteed to evaluate to `true`. Thus, these - * filters will not need to be evaluated again on the returned data. - */ - def listFiles(filters: Seq[Expression]): Seq[Partition] - - /** Returns the list of files that will be read when scanning this relation. */ - def inputFiles: Array[String] - - /** Refresh any cached file listings */ - def refresh(): Unit - - /** Sum of table file sizes, in bytes */ - def sizeInBytes: Long -} - -/** - * A [[BasicFileCatalog]] which can enumerate all of the files comprising a relation and, from - * those, infer the relation's partition specification. - */ -// TODO: Consider a more descriptive, appropriate name which suggests this is a file catalog for -// which it is safe to list all of its files? -trait FileCatalog extends BasicFileCatalog { - - /** Returns the specification of the partitions inferred from the data. */ - def partitionSpec(): PartitionSpec - - /** Returns all the valid files. */ - def allFiles(): Seq[FileStatus] - - /** Returns the list of files that will be read when scanning this relation. */ - override def inputFiles: Array[String] = - allFiles().map(_.getPath.toUri.toString).toArray - - override def sizeInBytes: Long = allFiles().map(_.getLen).sum -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index db889edf032d6..afad8898089bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.StructType * Acts as a container for all of the metadata required to read from a datasource. All discovery, * resolution and merging logic for schemas and partitions has been removed. * - * @param location A [[BasicFileCatalog]] that can enumerate the locations of all the files that + * @param location A [[FileCatalog]] that can enumerate the locations of all the files that * comprise this relation. * @param partitionSchema The schema of the columns (if any) that are used to partition the relation * @param dataSchema The schema of any remaining columns. Note that if any partition columns are @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType * @param options Configuration used when reading / writing data. */ case class HadoopFsRelation( - location: BasicFileCatalog, + location: FileCatalog, partitionSchema: StructType, dataSchema: StructType, bucketSpec: Option[BucketSpec], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index b2508115c282f..5c8eff7ec46b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -17,14 +17,21 @@ package org.apache.spark.sql.execution.datasources +import java.io.FileNotFoundException + import scala.collection.mutable -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.SerializableConfiguration /** @@ -38,22 +45,24 @@ import org.apache.spark.sql.types.{StringType, StructType} abstract class PartitioningAwareFileCatalog( sparkSession: SparkSession, parameters: Map[String, String], - partitionSchema: Option[StructType]) - extends SessionFileCatalog(sparkSession) with FileCatalog { + partitionSchema: Option[StructType]) extends FileCatalog with Logging { import PartitioningAwareFileCatalog.BASE_PATH_PARAM - override protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) + /** Returns the specification of the partitions inferred from the data. */ + def partitionSpec(): PartitionSpec + + protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] - override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) { - Partition(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil + PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil } else { prunePartitions(filters, partitionSpec()).map { - case PartitionDirectory(values, path) => + case PartitionPath(values, path) => val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match { case Some(existingDir) => // Directory has children files in it, return them @@ -63,14 +72,20 @@ abstract class PartitioningAwareFileCatalog( // Directory does not exist, or has no children files Nil } - Partition(values, files) + PartitionDirectory(values, files) } } logTrace("Selected files after partition pruning:\n\t" + selectedPartitions.mkString("\n\t")) selectedPartitions } - override def allFiles(): Seq[FileStatus] = { + /** Returns the list of files that will be read when scanning this relation. */ + override def inputFiles: Array[String] = + allFiles().map(_.getPath.toUri.toString).toArray + + override def sizeInBytes: Long = allFiles().map(_.getLen).sum + + def allFiles(): Seq[FileStatus] = { if (partitionSpec().partitionColumns.isEmpty) { // For each of the root input paths, get the list of files inside them rootPaths.flatMap { path => @@ -139,7 +154,7 @@ abstract class PartitioningAwareFileCatalog( private def prunePartitions( predicates: Seq[Expression], - partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { + partitionSpec: PartitionSpec): Seq[PartitionPath] = { val PartitionSpec(partitionColumns, partitions) = partitionSpec val partitionColumnNames = partitionColumns.map(_.name).toSet val partitionPruningPredicates = predicates.filter { @@ -156,7 +171,7 @@ abstract class PartitioningAwareFileCatalog( }) val selected = partitions.filter { - case PartitionDirectory(values, _) => boundPredicate(values) + case PartitionPath(values, _) => boundPredicate(values) } logInfo { val total = partitions.length @@ -214,8 +229,186 @@ abstract class PartitioningAwareFileCatalog( val name = path.getName !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } + + /** + * List leaf files of given paths. This method will submit a Spark job to do parallel + * listing whenever there is a path having more files than the parallel partition discovery + * discovery threshold. + * + * This is publicly visible for testing. + */ + def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + val files = + if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + PartitioningAwareFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) + } else { + PartitioningAwareFileCatalog.listLeafFilesInSerial(paths, hadoopConf) + } + + HiveCatalogMetrics.incrementFilesDiscovered(files.size) + mutable.LinkedHashSet(files: _*) + } } -object PartitioningAwareFileCatalog { +object PartitioningAwareFileCatalog extends Logging { val BASE_PATH_PARAM = "basePath" + + /** A serializable variant of HDFS's BlockLocation. */ + private case class SerializableBlockLocation( + names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + + /** A serializable variant of HDFS's FileStatus. */ + private case class SerializableFileStatus( + path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long, + blockLocations: Array[SerializableBlockLocation]) + + /** + * List a collection of path recursively. + */ + private def listLeafFilesInSerial( + paths: Seq[Path], + hadoopConf: Configuration): Seq[FileStatus] = { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val filter = FileInputFormat.getInputPathFilter(jobConf) + + paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + listLeafFiles0(fs, path, filter) + } + } + + /** + * List a collection of path recursively in parallel (using Spark executors). + * Each task launched will use [[listLeafFilesInSerial]] to list. + */ + private def listLeafFilesInParallel( + paths: Seq[Path], + hadoopConf: Configuration, + sparkSession: SparkSession): Seq[FileStatus] = { + assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) + logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") + + val sparkContext = sparkSession.sparkContext + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, 10000) + + val statuses = sparkContext + .parallelize(serializedPaths, numParallelism) + .mapPartitions { paths => + val hadoopConf = serializableConfiguration.value + listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator + }.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + }.collect() + + // Turn SerializableFileStatus back to Status + statuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), + blockLocations) + } + } + + /** + * List a single path, provided as a FileStatus, in serial. + */ + private def listLeafFiles0( + fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { + logTrace(s"Listing $path") + val name = path.getName.toLowerCase + if (shouldFilterOut(name)) { + Seq.empty[FileStatus] + } else { + // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses = try fs.listStatus(path) catch { + case _: FileNotFoundException => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + val allLeafStatuses = { + val (dirs, files) = statuses.partition(_.isDirectory) + val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) + if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats + } + + allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { + case f: LocatedFileStatus => + f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. + case f => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + val locations = fs.getFileBlockLocations(f, 0, f.getLen) + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs + } + } + } + + /** Checks if we should filter out this path name. */ + def shouldFilterOut(pathName: String): Boolean = { + // We filter everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && + !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 381261cf65ca0..81bdabb7afdab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -34,8 +34,8 @@ import org.apache.spark.sql.types._ // TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. -object PartitionDirectory { - def apply(values: InternalRow, path: String): PartitionDirectory = +object PartitionPath { + def apply(values: InternalRow, path: String): PartitionPath = apply(values, new Path(path)) } @@ -43,14 +43,14 @@ object PartitionDirectory { * Holds a directory in a partitioned collection of files as well as as the partition values * in the form of a Row. Before scanning, the files at `path` need to be enumerated. */ -case class PartitionDirectory(values: InternalRow, path: Path) +case class PartitionPath(values: InternalRow, path: Path) case class PartitionSpec( partitionColumns: StructType, - partitions: Seq[PartitionDirectory]) + partitions: Seq[PartitionPath]) object PartitionSpec { - val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionDirectory]) + val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionPath]) } object PartitioningUtils { @@ -142,7 +142,7 @@ object PartitioningUtils { // Finally, we create `Partition`s based on paths and resolved partition values. val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map { case (PartitionValues(_, literals), (path, _)) => - PartitionDirectory(InternalRow.fromSeq(literals.map(_.value)), path) + PartitionPath(InternalRow.fromSeq(literals.map(_.value)), path) } PartitionSpec(StructType(fields), partitions) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala deleted file mode 100644 index 4807a92c2e6b8..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalog.scala +++ /dev/null @@ -1,225 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import java.io.FileNotFoundException - -import scala.collection.mutable - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs._ -import org.apache.hadoop.mapred.{FileInputFormat, JobConf} - -import org.apache.spark.internal.Logging -import org.apache.spark.metrics.source.HiveCatalogMetrics -import org.apache.spark.sql.SparkSession -import org.apache.spark.util.SerializableConfiguration - - -/** - * A base class for [[BasicFileCatalog]]s that need a [[SparkSession]] and the ability to find leaf - * files in a list of HDFS paths. - * - * @param sparkSession a [[SparkSession]] - * @param ignoreFileNotFound (see [[ListingFileCatalog]]) - */ -abstract class SessionFileCatalog(sparkSession: SparkSession) - extends BasicFileCatalog with Logging { - protected val hadoopConf: Configuration - - /** - * List leaf files of given paths. This method will submit a Spark job to do parallel - * listing whenever there is a path having more files than the parallel partition discovery - * discovery threshold. - * - * This is publicly visible for testing. - */ - def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - val files = - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - SessionFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) - } else { - SessionFileCatalog.listLeafFilesInSerial(paths, hadoopConf) - } - - HiveCatalogMetrics.incrementFilesDiscovered(files.size) - mutable.LinkedHashSet(files: _*) - } -} - -object SessionFileCatalog extends Logging { - - /** A serializable variant of HDFS's BlockLocation. */ - private case class SerializableBlockLocation( - names: Array[String], - hosts: Array[String], - offset: Long, - length: Long) - - /** A serializable variant of HDFS's FileStatus. */ - private case class SerializableFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long, - blockLocations: Array[SerializableBlockLocation]) - - /** - * List a collection of path recursively. - */ - private def listLeafFilesInSerial( - paths: Seq[Path], - hadoopConf: Configuration): Seq[FileStatus] = { - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val filter = FileInputFormat.getInputPathFilter(jobConf) - - paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - listLeafFiles0(fs, path, filter) - } - } - - /** - * List a collection of path recursively in parallel (using Spark executors). - * Each task launched will use [[listLeafFilesInSerial]] to list. - */ - private def listLeafFilesInParallel( - paths: Seq[Path], - hadoopConf: Configuration, - sparkSession: SparkSession): Seq[FileStatus] = { - assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) - logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") - - val sparkContext = sparkSession.sparkContext - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val serializedPaths = paths.map(_.toString) - - // Set the number of parallelism to prevent following file listing from generating many tasks - // in case of large #defaultParallelism. - val numParallelism = Math.min(paths.size, 10000) - - val statuses = sparkContext - .parallelize(serializedPaths, numParallelism) - .mapPartitions { paths => - val hadoopConf = serializableConfiguration.value - listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator - }.map { status => - // Turn FileStatus into SerializableFileStatus so we can send it back to the driver - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - SerializableBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[SerializableBlockLocation] - } - - SerializableFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) - }.collect() - - // Turn SerializableFileStatus back to Status - statuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) - } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), - blockLocations) - } - } - - /** - * List a single path, provided as a FileStatus, in serial. - */ - private def listLeafFiles0( - fs: FileSystem, path: Path, filter: PathFilter): Seq[FileStatus] = { - logTrace(s"Listing $path") - val name = path.getName.toLowerCase - if (shouldFilterOut(name)) { - Seq.empty[FileStatus] - } else { - // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist - // Note that statuses only include FileStatus for the files and dirs directly under path, - // and does not include anything else recursively. - val statuses = try fs.listStatus(path) catch { - case _: FileNotFoundException => - logWarning(s"The directory $path was not found. Was it deleted very recently?") - Array.empty[FileStatus] - } - - val allLeafStatuses = { - val (dirs, files) = statuses.partition(_.isDirectory) - val stats = files ++ dirs.flatMap(dir => listLeafFiles0(fs, dir.getPath, filter)) - if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats - } - - allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { - case f: LocatedFileStatus => - f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of - // paths exceeds threshold. - case f => - // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), - // which is very slow on some file system (RawLocalFileSystem, which is launch a - // subprocess and parse the stdout). - val locations = fs.getFileBlockLocations(f, 0, f.getLen) - val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, - f.getModificationTime, 0, null, null, null, null, f.getPath, locations) - if (f.isSymlink) { - lfs.setSymlink(f.getSymlink) - } - lfs - } - } - } - - /** Checks if we should filter out this path name. */ - def shouldFilterOut(pathName: String): Boolean = { - // We filter everything that starts with _ and ., except _common_metadata and _metadata - // because Parquet needs to find those metadata files from leaf files returned by this method. - // We should refactor this logic to not mix metadata files with data files. - ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && - !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala index a5c41b244589b..5648ab480a98a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types.StructType /** - * A [[BasicFileCatalog]] for a metastore catalog table. + * A [[FileCatalog]] for a metastore catalog table. * * @param sparkSession a [[SparkSession]] * @param db the table's database name @@ -38,10 +38,9 @@ class TableFileCatalog( db: String, table: String, partitionSchema: Option[StructType], - override val sizeInBytes: Long) - extends SessionFileCatalog(sparkSession) { + override val sizeInBytes: Long) extends FileCatalog { - override protected val hadoopConf = sparkSession.sessionState.newHadoopConf + protected val hadoopConf = sparkSession.sessionState.newHadoopConf private val externalCatalog = sparkSession.sharedState.externalCatalog @@ -51,7 +50,7 @@ class TableFileCatalog( override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq - override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { filterPartitions(filters).listFiles(Nil) } @@ -79,7 +78,7 @@ class TableFileCatalog( case Some(schema) => val selectedPartitions = externalCatalog.listPartitionsByFilter(db, table, filters) val partitions = selectedPartitions.map { p => - PartitionDirectory(p.toRow(schema), p.storage.locationUri.get) + PartitionPath(p.toRow(schema), p.storage.locationUri.get) } val partitionSpec = PartitionSpec(schema, partitions) new PrunedTableFileCatalog( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 2695974b84b00..9c43169cbf898 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -81,6 +81,16 @@ class FileCatalogSuite extends SharedSQLContext { } } + test("PartitioningAwareFileCatalog - file filtering") { + assert(!PartitioningAwareFileCatalog.shouldFilterOut("abcd")) + assert(PartitioningAwareFileCatalog.shouldFilterOut(".ab")) + assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd")) + assert(!PartitioningAwareFileCatalog.shouldFilterOut("_metadata")) + assert(!PartitioningAwareFileCatalog.shouldFilterOut("_common_metadata")) + assert(PartitioningAwareFileCatalog.shouldFilterOut("_ab_metadata")) + assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd_common_metadata")) + } + test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { class MockCatalog( override val rootPaths: Seq[Path]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala deleted file mode 100644 index df509583377ae..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SessionFileCatalogSuite.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import org.apache.spark.SparkFunSuite - -class SessionFileCatalogSuite extends SparkFunSuite { - - test("file filtering") { - assert(!SessionFileCatalog.shouldFilterOut("abcd")) - assert(SessionFileCatalog.shouldFilterOut(".ab")) - assert(SessionFileCatalog.shouldFilterOut("_cd")) - - assert(!SessionFileCatalog.shouldFilterOut("_metadata")) - assert(!SessionFileCatalog.shouldFilterOut("_common_metadata")) - assert(SessionFileCatalog.shouldFilterOut("_ab_metadata")) - assert(SessionFileCatalog.shouldFilterOut("_cd_common_metadata")) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 2ef66baee1eac..f2a209e91962d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -30,7 +30,8 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.execution.datasources.{FileCatalog, HadoopFsRelation, LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -632,10 +633,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(HadoopFsRelation(location: FileCatalog, _, _, _, _, _), _, _) => - assert(location.partitionSpec === PartitionSpec.emptySpec) + case LogicalRelation( + HadoopFsRelation(location: PartitioningAwareFileCatalog, _, _, _, _, _), _, _) => + assert(location.partitionSpec() === PartitionSpec.emptySpec) }.getOrElse { - fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") + fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 4a2aaa7d4f6ca..16e1e37b2fb02 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.{Partition => _, _} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ From bfe7885aee2f406c1bbde08e30809a0b4bb070d2 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 18 Oct 2016 13:36:00 -0700 Subject: [PATCH 121/177] [SPARK-17985][CORE] Bump commons-lang3 version to 3.5. ## What changes were proposed in this pull request? `SerializationUtils.clone()` of commons-lang3 (<3.5) has a bug that breaks thread safety, which gets stack sometimes caused by race condition of initializing hash map. See https://issues.apache.org/jira/browse/LANG-1251. ## How was this patch tested? Existing tests. Author: Takuya UESHIN Closes #15525 from ueshin/issues/SPARK-17985. --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- docs/streaming-flume-integration.md | 4 ++-- pom.xml | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index b30f8c347c0af..525dcef5b7d99 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -33,7 +33,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math-2.1.jar commons-math3-3.4.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 5b3a7651dd299..562fe6461e753 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -36,7 +36,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index e323efe30f64b..747521aa2a566 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -36,7 +36,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 77d97e5365b9f..afd4502c59d33 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -40,7 +40,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 572edfa0cc29e..687b855b649d8 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -40,7 +40,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index 767e1f9402e01..a5d36da5b6de9 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -115,11 +115,11 @@ Configuring Flume on the chosen machine requires the following two steps. artifactId = scala-library version = {{site.SCALA_VERSION}} - (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.3.2/commons-lang3-3.3.2.jar)). + (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.5/commons-lang3-3.5.jar)). groupId = org.apache.commons artifactId = commons-lang3 - version = 3.3.2 + version = 3.5 2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. diff --git a/pom.xml b/pom.xml index 7d13c51b2a596..aaf7cfa7eb2ad 100644 --- a/pom.xml +++ b/pom.xml @@ -168,7 +168,7 @@ 2.6 - 3.3.2 + 3.5 3.2.10 3.0.0 2.22.2 From 20dd11096cfda51e47b9dbe3b715a12ccbb4ce1d Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Tue, 18 Oct 2016 13:38:14 -0700 Subject: [PATCH 122/177] [MINOR][DOC] Add more built-in sources in sql-programming-guide.md ## What changes were proposed in this pull request? Add more built-in sources in sql-programming-guide.md. ## How was this patch tested? Manually. Author: Weiqing Yang Closes #15522 from weiqingy/dsDoc. --- docs/sql-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index dcc828cc69fed..3f1b73a830eca 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -422,8 +422,8 @@ In the simplest form, the default data source (`parquet` unless otherwise config You can also manually specify the data source that will be used along with any extra options that you would like to pass to the data source. Data sources are specified by their fully qualified name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short -names (`json`, `parquet`, `jdbc`). DataFrames loaded from any data source type can be converted into other types -using this syntax. +names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data +source type can be converted into other types using this syntax.
    From 4518642abd71bb1213a9efd72732102abf0bf7e7 Mon Sep 17 00:00:00 2001 From: Guoqiang Li Date: Tue, 18 Oct 2016 13:46:57 -0700 Subject: [PATCH 123/177] [SPARK-17930][CORE] The SerializerInstance instance used when deserializing a TaskResult is not reused ## What changes were proposed in this pull request? The following code is called when the DirectTaskResult instance is deserialized ```scala def value(): T = { if (valueObjectDeserialized) { valueObject } else { // Each deserialization creates a new instance of SerializerInstance, which is very time-consuming val resultSer = SparkEnv.get.serializer.newInstance() valueObject = resultSer.deserialize(valueBytes) valueObjectDeserialized = true valueObject } } ``` In the case of stage has a lot of tasks, reuse SerializerInstance instance can improve the scheduling performance of three times The test data is TPC-DS 2T (Parquet) and SQL statement as follows (query 2): ```sql select i_item_id, avg(ss_quantity) agg1, avg(ss_list_price) agg2, avg(ss_coupon_amt) agg3, avg(ss_sales_price) agg4 from store_sales, customer_demographics, date_dim, item, promotion where ss_sold_date_sk = d_date_sk and ss_item_sk = i_item_sk and ss_cdemo_sk = cd_demo_sk and ss_promo_sk = p_promo_sk and cd_gender = 'M' and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 2001 group by i_item_id order by i_item_id limit 100; ``` `spark-defaults.conf` file: ``` spark.master yarn-client spark.executor.instances 20 spark.driver.memory 16g spark.executor.memory 30g spark.executor.cores 5 spark.default.parallelism 100 spark.sql.shuffle.partitions 100000 spark.serializer org.apache.spark.serializer.KryoSerializer spark.driver.maxResultSize 0 spark.rpc.netty.dispatcher.numThreads 8 spark.executor.extraJavaOptions -XX:+UseG1GC -XX:+UseStringDeduplication -XX:G1HeapRegionSize=16M -XX:MetaspaceSize=256M spark.cleaner.referenceTracking.blocking true spark.cleaner.referenceTracking.blocking.shuffle true ``` Performance test results are as follows [SPARK-17930](https://github.com/witgo/spark/tree/SPARK-17930)| [ed14633](https://github.com/witgo/spark/commit/ed1463341455830b8867b721a1b34f291139baf3]) ------------ | ------------- 54.5 s|231.7 s ## How was this patch tested? Existing tests. Author: Guoqiang Li Closes #15512 from witgo/SPARK-17930. --- .../scala/org/apache/spark/scheduler/TaskResult.scala | 9 +++++---- .../org/apache/spark/scheduler/TaskResultGetter.scala | 10 +++++++++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 77fda6fcff959..366b92c5f2ada 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkEnv +import org.apache.spark.serializer.SerializerInstance import org.apache.spark.storage.BlockId import org.apache.spark.util.{AccumulatorV2, Utils} @@ -77,14 +78,14 @@ private[spark] class DirectTaskResult[T]( * * After the first time, `value()` is trivial and just returns the deserialized `valueObject`. */ - def value(): T = { + def value(resultSer: SerializerInstance = null): T = { if (valueObjectDeserialized) { valueObject } else { // This should not run when holding a lock because it may cost dozens of seconds for a large - // value. - val resultSer = SparkEnv.get.serializer.newInstance() - valueObject = resultSer.deserialize(valueBytes) + // value + val ser = if (resultSer == null) SparkEnv.get.serializer.newInstance() else resultSer + valueObject = ser.deserialize(valueBytes) valueObjectDeserialized = true valueObject } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 1c3fcbd4612a0..b1addc128e696 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -48,6 +48,12 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } } + protected val taskResultSerializer = new ThreadLocal[SerializerInstance] { + override def initialValue(): SerializerInstance = { + sparkEnv.serializer.newInstance() + } + } + def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, @@ -63,7 +69,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul // deserialize "value" without holding any lock so that it won't block other threads. // We should call it here, so that when it's called again in // "TaskSetManager.handleSuccessfulTask", it does not need to deserialize the value. - directResult.value() + directResult.value(taskResultSerializer.get()) (directResult, serializedData.limit()) case IndirectTaskResult(blockId, size) => if (!taskSetManager.canFetchMoreResults(size)) { @@ -84,6 +90,8 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get.toByteBuffer) + // force deserialization of referenced value + deserializedResult.value(taskResultSerializer.get()) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } From b3130c7b6a1ab4975023f08c3ab02ee8d2c7e995 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 18 Oct 2016 13:49:02 -0700 Subject: [PATCH 124/177] [SPARK-17955][SQL] Make DataFrameReader.jdbc call DataFrameReader.format("jdbc").load ## What changes were proposed in this pull request? This PR proposes to make `DataFrameReader.jdbc` call `DataFrameReader.format("jdbc").load` consistently with other APIs in `DataFrameReader`/`DataFrameWriter` and avoid calling `sparkSession.baseRelationToDataFrame(..)` here and there. The changes were mostly copied from `DataFrameWriter.jdbc()` which was recently updated. ```diff - val params = extraOptions.toMap ++ connectionProperties.asScala.toMap - val options = new JDBCOptions(url, table, params) - val relation = JDBCRelation(parts, options)(sparkSession) - sparkSession.baseRelationToDataFrame(relation) + this.extraOptions = this.extraOptions ++ connectionProperties.asScala + // explicit url and dbtable should override all + this.extraOptions += ("url" -> url, "dbtable" -> table) + format("jdbc").load() ``` ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon Closes #15499 from HyukjinKwon/SPARK-17955. --- .../main/scala/org/apache/spark/sql/DataFrameReader.scala | 8 ++++---- .../main/scala/org/apache/spark/sql/DataFrameWriter.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index ac3358592202f..b7b2203cdd85b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -232,10 +232,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { parts: Array[Partition], connectionProperties: Properties): DataFrame = { // connectionProperties should override settings in extraOptions. - val params = extraOptions.toMap ++ connectionProperties.asScala.toMap - val options = new JDBCOptions(url, table, params) - val relation = JDBCRelation(parts, options)(sparkSession) - sparkSession.baseRelationToDataFrame(relation) + this.extraOptions = this.extraOptions ++ connectionProperties.asScala + // explicit url and dbtable should override all + this.extraOptions += ("url" -> url, "dbtable" -> table) + format("jdbc").load() } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 35ef050dcb169..5be3277651d02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -426,8 +426,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { assertNotPartitioned("jdbc") assertNotBucketed("jdbc") - // connectionProperties should override settings in extraOptions - this.extraOptions = this.extraOptions ++ (connectionProperties.asScala) + // connectionProperties should override settings in extraOptions. + this.extraOptions = this.extraOptions ++ connectionProperties.asScala // explicit url and dbtable should override all this.extraOptions += ("url" -> url, "dbtable" -> table) format("jdbc").save() From cd662bc7a2050264f40650442858a85c4827b608 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 18 Oct 2016 13:56:35 -0700 Subject: [PATCH 125/177] Revert "[SPARK-17985][CORE] Bump commons-lang3 version to 3.5." This reverts commit bfe7885aee2f406c1bbde08e30809a0b4bb070d2. The commit caused build failures on Hadoop 2.2 profile: ``` [error] /scratch/rxin/spark/core/src/main/scala/org/apache/spark/util/Utils.scala:1489: value read is not a member of object org.apache.commons.io.IOUtils [error] var numBytes = IOUtils.read(gzInputStream, buf) [error] ^ [error] /scratch/rxin/spark/core/src/main/scala/org/apache/spark/util/Utils.scala:1492: value read is not a member of object org.apache.commons.io.IOUtils [error] numBytes = IOUtils.read(gzInputStream, buf) [error] ^ ``` --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- docs/streaming-flume-integration.md | 4 ++-- pom.xml | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 525dcef5b7d99..b30f8c347c0af 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -33,7 +33,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.5.jar +commons-lang3-3.3.2.jar commons-logging-1.1.3.jar commons-math-2.1.jar commons-math3-3.4.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 562fe6461e753..5b3a7651dd299 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -36,7 +36,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.5.jar +commons-lang3-3.3.2.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 747521aa2a566..e323efe30f64b 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -36,7 +36,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.5.jar +commons-lang3-3.3.2.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index afd4502c59d33..77d97e5365b9f 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -40,7 +40,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.5.jar +commons-lang3-3.3.2.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 687b855b649d8..572edfa0cc29e 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -40,7 +40,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.5.jar +commons-lang3-3.3.2.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index a5d36da5b6de9..767e1f9402e01 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -115,11 +115,11 @@ Configuring Flume on the chosen machine requires the following two steps. artifactId = scala-library version = {{site.SCALA_VERSION}} - (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.5/commons-lang3-3.5.jar)). + (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.3.2/commons-lang3-3.3.2.jar)). groupId = org.apache.commons artifactId = commons-lang3 - version = 3.5 + version = 3.3.2 2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. diff --git a/pom.xml b/pom.xml index aaf7cfa7eb2ad..7d13c51b2a596 100644 --- a/pom.xml +++ b/pom.xml @@ -168,7 +168,7 @@ 2.6 - 3.5 + 3.3.2 3.2.10 3.0.0 2.22.2 From cd106b050ff789b6de539956a7f01159ab15c820 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 18 Oct 2016 14:01:49 -0700 Subject: [PATCH 126/177] [SPARK-17841][STREAMING][KAFKA] drain commitQueue ## What changes were proposed in this pull request? Actually drain commit queue rather than just iterating it. iterator() on a concurrent linked queue won't remove items from the queue, poll() will. ## How was this patch tested? Unit tests Author: cody koeninger Closes #15407 from koeninger/SPARK-17841. --- .../spark/streaming/kafka010/DirectKafkaInputDStream.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index 432537ebf05b2..7e57bb18cbd50 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -282,13 +282,13 @@ private[spark] class DirectKafkaInputDStream[K, V]( protected def commitAll(): Unit = { val m = new ju.HashMap[TopicPartition, OffsetAndMetadata]() - val it = commitQueue.iterator() - while (it.hasNext) { - val osr = it.next + var osr = commitQueue.poll() + while (null != osr) { val tp = osr.topicPartition val x = m.get(tp) val offset = if (null == x) { osr.untilOffset } else { Math.max(x.offset, osr.untilOffset) } m.put(tp, new OffsetAndMetadata(offset)) + osr = commitQueue.poll() } if (!m.isEmpty) { consumer.commitAsync(m, commitCallback.get) From 1e35e969305555dda02cb0788c8143e5f2e1944b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 18 Oct 2016 14:25:10 -0700 Subject: [PATCH 127/177] [SPARK-17817] [PYSPARK] [FOLLOWUP] PySpark RDD Repartitioning Results in Highly Skewed Partition Sizes ## What changes were proposed in this pull request? This change is a followup for #15389 which calls `_to_java_object_rdd()` to solve this issue. Due to the concern of the possible expensive cost of the call, we can choose to decrease the batch size to solve this issue too. Simple benchmark: import time num_partitions = 20000 a = sc.parallelize(range(int(1e6)), 2) start = time.time() l = a.repartition(num_partitions).glom().map(len).collect() end = time.time() print(end - start) Before: 419.447577953 _to_java_object_rdd(): 421.916361094 decreasing the batch size: 423.712255955 ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #15445 from viirya/repartition-batch-size. --- python/pyspark/rdd.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 0e2ae19ca39aa..2de2c2fd1a60b 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2029,12 +2029,12 @@ def coalesce(self, numPartitions, shuffle=False): [[1, 2, 3, 4, 5]] """ if shuffle: - # In Scala's repartition code, we will distribute elements evenly across output - # partitions. However, the RDD from Python is serialized as a single binary data, - # so the distribution fails and produces highly skewed partitions. We need to - # convert it to a RDD of java object before repartitioning. - data_java_rdd = self._to_java_object_rdd().coalesce(numPartitions, shuffle) - jrdd = self.ctx._jvm.SerDeUtil.javaToPython(data_java_rdd) + # Decrease the batch size in order to distribute evenly the elements across output + # partitions. Otherwise, repartition will possibly produce highly skewed partitions. + batchSize = min(10, self.ctx._batchSize or 1024) + ser = BatchedSerializer(PickleSerializer(), batchSize) + selfCopy = self._reserialize(ser) + jrdd = selfCopy._jrdd.coalesce(numPartitions, shuffle) else: jrdd = self._jrdd.coalesce(numPartitions, shuffle) return RDD(jrdd, self.ctx, self._jrdd_deserializer) From 941b3f9aca59e62c078508a934f8c2221ced96ce Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 18 Oct 2016 17:32:16 -0700 Subject: [PATCH 128/177] [SPARK-17731][SQL][STREAMING][FOLLOWUP] Refactored StreamingQueryListener APIs ## What changes were proposed in this pull request? As per rxin request, here are further API changes - Changed `Stream(Started/Progress/Terminated)` events to `Stream*Event` - Changed the fields in `StreamingQueryListener.on***` from `query*` to `event` ## How was this patch tested? Existing unit tests. Author: Tathagata Das Closes #15530 from tdas/SPARK-17731-1. --- project/MimaExcludes.scala | 9 +++++++++ .../sql/execution/streaming/StreamExecution.scala | 15 ++++++++------- .../streaming/StreamingQueryListenerBus.scala | 8 ++++---- .../sql/streaming/StreamingQueryListener.scala | 14 +++++++------- .../apache/spark/sql/streaming/StreamTest.scala | 6 +++--- .../streaming/StreamingQueryListenerSuite.scala | 13 +++++++------ .../spark/sql/streaming/StreamingQuerySuite.scala | 6 +++--- 7 files changed, 41 insertions(+), 30 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1349af4219c16..facf034ea7e7d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -68,6 +68,15 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryStarted"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated"), // [SPARK-17338][SQL] add global temp view ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 9144736c940f5..ba8cf808e339c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -165,7 +165,7 @@ class StreamExecution( new Path(new Path(checkpointRoot), name).toUri.toString /** - * Starts the execution. This returns only after the thread has started and [[QueryStarted]] event + * Starts the execution. This returns only after the thread has started and [[QueryStartedEvent]] * has been posted to all the listeners. */ def start(): Unit = { @@ -177,9 +177,10 @@ class StreamExecution( /** * Repeatedly attempts to run batches as data arrives. * - * Note that this method ensures that [[QueryStarted]] and [[QueryTerminated]] events are posted - * such that listeners are guaranteed to get a start event before a termination. Furthermore, this - * method also ensures that [[QueryStarted]] event is posted before the `start()` method returns. + * Note that this method ensures that [[QueryStartedEvent]] and [[QueryTerminatedEvent]] are + * posted such that listeners are guaranteed to get a start event before a termination. + * Furthermore, this method also ensures that [[QueryStartedEvent]] event is posted before the + * `start()` method returns. */ private def runBatches(): Unit = { try { @@ -190,7 +191,7 @@ class StreamExecution( sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) } updateStatus() - postEvent(new QueryStarted(currentStatus)) // Assumption: Does not throw exception. + postEvent(new QueryStartedEvent(currentStatus)) // Assumption: Does not throw exception. // Unblock starting thread startLatch.countDown() @@ -232,7 +233,7 @@ class StreamExecution( // Update metrics and notify others streamMetrics.reportTriggerFinished() updateStatus() - postEvent(new QueryProgress(currentStatus)) + postEvent(new QueryProgressEvent(currentStatus)) isTerminated }) } catch { @@ -260,7 +261,7 @@ class StreamExecution( // Notify others sparkSession.streams.notifyQueryTermination(StreamExecution.this) postEvent( - new QueryTerminated(currentStatus, exception.map(_.cause).map(Utils.exceptionString))) + new QueryTerminatedEvent(currentStatus, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index 1e663956f980b..fc2190d39da4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -40,7 +40,7 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) */ def post(event: StreamingQueryListener.Event) { event match { - case s: QueryStarted => + case s: QueryStartedEvent => postToAll(s) case _ => sparkListenerBus.post(event) @@ -59,11 +59,11 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) listener: StreamingQueryListener, event: StreamingQueryListener.Event): Unit = { event match { - case queryStarted: QueryStarted => + case queryStarted: QueryStartedEvent => listener.onQueryStarted(queryStarted) - case queryProgress: QueryProgress => + case queryProgress: QueryProgressEvent => listener.onQueryProgress(queryProgress) - case queryTerminated: QueryTerminated => + case queryTerminated: QueryTerminatedEvent => listener.onQueryTerminated(queryTerminated) case _ => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 69790e33b2168..9e311fae842be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -41,7 +41,7 @@ abstract class StreamingQueryListener { * don't block this method as it will block your query. * @since 2.0.0 */ - def onQueryStarted(queryStarted: QueryStarted): Unit + def onQueryStarted(event: QueryStartedEvent): Unit /** * Called when there is some status update (ingestion rate updated, etc.) @@ -49,16 +49,16 @@ abstract class StreamingQueryListener { * @note This method is asynchronous. The status in [[StreamingQuery]] will always be * latest no matter when this method is called. Therefore, the status of [[StreamingQuery]] * may be changed before/when you process the event. E.g., you may find [[StreamingQuery]] - * is terminated when you are processing [[QueryProgress]]. + * is terminated when you are processing [[QueryProgressEvent]]. * @since 2.0.0 */ - def onQueryProgress(queryProgress: QueryProgress): Unit + def onQueryProgress(event: QueryProgressEvent): Unit /** * Called when a query is stopped, with or without error. * @since 2.0.0 */ - def onQueryTerminated(queryTerminated: QueryTerminated): Unit + def onQueryTerminated(event: QueryTerminatedEvent): Unit } @@ -84,7 +84,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryStarted private[sql](val queryStatus: StreamingQueryStatus) extends Event + class QueryStartedEvent private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -92,7 +92,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryProgress private[sql](val queryStatus: StreamingQueryStatus) extends Event + class QueryProgressEvent private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -104,7 +104,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryTerminated private[sql]( + class QueryTerminatedEvent private[sql]( val queryStatus: StreamingQueryStatus, val exception: Option[String]) extends Event } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 8dfeb8da4b826..742833065144d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -684,20 +684,20 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } - override def onQueryStarted(queryStarted: QueryStarted): Unit = { + override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { asyncTestWaiter { startStatus = queryStarted.queryStatus } } - override def onQueryProgress(queryProgress: QueryProgress): Unit = { + override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryProgress called before onQueryStarted") synchronized { progressStatuses += queryProgress.queryStatus } } } - override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { + override def onQueryTerminated(queryTerminated: QueryTerminatedEvent): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryTerminated called before onQueryStarted") terminationStatus = queryTerminated.queryStatus diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 623f66a778eac..ff843865a017e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -177,30 +177,31 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStarted = new StreamingQueryListener.QueryStarted(StreamingQueryStatus.testStatus) + val queryStarted = new StreamingQueryListener.QueryStartedEvent(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[StreamingQueryListener.QueryStarted] + .asInstanceOf[StreamingQueryListener.QueryStartedEvent] assertStreamingQueryInfoEquals(queryStarted.queryStatus, newQueryStarted.queryStatus) } test("QueryProgress serialization") { - val queryProcess = new StreamingQueryListener.QueryProgress(StreamingQueryStatus.testStatus) + val queryProcess = new StreamingQueryListener.QueryProgressEvent( + StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[StreamingQueryListener.QueryProgress] + .asInstanceOf[StreamingQueryListener.QueryProgressEvent] assertStreamingQueryInfoEquals(queryProcess.queryStatus, newQueryProcess.queryStatus) } test("QueryTerminated serialization") { val exception = new RuntimeException("exception") - val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( + val queryQueryTerminated = new StreamingQueryListener.QueryTerminatedEvent( StreamingQueryStatus.testStatus, Some(exception.getMessage)) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[StreamingQueryListener.QueryTerminated] + .asInstanceOf[StreamingQueryListener.QueryTerminatedEvent] assertStreamingQueryInfoEquals(queryQueryTerminated.queryStatus, newQueryTerminated.queryStatus) assert(queryQueryTerminated.exception === newQueryTerminated.exception) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 9f8e2db966367..92020be9789fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -290,11 +290,11 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { // A StreamingQueryListener that gets the query status after the first completed trigger val listener = new StreamingQueryListener { @volatile var firstStatus: StreamingQueryStatus = null - override def onQueryStarted(queryStarted: QueryStarted): Unit = { } - override def onQueryProgress(queryProgress: QueryProgress): Unit = { + override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { } + override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = { if (firstStatus == null) firstStatus = queryProgress.queryStatus } - override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { } + override def onQueryTerminated(queryTerminated: QueryTerminatedEvent): Unit = { } } try { From 5f20ae0394388574a3767daf7f499c89658f61e1 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 19 Oct 2016 10:20:12 +0800 Subject: [PATCH 129/177] [SPARK-17980][SQL] Fix refreshByPath for converted Hive tables ## What changes were proposed in this pull request? There was a bug introduced in https://github.com/apache/spark/pull/14690 which broke refreshByPath with converted hive tables (though, it turns out it was very difficult to refresh converted hive tables anyways, since you had to specify the exact path of one of the partitions). This changes refreshByPath to invalidate by prefix instead of exact match, and fixes the issue. cc sameeragarwal for refreshByPath changes mallman ## How was this patch tested? Extended unit test. Author: Eric Liang Closes #15521 from ericl/fix-caching. --- .../apache/spark/sql/catalog/Catalog.scala | 3 ++- .../spark/sql/execution/CacheManager.scala | 5 +++-- .../datasources/TableFileCatalog.scala | 18 ++++++++++++---- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../sql/hive/HiveMetadataCacheSuite.scala | 21 +++++++++++++++++-- 5 files changed, 39 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 889b8a02784d6..aecdda1c36498 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -343,7 +343,8 @@ abstract class Catalog { /** * Invalidate and refresh all the cached data (and the associated metadata) for any dataframe that - * contains the given data source path. + * contains the given data source path. Path matching is by prefix, i.e. "/" would invalidate + * everything that is cached. * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 92fd366e101fd..fb72c679e3628 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -185,9 +185,10 @@ class CacheManager extends Logging { plan match { case lr: LogicalRelation => lr.relation match { case hr: HadoopFsRelation => + val prefixToInvalidate = qualifiedPath.toString val invalidate = hr.location.rootPaths - .map(_.makeQualified(fs.getUri, fs.getWorkingDirectory)) - .contains(qualifiedPath) + .map(_.makeQualified(fs.getUri, fs.getWorkingDirectory).toString) + .exists(_.startsWith(prefixToInvalidate)) if (invalidate) hr.location.refresh() invalidate case _ => false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala index 5648ab480a98a..fc08c3798ee06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -48,13 +48,18 @@ class TableFileCatalog( private val baseLocation = catalogTable.storage.locationUri + // Populated on-demand by calls to cachedAllPartitions + private var cachedAllPartitions: ListingFileCatalog = null + override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { filterPartitions(filters).listFiles(Nil) } - override def refresh(): Unit = {} + override def refresh(): Unit = synchronized { + cachedAllPartitions = null + } /** * Returns a [[ListingFileCatalog]] for this table restricted to the subset of partitions @@ -64,7 +69,7 @@ class TableFileCatalog( */ def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = { if (filters.isEmpty) { - cachedAllPartitions + allPartitions } else { filterPartitions0(filters) } @@ -89,9 +94,14 @@ class TableFileCatalog( } // Not used in the hot path of queries when metastore partition pruning is enabled - lazy val cachedAllPartitions: ListingFileCatalog = filterPartitions0(Nil) + def allPartitions: ListingFileCatalog = synchronized { + if (cachedAllPartitions == null) { + cachedAllPartitions = filterPartitions0(Nil) + } + cachedAllPartitions + } - override def inputFiles: Array[String] = cachedAllPartitions.inputFiles + override def inputFiles: Array[String] = allPartitions.inputFiles } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 16e1e37b2fb02..c909eb5d20bcd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -235,7 +235,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log if (lazyPruningEnabled) { catalog } else { - catalog.cachedAllPartitions + catalog.allPartitions } } val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala index 7af81a3a90504..2ca1cd4c07fdb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -80,9 +80,13 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi val df = spark.sql("select * from test") assert(sql("select * from test").count() == 5) + def deleteRandomFile(): Unit = { + val p = new Path(spark.table("test").inputFiles.head) + assert(p.getFileSystem(hiveContext.sessionState.newHadoopConf()).delete(p, true)) + } + // Delete a file, then assert that we tried to read it. This means the table was cached. - val p = new Path(spark.table("test").inputFiles.head) - assert(p.getFileSystem(hiveContext.sessionState.newHadoopConf()).delete(p, true)) + deleteRandomFile() val e = intercept[SparkException] { sql("select * from test").count() } @@ -91,6 +95,19 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi // Test refreshing the cache. spark.catalog.refreshTable("test") assert(sql("select * from test").count() == 4) + assert(spark.table("test").inputFiles.length == 4) + + // Test refresh by path separately since it goes through different code paths than + // refreshTable does. + deleteRandomFile() + spark.catalog.cacheTable("test") + spark.catalog.refreshByPath("/some-invalid-path") // no-op + val e2 = intercept[SparkException] { + sql("select * from test").count() + } + assert(e2.getMessage.contains("FileNotFoundException")) + spark.catalog.refreshByPath(dir.getAbsolutePath) + assert(sql("select * from test").count() == 3) } } } From 2629cd74602cfe77188b76428fed62a7a7149315 Mon Sep 17 00:00:00 2001 From: Yu Peng Date: Tue, 18 Oct 2016 19:43:08 -0700 Subject: [PATCH 130/177] [SPARK-17711][TEST-HADOOP2.2] Fix hadoop2.2 compilation error ## What changes were proposed in this pull request? Fix hadoop2.2 compilation error. ## How was this patch tested? Existing tests. cc tdas zsxwing Author: Yu Peng Closes #15537 from loneknightpy/fix-17711. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index a4da138e71992..7fba901b85695 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -42,7 +42,6 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.google.common.io.{ByteStreams, Files => GFiles} import com.google.common.net.InetAddresses -import org.apache.commons.io.IOUtils import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} @@ -1486,10 +1485,10 @@ private[spark] object Utils extends Logging { val gzInputStream = new GZIPInputStream(new FileInputStream(file)) val bufSize = 1024 val buf = new Array[Byte](bufSize) - var numBytes = IOUtils.read(gzInputStream, buf) + var numBytes = ByteStreams.read(gzInputStream, buf, 0, bufSize) while (numBytes > 0) { fileSize += numBytes - numBytes = IOUtils.read(gzInputStream, buf) + numBytes = ByteStreams.read(gzInputStream, buf, 0, bufSize) } fileSize } catch { From 4329c5cea4d235dc582fdb7cbdb822f62e650f5d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 18 Oct 2016 20:23:13 -0700 Subject: [PATCH 131/177] [SPARK-17873][SQL] ALTER TABLE RENAME TO should allow users to specify database in destination table name(but have to be same as source table) ## What changes were proposed in this pull request? Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`. However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO. This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc. ## How was this patch tested? The added back tests and some new tests. Author: Wenchen Fan Closes #15434 from cloud-fan/revert. --- .../sql/catalyst/catalog/SessionCatalog.scala | 18 +++++++- .../catalog/SessionCatalogSuite.scala | 23 ++++++---- .../spark/sql/execution/SparkSqlParser.scala | 10 +--- .../spark/sql/execution/command/tables.scala | 7 ++- .../execution/command/DDLCommandSuite.scala | 18 ++++---- .../sql/execution/command/DDLSuite.scala | 46 +++++++++++++++++-- 6 files changed, 87 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index fe41c41a6eb20..9711131d88a05 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -462,11 +462,20 @@ class SessionCatalog( * If a database is specified in `oldName`, this will rename the table in that database. * If no database is specified, this will first attempt to rename a temporary table with * the same name, then, if that does not exist, rename the table in the current database. + * + * This assumes the database specified in `newName` matches the one in `oldName`. */ - def renameTable(oldName: TableIdentifier, newName: String): Unit = synchronized { + def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = synchronized { val db = formatDatabaseName(oldName.database.getOrElse(currentDb)) + newName.database.map(formatDatabaseName).foreach { newDb => + if (db != newDb) { + throw new AnalysisException( + s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") + } + } + val oldTableName = formatTableName(oldName.table) - val newTableName = formatTableName(newName) + val newTableName = formatTableName(newName.table) if (db == globalTempViewManager.database) { globalTempViewManager.rename(oldTableName, newTableName) } else { @@ -476,6 +485,11 @@ class SessionCatalog( requireTableNotExists(TableIdentifier(newTableName, Some(db))) externalCatalog.renameTable(db, oldTableName, newTableName) } else { + if (newName.database.isDefined) { + throw new AnalysisException( + s"RENAME TEMPORARY TABLE from '$oldName' to '$newName': cannot specify database " + + s"name '${newName.database.get}' in the destination table") + } if (tempTables.contains(newTableName)) { throw new AnalysisException(s"RENAME TEMPORARY TABLE from '$oldName' to '$newName': " + "destination table already exists") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 915ed8f8b1787..187611bc77460 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -273,27 +273,34 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) - sessionCatalog.renameTable(TableIdentifier("tbl1", Some("db2")), "tblone") + sessionCatalog.renameTable(TableIdentifier("tbl1", Some("db2")), TableIdentifier("tblone")) assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbl2")) - sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), "tbltwo") + sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo")) assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbltwo")) // Rename table without explicitly specifying database sessionCatalog.setCurrentDatabase("db2") - sessionCatalog.renameTable(TableIdentifier("tbltwo"), "table_two") + sessionCatalog.renameTable(TableIdentifier("tbltwo"), TableIdentifier("table_two")) assert(externalCatalog.listTables("db2").toSet == Set("tblone", "table_two")) + // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match + intercept[AnalysisException] { + sessionCatalog.renameTable( + TableIdentifier("tblone", Some("db2")), TableIdentifier("tblones", Some("db1"))) + } // The new table already exists intercept[TableAlreadyExistsException] { - sessionCatalog.renameTable(TableIdentifier("tblone", Some("db2")), "table_two") + sessionCatalog.renameTable( + TableIdentifier("tblone", Some("db2")), + TableIdentifier("table_two")) } } test("rename table when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[NoSuchDatabaseException] { - catalog.renameTable(TableIdentifier("tbl1", Some("unknown_db")), "tbl2") + catalog.renameTable(TableIdentifier("tbl1", Some("unknown_db")), TableIdentifier("tbl2")) } intercept[NoSuchTableException] { - catalog.renameTable(TableIdentifier("unknown_table", Some("db2")), "tbl2") + catalog.renameTable(TableIdentifier("unknown_table", Some("db2")), TableIdentifier("tbl2")) } } @@ -306,12 +313,12 @@ class SessionCatalogSuite extends SparkFunSuite { assert(sessionCatalog.getTempView("tbl1") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be renamed first - sessionCatalog.renameTable(TableIdentifier("tbl1"), "tbl3") + sessionCatalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3")) assert(sessionCatalog.getTempView("tbl1").isEmpty) assert(sessionCatalog.getTempView("tbl3") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is specified, temp tables are never renamed - sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), "tbl4") + sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4")) assert(sessionCatalog.getTempView("tbl3") == Option(tempTable)) assert(sessionCatalog.getTempView("tbl4").isEmpty) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8c68d1e3a2379..ea22b02d40b80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -689,15 +689,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { - val fromName = visitTableIdentifier(ctx.from) - val toName = visitTableIdentifier(ctx.to) - if (toName.database.isDefined) { - operationNotAllowed("Can not specify database in table/view name after RENAME TO", ctx) - } - AlterTableRenameCommand( - fromName, - toName.table, + visitTableIdentifier(ctx.from), + visitTableIdentifier(ctx.to), ctx.VIEW != null) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 424ef58d76c5e..403b479a0e1bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -146,7 +146,7 @@ case class CreateTableCommand(table: CatalogTable, ifNotExists: Boolean) extends */ case class AlterTableRenameCommand( oldName: TableIdentifier, - newName: String, + newName: TableIdentifier, isView: Boolean) extends RunnableCommand { @@ -159,7 +159,6 @@ case class AlterTableRenameCommand( } else { val table = catalog.getTableMetadata(oldName) DDLUtils.verifyAlterTableType(catalog, table, isView) - val newTblName = TableIdentifier(newName, oldName.database) // If an exception is thrown here we can just assume the table is uncached; // this can happen with Hive tables when the underlying catalog is in-memory. val wasCached = Try(sparkSession.catalog.isCached(oldName.unquotedString)).getOrElse(false) @@ -172,7 +171,7 @@ case class AlterTableRenameCommand( } // For datasource tables, we also need to update the "path" serde property if (DDLUtils.isDatasourceTable(table) && table.tableType == CatalogTableType.MANAGED) { - val newPath = catalog.defaultTablePath(newTblName) + val newPath = catalog.defaultTablePath(newName) val newTable = table.withNewStorage( properties = table.storage.properties ++ Map("path" -> newPath)) catalog.alterTable(newTable) @@ -182,7 +181,7 @@ case class AlterTableRenameCommand( catalog.refreshTable(oldName) catalog.renameTable(oldName, newName) if (wasCached) { - sparkSession.catalog.cacheTable(newTblName.unquotedString) + sparkSession.catalog.cacheTable(newName.unquotedString) } } Seq.empty[Row] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 547fb63813750..a3dbc9234f2f3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -387,20 +387,22 @@ class DDLCommandSuite extends PlanTest { val parsed_table = parser.parsePlan(sql_table) val parsed_view = parser.parsePlan(sql_view) val expected_table = AlterTableRenameCommand( - TableIdentifier("table_name", None), - "new_table_name", + TableIdentifier("table_name"), + TableIdentifier("new_table_name"), isView = false) val expected_view = AlterTableRenameCommand( - TableIdentifier("table_name", None), - "new_table_name", + TableIdentifier("table_name"), + TableIdentifier("new_table_name"), isView = true) comparePlans(parsed_table, expected_table) comparePlans(parsed_view, expected_view) + } - val e = intercept[ParseException]( - parser.parsePlan("ALTER TABLE db1.tbl RENAME TO db1.tbl2") - ) - assert(e.getMessage.contains("Can not specify database in table/view name after RENAME TO")) + test("alter table: rename table with database") { + val query = "ALTER TABLE db1.tbl RENAME TO db1.tbl2" + val plan = parseAs[AlterTableRenameCommand](query) + assert(plan.oldName == TableIdentifier("tbl", Some("db1"))) + assert(plan.newName == TableIdentifier("tbl2", Some("db1"))) } // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 097dc2441351f..c8b8e9ebabc75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -665,16 +665,27 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { createDatabase(catalog, "dbx") createDatabase(catalog, "dby") createTable(catalog, tableIdent1) + assert(catalog.listTables("dbx") == Seq(tableIdent1)) - sql("ALTER TABLE dbx.tab1 RENAME TO tab2") + sql("ALTER TABLE dbx.tab1 RENAME TO dbx.tab2") assert(catalog.listTables("dbx") == Seq(tableIdent2)) + + // The database in destination table name can be omitted, and we will use the database of source + // table for it. + sql("ALTER TABLE dbx.tab2 RENAME TO tab1") + assert(catalog.listTables("dbx") == Seq(tableIdent1)) + catalog.setCurrentDatabase("dbx") // rename without explicitly specifying database - sql("ALTER TABLE tab2 RENAME TO tab1") - assert(catalog.listTables("dbx") == Seq(tableIdent1)) + sql("ALTER TABLE tab1 RENAME TO tab2") + assert(catalog.listTables("dbx") == Seq(tableIdent2)) // table to rename does not exist intercept[AnalysisException] { - sql("ALTER TABLE dbx.does_not_exist RENAME TO tab2") + sql("ALTER TABLE dbx.does_not_exist RENAME TO dbx.tab2") + } + // destination database is different + intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 RENAME TO dby.tab2") } } @@ -696,6 +707,31 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(spark.table("teachers").collect().toSeq == df.collect().toSeq) } + test("rename temporary table - destination table with database name") { + withTempView("tab1") { + sql( + """ + |CREATE TEMPORARY TABLE tab1 + |USING org.apache.spark.sql.sources.DDLScanSource + |OPTIONS ( + | From '1', + | To '10', + | Table 'test1' + |) + """.stripMargin) + + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab1 RENAME TO default.tab2") + } + assert(e.getMessage.contains( + "RENAME TEMPORARY TABLE from '`tab1`' to '`default`.`tab2`': " + + "cannot specify database name 'default' in the destination table")) + + val catalog = spark.sessionState.catalog + assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"))) + } + } + test("rename temporary table") { withTempView("tab1", "tab2") { spark.range(10).createOrReplaceTempView("tab1") @@ -736,7 +772,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("ALTER TABLE tab1 RENAME TO tab2") } assert(e.getMessage.contains( - "RENAME TEMPORARY TABLE from '`tab1`' to 'tab2': destination table already exists")) + "RENAME TEMPORARY TABLE from '`tab1`' to '`tab2`': destination table already exists")) val catalog = spark.sessionState.catalog assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"), TableIdentifier("tab2"))) From f39852e59883c214b0d007faffb406570ea3084b Mon Sep 17 00:00:00 2001 From: Tommy YU Date: Tue, 18 Oct 2016 21:15:32 -0700 Subject: [PATCH 132/177] [SPARK-18001][DOCUMENT] fix broke link to SparkDataFrame ## What changes were proposed in this pull request? In http://spark.apache.org/docs/latest/sql-programming-guide.html, Section "Untyped Dataset Operations (aka DataFrame Operations)" Link to R DataFrame doesn't work that return The requested URL /docs/latest/api/R/DataFrame.html was not found on this server. Correct link is SparkDataFrame.html for spark 2.0 ## How was this patch tested? Manual checked. Author: Tommy YU Closes #15543 from Wenpei/spark-18001. --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3f1b73a830eca..d334a86bc73d7 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -140,7 +140,7 @@ As an example, the following creates a DataFrame based on the content of a JSON ## Untyped Dataset Operations (aka DataFrame Operations) -DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/DataFrame.html). +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. From 9540357ada7df1acfefa7b775c82675cd475244c Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 19 Oct 2016 10:06:43 +0100 Subject: [PATCH 133/177] [SPARK-17985][CORE] Bump commons-lang3 version to 3.5. ## What changes were proposed in this pull request? `SerializationUtils.clone()` of commons-lang3 (<3.5) has a bug that breaks thread safety, which gets stack sometimes caused by race condition of initializing hash map. See https://issues.apache.org/jira/browse/LANG-1251. ## How was this patch tested? Existing tests. Author: Takuya UESHIN Closes #15548 from ueshin/issues/SPARK-17985. --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- docs/streaming-flume-integration.md | 4 ++-- pom.xml | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index b30f8c347c0af..525dcef5b7d99 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -33,7 +33,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math-2.1.jar commons-math3-3.4.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 5b3a7651dd299..562fe6461e753 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -36,7 +36,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index e323efe30f64b..747521aa2a566 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -36,7 +36,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 77d97e5365b9f..afd4502c59d33 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -40,7 +40,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 572edfa0cc29e..687b855b649d8 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -40,7 +40,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.3.2.jar +commons-lang3-3.5.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-2.2.jar diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index 767e1f9402e01..a5d36da5b6de9 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -115,11 +115,11 @@ Configuring Flume on the chosen machine requires the following two steps. artifactId = scala-library version = {{site.SCALA_VERSION}} - (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.3.2/commons-lang3-3.3.2.jar)). + (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.5/commons-lang3-3.5.jar)). groupId = org.apache.commons artifactId = commons-lang3 - version = 3.3.2 + version = 3.5 2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. diff --git a/pom.xml b/pom.xml index 7d13c51b2a596..aaf7cfa7eb2ad 100644 --- a/pom.xml +++ b/pom.xml @@ -168,7 +168,7 @@ 2.6 - 3.3.2 + 3.5 3.2.10 3.0.0 2.22.2 From 444c2d22e38a8a78135adf0d3a3774f0e9fc866c Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Wed, 19 Oct 2016 13:01:33 -0700 Subject: [PATCH 134/177] [SPARK-10541][WEB UI] Allow ApplicationHistoryProviders to provide their own text when there aren't any complete apps ## What changes were proposed in this pull request? I've added a method to `ApplicationHistoryProvider` that returns the html paragraph to display when there are no applications. This allows providers other than `FsHistoryProvider` to determine what is printed. The current hard coded text is now moved into `FsHistoryProvider` since it assumed that's what was being used before. I chose to make the function return html rather than text because the current text block had inline html in it and it allows a new implementation of `ApplicationHistoryProvider` more versatility. I did not see any security issues with this since injecting html here requires implementing `ApplicationHistoryProvider` and can't be done outside of code. ## How was this patch tested? Manual testing and dev/run-tests No visible changes to the UI Author: Alex Bozarth Closes #15490 from ajbozarth/spark10541. --- .../deploy/history/ApplicationHistoryProvider.scala | 6 ++++++ .../spark/deploy/history/FsHistoryProvider.scala | 12 ++++++++++++ .../apache/spark/deploy/history/HistoryPage.scala | 8 +------- .../apache/spark/deploy/history/HistoryServer.scala | 8 ++++++++ 4 files changed, 27 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index ad7a0972ef9d1..06530ff836466 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.history import java.util.zip.ZipOutputStream +import scala.xml.Node + import org.apache.spark.SparkException import org.apache.spark.ui.SparkUI @@ -114,4 +116,8 @@ private[history] abstract class ApplicationHistoryProvider { */ def getApplicationInfo(appId: String): Option[ApplicationHistoryInfo] + /** + * @return html text to display when the application list is empty + */ + def getEmptyListingHtml(): Seq[Node] = Seq.empty } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 3c2d169f3270e..530cc5252214b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -23,6 +23,7 @@ import java.util.concurrent.{Executors, ExecutorService, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable +import scala.xml.Node import com.google.common.io.ByteStreams import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} @@ -262,6 +263,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + override def getEmptyListingHtml(): Seq[Node] = { +

    + Did you specify the correct logging directory? Please verify your setting of + spark.history.fs.logDirectory + listed above and whether you have the permissions to access it. +
    + It is also possible that your application did not run to + completion or did not stop the SparkContext. +

    + } + override def getConfig(): Map[String, String] = { val safeMode = if (isFsInSafeMode()) { Map("HDFS State" -> "In safe mode, application logs not available.") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 95b72224e0f94..96b9ecf43b14c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -47,13 +47,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") } else if (requestedIncomplete) {

    No incomplete applications found!

    } else { -

    No completed applications found!

    ++ -

    Did you specify the correct logging directory? - Please verify your setting of - spark.history.fs.logDirectory and whether you have the permissions to - access it.
    It is also possible that your application did not run to - completion or did not stop the SparkContext. -

    +

    No completed applications found!

    ++ parent.emptyListingHtml } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 087c69e6489dd..3175b36b3e56f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -22,6 +22,7 @@ import java.util.zip.ZipOutputStream import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.util.control.NonFatal +import scala.xml.Node import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} @@ -193,6 +194,13 @@ class HistoryServer( provider.writeEventLogs(appId, attemptId, zipStream) } + /** + * @return html text to display when the application list is empty + */ + def emptyListingHtml(): Seq[Node] = { + provider.getEmptyListingHtml() + } + /** * Returns the provider configuration to show in the listing page. * From 4b2011ec9da1245923b5cbd883240fef0dbf3ef0 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 19 Oct 2016 19:36:21 -0700 Subject: [PATCH 135/177] [SPARK-17989][SQL] Check ascendingOrder type in sort_array function rather than throwing ClassCastException ## What changes were proposed in this pull request? This PR proposes to check the second argument, `ascendingOrder` rather than throwing `ClassCastException` exception message. ```sql select sort_array(array('b', 'd'), '1'); ``` **Before** ``` 16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')] java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85) at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185) at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297) ``` **After** ``` Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7; ``` ## How was this patch tested? Unit test in `DataFrameFunctionsSuite`. Author: hyukjinkwon Closes #15532 from HyukjinKwon/SPARK-17989. --- .../expressions/collectionOperations.scala | 8 ++++++- .../test/resources/sql-tests/inputs/array.sql | 6 ++++++ .../resources/sql-tests/results/array.sql.out | 21 ++++++++++++++++--- 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index c0200299376ca..f56bb39d10791 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -124,7 +124,13 @@ case class SortArray(base: Expression, ascendingOrder: Expression) override def checkInputDataTypes(): TypeCheckResult = base.dataType match { case ArrayType(dt, _) if RowOrdering.isOrderable(dt) => - TypeCheckResult.TypeCheckSuccess + ascendingOrder match { + case Literal(_: Boolean, BooleanType) => + TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure( + "Sort order in second argument requires a boolean literal.") + } case ArrayType(dt, _) => TypeCheckResult.TypeCheckFailure( s"$prettyName does not support sorting array of type ${dt.simpleString}") diff --git a/sql/core/src/test/resources/sql-tests/inputs/array.sql b/sql/core/src/test/resources/sql-tests/inputs/array.sql index 4038a0da41d2b..984321ab795fc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/array.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/array.sql @@ -71,6 +71,12 @@ select sort_array(timestamp_array) from primitive_arrays; +-- sort_array with an invalid string literal for the argument of sort order. +select sort_array(array('b', 'd'), '1'); + +-- sort_array with an invalid null literal casted as boolean for the argument of sort order. +select sort_array(array('b', 'd'), cast(NULL as boolean)); + -- size select size(boolean_array), diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 4a1d149c1f362..499a3d5fb72f6 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -124,8 +124,23 @@ struct,sort_array(tinyint_array, -- !query 8 output [true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00.0,2016-11-15 20:54:00.0] - -- !query 9 +select sort_array(array('b', 'd'), '1') +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 + +-- !query 10 +select sort_array(array('b', 'd'), cast(NULL as boolean)) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot resolve 'sort_array(array('b', 'd'), CAST(NULL AS BOOLEAN))' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 + +-- !query 11 select size(boolean_array), size(tinyint_array), @@ -138,7 +153,7 @@ select size(date_array), size(timestamp_array) from primitive_arrays --- !query 9 schema +-- !query 11 schema struct --- !query 9 output +-- !query 11 output 1 2 2 2 2 2 2 2 2 2 From f313117bc93b0bf560528b316d3e6947caa96296 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 19 Oct 2016 22:22:35 -0700 Subject: [PATCH 136/177] [SPARK-18012][SQL] Simplify WriterContainer ## What changes were proposed in this pull request? This patch refactors WriterContainer to simplify the logic and make control flow more obvious.The previous code setup made it pretty difficult to track the actual dependencies on variables and setups because the driver side and the executor side were using the same set of variables. ## How was this patch tested? N/A - this should be covered by existing tests. Author: Reynold Xin Closes #15551 from rxin/writercontainer-refactor. --- .../InsertIntoHadoopFsRelationCommand.scala | 79 +-- .../execution/datasources/WriteOutput.scala | 480 ++++++++++++++++++ .../datasources/WriterContainer.scala | 445 ---------------- .../apache/spark/sql/internal/SQLConf.scala | 9 - 4 files changed, 492 insertions(+), 521 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 99ca3df673568..22dbe7149531c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -20,18 +20,12 @@ package org.apache.spark.sql.execution.datasources import java.io.IOException import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat -import org.apache.spark._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.internal.SQLConf /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. @@ -40,20 +34,6 @@ import org.apache.spark.sql.internal.SQLConf * implementation of [[HadoopFsRelation]] should use this UUID together with task id to generate * unique file path for each task output file. This UUID is passed to executor side via a * property named `spark.sql.sources.writeJobUUID`. - * - * Different writer containers, [[DefaultWriterContainer]] and [[DynamicPartitionWriterContainer]] - * are used to write to normal tables and tables with dynamic partitions. - * - * Basic work flow of this command is: - * - * 1. Driver side setup, including output committer initialization and data source specific - * preparation work for the write job to be issued. - * 2. Issues a write job consists of one or more executor side tasks, each of which writes all - * rows within an RDD partition. - * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any - * exception is thrown during task commitment, also aborts that task. - * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is - * thrown during job commitment, also aborts the job. */ case class InsertIntoHadoopFsRelationCommand( outputPath: Path, @@ -103,52 +83,17 @@ case class InsertIntoHadoopFsRelationCommand( val isAppend = pathExists && (mode == SaveMode.Append) if (doInsertion) { - val job = Job.getInstance(hadoopConf) - job.setOutputKeyClass(classOf[Void]) - job.setOutputValueClass(classOf[InternalRow]) - FileOutputFormat.setOutputPath(job, qualifiedOutputPath) - - val partitionSet = AttributeSet(partitionColumns) - val dataColumns = query.output.filterNot(partitionSet.contains) - - val queryExecution = Dataset.ofRows(sparkSession, query).queryExecution - SQLExecution.withNewExecutionId(sparkSession, queryExecution) { - val relation = - WriteRelation( - sparkSession, - dataColumns.toStructType, - qualifiedOutputPath.toString, - fileFormat.prepareWrite(sparkSession, _, options, dataColumns.toStructType), - bucketSpec) - - val writerContainer = if (partitionColumns.isEmpty && bucketSpec.isEmpty) { - new DefaultWriterContainer(relation, job, isAppend) - } else { - new DynamicPartitionWriterContainer( - relation, - job, - partitionColumns = partitionColumns, - dataColumns = dataColumns, - inputSchema = query.output, - PartitioningUtils.DEFAULT_PARTITION_NAME, - sparkSession.sessionState.conf.partitionMaxFiles, - isAppend) - } - - // This call shouldn't be put into the `try` block below because it only initializes and - // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. - writerContainer.driverSideSetup() - - try { - sparkSession.sparkContext.runJob(queryExecution.toRdd, writerContainer.writeRows _) - writerContainer.commitJob() - refreshFunction() - } catch { case cause: Throwable => - logError("Aborting job.", cause) - writerContainer.abortJob() - throw new SparkException("Job aborted.", cause) - } - } + WriteOutput.write( + sparkSession, + query, + fileFormat, + qualifiedOutputPath, + hadoopConf, + partitionColumns, + bucketSpec, + refreshFunction, + options, + isAppend) } else { logInfo("Skipping insertion into a relation that already exists.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala new file mode 100644 index 0000000000000..54d0f3bd6291a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala @@ -0,0 +1,480 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Date, UUID} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat} +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.mapred.SparkHadoopMapRedUtil +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.{SQLExecution, UnsafeKVExternalSorter} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.util.{SerializableConfiguration, Utils} +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter + + +/** A helper object for writing data out to a location. */ +object WriteOutput extends Logging { + + /** A shared job description for all the write tasks. */ + private class WriteJobDescription( + val serializableHadoopConf: SerializableConfiguration, + val outputWriterFactory: OutputWriterFactory, + val allColumns: Seq[Attribute], + val partitionColumns: Seq[Attribute], + val nonPartitionColumns: Seq[Attribute], + val bucketSpec: Option[BucketSpec], + val isAppend: Boolean, + val path: String, + val outputFormatClass: Class[_ <: OutputFormat[_, _]]) + extends Serializable { + + assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ nonPartitionColumns), + s""" + |All columns: ${allColumns.mkString(", ")} + |Partition columns: ${partitionColumns.mkString(", ")} + |Non-partition columns: ${nonPartitionColumns.mkString(", ")} + """.stripMargin) + } + + /** + * Basic work flow of this command is: + * 1. Driver side setup, including output committer initialization and data source specific + * preparation work for the write job to be issued. + * 2. Issues a write job consists of one or more executor side tasks, each of which writes all + * rows within an RDD partition. + * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any + * exception is thrown during task commitment, also aborts that task. + * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is + * thrown during job commitment, also aborts the job. + */ + def write( + sparkSession: SparkSession, + plan: LogicalPlan, + fileFormat: FileFormat, + outputPath: Path, + hadoopConf: Configuration, + partitionColumns: Seq[Attribute], + bucketSpec: Option[BucketSpec], + refreshFunction: () => Unit, + options: Map[String, String], + isAppend: Boolean): Unit = { + + val job = Job.getInstance(hadoopConf) + job.setOutputKeyClass(classOf[Void]) + job.setOutputValueClass(classOf[InternalRow]) + FileOutputFormat.setOutputPath(job, outputPath) + + val partitionSet = AttributeSet(partitionColumns) + val dataColumns = plan.output.filterNot(partitionSet.contains) + val queryExecution = Dataset.ofRows(sparkSession, plan).queryExecution + + // Note: prepareWrite has side effect. It sets "job". + val outputWriterFactory = + fileFormat.prepareWrite(sparkSession, job, options, dataColumns.toStructType) + + val description = new WriteJobDescription( + serializableHadoopConf = new SerializableConfiguration(job.getConfiguration), + outputWriterFactory = outputWriterFactory, + allColumns = plan.output, + partitionColumns = partitionColumns, + nonPartitionColumns = dataColumns, + bucketSpec = bucketSpec, + isAppend = isAppend, + path = outputPath.toString, + outputFormatClass = job.getOutputFormatClass) + + SQLExecution.withNewExecutionId(sparkSession, queryExecution) { + // This call shouldn't be put into the `try` block below because it only initializes and + // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. + val committer = setupDriverCommitter(job, outputPath.toString, isAppend) + + try { + sparkSession.sparkContext.runJob(queryExecution.toRdd, + (taskContext: TaskContext, iter: Iterator[InternalRow]) => { + executeTask( + description = description, + sparkStageId = taskContext.stageId(), + sparkPartitionId = taskContext.partitionId(), + sparkAttemptNumber = taskContext.attemptNumber(), + iterator = iter) + }) + + committer.commitJob(job) + logInfo(s"Job ${job.getJobID} committed.") + refreshFunction() + } catch { case cause: Throwable => + logError(s"Aborting job ${job.getJobID}.", cause) + committer.abortJob(job, JobStatus.State.FAILED) + throw new SparkException("Job aborted.", cause) + } + } + } + + /** Writes data out in a single Spark task. */ + private def executeTask( + description: WriteJobDescription, + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + iterator: Iterator[InternalRow]): Unit = { + + val jobId = SparkHadoopWriter.createJobID(new Date, sparkStageId) + val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) + val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) + + // Set up the attempt context required to use in the output committer. + val taskAttemptContext: TaskAttemptContext = { + // Set up the configuration object + val hadoopConf = description.serializableHadoopConf.value + hadoopConf.set("mapred.job.id", jobId.toString) + hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + hadoopConf.set("mapred.task.id", taskAttemptId.toString) + hadoopConf.setBoolean("mapred.task.is.map", true) + hadoopConf.setInt("mapred.task.partition", 0) + + new TaskAttemptContextImpl(hadoopConf, taskAttemptId) + } + + val committer = newOutputCommitter( + description.outputFormatClass, taskAttemptContext, description.path, description.isAppend) + committer.setupTask(taskAttemptContext) + + // Figure out where we need to write data to for staging. + // For FileOutputCommitter it has its own staging path called "work path". + val stagingPath = committer match { + case f: FileOutputCommitter => f.getWorkPath.toString + case _ => description.path + } + + val writeTask = + if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty) { + new SingleDirectoryWriteTask(description, taskAttemptContext, stagingPath) + } else { + new DynamicPartitionWriteTask(description, taskAttemptContext, stagingPath) + } + + try { + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + // Execute the task to write rows out + writeTask.execute(iterator) + writeTask.releaseResources() + + // Commit the task + SparkHadoopMapRedUtil.commitTask(committer, taskAttemptContext, jobId.getId, taskId.getId) + })(catchBlock = { + // If there is an error, release resource and then abort the task + try { + writeTask.releaseResources() + } finally { + committer.abortTask(taskAttemptContext) + logError(s"Job $jobId aborted.") + } + }) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } + } + + /** + * A simple trait for writing out data in a single Spark task, without any concerns about how + * to commit or abort tasks. Exceptions thrown by the implementation of this trait will + * automatically trigger task aborts. + */ + private trait ExecuteWriteTask { + def execute(iterator: Iterator[InternalRow]): Unit + def releaseResources(): Unit + } + + /** Writes data to a single directory (used for non-dynamic-partition writes). */ + private class SingleDirectoryWriteTask( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + stagingPath: String) extends ExecuteWriteTask { + + private[this] var outputWriter: OutputWriter = { + val outputWriter = description.outputWriterFactory.newInstance( + path = stagingPath, + bucketId = None, + dataSchema = description.nonPartitionColumns.toStructType, + context = taskAttemptContext) + outputWriter.initConverter(dataSchema = description.nonPartitionColumns.toStructType) + outputWriter + } + + override def execute(iter: Iterator[InternalRow]): Unit = { + while (iter.hasNext) { + val internalRow = iter.next() + outputWriter.writeInternal(internalRow) + } + } + + override def releaseResources(): Unit = { + if (outputWriter != null) { + outputWriter.close() + outputWriter = null + } + } + } + + /** + * Writes data to using dynamic partition writes, meaning this single function can write to + * multiple directories (partitions) or files (bucketing). + */ + private class DynamicPartitionWriteTask( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + stagingPath: String) extends ExecuteWriteTask { + + // currentWriter is initialized whenever we see a new key + private var currentWriter: OutputWriter = _ + + private val bucketColumns: Seq[Attribute] = description.bucketSpec.toSeq.flatMap { + spec => spec.bucketColumnNames.map(c => description.allColumns.find(_.name == c).get) + } + + private val sortColumns: Seq[Attribute] = description.bucketSpec.toSeq.flatMap { + spec => spec.sortColumnNames.map(c => description.allColumns.find(_.name == c).get) + } + + private def bucketIdExpression: Option[Expression] = description.bucketSpec.map { spec => + // Use `HashPartitioning.partitionIdExpression` as our bucket id expression, so that we can + // guarantee the data distribution is same between shuffle and bucketed data source, which + // enables us to only shuffle one side when join a bucketed table and a normal one. + HashPartitioning(bucketColumns, spec.numBuckets).partitionIdExpression + } + + /** Expressions that given a partition key build a string like: col1=val/col2=val/... */ + private def partitionStringExpression: Seq[Expression] = { + description.partitionColumns.zipWithIndex.flatMap { case (c, i) => + val escaped = ScalaUDF( + PartitioningUtils.escapePathName _, + StringType, + Seq(Cast(c, StringType)), + Seq(StringType)) + val str = If(IsNull(c), Literal(PartitioningUtils.DEFAULT_PARTITION_NAME), escaped) + val partitionName = Literal(c.name + "=") :: str :: Nil + if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName + } + } + + private def getBucketIdFromKey(key: InternalRow): Option[Int] = + description.bucketSpec.map { _ => key.getInt(description.partitionColumns.length) } + + /** + * Open and returns a new OutputWriter given a partition key and optional bucket id. + * If bucket id is specified, we will append it to the end of the file name, but before the + * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet + */ + private def newOutputWriter( + key: InternalRow, + getPartitionString: UnsafeProjection): OutputWriter = { + val path = + if (description.partitionColumns.nonEmpty) { + val partitionPath = getPartitionString(key).getString(0) + new Path(stagingPath, partitionPath).toString + } else { + stagingPath + } + val bucketId = getBucketIdFromKey(key) + + val newWriter = description.outputWriterFactory.newInstance( + path = path, + bucketId = bucketId, + dataSchema = description.nonPartitionColumns.toStructType, + context = taskAttemptContext) + newWriter.initConverter(description.nonPartitionColumns.toStructType) + newWriter + } + + override def execute(iter: Iterator[InternalRow]): Unit = { + // We should first sort by partition columns, then bucket id, and finally sorting columns. + val sortingExpressions: Seq[Expression] = + description.partitionColumns ++ bucketIdExpression ++ sortColumns + val getSortingKey = UnsafeProjection.create(sortingExpressions, description.allColumns) + + val sortingKeySchema = StructType(sortingExpressions.map { + case a: Attribute => StructField(a.name, a.dataType, a.nullable) + // The sorting expressions are all `Attribute` except bucket id. + case _ => StructField("bucketId", IntegerType, nullable = false) + }) + + // Returns the data columns to be written given an input row + val getOutputRow = UnsafeProjection.create( + description.nonPartitionColumns, description.allColumns) + + // Returns the partition path given a partition key. + val getPartitionString = + UnsafeProjection.create(Seq(Concat(partitionStringExpression)), description.partitionColumns) + + // Sorts the data before write, so that we only need one writer at the same time. + val sorter = new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(description.nonPartitionColumns), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get().taskMemoryManager().pageSizeBytes, + SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) + + while (iter.hasNext) { + val currentRow = iter.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + logInfo(s"Sorting complete. Writing out partition files one at a time.") + + val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { + identity + } else { + UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { + case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + }) + } + + val sortedIterator = sorter.sortedIterator() + + // If anything below fails, we should abort the task. + var currentKey: UnsafeRow = null + while (sortedIterator.next()) { + val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + currentKey = nextKey.copy() + logDebug(s"Writing partition: $currentKey") + + currentWriter = newOutputWriter(currentKey, getPartitionString) + } + currentWriter.writeInternal(sortedIterator.getValue) + } + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + } + + override def releaseResources(): Unit = { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + } + } + + private def setupDriverCommitter(job: Job, path: String, isAppend: Boolean): OutputCommitter = { + // Setup IDs + val jobId = SparkHadoopWriter.createJobID(new Date, 0) + val taskId = new TaskID(jobId, TaskType.MAP, 0) + val taskAttemptId = new TaskAttemptID(taskId, 0) + + // Set up the configuration object + job.getConfiguration.set("mapred.job.id", jobId.toString) + job.getConfiguration.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + job.getConfiguration.set("mapred.task.id", taskAttemptId.toString) + job.getConfiguration.setBoolean("mapred.task.is.map", true) + job.getConfiguration.setInt("mapred.task.partition", 0) + + // This UUID is sent to executor side together with the serialized `Configuration` object within + // the `Job` instance. `OutputWriters` on the executor side should use this UUID to generate + // unique task output files. + // This UUID is used to avoid output file name collision between different appending write jobs. + // These jobs may belong to different SparkContext instances. Concrete data source + // implementations may use this UUID to generate unique file names (e.g., + // `part-r--.parquet`). The reason why this ID is used to identify a job + // rather than a single task output file is that, speculative tasks must generate the same + // output file name as the original task. + job.getConfiguration.set(WriterContainer.DATASOURCE_WRITEJOBUUID, UUID.randomUUID().toString) + + val taskAttemptContext = new TaskAttemptContextImpl(job.getConfiguration, taskAttemptId) + val outputCommitter = newOutputCommitter( + job.getOutputFormatClass, taskAttemptContext, path, isAppend) + outputCommitter.setupJob(job) + outputCommitter + } + + private def newOutputCommitter( + outputFormatClass: Class[_ <: OutputFormat[_, _]], + context: TaskAttemptContext, + path: String, + isAppend: Boolean): OutputCommitter = { + val defaultOutputCommitter = outputFormatClass.newInstance().getOutputCommitter(context) + + if (isAppend) { + // If we are appending data to an existing dir, we will only use the output committer + // associated with the file output format since it is not safe to use a custom + // committer for appending. For example, in S3, direct parquet output committer may + // leave partial data in the destination dir when the appending job fails. + // See SPARK-8578 for more details + logInfo( + s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " + + "for appending.") + defaultOutputCommitter + } else { + val configuration = context.getConfiguration + val clazz = + configuration.getClass(SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) + + if (clazz != null) { + logInfo(s"Using user defined output committer class ${clazz.getCanonicalName}") + + // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat + // has an associated output committer. To override this output committer, + // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. + // If a data source needs to override the output committer, it needs to set the + // output committer in prepareForWrite method. + if (classOf[FileOutputCommitter].isAssignableFrom(clazz)) { + // The specified output committer is a FileOutputCommitter. + // So, we will use the FileOutputCommitter-specified constructor. + val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) + ctor.newInstance(new Path(path), context) + } else { + // The specified output committer is just an OutputCommitter. + // So, we will use the no-argument constructor. + val ctor = clazz.getDeclaredConstructor() + ctor.newInstance() + } + } else { + // If output committer class is not set, we will use the one associated with the + // file output format. + logInfo( + s"Using output committer class ${defaultOutputCommitter.getClass.getCanonicalName}") + defaultOutputCommitter + } + } + } +} + +object WriterContainer { + val DATASOURCE_WRITEJOBUUID = "spark.sql.sources.writeJobUUID" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala deleted file mode 100644 index 253aa4405defa..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ /dev/null @@ -1,445 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import java.util.{Date, UUID} - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => MapReduceFileOutputCommitter} -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl - -import org.apache.spark._ -import org.apache.spark.internal.Logging -import org.apache.spark.mapred.SparkHadoopMapRedUtil -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.UnsafeKVExternalSorter -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} -import org.apache.spark.util.{SerializableConfiguration, Utils} -import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter - - -/** A container for all the details required when writing to a table. */ -private[datasources] case class WriteRelation( - sparkSession: SparkSession, - dataSchema: StructType, - path: String, - prepareJobForWrite: Job => OutputWriterFactory, - bucketSpec: Option[BucketSpec]) - -object WriterContainer { - val DATASOURCE_WRITEJOBUUID = "spark.sql.sources.writeJobUUID" -} - -private[datasources] abstract class BaseWriterContainer( - @transient val relation: WriteRelation, - @transient private val job: Job, - isAppend: Boolean) - extends Logging with Serializable { - - protected val dataSchema = relation.dataSchema - - protected val serializableConf = - new SerializableConfiguration(job.getConfiguration) - - // This UUID is used to avoid output file name collision between different appending write jobs. - // These jobs may belong to different SparkContext instances. Concrete data source implementations - // may use this UUID to generate unique file names (e.g., `part-r--.parquet`). - // The reason why this ID is used to identify a job rather than a single task output file is - // that, speculative tasks must generate the same output file name as the original task. - private val uniqueWriteJobId = UUID.randomUUID() - - // This is only used on driver side. - @transient private val jobContext: JobContext = job - - // The following fields are initialized and used on both driver and executor side. - @transient protected var outputCommitter: OutputCommitter = _ - @transient private var jobId: JobID = _ - @transient private var taskId: TaskID = _ - @transient private var taskAttemptId: TaskAttemptID = _ - @transient protected var taskAttemptContext: TaskAttemptContext = _ - - protected val outputPath: String = relation.path - - protected var outputWriterFactory: OutputWriterFactory = _ - - private var outputFormatClass: Class[_ <: OutputFormat[_, _]] = _ - - def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit - - def driverSideSetup(): Unit = { - setupIDs(0, 0, 0) - setupConf() - - // This UUID is sent to executor side together with the serialized `Configuration` object within - // the `Job` instance. `OutputWriters` on the executor side should use this UUID to generate - // unique task output files. - job.getConfiguration.set(WriterContainer.DATASOURCE_WRITEJOBUUID, uniqueWriteJobId.toString) - - // Order of the following two lines is important. For Hadoop 1, TaskAttemptContext constructor - // clones the Configuration object passed in. If we initialize the TaskAttemptContext first, - // configurations made in prepareJobForWrite(job) are not populated into the TaskAttemptContext. - // - // Also, the `prepareJobForWrite` call must happen before initializing output format and output - // committer, since their initialization involve the job configuration, which can be potentially - // decorated in `prepareJobForWrite`. - outputWriterFactory = relation.prepareJobForWrite(job) - taskAttemptContext = new TaskAttemptContextImpl(serializableConf.value, taskAttemptId) - - outputFormatClass = job.getOutputFormatClass - outputCommitter = newOutputCommitter(taskAttemptContext) - outputCommitter.setupJob(jobContext) - } - - def executorSideSetup(taskContext: TaskContext): Unit = { - setupIDs(taskContext.stageId(), taskContext.partitionId(), taskContext.attemptNumber()) - setupConf() - taskAttemptContext = new TaskAttemptContextImpl(serializableConf.value, taskAttemptId) - outputCommitter = newOutputCommitter(taskAttemptContext) - outputCommitter.setupTask(taskAttemptContext) - } - - protected def getWorkPath: String = { - outputCommitter match { - // FileOutputCommitter writes to a temporary location returned by `getWorkPath`. - case f: MapReduceFileOutputCommitter => f.getWorkPath.toString - case _ => outputPath - } - } - - protected def newOutputWriter(path: String, bucketId: Option[Int] = None): OutputWriter = { - try { - outputWriterFactory.newInstance(path, bucketId, dataSchema, taskAttemptContext) - } catch { - case e: org.apache.hadoop.fs.FileAlreadyExistsException => - if (outputCommitter.getClass.getName.contains("Direct")) { - // SPARK-11382: DirectParquetOutputCommitter is not idempotent, meaning on retry - // attempts, the task will fail because the output file is created from a prior attempt. - // This often means the most visible error to the user is misleading. Augment the error - // to tell the user to look for the actual error. - throw new SparkException("The output file already exists but this could be due to a " + - "failure from an earlier attempt. Look through the earlier logs or stage page for " + - "the first error.\n File exists error: " + e, e) - } else { - throw e - } - } - } - - private def newOutputCommitter(context: TaskAttemptContext): OutputCommitter = { - val defaultOutputCommitter = outputFormatClass.newInstance().getOutputCommitter(context) - - if (isAppend) { - // If we are appending data to an existing dir, we will only use the output committer - // associated with the file output format since it is not safe to use a custom - // committer for appending. For example, in S3, direct parquet output committer may - // leave partial data in the destination dir when the appending job fails. - // - // See SPARK-8578 for more details - logInfo( - s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " + - "for appending.") - defaultOutputCommitter - } else { - val configuration = context.getConfiguration - val committerClass = configuration.getClass( - SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) - - Option(committerClass).map { clazz => - logInfo(s"Using user defined output committer class ${clazz.getCanonicalName}") - - // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat - // has an associated output committer. To override this output committer, - // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. - // If a data source needs to override the output committer, it needs to set the - // output committer in prepareForWrite method. - if (classOf[MapReduceFileOutputCommitter].isAssignableFrom(clazz)) { - // The specified output committer is a FileOutputCommitter. - // So, we will use the FileOutputCommitter-specified constructor. - val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) - ctor.newInstance(new Path(outputPath), context) - } else { - // The specified output committer is just an OutputCommitter. - // So, we will use the no-argument constructor. - val ctor = clazz.getDeclaredConstructor() - ctor.newInstance() - } - }.getOrElse { - // If output committer class is not set, we will use the one associated with the - // file output format. - logInfo( - s"Using output committer class ${defaultOutputCommitter.getClass.getCanonicalName}") - defaultOutputCommitter - } - } - } - - private def setupIDs(jobId: Int, splitId: Int, attemptId: Int): Unit = { - this.jobId = SparkHadoopWriter.createJobID(new Date, jobId) - this.taskId = new TaskID(this.jobId, TaskType.MAP, splitId) - this.taskAttemptId = new TaskAttemptID(taskId, attemptId) - } - - private def setupConf(): Unit = { - serializableConf.value.set("mapred.job.id", jobId.toString) - serializableConf.value.set("mapred.tip.id", taskAttemptId.getTaskID.toString) - serializableConf.value.set("mapred.task.id", taskAttemptId.toString) - serializableConf.value.setBoolean("mapred.task.is.map", true) - serializableConf.value.setInt("mapred.task.partition", 0) - } - - def commitTask(): Unit = { - SparkHadoopMapRedUtil.commitTask(outputCommitter, taskAttemptContext, jobId.getId, taskId.getId) - } - - def abortTask(): Unit = { - if (outputCommitter != null) { - outputCommitter.abortTask(taskAttemptContext) - } - logError(s"Task attempt $taskAttemptId aborted.") - } - - def commitJob(): Unit = { - outputCommitter.commitJob(jobContext) - logInfo(s"Job $jobId committed.") - } - - def abortJob(): Unit = { - if (outputCommitter != null) { - outputCommitter.abortJob(jobContext, JobStatus.State.FAILED) - } - logError(s"Job $jobId aborted.") - } -} - -/** - * A writer that writes all of the rows in a partition to a single file. - */ -private[datasources] class DefaultWriterContainer( - relation: WriteRelation, - job: Job, - isAppend: Boolean) - extends BaseWriterContainer(relation, job, isAppend) { - - def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - executorSideSetup(taskContext) - var writer = newOutputWriter(getWorkPath) - writer.initConverter(dataSchema) - - // If anything below fails, we should abort the task. - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - while (iterator.hasNext) { - val internalRow = iterator.next() - writer.writeInternal(internalRow) - } - commitTask() - }(catchBlock = abortTask()) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } - - def commitTask(): Unit = { - try { - if (writer != null) { - writer.close() - writer = null - } - super.commitTask() - } catch { - case cause: Throwable => - // This exception will be handled in `InsertIntoHadoopFsRelation.insert$writeRows`, and - // will cause `abortTask()` to be invoked. - throw new RuntimeException("Failed to commit task", cause) - } - } - - def abortTask(): Unit = { - try { - if (writer != null) { - writer.close() - } - } finally { - super.abortTask() - } - } - } -} - -/** - * A writer that dynamically opens files based on the given partition columns. Internally this is - * done by maintaining a HashMap of open files until `maxFiles` is reached. If this occurs, the - * writer externally sorts the remaining rows and then writes out them out one file at a time. - */ -private[datasources] class DynamicPartitionWriterContainer( - relation: WriteRelation, - job: Job, - partitionColumns: Seq[Attribute], - dataColumns: Seq[Attribute], - inputSchema: Seq[Attribute], - defaultPartitionName: String, - maxOpenFiles: Int, - isAppend: Boolean) - extends BaseWriterContainer(relation, job, isAppend) { - - private val bucketSpec = relation.bucketSpec - - private val bucketColumns: Seq[Attribute] = bucketSpec.toSeq.flatMap { - spec => spec.bucketColumnNames.map(c => inputSchema.find(_.name == c).get) - } - - private val sortColumns: Seq[Attribute] = bucketSpec.toSeq.flatMap { - spec => spec.sortColumnNames.map(c => inputSchema.find(_.name == c).get) - } - - private def bucketIdExpression: Option[Expression] = bucketSpec.map { spec => - // Use `HashPartitioning.partitionIdExpression` as our bucket id expression, so that we can - // guarantee the data distribution is same between shuffle and bucketed data source, which - // enables us to only shuffle one side when join a bucketed table and a normal one. - HashPartitioning(bucketColumns, spec.numBuckets).partitionIdExpression - } - - // Expressions that given a partition key build a string like: col1=val/col2=val/... - private def partitionStringExpression: Seq[Expression] = { - partitionColumns.zipWithIndex.flatMap { case (c, i) => - val escaped = - ScalaUDF( - PartitioningUtils.escapePathName _, - StringType, - Seq(Cast(c, StringType)), - Seq(StringType)) - val str = If(IsNull(c), Literal(defaultPartitionName), escaped) - val partitionName = Literal(c.name + "=") :: str :: Nil - if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName - } - } - - private def getBucketIdFromKey(key: InternalRow): Option[Int] = bucketSpec.map { _ => - key.getInt(partitionColumns.length) - } - - /** - * Open and returns a new OutputWriter given a partition key and optional bucket id. - * If bucket id is specified, we will append it to the end of the file name, but before the - * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet - */ - private def newOutputWriter( - key: InternalRow, - getPartitionString: UnsafeProjection): OutputWriter = { - val path = if (partitionColumns.nonEmpty) { - val partitionPath = getPartitionString(key).getString(0) - new Path(getWorkPath, partitionPath).toString - } else { - getWorkPath - } - val bucketId = getBucketIdFromKey(key) - val newWriter = super.newOutputWriter(path, bucketId) - newWriter.initConverter(dataSchema) - newWriter - } - - def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - executorSideSetup(taskContext) - - // We should first sort by partition columns, then bucket id, and finally sorting columns. - val sortingExpressions: Seq[Expression] = partitionColumns ++ bucketIdExpression ++ sortColumns - val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) - - val sortingKeySchema = StructType(sortingExpressions.map { - case a: Attribute => StructField(a.name, a.dataType, a.nullable) - // The sorting expressions are all `Attribute` except bucket id. - case _ => StructField("bucketId", IntegerType, nullable = false) - }) - - // Returns the data columns to be written given an input row - val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) - - // Returns the partition path given a partition key. - val getPartitionString = - UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) - - // Sorts the data before write, so that we only need one writer at the same time. - // TODO: inject a local sort operator in planning. - val sorter = new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes, - SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", - UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) - - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) - } - logInfo(s"Sorting complete. Writing out partition files one at a time.") - - val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { - identity - } else { - UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { - case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) - }) - } - - val sortedIterator = sorter.sortedIterator() - - // If anything below fails, we should abort the task. - var currentWriter: OutputWriter = null - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - var currentKey: UnsafeRow = null - while (sortedIterator.next()) { - val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] - if (currentKey != nextKey) { - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - currentKey = nextKey.copy() - logDebug(s"Writing partition: $currentKey") - - currentWriter = newOutputWriter(currentKey, getPartitionString) - } - currentWriter.writeInternal(sortedIterator.getValue) - } - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - - commitTask() - }(catchBlock = { - if (currentWriter != null) { - currentWriter.close() - } - abortTask() - }) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8afd39d657865..9061b1b9a2638 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -339,13 +339,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val PARTITION_MAX_FILES = - SQLConfigBuilder("spark.sql.sources.maxConcurrentWrites") - .doc("The maximum number of concurrent files to open before falling back on sorting when " + - "writing out files using dynamic partitioning.") - .intConf - .createWithDefault(1) - val BUCKETING_ENABLED = SQLConfigBuilder("spark.sql.sources.bucketing.enabled") .doc("When false, we will treat bucketed table as normal table") .booleanConf @@ -733,8 +726,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def partitionColumnTypeInferenceEnabled: Boolean = getConf(SQLConf.PARTITION_COLUMN_TYPE_INFERENCE) - def partitionMaxFiles: Int = getConf(PARTITION_MAX_FILES) - def parallelPartitionDiscoveryThreshold: Int = getConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD) From 39755169fb5bb07332eef263b4c18ede1528812d Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 19 Oct 2016 23:41:38 -0700 Subject: [PATCH 137/177] [SPARK-18003][SPARK CORE] Fix bug of RDD zipWithIndex & zipWithUniqueId index value overflowing ## What changes were proposed in this pull request? - Fix bug of RDD `zipWithIndex` generating wrong result when one partition contains more than 2147483647 records. - Fix bug of RDD `zipWithUniqueId` generating wrong result when one partition contains more than 2147483647 records. ## How was this patch tested? test added. Author: WeichenXu Closes #15550 from WeichenXu123/fix_rdd_zipWithIndex_overflow. --- .../src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../org/apache/spark/rdd/ZippedWithIndexRDD.scala | 5 ++--- .../main/scala/org/apache/spark/util/Utils.scala | 15 +++++++++++++++ .../scala/org/apache/spark/util/UtilsSuite.scala | 7 +++++++ 4 files changed, 25 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6dc334ceb52ea..be119578d2c35 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1278,7 +1278,7 @@ abstract class RDD[T: ClassTag]( def zipWithUniqueId(): RDD[(T, Long)] = withScope { val n = this.partitions.length.toLong this.mapPartitionsWithIndex { case (k, iter) => - iter.zipWithIndex.map { case (item, i) => + Utils.getIteratorZipWithIndex(iter, 0L).map { case (item, i) => (item, i * n + k) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala index b5738b9a95c36..b0e5ba0865c63 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala @@ -64,8 +64,7 @@ class ZippedWithIndexRDD[T: ClassTag](prev: RDD[T]) extends RDD[(T, Long)](prev) override def compute(splitIn: Partition, context: TaskContext): Iterator[(T, Long)] = { val split = splitIn.asInstanceOf[ZippedWithIndexRDDPartition] - firstParent[T].iterator(split.prev, context).zipWithIndex.map { x => - (x._1, split.startIndex + x._2) - } + val parentIter = firstParent[T].iterator(split.prev, context) + Utils.getIteratorZipWithIndex(parentIter, split.startIndex) } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 7fba901b85695..bfc609419ccdb 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1759,6 +1759,21 @@ private[spark] object Utils extends Logging { count } + /** + * Generate a zipWithIndex iterator, avoid index value overflowing problem + * in scala's zipWithIndex + */ + def getIteratorZipWithIndex[T](iterator: Iterator[T], startIndex: Long): Iterator[(T, Long)] = { + new Iterator[(T, Long)] { + var index: Long = startIndex - 1L + def hasNext: Boolean = iterator.hasNext + def next(): (T, Long) = { + index += 1L + (iterator.next(), index) + } + } + } + /** * Creates a symlink. * diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index b427f7fb50158..4dda80f10a08a 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -396,6 +396,13 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.getIteratorSize(iterator) === 5L) } + test("getIteratorZipWithIndex") { + val iterator = Utils.getIteratorZipWithIndex(Iterator(0, 1, 2), -1L + Int.MaxValue) + assert(iterator.toArray === Array( + (0, -1L + Int.MaxValue), (1, 0L + Int.MaxValue), (2, 1L + Int.MaxValue) + )) + } + test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files val parent: File = Utils.createTempDir() From 4bd17c4606764242bc29888b8eedc8e4b5a00f46 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 19 Oct 2016 23:55:05 -0700 Subject: [PATCH 138/177] [SPARK-17991][SQL] Enable metastore partition pruning by default. ## What changes were proposed in this pull request? This should apply to non-converted metastore relations. WIP to see if this causes any test failures. ## How was this patch tested? Existing tests. Author: Eric Liang Closes #15475 from ericl/try-enabling-pruning. --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9061b1b9a2638..ebf4fad5cbcff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -267,7 +267,7 @@ object SQLConf { .doc("When true, some predicates will be pushed down into the Hive metastore so that " + "unmatching partitions can be eliminated earlier.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val HIVE_FILESOURCE_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.hive.filesourcePartitionPruning") From c2c107abad8b462218d33c70b946e840663228a1 Mon Sep 17 00:00:00 2001 From: Mike Ihbe Date: Thu, 20 Oct 2016 09:49:58 +0100 Subject: [PATCH 139/177] [SPARK-11653][DEPLOY] Allow spark-daemon.sh to run in the foreground ## What changes were proposed in this pull request? Add a SPARK_NO_DAEMONIZE environment variable flag to spark-daemon.sh that causes the process it would run to be run in the foreground. It looks like there has been some prior work in https://github.com/apache/spark/pull/3881, but there was some talk about these being refactored. I'm not sure if that happened or not, but that PR is almost 2 years old at this point so it was worth revisiting. ## How was this patch tested? ./dev/run-tests still seems to work. It doesn't look like these scripts have tests, but if I missed them just let me know. Author: Mike Ihbe Closes #15338 from mikejihbe/SPARK-11653. --- conf/spark-env.sh.template | 1 + sbin/spark-daemon.sh | 54 ++++++++++++++++++++++---------------- 2 files changed, 33 insertions(+), 22 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index c750c72d19880..5c1e876ef9afc 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -63,3 +63,4 @@ # - SPARK_PID_DIR Where the pid file is stored. (Default: /tmp) # - SPARK_IDENT_STRING A string representing this instance of spark. (Default: $USER) # - SPARK_NICENESS The scheduling priority for daemons. (Default: 0) +# - SPARK_NO_DAEMONIZE Run the proposed command in the foreground. It will not output a PID file. diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 59823571124f1..061019a55e997 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -27,6 +27,7 @@ # SPARK_PID_DIR The pid files are stored. /tmp by default. # SPARK_IDENT_STRING A string representing this instance of spark. $USER by default # SPARK_NICENESS The scheduling priority for daemons. Defaults to 0. +# SPARK_NO_DAEMONIZE If set, will run the proposed command in the foreground. It will not output a PID file. ## usage="Usage: spark-daemon.sh [--config ] (start|stop|submit|status) " @@ -122,6 +123,35 @@ if [ "$SPARK_NICENESS" = "" ]; then export SPARK_NICENESS=0 fi +execute_command() { + local command="$@" + if [ -z ${SPARK_NO_DAEMONIZE+set} ]; then + nohup -- $command >> $log 2>&1 < /dev/null & + newpid="$!" + + echo "$newpid" > "$pid" + + # Poll for up to 5 seconds for the java process to start + for i in {1..10} + do + if [[ $(ps -p "$newpid" -o comm=) =~ "java" ]]; then + break + fi + sleep 0.5 + done + + sleep 2 + # Check if the process has died; in that case we'll tail the log so the user can see + if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then + echo "failed to launch $command:" + tail -2 "$log" | sed 's/^/ /' + echo "full log in $log" + fi + else + $command + fi +} + run_command() { mode="$1" shift @@ -146,13 +176,11 @@ run_command() { case "$mode" in (class) - nohup nice -n "$SPARK_NICENESS" "${SPARK_HOME}"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & - newpid="$!" + execute_command nice -n "$SPARK_NICENESS" "${SPARK_HOME}"/bin/spark-class $command $@ ;; (submit) - nohup nice -n "$SPARK_NICENESS" "${SPARK_HOME}"/bin/spark-submit --class $command "$@" >> "$log" 2>&1 < /dev/null & - newpid="$!" + execute_command nice -n "$SPARK_NICENESS" bash "${SPARK_HOME}"/bin/spark-submit --class $command $@ ;; (*) @@ -161,24 +189,6 @@ run_command() { ;; esac - echo "$newpid" > "$pid" - - #Poll for up to 5 seconds for the java process to start - for i in {1..10} - do - if [[ $(ps -p "$newpid" -o comm=) =~ "java" ]]; then - break - fi - sleep 0.5 - done - - sleep 2 - # Check if the process has died; in that case we'll tail the log so the user can see - if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then - echo "failed to launch $command:" - tail -2 "$log" | sed 's/^/ /' - echo "full log in $log" - fi } case $option in From 986a3b8b5bedb1d64e2cf7c95bfdf5505f3e8c69 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 20 Oct 2016 09:53:12 +0100 Subject: [PATCH 140/177] [SPARK-17796][SQL] Support wildcard character in filename for LOAD DATA LOCAL INPATH ## What changes were proposed in this pull request? Currently, Spark 2.0 raises an `input path does not exist` AnalysisException if the file name contains '*'. It is misleading since it occurs when there exist some matched files. Also, it was a supported feature in Spark 1.6.2. This PR aims to support wildcard characters in filename for `LOAD DATA LOCAL INPATH` SQL command like Spark 1.6.2. **Reported Error Scenario** ```scala scala> sql("CREATE TABLE t(a string)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("LOAD DATA LOCAL INPATH '/tmp/x*' INTO TABLE t") org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /tmp/x*; ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun Closes #15376 from dongjoon-hyun/SPARK-17796. --- .../spark/sql/execution/command/tables.scala | 23 +++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 30 +++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 403b479a0e1bc..4c0675adb4973 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.command import java.io.File import java.net.URI +import java.nio.file.FileSystems import java.util.Date import scala.collection.mutable.ArrayBuffer @@ -245,7 +246,27 @@ case class LoadDataCommand( val loadPath = if (isLocal) { val uri = Utils.resolveURI(path) - if (!new File(uri.getPath()).exists()) { + val filePath = uri.getPath() + val exists = if (filePath.contains("*")) { + val fileSystem = FileSystems.getDefault + val pathPattern = fileSystem.getPath(filePath) + val dir = pathPattern.getParent.toString + if (dir.contains("*")) { + throw new AnalysisException( + s"LOAD DATA input path allows only filename wildcard: $path") + } + + val files = new File(dir).listFiles() + if (files == null) { + false + } else { + val matcher = fileSystem.getPathMatcher("glob:" + pathPattern.toAbsolutePath) + files.exists(f => matcher.matches(fileSystem.getPath(f.getAbsolutePath))) + } + } else { + new File(filePath).exists() + } + if (!exists) { throw new AnalysisException(s"LOAD DATA input path does not exist: $path") } uri diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index e26b6b57ef569..495b4f874a1d6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.hive.execution +import java.io.{File, PrintWriter} +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import scala.sys.process.{Process, ProcessLogger} import scala.util.Try +import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.apache.spark.sql._ @@ -1917,6 +1920,33 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH") { + withTempDir { dir => + for (i <- 1 to 3) { + Files.write(s"$i", new File(s"$dir/part-r-0000$i"), StandardCharsets.UTF_8) + } + for (i <- 5 to 7) { + Files.write(s"$i", new File(s"$dir/part-s-0000$i"), StandardCharsets.UTF_8) + } + + withTable("load_t") { + sql("CREATE TABLE load_t (a STRING)") + sql(s"LOAD DATA LOCAL INPATH '$dir/*part-r*' INTO TABLE load_t") + checkAnswer(sql("SELECT * FROM load_t"), Seq(Row("1"), Row("2"), Row("3"))) + + val m = intercept[AnalysisException] { + sql("LOAD DATA LOCAL INPATH '/non-exist-folder/*part*' INTO TABLE load_t") + }.getMessage + assert(m.contains("LOAD DATA input path does not exist")) + + val m2 = intercept[AnalysisException] { + sql(s"LOAD DATA LOCAL INPATH '$dir*/*part*' INTO TABLE load_t") + }.getMessage + assert(m2.contains("LOAD DATA input path allows only filename wildcard")) + } + } + } + def testCommandAvailable(command: String): Boolean = { val attempt = Try(Process(command).run(ProcessLogger(_ => ())).exitValue()) attempt.isSuccess && attempt.get == 0 From e895bc25481f73b433a3cc3ad46df066ec602862 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 20 Oct 2016 19:39:25 +0800 Subject: [PATCH 141/177] [SPARK-17860][SQL] SHOW COLUMN's database conflict check should respect case sensitivity configuration ## What changes were proposed in this pull request? SHOW COLUMNS command validates the user supplied database name with database name from qualified table name name to make sure both of them are consistent. This comparison should respect case sensitivity. ## How was this patch tested? Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure. Author: Dilip Biswal Closes #15423 from dilipbiswal/dkb_show_column_fix. --- .../spark/sql/execution/SparkSqlParser.scala | 12 +- .../spark/sql/execution/command/tables.scala | 14 +- .../sql-tests/inputs/show_columns.sql | 58 +++++ .../sql-tests/results/show_columns.sql.out | 217 ++++++++++++++++++ .../apache/spark/sql/SQLQueryTestSuite.scala | 1 + .../execution/command/DDLCommandSuite.scala | 18 +- .../sql/execution/command/DDLSuite.scala | 17 ++ .../sql/hive/execution/HiveCommandSuite.scala | 23 +- .../hive/execution/HiveComparisonTest.scala | 2 +- 9 files changed, 318 insertions(+), 44 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/show_columns.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/show_columns.sql.out diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ea22b02d40b80..1cc166d5a7a9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -168,17 +168,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitShowColumns(ctx: ShowColumnsContext): LogicalPlan = withOrigin(ctx) { - val table = visitTableIdentifier(ctx.tableIdentifier) - - val lookupTable = Option(ctx.db) match { - case None => table - case Some(db) if table.database.exists(_ != db) => - operationNotAllowed( - s"SHOW COLUMNS with conflicting databases: '$db' != '${table.database.get}'", - ctx) - case Some(db) => TableIdentifier(table.identifier, Some(db.getText)) - } - ShowColumnsCommand(lookupTable) + ShowColumnsCommand(Option(ctx.db).map(_.getText), visitTableIdentifier(ctx.tableIdentifier)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 4c0675adb4973..aec25430b719d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -671,14 +671,24 @@ case class ShowTablePropertiesCommand(table: TableIdentifier, propertyKey: Optio * SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]; * }}} */ -case class ShowColumnsCommand(tableName: TableIdentifier) extends RunnableCommand { +case class ShowColumnsCommand( + databaseName: Option[String], + tableName: TableIdentifier) extends RunnableCommand { override val output: Seq[Attribute] = { AttributeReference("col_name", StringType, nullable = false)() :: Nil } override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - val table = catalog.getTempViewOrPermanentTableMetadata(tableName) + val resolver = sparkSession.sessionState.conf.resolver + val lookupTable = databaseName match { + case None => tableName + case Some(db) if tableName.database.exists(!resolver(_, db)) => + throw new AnalysisException( + s"SHOW COLUMNS with conflicting databases: '$db' != '${tableName.database.get}'") + case Some(db) => TableIdentifier(tableName.identifier, Some(db)) + } + val table = catalog.getTempViewOrPermanentTableMetadata(lookupTable) table.schema.map { c => Row(c.name) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql new file mode 100644 index 0000000000000..3894082255088 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql @@ -0,0 +1,58 @@ +CREATE DATABASE showdb; + +USE showdb; + +CREATE TABLE showcolumn1 (col1 int, `col 2` int); +CREATE TABLE showcolumn2 (price int, qty int) partitioned by (year int, month int); +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING parquet; +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5`; + + +-- only table name +SHOW COLUMNS IN showcolumn1; + +-- qualified table name +SHOW COLUMNS IN showdb.showcolumn1; + +-- table name and database name +SHOW COLUMNS IN showcolumn1 FROM showdb; + +-- partitioned table +SHOW COLUMNS IN showcolumn2 IN showdb; + +-- Non-existent table. Raise an error in this case +SHOW COLUMNS IN badtable FROM showdb; + +-- database in table identifier and database name in different case +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB; + +-- different database name in table identifier and database name. +-- Raise an error in this case. +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb; + +-- show column on temporary view +SHOW COLUMNS IN showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showdb.showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showcolumn3 FROM showdb; + +-- error global temp view needs to be qualified +SHOW COLUMNS IN showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN global_temp.showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN showcolumn4 FROM global_temp; + +DROP TABLE showcolumn1; +DROP TABLE showColumn2; +DROP VIEW showcolumn3; +DROP VIEW global_temp.showcolumn4; + +use default; + +DROP DATABASE showdb; diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out new file mode 100644 index 0000000000000..832e6e25bb2bd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -0,0 +1,217 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 25 + + +-- !query 0 +CREATE DATABASE showdb +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +USE showdb +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE showcolumn1 (col1 int, `col 2` int) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TABLE showcolumn2 (price int, qty int) partitioned by (year int, month int) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING parquet +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5` +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SHOW COLUMNS IN showcolumn1 +-- !query 6 schema +struct +-- !query 6 output +col 2 +col1 + + +-- !query 7 +SHOW COLUMNS IN showdb.showcolumn1 +-- !query 7 schema +struct +-- !query 7 output +col 2 +col1 + + +-- !query 8 +SHOW COLUMNS IN showcolumn1 FROM showdb +-- !query 8 schema +struct +-- !query 8 output +col 2 +col1 + + +-- !query 9 +SHOW COLUMNS IN showcolumn2 IN showdb +-- !query 9 schema +struct +-- !query 9 output +month +price +qty +year + + +-- !query 10 +SHOW COLUMNS IN badtable FROM showdb +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'badtable' not found in database 'showdb'; + + +-- !query 11 +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB +-- !query 11 schema +struct +-- !query 11 output +col 2 +col1 + + +-- !query 12 +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +SHOW COLUMNS with conflicting databases: 'baddb' != 'showdb'; + + +-- !query 13 +SHOW COLUMNS IN showcolumn3 +-- !query 13 schema +struct +-- !query 13 output +col 4 +col3 + + +-- !query 14 +SHOW COLUMNS IN showdb.showcolumn3 +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn3' not found in database 'showdb'; + + +-- !query 15 +SHOW COLUMNS IN showcolumn3 FROM showdb +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn3' not found in database 'showdb'; + + +-- !query 16 +SHOW COLUMNS IN showcolumn4 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn4' not found in database 'showdb'; + + +-- !query 17 +SHOW COLUMNS IN global_temp.showcolumn4 +-- !query 17 schema +struct +-- !query 17 output +col 5 +col1 + + +-- !query 18 +SHOW COLUMNS IN showcolumn4 FROM global_temp +-- !query 18 schema +struct +-- !query 18 output +col 5 +col1 + + +-- !query 19 +DROP TABLE showcolumn1 +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +DROP TABLE showColumn2 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DROP VIEW showcolumn3 +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +DROP VIEW global_temp.showcolumn4 +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +use default +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +DROP DATABASE showdb +-- !query 24 schema +struct<> +-- !query 24 output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 02841d7bb03ff..6857dd37286dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} +import org.apache.spark.sql.execution.command.ShowColumnsCommand import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index a3dbc9234f2f3..d31e7aeb3a78a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -824,22 +824,24 @@ class DDLCommandSuite extends PlanTest { val sql1 = "SHOW COLUMNS FROM t1" val sql2 = "SHOW COLUMNS IN db1.t1" val sql3 = "SHOW COLUMNS FROM t1 IN db1" - val sql4 = "SHOW COLUMNS FROM db1.t1 IN db1" - val sql5 = "SHOW COLUMNS FROM db1.t1 IN db2" + val sql4 = "SHOW COLUMNS FROM db1.t1 IN db2" val parsed1 = parser.parsePlan(sql1) - val expected1 = ShowColumnsCommand(TableIdentifier("t1", None)) + val expected1 = ShowColumnsCommand(None, TableIdentifier("t1", None)) val parsed2 = parser.parsePlan(sql2) - val expected2 = ShowColumnsCommand(TableIdentifier("t1", Some("db1"))) + val expected2 = ShowColumnsCommand(None, TableIdentifier("t1", Some("db1"))) val parsed3 = parser.parsePlan(sql3) - val parsed4 = parser.parsePlan(sql3) + val expected3 = ShowColumnsCommand(Some("db1"), TableIdentifier("t1", None)) + val parsed4 = parser.parsePlan(sql4) + val expected4 = ShowColumnsCommand(Some("db2"), TableIdentifier("t1", Some("db1"))) + comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) - comparePlans(parsed3, expected2) - comparePlans(parsed4, expected2) - assertUnsupported(sql5) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) } + test("show partitions") { val sql1 = "SHOW PARTITIONS t1" val sql2 = "SHOW PARTITIONS db1.t1" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index c8b8e9ebabc75..a6da8a86c1623 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1749,4 +1749,21 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(sql("show user functions").count() === 1L) } } + + test("show columns - negative test") { + // When case sensitivity is true, the user supplied database name in table identifier + // should match the supplied database name in case sensitive way. + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTempDatabase { db => + val tabName = s"$db.showcolumn" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(col1 int, col2 string) USING parquet ") + val message = intercept[AnalysisException] { + sql(s"SHOW COLUMNS IN $db.showcolumn FROM ${db.toUpperCase}") + }.getMessage + assert(message.contains("SHOW COLUMNS with conflicting databases")) + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 2c772ce2155ef..ad1e9b17a9f71 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -336,28 +337,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } - test("show columns") { - checkAnswer( - sql("SHOW COLUMNS IN parquet_tab3"), - Row("col1") :: Row("col 2") :: Nil) - - checkAnswer( - sql("SHOW COLUMNS IN default.parquet_tab3"), - Row("col1") :: Row("col 2") :: Nil) - - checkAnswer( - sql("SHOW COLUMNS IN parquet_tab3 FROM default"), - Row("col1") :: Row("col 2") :: Nil) - - checkAnswer( - sql("SHOW COLUMNS IN parquet_tab4 IN default"), - Row("price") :: Row("qty") :: Row("year") :: Row("month") :: Nil) - - val message = intercept[NoSuchTableException] { - sql("SHOW COLUMNS IN badtable FROM default") - }.getMessage - assert(message.contains("'badtable' not found in database")) - } test("show partitions - show everything") { checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 80e75aa898c38..13ceed7c79e35 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -167,7 +167,7 @@ abstract class HiveComparisonTest // and does not return it as a query answer. case _: SetCommand => Seq("0") case _: ExplainCommand => answer - case _: DescribeTableCommand | ShowColumnsCommand(_) => + case _: DescribeTableCommand | ShowColumnsCommand(_, _) => // Filter out non-deterministic lines and lines which do not have actual results but // can introduce problems because of the way Hive formats these lines. // Then, remove empty lines. Do not sort the results. From fb0894b3a87331a731129ad3fc7ebe598d90a6ee Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Thu, 20 Oct 2016 09:50:55 -0700 Subject: [PATCH 142/177] [SPARK-17698][SQL] Join predicates should not contain filter clauses ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-17698 `ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below: [0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91 eg. ``` val df = (1 until 10).toDF("id").coalesce(1) hc.sql("DROP TABLE IF EXISTS table1").collect df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1") hc.sql("DROP TABLE IF EXISTS table2").collect df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2") sqlContext.sql(""" SELECT a.id, b.id FROM table1 a FULL OUTER JOIN table2 b ON a.id = b.id AND a.id='1' AND b.id='1' """).explain(true) ``` BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job. ``` SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter :- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200) : +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200) +- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct ``` AFTER : ``` SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0)) :- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct ``` ## How was this patch tested? - Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses` - Ran all the tests in `BucketedReadSuite` Author: Tejas Patil Closes #15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause. --- .../sql/catalyst/expressions/predicates.scala | 5 +- .../spark/sql/catalyst/optimizer/joins.scala | 4 +- .../sql/catalyst/planning/patterns.scala | 2 + .../spark/sql/sources/BucketedReadSuite.scala | 124 ++++++++++++++---- 4 files changed, 109 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 799858a6865e5..9394e39aadd9d 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 @@ -84,8 +84,9 @@ trait PredicateHelper { * * For example consider a join between two relations R(a, b) and S(c, d). * - * `canEvaluate(EqualTo(a,b), R)` returns `true` where as `canEvaluate(EqualTo(a,c), R)` returns - * `false`. + * - `canEvaluate(EqualTo(a,b), R)` returns `true` + * - `canEvaluate(EqualTo(a,c), R)` returns `false` + * - `canEvaluate(Literal(1), R)` returns `true` as literals CAN be evaluated on any plan */ protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = expr.references.subsetOf(plan.outputSet) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 2626057e492ef..180ad2e0ad1fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -65,7 +65,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val conditionalJoin = rest.find { planJoinPair => val plan = planJoinPair._1 val refs = left.outputSet ++ plan.outputSet - conditions.filterNot(canEvaluate(_, left)).filterNot(canEvaluate(_, plan)) + conditions + .filterNot(l => l.references.nonEmpty && canEvaluate(l, left)) + .filterNot(r => r.references.nonEmpty && canEvaluate(r, plan)) .exists(_.references.subsetOf(refs)) } // pick the next one if no condition left diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index bdae56881bf46..c5f92c59c88f4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -112,6 +112,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { // as join keys. val predicates = condition.map(splitConjunctivePredicates).getOrElse(Nil) val joinKeys = predicates.flatMap { + case EqualTo(l, r) if l.references.isEmpty || r.references.isEmpty => None case EqualTo(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => Some((l, r)) case EqualTo(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => Some((r, l)) // Replace null with default value for joining key, then those rows with null in it could @@ -125,6 +126,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { case other => None } val otherPredicates = predicates.filterNot { + case EqualTo(l, r) if l.references.isEmpty || r.references.isEmpty => false case EqualTo(l, r) => canEvaluate(l, left) && canEvaluate(r, right) || canEvaluate(l, right) && canEvaluate(r, left) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 3ff85176de10e..9ed454e578d69 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -235,7 +235,8 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet private def testBucketing( bucketSpecLeft: Option[BucketSpec], bucketSpecRight: Option[BucketSpec], - joinColumns: Seq[String], + joinType: String = "inner", + joinCondition: (DataFrame, DataFrame) => Column, shuffleLeft: Boolean, shuffleRight: Boolean, sortLeft: Boolean = true, @@ -268,12 +269,12 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { val t1 = spark.table("bucketed_table1") val t2 = spark.table("bucketed_table2") - val joined = t1.join(t2, joinCondition(t1, t2, joinColumns)) + val joined = t1.join(t2, joinCondition(t1, t2), joinType) // First check the result is corrected. checkAnswer( joined.sort("bucketed_table1.k", "bucketed_table2.k"), - df1.join(df2, joinCondition(df1, df2, joinColumns)).sort("df1.k", "df2.k")) + df1.join(df2, joinCondition(df1, df2), joinType).sort("df1.k", "df2.k")) assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoinExec]) val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoinExec] @@ -297,56 +298,102 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet } } - private def joinCondition(left: DataFrame, right: DataFrame, joinCols: Seq[String]): Column = { + private def joinCondition(joinCols: Seq[String]) (left: DataFrame, right: DataFrame): Column = { joinCols.map(col => left(col) === right(col)).reduce(_ && _) } test("avoid shuffle when join 2 bucketed tables") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) - testBucketing(bucketSpec, bucketSpec, Seq("i", "j"), shuffleLeft = false, shuffleRight = false) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false + ) } // Enable it after fix https://issues.apache.org/jira/browse/SPARK-12704 ignore("avoid shuffle when join keys are a super-set of bucket keys") { val bucketSpec = Some(BucketSpec(8, Seq("i"), Nil)) - testBucketing(bucketSpec, bucketSpec, Seq("i", "j"), shuffleLeft = false, shuffleRight = false) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false + ) } test("only shuffle one side when join bucketed table and non-bucketed table") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) - testBucketing(bucketSpec, None, Seq("i", "j"), shuffleLeft = false, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = None, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = true + ) } test("only shuffle one side when 2 bucketed tables have different bucket number") { val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Nil)) val bucketSpec2 = Some(BucketSpec(5, Seq("i", "j"), Nil)) - testBucketing(bucketSpec1, bucketSpec2, Seq("i", "j"), shuffleLeft = false, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = true + ) } test("only shuffle one side when 2 bucketed tables have different bucket keys") { val bucketSpec1 = Some(BucketSpec(8, Seq("i"), Nil)) val bucketSpec2 = Some(BucketSpec(8, Seq("j"), Nil)) - testBucketing(bucketSpec1, bucketSpec2, Seq("i"), shuffleLeft = false, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i")), + shuffleLeft = false, + shuffleRight = true + ) } test("shuffle when join keys are not equal to bucket keys") { val bucketSpec = Some(BucketSpec(8, Seq("i"), Nil)) - testBucketing(bucketSpec, bucketSpec, Seq("j"), shuffleLeft = true, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("j")), + shuffleLeft = true, + shuffleRight = true + ) } test("shuffle when join 2 bucketed tables with bucketing disabled") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) withSQLConf(SQLConf.BUCKETING_ENABLED.key -> "false") { - testBucketing(bucketSpec, bucketSpec, Seq("i", "j"), shuffleLeft = true, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = true, + shuffleRight = true + ) } } test("avoid shuffle and sort when bucket and sort columns are join keys") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) testBucketing( - bucketSpec, bucketSpec, Seq("i", "j"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = false + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = false ) } @@ -354,9 +401,13 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet val bucketSpec1 = Some(BucketSpec(8, Seq("i"), Seq("i", "j"))) val bucketSpec2 = Some(BucketSpec(8, Seq("i"), Seq("i", "k"))) testBucketing( - bucketSpec1, bucketSpec2, Seq("i"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = false + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = false ) } @@ -364,9 +415,13 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) val bucketSpec2 = Some(BucketSpec(8, Seq("i", "j"), Seq("k"))) testBucketing( - bucketSpec1, bucketSpec2, Seq("i", "j"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = true + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = true ) } @@ -374,9 +429,13 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) val bucketSpec2 = Some(BucketSpec(8, Seq("i", "j"), Seq("j", "i"))) testBucketing( - bucketSpec1, bucketSpec2, Seq("i", "j"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = true + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = true ) } @@ -408,6 +467,25 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet } } + test("SPARK-17698 Join predicates should not contain filter clauses") { + val bucketSpec = Some(BucketSpec(8, Seq("i"), Seq("i"))) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinType = "fullouter", + joinCondition = (left: DataFrame, right: DataFrame) => { + val joinPredicates = Seq("i").map(col => left(col) === right(col)).reduce(_ && _) + val filterLeft = left("i") === Literal("1") + val filterRight = right("i") === Literal("1") + joinPredicates && filterLeft && filterRight + }, + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = false + ) + } + test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") From 84b245f2dd31c1cebbf12458bf11f67e287e93f4 Mon Sep 17 00:00:00 2001 From: Koert Kuipers Date: Thu, 20 Oct 2016 10:08:12 -0700 Subject: [PATCH 143/177] [SPARK-15780][SQL] Support mapValues on KeyValueGroupedDataset ## What changes were proposed in this pull request? Add mapValues to KeyValueGroupedDataset ## How was this patch tested? New test in DatasetSuite for groupBy function, mapValues, flatMap Author: Koert Kuipers Closes #13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues. --- .../sql/catalyst/plans/logical/object.scala | 13 ++++++ .../spark/sql/KeyValueGroupedDataset.scala | 42 +++++++++++++++++++ .../org/apache/spark/sql/DatasetSuite.scala | 11 +++++ 3 files changed, 66 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index fefe5a3953a6e..0ab4c9016623e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -230,6 +230,19 @@ object AppendColumns { encoderFor[U].namedExpressions, child) } + + def apply[T : Encoder, U : Encoder]( + func: T => U, + inputAttributes: Seq[Attribute], + child: LogicalPlan): AppendColumns = { + new AppendColumns( + func.asInstanceOf[Any => Any], + implicitly[Encoder[T]].clsTag.runtimeClass, + implicitly[Encoder[T]].schema, + UnresolvedDeserializer(encoderFor[T].deserializer, inputAttributes), + encoderFor[U].namedExpressions, + child) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 828eb94efe598..4cb0313aa9037 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -66,6 +66,48 @@ class KeyValueGroupedDataset[K, V] private[sql]( dataAttributes, groupingAttributes) + /** + * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied + * to the data. The grouping key is unchanged by this. + * + * {{{ + * // Create values grouped by key from a Dataset[(K, V)] + * ds.groupByKey(_._1).mapValues(_._2) // Scala + * }}} + * + * @since 2.1.0 + */ + def mapValues[W : Encoder](func: V => W): KeyValueGroupedDataset[K, W] = { + val withNewData = AppendColumns(func, dataAttributes, logicalPlan) + val projected = Project(withNewData.newColumns ++ groupingAttributes, withNewData) + val executed = sparkSession.sessionState.executePlan(projected) + + new KeyValueGroupedDataset( + encoderFor[K], + encoderFor[W], + executed, + withNewData.newColumns, + groupingAttributes) + } + + /** + * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied + * to the data. The grouping key is unchanged by this. + * + * {{{ + * // Create Integer values grouped by String key from a Dataset> + * Dataset> ds = ...; + * KeyValueGroupedDataset grouped = + * ds.groupByKey(t -> t._1, Encoders.STRING()).mapValues(t -> t._2, Encoders.INT()); // Java 8 + * }}} + * + * @since 2.1.0 + */ + def mapValues[W](func: MapFunction[V, W], encoder: Encoder[W]): KeyValueGroupedDataset[K, W] = { + implicit val uEnc = encoder + mapValues { (v: V) => func.call(v) } + } + /** * Returns a [[Dataset]] that contains each unique key. This is equivalent to doing mapping * over the Dataset to extract the keys and then running a distinct operation on those. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 5fce9b4fe97ea..cc367acae2ba4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -336,6 +336,17 @@ class DatasetSuite extends QueryTest with SharedSQLContext { "a", "30", "b", "3", "c", "1") } + test("groupBy function, mapValues, flatMap") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + val keyValue = ds.groupByKey(_._1).mapValues(_._2) + val agged = keyValue.mapGroups { case (g, iter) => (g, iter.sum) } + checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) + + val keyValue1 = ds.groupByKey(t => (t._1, "key")).mapValues(t => (t._2, "value")) + val agged1 = keyValue1.mapGroups { case (g, iter) => (g._1, iter.map(_._1).sum) } + checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) + } + test("groupBy function, reduce") { val ds = Seq("abc", "xyz", "hello").toDS() val agged = ds.groupByKey(_.length).reduceGroups(_ + _) From 947f4f25273161dc4719419a35613a71c2e2a150 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 20 Oct 2016 10:50:34 -0700 Subject: [PATCH 144/177] [SPARK-17999][KAFKA][SQL] Add getPreferredLocations for KafkaSourceRDD ## What changes were proposed in this pull request? The newly implemented Structured Streaming `KafkaSource` did calculate the preferred locations for each topic partition, but didn't offer this information through RDD's `getPreferredLocations` method. So here propose to add this method in `KafkaSourceRDD`. ## How was this patch tested? Manual verification. Author: jerryshao Closes #15545 from jerryshao/SPARK-17999. --- .../scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 496af7e39abab..802dd040aed93 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -112,6 +112,11 @@ private[kafka010] class KafkaSourceRDD( buf.toArray } + override def getPreferredLocations(split: Partition): Seq[String] = { + val part = split.asInstanceOf[KafkaSourceRDDPartition] + part.offsetRange.preferredLoc.map(Seq(_)).getOrElse(Seq.empty) + } + override def compute( thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { From 7f9ec19eae60abe589ffd22259a9065e7e353a57 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 20 Oct 2016 12:18:56 -0700 Subject: [PATCH 145/177] [SPARK-18021][SQL] Refactor file name specification for data sources ## What changes were proposed in this pull request? Currently each data source OutputWriter is responsible for specifying the entire file name for each file output. This, however, does not make any sense because we rely on file naming schemes for certain behaviors in Spark SQL, e.g. bucket id. The current approach allows individual data sources to break the implementation of bucketing. On the flip side, we also don't want to move file naming entirely out of data sources, because different data sources do want to specify different extensions. This patch divides file name specification into two parts: the first part is a prefix specified by the caller of OutputWriter (in WriteOutput), and the second part is the suffix that can be specified by the OutputWriter itself. Note that a side effect of this change is that now all file based data sources also support bucketing automatically. There are also some other minor cleanups: - Removed the UUID passed through generic Configuration string - Some minor rewrites for better clarity - Renamed "path" in multiple places to "stagingDir", to more accurately reflect its meaning ## How was this patch tested? This should be covered by existing data source tests. Author: Reynold Xin Closes #15562 from rxin/SPARK-18021. --- .../ml/source/libsvm/LibSVMRelation.scala | 16 ++---- .../execution/datasources/OutputWriter.scala | 17 ++++-- .../execution/datasources/WriteOutput.scala | 56 +++++++++---------- .../datasources/csv/CSVRelation.scala | 18 +++--- .../datasources/json/JsonFileFormat.scala | 17 ++---- .../parquet/ParquetFileFormat.scala | 7 +-- .../parquet/ParquetOutputWriter.scala | 32 +++-------- .../datasources/text/TextFileFormat.scala | 21 +++---- .../spark/sql/hive/orc/OrcFileFormat.scala | 21 +++---- .../sql/sources/BucketedWriteSuite.scala | 5 -- .../sql/sources/CommitFailureTestSource.scala | 6 +- .../sql/sources/SimpleTextRelation.scala | 26 ++++----- 12 files changed, 99 insertions(+), 143 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 8577803743c8e..fff86686b550c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -40,7 +40,8 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration private[libsvm] class LibSVMOutputWriter( - path: String, + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter { @@ -50,11 +51,7 @@ private[libsvm] class LibSVMOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") + new Path(stagingDir, fileNamePrefix + extension) } }.getRecordWriter(context) } @@ -132,12 +129,11 @@ private[libsvm] class LibSVMFileFormat extends TextBasedFileFormat with DataSour dataSchema: StructType): OutputWriterFactory = { new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - if (bucketId.isDefined) { sys.error("LibSVM doesn't support bucketing") } - new LibSVMOutputWriter(path, dataSchema, context) + new LibSVMOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala index d2eec7b1413f8..f4cefdab077e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala @@ -34,18 +34,23 @@ abstract class OutputWriterFactory extends Serializable { * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side * to instantiate new [[OutputWriter]]s. * - * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that - * this may not point to the final output file. For example, `FileOutputFormat` writes to - * temporary directories and then merge written files back to the final destination. In - * this case, `path` points to a temporary output file under the temporary directory. + * @param stagingDir Base path (directory) of the file to which this [[OutputWriter]] is supposed + * to write. Note that this may not point to the final output file. For + * example, `FileOutputFormat` writes to temporary directories and then merge + * written files back to the final destination. In this case, `path` points to + * a temporary output file under the temporary directory. + * @param fileNamePrefix Prefix of the file name. The returned OutputWriter must make sure this + * prefix is used in the actual file name. For example, if the prefix is + * "part-1-2-3", then the file name must start with "part_1_2_3" but can + * end in arbitrary extension. * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the relation being written is partitioned. * @param context The Hadoop MapReduce task context. * @since 1.4.0 */ def newInstance( - path: String, - bucketId: Option[Int], // TODO: This doesn't belong here... + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala index 54d0f3bd6291a..bd56e511d0ccf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala @@ -46,6 +46,7 @@ object WriteOutput extends Logging { /** A shared job description for all the write tasks. */ private class WriteJobDescription( + val uuid: String, // prevent collision between different (appending) write jobs val serializableHadoopConf: SerializableConfiguration, val outputWriterFactory: OutputWriterFactory, val allColumns: Seq[Attribute], @@ -102,6 +103,7 @@ object WriteOutput extends Logging { fileFormat.prepareWrite(sparkSession, job, options, dataColumns.toStructType) val description = new WriteJobDescription( + uuid = UUID.randomUUID().toString, serializableHadoopConf = new SerializableConfiguration(job.getConfiguration), outputWriterFactory = outputWriterFactory, allColumns = plan.output, @@ -213,6 +215,11 @@ object WriteOutput extends Logging { private trait ExecuteWriteTask { def execute(iterator: Iterator[InternalRow]): Unit def releaseResources(): Unit + + final def filePrefix(split: Int, uuid: String, bucketId: Option[Int]): String = { + val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") + f"part-r-$split%05d-$uuid$bucketString" + } } /** Writes data to a single directory (used for non-dynamic-partition writes). */ @@ -222,9 +229,11 @@ object WriteOutput extends Logging { stagingPath: String) extends ExecuteWriteTask { private[this] var outputWriter: OutputWriter = { + val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId + val outputWriter = description.outputWriterFactory.newInstance( - path = stagingPath, - bucketId = None, + stagingDir = stagingPath, + fileNamePrefix = filePrefix(split, description.uuid, None), dataSchema = description.nonPartitionColumns.toStructType, context = taskAttemptContext) outputWriter.initConverter(dataSchema = description.nonPartitionColumns.toStructType) @@ -287,29 +296,31 @@ object WriteOutput extends Logging { } } - private def getBucketIdFromKey(key: InternalRow): Option[Int] = - description.bucketSpec.map { _ => key.getInt(description.partitionColumns.length) } - /** * Open and returns a new OutputWriter given a partition key and optional bucket id. * If bucket id is specified, we will append it to the end of the file name, but before the * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet */ - private def newOutputWriter( - key: InternalRow, - getPartitionString: UnsafeProjection): OutputWriter = { + private def newOutputWriter(key: InternalRow, partString: UnsafeProjection): OutputWriter = { val path = if (description.partitionColumns.nonEmpty) { - val partitionPath = getPartitionString(key).getString(0) + val partitionPath = partString(key).getString(0) new Path(stagingPath, partitionPath).toString } else { stagingPath } - val bucketId = getBucketIdFromKey(key) + // If the bucket spec is defined, the bucket column is right after the partition columns + val bucketId = if (description.bucketSpec.isDefined) { + Some(key.getInt(description.partitionColumns.length)) + } else { + None + } + + val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId val newWriter = description.outputWriterFactory.newInstance( - path = path, - bucketId = bucketId, + stagingDir = path, + fileNamePrefix = filePrefix(split, description.uuid, bucketId), dataSchema = description.nonPartitionColumns.toStructType, context = taskAttemptContext) newWriter.initConverter(description.nonPartitionColumns.toStructType) @@ -319,7 +330,7 @@ object WriteOutput extends Logging { override def execute(iter: Iterator[InternalRow]): Unit = { // We should first sort by partition columns, then bucket id, and finally sorting columns. val sortingExpressions: Seq[Expression] = - description.partitionColumns ++ bucketIdExpression ++ sortColumns + description.partitionColumns ++ bucketIdExpression ++ sortColumns val getSortingKey = UnsafeProjection.create(sortingExpressions, description.allColumns) val sortingKeySchema = StructType(sortingExpressions.map { @@ -333,8 +344,8 @@ object WriteOutput extends Logging { description.nonPartitionColumns, description.allColumns) // Returns the partition path given a partition key. - val getPartitionString = - UnsafeProjection.create(Seq(Concat(partitionStringExpression)), description.partitionColumns) + val getPartitionString = UnsafeProjection.create( + Seq(Concat(partitionStringExpression)), description.partitionColumns) // Sorts the data before write, so that we only need one writer at the same time. val sorter = new UnsafeKVExternalSorter( @@ -405,17 +416,6 @@ object WriteOutput extends Logging { job.getConfiguration.setBoolean("mapred.task.is.map", true) job.getConfiguration.setInt("mapred.task.partition", 0) - // This UUID is sent to executor side together with the serialized `Configuration` object within - // the `Job` instance. `OutputWriters` on the executor side should use this UUID to generate - // unique task output files. - // This UUID is used to avoid output file name collision between different appending write jobs. - // These jobs may belong to different SparkContext instances. Concrete data source - // implementations may use this UUID to generate unique file names (e.g., - // `part-r--.parquet`). The reason why this ID is used to identify a job - // rather than a single task output file is that, speculative tasks must generate the same - // output file name as the original task. - job.getConfiguration.set(WriterContainer.DATASOURCE_WRITEJOBUUID, UUID.randomUUID().toString) - val taskAttemptContext = new TaskAttemptContextImpl(job.getConfiguration, taskAttemptId) val outputCommitter = newOutputCommitter( job.getOutputFormatClass, taskAttemptContext, path, isAppend) @@ -474,7 +474,3 @@ object WriteOutput extends Logging { } } } - -object WriterContainer { - val DATASOURCE_WRITEJOBUUID = "spark.sql.sources.writeJobUUID" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 55cb26d6513af..eefacbf05ba0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile, WriterContainer} +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.types._ object CSVRelation extends Logging { @@ -170,17 +170,17 @@ object CSVRelation extends Logging { private[csv] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - if (bucketId.isDefined) sys.error("csv doesn't support bucketing") - new CsvOutputWriter(path, dataSchema, context, params) + new CsvOutputWriter(stagingDir, fileNamePrefix, dataSchema, context, params) } } private[csv] class CsvOutputWriter( - path: String, + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext, params: CSVOptions) extends OutputWriter with Logging { @@ -199,11 +199,7 @@ private[csv] class CsvOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.csv$extension") + new Path(stagingDir, s"$fileNamePrefix.csv$extension") } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 9fe38ccc9fdc6..cdbb2f7292613 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -82,11 +82,11 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new JsonOutputWriter(path, parsedOptions, bucketId, dataSchema, context) + new JsonOutputWriter(stagingDir, parsedOptions, fileNamePrefix, dataSchema, context) } } } @@ -153,9 +153,9 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { } private[json] class JsonOutputWriter( - path: String, + stagingDir: String, options: JSONOptions, - bucketId: Option[Int], + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter with Logging { @@ -168,12 +168,7 @@ private[json] class JsonOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString.json$extension") + new Path(stagingDir, s"$fileNamePrefix.json$extension") } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 6faafed1e6290..87b944ba523ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -27,7 +27,7 @@ import scala.util.{Failure, Try} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit} +import org.apache.hadoop.mapreduce.lib.input.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.parquet.{Log => ApacheParquetLog} import org.apache.parquet.filter2.compat.FilterCompat @@ -45,7 +45,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -134,10 +133,10 @@ class ParquetFileFormat new OutputWriterFactory { override def newInstance( path: String, - bucketId: Option[Int], + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new ParquetOutputWriter(path, bucketId, context) + new ParquetOutputWriter(path, fileNamePrefix, context) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala index f89ce05d82d90..39c199784cd6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -26,7 +26,7 @@ import org.apache.parquet.hadoop.util.ContextUtil import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.{BucketingUtils, OutputWriter, OutputWriterFactory, WriterContainer} +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -122,13 +122,12 @@ private[parquet] class ParquetOutputWriterFactory( } /** Disable the use of the older API. */ - def newInstance( + override def newInstance( path: String, - bucketId: Option[Int], + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - throw new UnsupportedOperationException( - "this version of newInstance not supported for " + + throw new UnsupportedOperationException("this version of newInstance not supported for " + "ParquetOutputWriterFactory") } } @@ -136,33 +135,16 @@ private[parquet] class ParquetOutputWriterFactory( // NOTE: This class is instantiated and used on executor side only, no need to be serializable. private[parquet] class ParquetOutputWriter( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, context: TaskAttemptContext) extends OutputWriter { private val recordWriter: RecordWriter[Void, InternalRow] = { val outputFormat = { new ParquetOutputFormat[InternalRow]() { - // Here we override `getDefaultWorkFile` for two reasons: - // - // 1. To allow appending. We need to generate unique output file names to avoid - // overwriting existing files (either exist before the write job, or are just written - // by other tasks within the same write job). - // - // 2. To allow dynamic partitioning. Default `getDefaultWorkFile` uses - // `FileOutputCommitter.getWorkPath()`, which points to the base directory of all - // partitions in the case of dynamic partitioning. override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - // It has the `.parquet` extension at the end because (de)compression tools - // such as gunzip would not be able to decompress this as the compression - // is not applied on this whole file but on each "page" in Parquet format. - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") + new Path(stagingDir, fileNamePrefix + extension) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index 9f96667311015..6cd2351c5749a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -73,14 +73,11 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - if (bucketId.isDefined) { - throw new AnalysisException("Text doesn't support bucketing") - } - new TextOutputWriter(path, dataSchema, context) + new TextOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) } } } @@ -124,7 +121,11 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { } } -class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemptContext) +class TextOutputWriter( + stagingDir: String, + fileNamePrefix: String, + dataSchema: StructType, + context: TaskAttemptContext) extends OutputWriter { private[this] val buffer = new Text() @@ -132,11 +133,7 @@ class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemp private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.txt$extension") + new Path(stagingDir, s"$fileNamePrefix.txt$extension") } }.getRecordWriter(context) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 1af3280e18a89..1ceacb458ae6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -83,11 +83,11 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new OrcOutputWriter(path, bucketId, dataSchema, context) + new OrcOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) } } } @@ -210,8 +210,8 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) } private[orc] class OrcOutputWriter( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter { @@ -226,10 +226,7 @@ private[orc] class OrcOutputWriter( private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - val uniqueWriteJobId = conf.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val partition = taskAttemptId.getTaskID.getId - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") + val compressionExtension = { val name = conf.get(OrcRelation.ORC_COMPRESSION) OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") @@ -237,12 +234,12 @@ private[orc] class OrcOutputWriter( // It has the `.orc` extension at the end because (de)compression tools // such as gunzip would not be able to decompress this as the compression // is not applied on this whole file but on each "stream" in ORC format. - val filename = f"part-r-$partition%05d-$uniqueWriteJobId$bucketString$compressionExtension.orc" + val filename = s"$fileNamePrefix$compressionExtension.orc" new OrcOutputFormat().getRecordWriter( - new Path(path, filename).getFileSystem(conf), + new Path(stagingDir, filename).getFileSystem(conf), conf.asInstanceOf[JobConf], - new Path(path, filename).toString, + new Path(stagingDir, filename).toString, Reporter.NULL ).asInstanceOf[RecordWriter[NullWritable, Writable]] } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 997445114ba58..2eafe18b85844 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -54,11 +54,6 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle intercept[AnalysisException](df.write.bucketBy(2, "i").sortBy("j").saveAsTable("tt")) } - test("write bucketed data to unsupported data source") { - val df = Seq(Tuple1("a"), Tuple1("b")).toDF("i") - intercept[SparkException](df.write.bucketBy(3, "i").format("text").saveAsTable("tt")) - } - test("write bucketed data using save()") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala index 5a8a7f0ab5d7b..d5044684020e2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -39,11 +39,11 @@ class CommitFailureTestSource extends SimpleTextSource { dataSchema: StructType): OutputWriterFactory = new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(path, context) { + new SimpleTextOutputWriter(stagingDir, fileNamePrefix, context) { var failed = false TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) => failed = true diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 906de6bbcbee5..9e13b217ec305 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.{NullWritable, Text} import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat import org.apache.spark.sql.{sources, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, InternalRow} @@ -51,11 +51,11 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { SimpleTextRelation.lastHadoopConf = Option(job.getConfiguration) new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(path, context) + new SimpleTextOutputWriter(stagingDir, fileNamePrefix, context) } } } @@ -120,9 +120,11 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { } } -class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) extends OutputWriter { +class SimpleTextOutputWriter( + stagingDir: String, fileNamePrefix: String, context: TaskAttemptContext) + extends OutputWriter { private val recordWriter: RecordWriter[NullWritable, Text] = - new AppendingTextOutputFormat(new Path(path)).getRecordWriter(context) + new AppendingTextOutputFormat(new Path(stagingDir), fileNamePrefix).getRecordWriter(context) override def write(row: Row): Unit = { val serialized = row.toSeq.map { v => @@ -136,19 +138,15 @@ class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) extends } } -class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullWritable, Text] { - val numberFormat = NumberFormat.getInstance() +class AppendingTextOutputFormat(stagingDir: Path, fileNamePrefix: String) + extends TextOutputFormat[NullWritable, Text] { + val numberFormat = NumberFormat.getInstance() numberFormat.setMinimumIntegerDigits(5) numberFormat.setGroupingUsed(false) override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - val name = FileOutputFormat.getOutputName(context) - new Path(outputFile, s"$name-${numberFormat.format(split)}-$uniqueWriteJobId") + new Path(stagingDir, fileNamePrefix) } } From 2d14ab7e644b64ff911772e71f42653ba949cb07 Mon Sep 17 00:00:00 2001 From: Mark Grover Date: Thu, 20 Oct 2016 15:30:01 -0700 Subject: [PATCH 146/177] [DOCS] Update docs to not suggest to package Spark before running tests. ## What changes were proposed in this pull request? Update docs to not suggest to package Spark before running tests. ## How was this patch tested? Not creating a JIRA since this pretty small. We haven't had the need to run mvn package before mvn test since 1.6 at least, or so I am told. So, updating the docs to not be misguiding. Author: Mark Grover Closes #15572 from markgrover/doc_update. --- docs/building-spark.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index f5acee6b90059..ebe46a42a15c6 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -217,9 +217,8 @@ For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troub Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Note that tests should not be run as root or an admin user. -Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: +The following is an example of a command to run the tests: - ./build/mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package ./build/mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test The ScalaTest plugin also supports running only a specific Scala test suite as follows: @@ -233,9 +232,8 @@ or a Java test: ## Testing with SBT -Some of the tests require Spark to be packaged first, so always run `build/sbt package` the first time. The following is an example of a correct (build, test) sequence: +The following is an example of a command to run the tests: - ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver package ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: From 1bb99c4887e97ae5f55c8c2b392ba5ca72d6168b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 20 Oct 2016 20:44:32 -0700 Subject: [PATCH 147/177] [SPARK-18030][TESTS] Adds more checks to collect more info about FileStreamSourceSuite failure ## What changes were proposed in this pull request? My hunch is `mkdirs` fails. Just add more checks to collect more info. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15577 from zsxwing/SPARK-18030-debug. --- .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 aabdccaaf319d..b9e9da9a1ec53 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 @@ -664,7 +664,9 @@ class FileStreamSourceSuite extends FileStreamSourceTest { def createFile(content: String, src: File, tmp: File): Unit = { val tempFile = Utils.tempFileWith(new File(tmp, "text")) val finalFile = new File(src, tempFile.getName) - src.mkdirs() + require(!src.exists(), s"$src exists, dir: ${src.isDirectory}, file: ${src.isFile}") + require(src.mkdirs(), s"Cannot create $src") + require(src.isDirectory(), s"$src is not a directory") require(stringToFile(tempFile, content).renameTo(finalFile)) } From 3180272d2d49e440516085c0e4aebd5bad18bcad Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 20 Oct 2016 21:12:55 -0700 Subject: [PATCH 148/177] [SPARKR] fix warnings ## What changes were proposed in this pull request? Fix for a bunch of test warnings that were added recently. We need to investigate why warnings are not turning into errors. ``` Warnings ----------------------------------------------------------------------- 1. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Sepal_Length instead of Sepal.Length as column name 2. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Sepal_Width instead of Sepal.Width as column name 3. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Petal_Length instead of Petal.Length as column name 4. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Petal_Width instead of Petal.Width as column name Consider adding importFrom("utils", "object.size") to your NAMESPACE file. ``` ## How was this patch tested? unit tests Author: Felix Cheung Closes #15560 from felixcheung/rwarnings. --- R/pkg/NAMESPACE | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 267a38c21530b..5960c6206a6f1 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -3,7 +3,7 @@ importFrom("methods", "setGeneric", "setMethod", "setOldClass") importFrom("methods", "is", "new", "signature", "show") importFrom("stats", "gaussian", "setNames") -importFrom("utils", "download.file", "packageVersion", "untar") +importFrom("utils", "download.file", "object.size", "packageVersion", "untar") # Disable native libraries till we figure out how to package it # See SPARKR-7839 diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index af81d0586e0a6..1c806869e9fbe 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -212,7 +212,7 @@ test_that("createDataFrame uses files for large objects", { # To simulate a large file scenario, we set spark.r.maxAllocationLimit to a smaller value conf <- callJMethod(sparkSession, "conf") callJMethod(conf, "set", "spark.r.maxAllocationLimit", "100") - df <- createDataFrame(iris) + df <- suppressWarnings(createDataFrame(iris)) # Resetting the conf back to default value callJMethod(conf, "set", "spark.r.maxAllocationLimit", toString(.Machine$integer.max / 10)) From 57e97fcbd6fe62af4acd60896feeacfa21efc222 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 21 Oct 2016 12:27:53 +0800 Subject: [PATCH 149/177] [SPARK-18029][SQL] PruneFileSourcePartitions should not change the output of LogicalRelation ## What changes were proposed in this pull request? In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it. ## How was this patch tested? the new `PruneFileSourcePartitionsSuite` Author: Wenchen Fan Closes #15569 from cloud-fan/partition-bug. --- .../sql/catalyst/catalog/interface.scala | 4 +- .../PruneFileSourcePartitions.scala | 4 +- .../spark/sql/hive/HiveDataFrameSuite.scala | 7 +- .../sql/hive/HiveMetadataCacheSuite.scala | 3 +- .../PruneFileSourcePartitionsSuite.scala | 74 +++++++++++++++++++ 5 files changed, 85 insertions(+), 7 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 1a57a7707caa1..a97ed701c4207 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -102,8 +102,8 @@ case class CatalogTablePartition( * Given the partition schema, returns a row with that schema holding the partition values. */ def toRow(partitionSchema: StructType): InternalRow = { - InternalRow.fromSeq(partitionSchema.map { case StructField(name, dataType, _, _) => - Cast(Literal(spec(name)), dataType).eval() + InternalRow.fromSeq(partitionSchema.map { field => + Cast(Literal(spec(field.name)), field.dataType).eval() }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 29121a47d92d1..8689017c3ed75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -59,7 +59,9 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { val prunedFileCatalog = tableFileCatalog.filterPartitions(partitionKeyFilters.toSeq) val prunedFsRelation = fsRelation.copy(location = prunedFileCatalog)(sparkSession) - val prunedLogicalRelation = logicalRelation.copy(relation = prunedFsRelation) + val prunedLogicalRelation = logicalRelation.copy( + relation = prunedFsRelation, + expectedOutputAttributes = Some(logicalRelation.output)) // Keep partition-pruning predicates so that they are visible in physical planning val filterExpression = filters.reduceLeft(And) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala index f65e74de87a57..15523437a3404 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.QueryTest class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("table name with schema") { @@ -78,7 +79,7 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt } test("lazy partition pruning reads only necessary partition data") { - withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> "true") { + withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "true") { withTable("test") { withTempDir { dir => setupPartitionedTable("test", dir) @@ -114,7 +115,7 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt } test("all partitions read and cached when filesource partition pruning is off") { - withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> "false") { + withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "false") { withTable("test") { withTempDir { dir => setupPartitionedTable("test", dir) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala index 2ca1cd4c07fdb..d290fe9962db2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils /** @@ -62,7 +63,7 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi def testCaching(pruningEnabled: Boolean): Unit = { test(s"partitioned table is cached when partition pruning is $pruningEnabled") { - withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> pruningEnabled.toString) { + withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> pruningEnabled.toString) { withTable("test") { withTempDir { dir => spark.range(5).selectExpr("id", "id as f1", "id as f2").write diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala new file mode 100644 index 0000000000000..346ea0ca4367e --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions, TableFileCatalog} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType + +class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("PruneFileSourcePartitions", Once, PruneFileSourcePartitions) :: Nil + } + + test("PruneFileSourcePartitions should not change the output of LogicalRelation") { + withTable("test") { + withTempDir { dir => + sql( + s""" + |CREATE EXTERNAL TABLE test(i int) + |PARTITIONED BY (p int) + |STORED AS parquet + |LOCATION '${dir.getAbsolutePath}'""".stripMargin) + + val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") + val tableFileCatalog = new TableFileCatalog( + spark, + tableMeta.database, + tableMeta.identifier.table, + Some(tableMeta.partitionSchema), + 0) + + val dataSchema = StructType(tableMeta.schema.filterNot { f => + tableMeta.partitionColumnNames.contains(f.name) + }) + val relation = HadoopFsRelation( + location = tableFileCatalog, + partitionSchema = tableMeta.partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = new ParquetFileFormat(), + options = Map.empty)(sparkSession = spark) + + val logicalRelation = LogicalRelation(relation, catalogTable = Some(tableMeta)) + val query = Project(Seq('i, 'p), Filter('p === 1, logicalRelation)).analyze + + val optimized = Optimize.execute(query) + assert(optimized.missingInput.isEmpty) + } + } + } +} From 595893d33a26c838c8c5c0c599fbee7fa61cbdff Mon Sep 17 00:00:00 2001 From: Jagadeesan Date: Fri, 21 Oct 2016 09:48:24 +0100 Subject: [PATCH 150/177] [SPARK-17960][PYSPARK][UPGRADE TO PY4J 0.10.4] ## What changes were proposed in this pull request? 1) Upgrade the Py4J version on the Java side 2) Update the py4j src zip file we bundle with Spark ## How was this patch tested? Existing doctests & unit tests pass Author: Jagadeesan Closes #15514 from jagadeesanas2/SPARK-17960. --- LICENSE | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/docs/Makefile | 2 +- python/lib/py4j-0.10.3-src.zip | Bin 91275 -> 0 bytes python/lib/py4j-0.10.4-src.zip | Bin 0 -> 74096 bytes sbin/spark-config.sh | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 2 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 16 files changed, 14 insertions(+), 14 deletions(-) delete mode 100644 python/lib/py4j-0.10.3-src.zip create mode 100644 python/lib/py4j-0.10.4-src.zip diff --git a/LICENSE b/LICENSE index d68609cc28733..7950dd6ceb6db 100644 --- a/LICENSE +++ b/LICENSE @@ -263,7 +263,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.3 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.4 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) diff --git a/bin/pyspark b/bin/pyspark index 7590309b442ed..d6b3ab0a44321 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -57,7 +57,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.3-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 1217a4f2f97a2..f211c0873ad2f 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.3-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.4-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 205bbc588be09..eac99ab82a2e4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -331,7 +331,7 @@ net.sf.py4j py4j - 0.10.3 + 0.10.4 org.apache.spark diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 701097ace8974..c4e55b5e89027 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -32,7 +32,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.3-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.4-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 525dcef5b7d99..99279a4ca8be9 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -140,7 +140,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 562fe6461e753..f094b4a7e167a 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -147,7 +147,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 747521aa2a566..7f0ef98680a15 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -147,7 +147,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index afd4502c59d33..4a27bf3deecb6 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -155,7 +155,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 687b855b649d8..151670a8e23e4 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -156,7 +156,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/python/docs/Makefile b/python/docs/Makefile index de86e97d862f0..5e4cfb8ab6fe3 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -7,7 +7,7 @@ SPHINXBUILD ?= sphinx-build PAPER ?= BUILDDIR ?= _build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.3-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.4-src.zip) # User-friendly check for sphinx-build ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) diff --git a/python/lib/py4j-0.10.3-src.zip b/python/lib/py4j-0.10.3-src.zip deleted file mode 100644 index bc54f33af1515c0676bd831bc5a02f112b28e0a3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 91275 zcmdRVV~{4@)@9kYZChQoZ9ZkA%Qm{pwr$(C?JnDP&3o@R_lt>{ckayJn;9n~|E!3O z$hFSi`=p`_C>RXTKQ2Le45@#<{LcqC5E77s2b&e6nkF2OZfCnj%U|me7!64AF9;eC z)L(a=%$MZ<3PJi8#K6GP&eGYyfWg7zUub#L%zvU))S*h*Zk3@VWvAt36%M6o$7j@j z#VF9yQ`1XF#b}I9(vLFIC`roFOwUl$OGSm<5RX3q>P^eUhSFX=baiiU6)6-;=w&q# zr4nKPG6fQf3;?<+_m>s+f31I*_Fvw}{BPbs|7Dtift-+tf{NxpJ(~W@n}4i-dGz-% zg#Q)x-_iKrv40;8-2Vg1VDN7OV+=qyHU}61ZscUuZxGdgDf)xNVu4a3;R4@65JB7$ zG9!WGi3kVg%Em@x3tog22O|?Rq-W+-q-ZL}CX{PvrfVjqYQ(3dj4PJf0{{RETdVBE z)Wk#+o6GE+oa_V~6+@GqOr^w}448RCTT6>GGl1b~@%e?tLB6%o!CtwwA;7NC+R((x z>d3@s*W_$(_#fefgz`^x0!sKRw0r+EynpZOe`nsv(Ae6+-qHDQGQXrL6SKjA)cvl3 zG-hbpiyV%(Zlr{HLMt{)LKO?{2?Y{D8?`biT_~*FQQ?=fE5ahC)XcdBRw^IL_Hpz% zH9I3kO*3rIAXVbV=&Q@;^bgU%+oTiQOVr?`)tYT zkR3hp8f-1y&Eo|o#Es1R_Ui7?t9~o~;xN>^r&|}p%kwWs+9$^Y?lP_BbZ2S1=IPsMoRL+B$G7{n+hb$>T2SBy6kZ zY$(ISP1Hyj%=AuVykaB_3P#p?=_+KK}~t`WRvAbcihMaAuvbs(Z%uFN}vg(`YqmG4?GS#(c?Ds?-<`5+2xk zJ`xYY*(oz)L_m~{T$X0d^6%Ev&`pGV=Hll~m&>{vh(Mv(1;+wLG1$%7GpxqU=A$Hb zzig0u!77YK1(HM!*`&|{ym{V8e#Miv@1}5^JIn*aB6!j*VSDHilmO%^6o)!WsE;cj zVLLcEo-i!ojGg@=o7i>?aAMZ^pDQCtsTNC^sadr=C6SCkvD92nDmzJ!wx~~Z6=Y;l-XDnm( zHTKRX3)0V4>zz{zukhjPHLNankDu<$6$x52eC!o*!#3w~OBQRC6r)VzM|y~e~xF&=SUbpNw>Im_wl^p;(&%Q=SQv{qPp$O%DOB zRN>)Cqt}?YI+BLCXhYfMK1u+n$&xak0WP^=N0SXMR@mS;EJarAjUDmx*yyQX`u(pV z`jBzelkRj|d1ISUi+57!%%ho@XK4Rj3cs(iEy>w~qze??cpG@@V9Ms$Ao{dv$|n1i zo?L?y3|#c_n|0<<6U<+S&3+czTZkFsVphn%ezP{wWR9fsvpYq@xYpL4^~hOzOpzjs zv9rk5+y99PIwGI=pxA*!qhYiE1*X!Gz06)hD(TM>qSU zZSAfyMx1R~!I$KimeAWq?tjLZJUyf zUDmFMLy423I&iz0Q@F=Yo6|l2{cZB?@@ekKiOZ_nLMIQ&u+!v+A1ZiEbacODzDmP) zkQ;?3=*^=05+}UH*!Z5*u?ftR48>897R*J?s#E7ebf!YK`V5K(f99&}m$}M=W+6uN z{lVpR#9GJxy;g?<6IK)8k-EUfarrXGby418N+VhggO&DbmJmF;ctJe!+g;WA(fH zSXpd2s{Z2Y;X(w=*+@yPqa~&v9hwuxj4l9L|&iicCAs>m`l!wyc+8p1j;}FNd z^gA2HG`MR(V!6a(36+P)GgdU)s_f3P4ri!wjD|1T%I3g9?6%4+ECbP2&4k-!k4Tn< z!>C*!SCXk>N`f_3^lj|?nTi!x74DO;2f%PN3sof1^gQD#2j_Eh_0`KH(J33QIul&t zSL!ab&8E>NdnAzEWMZtyWBFw;xoA!i1*A)kNoPIbR!tyZ=QVOk{G-W`Cm{<>A!4$F z^n5FP1g3Y26GiEZX3xh4B|ZItdusFJBPJ#9;u0YI->XOUQ49H2fXk1fwm;2UkDBtO zLZAvTSjx+&x5?C*7~G{W5o&yU%_Y5aV)6W&qi+S3e<;~U5iqu$`KBaXjVW>=OUIRq zZS|AvTcon&2G>5O#=`D5UJfoUJbj%O1l;{xd?xfdCbXL>ukbZeUhpH_ho=G)8Y(=q-@w5uZsh+c2*medGx_iE$!P)4f+>R8k zj^EDBA=5-u4%IEEQ?v1un9uz>ijS{!HzmD@!;o5tJ@GUY&2}GV#X)g?FW~{kSPXoR zI|qVjlF|?C3;b!*&VMFRYs-`h-w5FZ+)adkQa7mqzyYMWRr1q4b)04q{QAlnBTrS* zKeEMR-FPU88$y|+oahx{8_5>$_^CuLgd(Q`2vhg9ieaok*+?17kG<8duJFVR`p6Y{({%c!hLaxxf!Fe? zt*%2+OqQFr;FE?tz)os*5VeT8)9l_kv*6`98gGtR++|`EwMP4<+Nl$+*YpWNzQ}fB zD5%KEd0f)j`GaU-TPN#!-YboN@L6pkITxt0{|BCCn=fCE=<7d7=(-W88dWJAh)RN3!9g0vZhGZm}xyd)#S&9 zLHGBUmtUp+h8b4cmE^EnsW20u=>@$soJ@{9W5`F(R^xe)+8Xb&fUv?547J*g>omWd zpcz+a5Er@c!c7lg6whsuRXT1&iNO)UG@Qpa=cf40{QE7skbREqhfysAl3H|^D_i9x zo#O_)HcnIv>NEE~7&I~ZFVzVKeONS#p-q>&*_Kg(A=A7x zTNXko2RDeCs`*COeoUnZ2?+167yhKevLaVWHOmC-f|n2Oltk76JfrI70i+qk^_?@=!d#ZXlwpo$jL-wzMn zLu5rGM%R4>yFDXh_P5-!P#y`1F-Tf~hJerzoV2>nweWJA@F-HDa}bHAvMM1>z>f`- zs*csyPG(R(=951|EvTYvt%xc!%upQu;4%&sV<@#r?9PM(iL@e#>#G$fPfEhJc(N4Q zz4^w|7#Do?KX-!_SKM*LO=$XApICO62OTyzs#%bih>~$al~5zR_I zvrn^KsbB7`?t4JSBc0uJ3|}cE!yUn;cy@Q)h}5GrGpkF5zz>ua8>8(OK+j>~cZ z-rUu|LHGGU$CCGj=)mSS#Go_kTq5CzhK@{oD3$eXhj?c{PCHESHd5`J1M$c=}L@f>Y$K8}> zVej)3V{pJ8^0uE+$AuB$9*ydThyqmf{U-_VQtI`MS(u;;WHzJ-;9yr@-9yXBOBWfz z5IxuSh*E9IXLc`VW!cs51d-&hJcV^lr!tUy;wp)R?~sDhARc(Nu0Ou=Q`n3l7KX7} z+mfZm`j+y+1`I20QCENTWn-z9kop_6kA=Y98bAz>vX!@kX5l1wIDATZ9|@xEWtfp0-4Sjop|ZLN&) z=SA8CC4S@Avvz3>vBKoOsIal5GdWu^2?Unv>58T{E8kRrKWlVtKwb*`(~l|MW{jL4jouc83BuqnQ)E z!gldopyX^&YGaujh*9d;f`fiD(hU(S*i+|+3Y{ znkLT0&9{WC6r}sN^d^%w9r@c;y1bAiE2r93;gOX57HW)IinFqb9b5^FRT*Jeqe=wr z5xpwhuRqBv#O=se9SUOoW@r1rD_#l^nGb{4r=j5Jjv)7Q>R6y}`L|r2;_^h$UN^Cu z&$r>6Q10S-*NN{XJ(SC?#wya7i=8MMzdA~pY5c<P(nAUV3n`)=o= zrxY@|nQ);&TlbopI^b=rPQ2k}_TyDakb`6{j4QpFYSzcV1-#ycMzK0s7QDwoiPW?` zd#?bDn!JF7UXiJb1=eY8t4diNHSI*6;)l8A>-Q+kE6(ig)e7y>t{=bT)X=ww0pSV* zZ+$kr94_>!M@o5pJLh#Hr#0Q*4lB@nn-cGvC60)Zm=sVYCiP!_n5`wvr92bn0eFbR zuJZ``xFe_)1_za%22g)+5lEiEx~vx^Fd?3~W=x+jJ=PjE0|mPlP4pR2YU7~IG$Yx@ zHIFTSm|Kf_x_H~!j%=|k*2IR@g;;Xi>1g*HTl!;DTzs&tt(R&i=5qUv}uC68;h~STp}am(OH60w2Ve{Zmk7B zDxLjw(0(&FXK@$hnGJ}@`7G)-!Lxhl>_bz@+~rk=r;}x84Sn!}fMB0L5m8}J=koEG zIVUrJp>(|I%R%^^6Fpf@GW#oggMOdHUi6;%HWaMM0B|mH~_Hvv{$fBd%U8dx8nhfQ-bwZVNH|7J~6mq|XGmnivO72e~@RSLMK8n&Mi|$nk+(=)mug=gC zQZA0P14EuQ?Bc8%fC^ts>wXdn{9x;}Rwdd;yF?aQlbG|!_2d5VkHPkDaU7F#L$BX} z&j32spssTpo^>~d8(sPw5+6X#wb#k?+`~~AVvD(kfZnjZg59w|)(2jWzMgKTZuU?L zCFVV~{|OD~{h#KE6+irt4~Ut;i91BBJKCS>r@DU#-^7U~yP!=M$OHG!e)J`f4a(Hv3G3TVYIrN>|q1?cr%7T?&xC zc#!M3&O9!Uy0)mO8FK!*3k*RM_aj(Og`IUX-^7wy&A49+aBEBtCTx4bSNHFXPr<8O(@nw7wlaKUpQ=yZnTiHlp}?z5I?Y zcBamYdLSDfaP4DK9$-o?e(o?vX-~CQoX|5JuJD@Pjnf7{LpmW}?@$+0Q&nHG`VMou z6i3zAm(}jtXR3L7O~%E6q*h3jo2jX(sot-ccro+!Xr4)90?+(~XdPegLFR0u&&H;Q zuv<*R*!Sj_*7bFDZ5$I!zl%yy$}tcf?9o`Ku%Gk-d|u8BHR}Uj5UqNbW+Fba+~Ta* zYX^2so+Mn&dC4HCKrfqwgX5w(T!nB@<2U_Vt!fdQJJvqGj*CC^W#`^5hKU{btpKP! zGTrdPau#;-6Xpl$MFK{2eo=*oKY=cRAJNfPD5m40K1N?=T!;ZB-UFs-EvaL1I zPb1}CwMtDd?9&YcY$0lgRFYxSKbPD`q>~gBrf1Xw+Obpn8!k0)Yk;5jE-SADF)#32 zigR3%72HT53QAr*#;e?I@iC1R%>8(m2-jykGTpr#KM9FU00XbD9}iDjvd|MowH_1^ zr5o$t{U0!?&nbIz4$c_((ytDlFwz7YFn&}&!WwY}#wDaUrV$TCEb})%g+@O+Q3S^9 zBXOm%2wyc(^V1omi)ozPIpC!AmGm6uah4Eu^S1t=*bnAc6PTR7Q+;|67jQe{jUUbY zjxivm((dELBCa_uxpxzoo=TK}tg6ERc1TL;t4nr4rpyZ+I62Q@JIQz<$c<(f+ zf^uweHUpR`Uc|9fcT@jOSj9}1f~3_D8zII@-Sbc2JcJ$)>Za3nuHZPbq*_#^sKpL9Li z)|&OX3wG~AUHXC%*Mm|%sf(w?7zOvj3}p5onA7JMb|d2bKhr2NLheZ^b~5r`uPAtp zl5FRjo35KEtn~L!Vq_g+6DPQH<)Tja!!L@D`1E-f!Kv_CAKi*!Uu z<HXyXP78qfbLJmHyDD*2l`asYoAa7;Ms4Ao z9n+orOWXE%uV?SheFOia!Rq;hS3rse1mvdi?;5Oxe=}J8fAk;!qnoV%7{dN@_G)Zz z>tOhIqs1omExQFKwC))dJQvPWRgy*GU>Z+m;vEp^0Kq*FxwwSP%5CKz4O#Ra%=-SA z`>WIT3odZK^%tkzy6^Y*YqRLyjC?Q>aPBRKnv>Y%pAe8%7u-23D}J;;>@ie_VF zz>{{kN8nI4>%8Oz1CM}}dQ*lk3jJMFK~b@FMP1OqaMh8^IUKYA@Ptd(21b+-T@&%N zZDCx~v?|n*#yjuIG{&fkVpuA6s+F^}fZxetrc6(fL>6hfCnzcld}reCsb(dyYpgEB zJwJ$`u9Pf!Bqzlf46X6=nMDJfus{Vk&uLffh;?gr9vxAEd~`$WJzP-9>AD_WUg!j3 zw|qrF9Uz3a>wsXRNOK@rXeSAcXB>Mv16v$+K~0utcJ#xvbV5I=3a6n*|Jo5k7+3R5 zX|NIoG(n$pcSUCwnbJ^H$9ZbJ%NcYvp~q`?KTZtBGDWRZT}>8*L6>#8898@YMh+9H zgtJ@Y7I-}-IhI;Ho)=l_pc0(OXCQTBBTP;KR9fA5q zC^_k#I+3+^YQyJ}xL_piXxMLZ#h@mBL%gQ$8k`BoYxHuJxzhymrEM63;PB(DNQX2g z=D<(63w(v*g8YLH9`UUSrs%^pZG5o`)kV1{`d7V1Uv57^I<*2}X>>PY5l8-7!$aT? z*M(^lMvp${@mHf)1yAQebDFy{sBe|rF$1fGK3c)ukww?-LZSMS^mj>r^$9mt`ae-k zHU`r_iqrCp*2_dzT%v|am|gmAoa^uww;fmVWbG_`=bj`@l`-^cEpV(HbY5E^J{dxD zR4_BcZCX?T+Duh_h;{;3!aa?@EqCNg+FJteBfJ?IB82%*d}9@;$P59N;Y9(FyV(~l zh_Z9RR`3z$Y^gnIjf8tru-JxfqW>oKfY`#X>mq)dP%qRn5aF*Z_p{F2!No$3| zBF>w-(pBn3<?2Q+6N*_GNeInK7QsxT-2~ zdMDjdC$HBNvZo$idN@j?}}F)@&nb7mucZ@#Ky0LT*%} zsk?WNba%oQbTPPq))=*BkY;r0{>;eKMl-a`$2!5JMJ}i$4|9I;X_sx7;`dbwm!CJm zhpSe)sR8NsG_WXt5|(g$-oPLk%fztoy{#d#%YJ#;&{ew04|uUwUa1<-af%9j^qz_D z^uLDx)(z?H-m#t;I}bRp5`rmKuA&m8kuj_V#yBGoksIH{K{JJOt6vB(lM!L`<_i2X z1ub0x?T$3aE#YJ0afn|Qs%hO8U$)k{Fe`p zAQRuiqAA)3{=n88D?UZ%H$D`F?3w?mwPQ@e6VlgEL=SW;!#e1wW#y6=FT^zxO9?wf zq<2~7d)3i>y&ceu@C^No?a=)=Y*-aipqYuPflx=BFdu44qtetQ^FF6G3?lOQrNdT% z5eS`-bb^G}IC0xCC_*Vo zI(!RDRGZvx2sN+H?&2!e3bce&;^%2}6FghE%z5>%4JZ_W);)`A5yGjDi|QLk#rKmb z(V0$)sBcAZ=r3at391#n@yvO+5gU#t4uR%y=(stXtK_u(;l^cAN66stu=_qIkFU5) z-q^8R1A~}@?-R-B_wz`^8RLiZlH~CRz7iBTxzt|M+327(i>5p^LwUk+#xEM*xsvO1 zAFqq~FT~M(=myArkHJ#`l`SU@OP+pNKwNexpTnVL`u9K2E~&f~1TXx>h4V=NjtlYr zh7124S^P)SA=ba>(8|!&(7@Q<#>Ujx+0x$5>2HVDw)PubC|_3weNH`+IxCXx@4HAg z7^7xe9S`wrzqY_JwuFmpdDLlT_?6`MXrFriOx}nU>f=$$Z_p`%irG4vab;rjXA&3j z8Z-L3_ff2|$=4vPZp>9+eh((gw`^Qynl7l3XfhnI$e8kmUN_J|?mcK9fHT#ue54Ot zlOZRNOe=FpwI&MJlNe*38%5JMHx&GMw_%GR*P+_Iq1Q3uR9JOat0mi4VK&`o&j`mh zAQ$UdIK?{-#GHk>)4xyb@?JA$e13Z7$pY;TRJsFwqMeColN>af^4-@UIjk$Bp1iZ^ z%eQEzQE&T2g29-qrC+OtGmt2jI2bbOLE;q|W13tEgG?PBy+_xS63TwU^paTxgYd+) zKS92lHxjznLmMYAI`gm!k>X7SLq^{pjX|-Z9Mh=#{ZJ<`h9RT>LYfzPV-L(zyod2} ztb3O~+6PgGR|DI3pINiSRHqxi4Qgwo?5oyR&Y;h^ITPSLe%&5F~0H}yg~ z*(~vAX3&UEus?|^KLn^uWyv>EpUxEf$ut`sJ2m2GS@(~DHGX>|%<~+5#5>*v^@&R@&hQ~t8yOdyg654)S(8QDfedIe5rjCq&rfz! z9&DF*f+9g}8f32L?76b@-ljEu5)#N65EYOI58RWA$rFu-6=Gy#wI7sgr0%4Zj?RU- zRH>R+BqmLHUhRhd^}KVB!UYmjM=b2QIBQMpL?X}LIHVU>qTuOpnCDQ;+z!B&V>1!SFX~zz7Ta7(~-=YRrfW?M1e3)Y7+JU~UnlC-~kMkpXp$ z8?Gds)XCT{iOTb2+ABrUKzNVeYy=s06*o%#APCP&x$hp=HK0!(-!*oGccb8pAO{Hi zA*~$)Dd$WNC(LUfU5zaX<_3NQp1i4)gFc3dzKM_a85n(TH(RsBzXMaxm znvei1l8o0@m;$-6fe4jI6a=7yVysEW*KM_kL-S{G|H;g?z{4nlp+8lNExZw0W7l9kwA_uIWKSXwO!1$ITj#U>R5; zqGffrZv%Nbn4mY!vk}4`%v`i)$Ce7?78X7%`P?5GTsO9&ge4nBHs}WpKN@%vnQ?%r z4{roGolE5rvlaxc9=cOPWyb^7Tmp9(&Tv`5u=gEw3`V9(h-SQgI*`qemPQsYC`AemV{x?nyNLobR1mch^3HI>+3h%j7sgWCpA&h%JSRW z>pcNL0c(6Nh9(R`H+E|F~B7PPkTuAYY4!D?^r+lb%j|bxw*t~kJzzjb!@K!tplRHk00&kB3YmjKay*O(D3}V)ie$!#v#gVhoFMq-ZTJlG&Z8(O;i&%x$FcV zfIZR&=Hhbki`}1k)sTqW?Uo|B$To;L{)-SMW=%j216fF+Xd1DO&lW9#U>nI4O>%@& zKs-B7_+mHkGgSFhm1e}yWth8t(h@N_Py0dH|ab##&KdEMuqyEWdJ^ zdsRq0A!<4sO*&0a+WR3n_NL1%#S}k2C<0$=sA^>E2=-WhQ?AvsR#O>OL1;}iHVS&O z5@ZfI9X<+D7z?5r!4L_k%r@5iy|Or6>-g}LhYQJJfnR%Xxy`NUx*yS*FpO4cFQ<1P zaBZsQaLU6PmT5F1;v#LTEorE^;BA24fek~d*efh;BlM+!n8hl88~g*wdn?ODIKjAK zLbo~EJVYR)2-K@J-Hvp2I*zfOl;FMLc{ff$9{fxJ9wI)z9S534l^d0NXkD4CGyRFz zmY>gDzeumnr0~4Jf8@3vt`=UAGBwvhsHHFtKH7cnj64%}{5DL4){|iEeATa(GnC0^ zby0YJduCPRLTJjxx{gwh??9;J^b9Rf1$HauOcISYnAcr&KRwjwXrC9e`nWe z{6vUsm2&cttFDiB*Hl#1WxdHnVLr#6VZ8ujB8qQ&s z;AgOlX*MB_92^Ac45+YFRlkVHz1_7>L4Rn2o*v3#VGT}R`3+~43}Frb;RLxE?{gLu zOv;cS24Wq(?njcO2^Au{=t$7?zsMC<5Ur&Z!@jm?fnf(YAazL z+^|AJfF;GBiQQqxmoJH@63T%&j3VZ^HPH9#X__$LQ z(Pe9}_N+DOCwV1}88c*?36F9yE+|_WTAz7KHc1%tA04*U_A`V&e6---{IRqRoljr- z1)JTO&8WhS@3(DY+w4bB=I<6e)vY<`QwFhTZ#~Tj1sHi}t0UU>(>)}Ms(gU8+VXm6 z_6oo48IEVSY?x3WXPzCvARJa^LFvaeDC^k2ojD$Gy^+GsFZ~1K0yP2IqXkU3De+; zn|r@fRn%X}jSdz*nIu$Pt{Fb>4v#cJqpmhO=e;uwFiHkeRBL8A>Kgy_)=-8p4B8bM z{Bbgvl88?IY1POd>&j#6XGe4+h|=4TG<&~6Axub#6r=N<9#GDz8t>rhj2!jViuIt` zkGW56?{_4gHqe=}`DG#By49O_wdrP}&G-YciXge;diBhndUV%yDV}QZKtLek|E?}2{hPY<|D$UAk1kLD`#kkO9W^&} zHgz-f_`A~7=iO^#*zx50jn?7B*I+bJ{35flWvR-sVl=W~FkV+mKK=X+lo%xuDFF=B zKYnTR-+Y}nj^7xQ^sQz!2sbqfm6gdJOcQamI@2@!GYS)woz^fTX`gB(b=C3YwlM@i; zwlNSnrVTC~)e&?u<3r_v2$d=1la6syGMN(|tg?3%#pyL0B>N~{x=zYd$4p&J>Edcx z7y>mBe6P$k)is8TXXZ-|O-rAR>7_8VCkj>4Y~cM1J>_E?{U286?46`gQ*Tyk8VUJx zm)(`>LyoDPyuRG?=+4BS2l}3~+r5#*IAv!k4}jFn9)AwC+YS;x9n1#4rE^AKX~Q!O ze{kj8o|N8_>6Dcrrna$+PTTXDural5v{@Y>fne|v7?_Wqst9(pPhcPew98%)UfPt` z&X1XVS1;$dnedt=S`}RdeZ401O5u3UkFCJcKipY99^XSP1?|;m#6EoV>mB93vtbeQ=tSG(LiJS*K=pwC$kU-V5Vox7D>2-rFfyvtTHo;Xm1-IVJ@gG6T zW3esFLYc#7K+YR4y_r-U)dqj=s*Pkvb@up=k4m7)?QN7D6HZ-bZiilkCo}-o7wikn zjOS|4HqlgxirQpouNsuH#MT34F8mt1&34B}H|Ofc``OZpLCkbR#Qyh>Ky3rw*JHm; zFOIjjo27_}i<`BE`?(=--YdS(55FRX9X|!Hx66r!dl~n~SAmbm8b5YKzs`@N3pOtg z-=}MVb@FZ|!I)@qM#hGT{e-zS+R~f78;ggjwFvV9CmRXrH1%i%gU*+;o0E|`xTY978xa58wYI`w;efC+OJO&0AY_U~+zK%} z|6~T$0{$+5aB8qCS=zcA5Kl1ZgO=pQ_m@G^N1O5ui&K+({}~plh-CZSD|ET*A-@S- z*F9Jd)sUjMvgDc6)fF_H(T#VjXxV7vKZedhpaX>9$1lf5xJHI*SfHc{wEO{)u4WHS z5s7Z7x1Bg@i#|F!U;_0#UrEV18mfmUAi&>cQU_uM>Xm^y2P})EM6AY_-*X7}vge!z zD)WuWMV?|i!$i&tp;>9*IjetLoodoxYEMYptJKzBt2QR8Ss2aCK7|xXFOF!q@P0Tm3YLv2@1VLXitk^(Z`Pec zHi!Dr3R)ZZ8sDpj4s9clT!y3@v{2DL&!}UvTt~`T&q5RtHyaWu`!uk7dkAUG@><>Yq4$e>rNT}`*$-?lrLfwS?+(z&b{JOAwE97(*kGR={|6r*xe7aGE;7AGn z<>3O(X3jYsZ*?&l+|HNy+Bh6?&Lu18V39<{EQ|EluXp~FF0(;GQ-)qMh-e#MHV;b` z$9+iFNCgpGG{@b#^paIb!N6SO`?WQb+ zqB#cHU+eFPC9i0ysbbsW!z{mIgApw_#SMRfs+6|n7-#0& z2Q>$8O*a@gRi&g;<)|=Ho|aCHt9UoDDUbOXnhOC8`~uyt;Lr-+OBI}dS%b8Xy7c86 zD3SRVg5*ybf(pv6PPcNjd0{?}xSjM~tQFd;caC}L!c!PfeBPW{!YFt`09dJtdpSwx zS{kSMbOxlmS@=U0u9@hQ$T;M_XPI5m($K?zP>Iuop+&b|$Lwly^ZelE=JiR=BFK)6 z@8M)@*hftiYlzS=@GylVNrRZ6UcCGqsfYCA{S^J9t*kfNy8|Hsq}OzJcatdvjYsB2 zmeMgG6$h>a6Vv^HuX#*-{-ZX#1mu);Fb~opHB-G*qR-Ek3xTU|9Xr!+O~E6p>h5WH zPlw?gr7bxeX7sIJ3EW#-??yotN=R&T{XYDuNoKx|_YSn?57W3XE^BO9ED1m_7lxYm zye!I^TC;T~0#7q*vgXO%n$L=JDxfTEO?ByYiBvO?gAOc=?n~aUe7xw3ncjL+_lQ@G z(Z4Ni!Ne2{I~~8*GO;-4)^Aa2@Novo&H23?l&zdz4=P>NXKfK&k9rv$X&K0^R^+M@ zfkfpg&a9wq>^!g?DFdoRY+-+_y&5=xaAl}-G^4+&t*By(@1s0I>lJ}#c8f!|ApJRZkYsOd_*%sZNHE#ckLhpHw}hY%7bD4z&&|W!`dQO zNL+}p*_{v6+Ttqb!De?zys5hzIJ$@)AXocTzoTU&+9$y@;nvB*Iphs~yaT5w!lw12 zs)@Nt`|jOOtX!75vKw}2oM)uPq1AQhsY7drKADz%FnQX)q-6FFZd zvzN)VFjwad?5s&G8jodjvW+0Os32ySes@1`x4Z48l$%!6at@1! zzP4ChYv;d}RRQRoJ1*^^Yh`~_N|{#wCa0de!NKwp0t;rs2+?FkPf`>iZqEvWIJ@l& z61Vz*M9@a&6%VKu3Nxq7!^z6&tO7t*25gQd*%uV)=`CC+(Zi9dM+C4Csk)YYS_1V&QMyveFg<9{11KirYY%~4;XPy0hw>p{xY9Hv`Px8DNz237 zk#gRHrynXj>aZv@e)X9t$lZl5);f;`So6-927D4ADj^SfAtn9N491G^hcT5M2P+B{ zhW*K*1pngphrrh_)yK(|Jf0P ziT<|mer1$qFK91A6xU!jr-j|TY`Cd@N>bFHQA?`iJiBX+kRf`ym52)A2B*;D$TB<) zi;g(#rl!(dvAE(?-~!woNX2>7v@R|KLi^?PT^^2BMfpFC1hqvymiL zm7ydaHiu9XQ9?ZeGD1P;#b}gfoZM=oX~^NCXHn5l9>!w~^90$7KsT;2kupAA1?6~d zoR@G`+poE6{JEbBt-r&;tHglVAjkA&9%1GMUCu%bWpG?LL9vYW=$Q)2U$ObbjX~7|bWE?Ip8Kwn4kwtwzZ_rYJB1Ju5@6x|9;koG3#X9PhR9 zd_S1%hwhNht0AlvT9kQC zPm>cg@nMT*u@j)eZ5>a<%8_8m`+MB4`)9Rru(SiZ%Y=Ap^@LIh8T2wFoShI1J#Kr`NXMJ$2J*E z(<`<^UEHa}v^3##V%oG6c!O6=n%CmPjUyl7%`!;%(I_5e&>h960vd(yfL#aoTr!*6 z7kq?~9+KBL_lG^LkHBC9KEkYag*dpn5(ray0%zQpGUdSZ@SqxH(#fn6WiV8FJ z33Fo_1x`yi5tlYc)h?N=6WvjCjUTYM7+-v%Bd;RblZBgDWM2-B7lZ-Yig!ynJT&>fHxV5#b;^%LBh19O~mN4T8QnO718^to1hvlD684jm7K(I@Lh zin;5ezTEq498HEBkWq*Hfx!Z$D-iGOAx`HBe3;V zXWkVlR^TN7sV38ol4;U#vVSrJPn2c<;wARuCdn#(Edy*g&{8UaFKGju4fXVXm0OX3 z*&KQrgfN{8sd-pB6U-5;m^LuUE{ZXC$@%J@I`!`4`!J%;)e~1AL3e=YC9fo;Ld+6% z6*iwo{6#uy&(;q^sHPPYam0zeJ{6_6VEypuK>33|YpU<>vTV#Z$>dSIlF^qNkoads z#)>YK>2l!Li?)I=J=xc}YeJ$d#CSS;Zb{lp)d5;2f67b)?1S}+8^qCGmyD1M#7BdS zz3dEIpiWnIst~QX`4E*V#2S23txY zi=w!F5-K36vNp<8g{I8;ChcQ84v8Em;EW3$WB7!V*BvHgZ&R{93x|E8c{y`9xp<))BtUm2P|XaLz+FSWICD z!>p!yJyUE-jV48>7Se4F)GHx=xIVd%LcWT2TR$FFCcCaVEAVf&GUHJ<=TKQwn@vLi zhX8{)HyzU=q`f@9KB{3!`8deno(ga3`8=XH!@ch*qZQ#%9ePq2(sMDHcOdb8G?7#$ z**tXEM9G}V?mX;S1w8P$`^@JC8P=`f11FC6kw-DRza4q?=i88h&QUSR?RCqNZ^aA3 zs+o88@$q-Z#gq4nj8d)dP2vpcvTY=drW}(Tkxf3Z#%`LKos+T&FWpRXE|bAB`%iQpk*%Q;DhsgZBt&tvRBaTg<&ZaX8pnE>SIpSG)TYPFe}d zOki3_P?}SZnq~&0Ukodp{56tKVuUeQP&9*3{Uuh-?r8RURGcDttY&mV=V33>s(2+jm9T6g0*?Doj#TcHQxThWOxiC)JtSBfygFfh1gsF>gNY{^9E)RudI|GM3{F9?CNznt>!^;E!a}`rkiQwE-Uw~?t>Wh3-ezcZCACqqPhr;mah`Og; zan{xCnUU>B1;c|`M=Lzvgu&{QaLY}?CuhJPA9?_3?b~&?Dmuu7!nHb~F|yuIX5nvc z4rsrB9`oC+PdP<#{a8N|6*nzduE=OU^33mnjNkTqtk=lO=gnAx325WF zMWL}q_ZC33e%vu5WOY5pioDL8WwhW0ehYL$c zUYM3>a+EZPibXN*leHW|yE8A(HQdc=o4SZS)^N;LFgTD!K9IkA&g!7|lNBB!;^fB5 z&)5DFX_1j6O*)1mQkVh6W0otU1wc5CrOw+e0CHCOy&(6VY8pHwFFgh2l9TNr5SpEq z)QEOco8GH#}aqNz|rsdXXqaf*V+)|Yn7kRv&F1@0+Otrd#t723? zlo7$Xt6e0{rwg{8ms~f^hy@xQl`|)RkVhXnQeyTC;>(L9%r0TkuFFTTO`F&C;>9V( zuGZUL4t1<}9-udkwFSNg10N`)^I`>yQ`e%Jl1>X|moahokWy&bGYBoMTA$naut722 z^>@M;$9JMI8AY8D8*2JYqcwgU_;CytP?B1pezyp0V^I&8*k&Z?dA@9QMex5 zYq9^VQ{GLQ>|wgMEDn#<`!cyQGszlr;Yym7Ce!89=h@@87m7x#^=VYpvIODgNI=R* z!E_82i`=GGH4TmZ>}T2R_ovw!Nl=m9ecpGFdg;L=d9*I3j;w1zA#0v zZ#_KQQ&lm#XyqpVv@`yucUc2Eeo(~NDWomhc^OaqQB?7;F58~$S-P*0t{eE!x7;DD zDxBcr_dZyV)8e!DNt(5yf24-u#vP>=joC2^cJ0x`8lyR;UH_#%M<^BVLIB3CgLvx_ zglqnofnmmOz2NTY<#v_cZHv!+!#24!!~_9P7=G|*3> zZN8Q|K9S{7bJxsTfnuCYm?DMAdc#L=iNIpT!QB|Ey z62n@fdI9ZNONs1EBWZeq$pailrqI$`)j|#Ydfv1`8nv^^%gSc^#Jg5^y?1-*{33!# za}TO>@Uj0dY+WLkhvK}O!*E<=olF7-wlI%ADNaUh${hzE%C1g~x3&MJLG*6PAT#%Ka!SW5ct91rdo28VjnQTY-51XSKOs^4+dna2y~ z%ytu+7%ws~5beKI{>=g0LGb8sd}b>3qb8d9T>-?MXo;{iPO*(6puu!P@@cEX)9lp@ zn$13XZwQPr5LD-Tg&WdQ-kRAYrGp^ws?5R`IKcL!THXbjaOOk zv2azY-0=bdaHjr{L|y}2RK|McR7A^0CH9S5Q>JGxX%i@pk+#DMIoYG}N*VDjD!f)S zV06B)IjW-QHegyqF``v<7A;26_~O_+ToFu)5K94^h1|9=cs0#JsTE~V+^PURqLB0z zT=ijURdLAS03r3QKvN?DhKC`PKseJvL~~z{dd^#DLbvXFQl=^0y?za&8E)4Rx=%cJ z@wW5b(3nS|Ag@*MNeO9(s8OrwLIfmLMq_YL*+mnT9Q7E8&BMf>2=*DiWU(cfMmr{f zt5v<*Cmsxm?qNC2YTLX%>&N#GyF~%S?4%vXEYNW2wbF0TJ~H0+1iFMXU@?9z}aE_ zpR6p*Ad@DfG5oqyIH+S7FTw2@V%gS>u7QTM)#(GFS@&SW*(*70G42FVzqw3EWO0(< z9%4pRqsl4WuO*b3ZqGsH;!P!=F^M|W7eZ+cm<4*>zi zkwO86Kfry@S`2iaOoLiqJw@R%>Q@IE(ed8WykHfDO$hVfwjbvwR;U2dl)LJ-q8H1X zKiWymGGCKsbF+y~oyn{n+{oS5gZ|LoS~CzLkc*%qaMtYYNkL)4j;s`Wpm&07Tp?}- z8@V+YQC^Y@Ht*gI)o|_TxS1_aSe_1b#Dnyxp^R|#P zsFG=3t-zUag{L8{|Rz>s-dlSI(&;+dTpODtF~$| zAnYn)Rs!7LHRtcKfoGb$#-s|z#lW;LTs0_4ja|24G6gpAhs9daiBmia(I!D(B%H~M z^OzqQyLn@JBzsM5>`q|AH;OdQ!K=nb>TWk)GFwld%l8X2xV3#%qe7awj8dU;X7#_D zNHgN;_EPJ|BxPxv)16~Zusf#~RqX>hTaLjB-eq2zHL8jS-3u5zf|%4#4(g@?a9QY5 zzFAk1v*-xD>t4c{F?Tr6BsIf#1g2rA|3)1NRriv{Q*T-k`+j6cH!Tt_9V1xk1$k?s zQd19?C~C57oKGH7-HBk{^!0hMz`-h`-)6!-%G;y>OBSCWd2`$3dPHoAX z4|}Ws-H*K4z+Bm0Ox#9=rcPjq-lkQ**++z?`^@GHP7R#BSDVND!P=m(>J0^#VNKQC ztzkfPiF>B^UK>FTBVCg5b}V)u=iD51lZSXY2G7x}hQPDr_rqChIbI-x=Uh}QyN^3$ zBDkgT+~vJlcV6d+-9E&-qe>5>UDD@bDI7I3c`LRmTOosj#l!=jaK=YBHQCOK!rEcT z=D8_vvEf(Fr?;vG&!m@B_y>=j&Lk|U+WUr-3H=%ia|p&1-mo83x4HBp9|H{ZgGU+6 z^bKD{Ia=b#FptpQ%R0vXN+x7Z`D1st_moVABqq-yYr6MXsqVG=~w2H0sg=PnK0^l;;QR@l-#_}OZHYw5-C49zd}O& zs2vW-3{6LIreY)t(rJBb-rz>&Lp)5~f~j&

    =vSC_}uBpBv(c6s8XJM{5qj)EZIJ z<5(LT&_HGtF|y4S1P8@0INWRy4mE32q`boZwRXkbg%{0Ihp!+d8JB^A+myAuhA*b{ z>yUEUYl(LMx;k@if>vE{u}Zs(p#;}3&oAZ5UhJQq$GJx7zK~d9ym_+K_$(C<3#%@5 zXbo-MArGc5Ph&i?`G%C}Ki$=CJFHfwhsC;oIAs1Q&& zLjwYISPVeczOuC-0^RJ?TI|>K{MtZqOkrrzqB~b>agIgiY|AB$q^mNhrb!4JT~HD! z^T?-c1;Mc@Mj9i#Ys=MiYu(Kw@xg6P*5_{EMA4u2FFW^~87LqYB;NFAOL=~9syCdD zk}Gk)^8@v?@Zj1kZd!_&y2#rzwO7307Kim%8(BAIW?Cr1<|{A+<74c_9IM}-exYO` z1v~@z%NtjJ8x9F#E%8BDPo@sA^vh19o1DExAPBazT8CMwXBG0@9nhCpQUO$9t(nNR>1jzY&my0WJGT(x2)6RV|s^M^`+tr6?HW5Gx`Kw1c0HRV3n^FPoyH*2zdkoTab9 zW3qH_-^tysm;lb}(N|L|-EhH-9)?FNZb+u?(I9wT44>}(Am|;}n9jw&WHFU-MjG29 zK5SC8uj}3lNM|i6+d-{iO|c%wgULgZu!1N2L?X8@rIh>%Qd^Iq>fo6&7aKfxO{-l>Fnq2V)Pl=N(nakEri z=LNt~0z&(Ia&#l&rTe#ez5Q*0KYL)O$8Tr1!wQL_@8{Z6V#6)JW2ewDH#eGS_&PH; zD;=e7F1$$@Oa~v90-2TH;#%&yc{X$9W<)*SUI2VM1zrH1guW1(fc;;@Yy`FWxpcLq z9*$}VK4#;fR;*i4yL&sC?2N}JjR8r;H1kQ|G=;>2r!jPtI#_|0jEQk3HKQ8_uF+ja zxZCdzHJN+${nu``?SH9~{o9_riECES-!?&-(4M@WRO++>!nC``NF`a1hDcBOBrBmD z$dwF!8ALPk^ailu6ZOmw^^miQeCg$WcK%E)=|ON-+OG-^JM)L?0Vv}EWkQs2$Nm%e z!>{#(QZNi~vB!LW3qnbCr*N}`PTa}v!`#)JItxdkc*n@YVx!PS@)h@9)r*PuG^D!E zzdscm1y+kTgfA-73yQLb;gw-1_Cdh{Dm4+NLyx&=$2<|+>lnsmz_=l|QEy)qzejQ7 zc9h<%n0s04UlhJIvGs74MUQWU>KwbdZ~>rA$dVssrY-{mv#8bn*ES;%JxA~Uda?$I zLqo}=)Ue!dS65+6(qOCgM)Jk|jRjt|cH6V~KT2~&MOqAYOwSd_S$4cv;Vym*A=na9 zVkD?e_h-LSE|&grlEj?~E-$6&=s6e^IaJqtf(bPsDG-s?)S2;3LEqf(m%G|`Slb)4 zowyxEn>XSqB{+9(=S4>lk@}Lx(ZMen#mY(bWhKyZleCYR%V6+LUd^x8%4Are(Ji;e znMc`PK0YTlyFZdW%N-?d`ufI+!boO%*b#;XP4_iHIgoFekmkYM(H23hNK$x^(@?C4 zQC9N=-8+7tk21*u$l2`94)Y^W5)epdJZ~I&>}=zuj~5v16|G#rtY5K@{bx}%K4t)K7qGK;f! zfa6L&{*WBJ3#X~UD$0^=>#f`QNZ5}vNf@5nm|IoQIK%6=Gjr$9ukmyp zO)PA|EUDuh88&BBm}b3Fk;;2pSnUtK!zT6<{@kexSpsw6=;SU2Ebhhv0!9tJCUPcn z&QoLLl{di%R`#lL5VA=8l*Q#}121DZS*`-0Wz}USKKd*>Txs70h+q9a1p#+12^jaW zk?Cg|5Tyg{!gVt7+Yr7N1*L=qSg*=WsEtQ=WoY5|1-E5F#iW*uK5ZfxtBsaQ;(66Cw1Dmfs z`qd#o?%kHnnQ=}&DwgJ>w!4!$HB)cPqup{m%Sv1mg~Y1aEdM0MisQ*x`B`b3$L4a` zR(=;DIFA U%{|V;-kAC|~ddl;nn@!THBb6^w>=N!StKMRJq%rNcwl+8U(3b%>Mjnp_N4h|xbQg{ zCt%Hp=SnaWw})a_CpeKY4xvV;WopA3%P&yC+|gS+*SKp-ZNKV|d_K7crI;7&pbXEB ze3Ms(p+W_eHA>(Y=v}4W);zFF(C3P=V5_3SXEQqurGK{+e=P8%1o|(ZpybSN3BI)_ zp&g`|o_TZsgs%nX`uG9SdW<*2fPk%tKVOU*BK>G!Y+V-<7tzBsXoxl_?loGE?FeA5 zpnf%m|2YpAYf+~y$G2xMiE%sX3WrYMa?p`MzLi-}fXZBA4bbIJyOCQXMttrs zWPFVfVGMI=7)&3stXv*Kr$Nj-mMGM*856sJTXH*LfFkNv#bub&y7@uM>@s~9DmK+d zh(5ifh~>VaQiKFvs$)l%1AT)$oLEdo6dP?=N94q(8qZ3w$Xb@kq)L!hyURibDq{)~ z*ll)V1ff*HXYSm#PJFJFRK4^J!Pxc9%Y|PSB=_0kF7C4Vp7S@<(@#g7j8dbeEF!X| z*ZiBe*R+cDCkzx^6J#lr*QBf861NOTeGyMEhMFx1_CPhQ4h zRyzUV|Ca(2?;>*Vw@@GeIJWfnpmYZcWHQimweTL+yJ*=c^+#of+ zq7ZBnihL`u4-;YX#4d(5FH=n{q$el>6P=yA@u!iYgK z5j*m7iR8X==J2$`!QSHp8Nw1H@=3rYg=io~HPqJ^eYG6{7E)OZYAhZFxA?bS`;IKG zErNISq+fK)SHDY(4Z1fx^_uxNZxq%;XXAR);X_Pe#(Vl=GLNs7CZQXpP|SO&Y9}ej zlLKkz_R{cX9z=wz?a*Bf(z*bTz-_o3zj)M26=nbBW{&v&8qFrJY{r~#&zxXpB6-o2 z-K?wD$hMk&8sK9-#p|}xCmT)4P;T{rJtQGTzp?!&a-VhU>(x1JpRjrR`r~8KeaVqx zD`BQ&%KhOD*W}ULo0PFph>VH$#k-k>XUo>za(mw=9sAs%ysB;^MsE5X>uNBmPSkVr zU>^t;R%e+yDEQw<_m$Sg+u}D8vrH@XXK?51u4h+pchbdg952r5Y!t(DGa4ks)2ql| z){sbO*1vkqf&y_^s)UmxW0wQ!wScO3kbr;yLwO3~<(Iq5QM?v6~nGEwM+x z^3<$3*_`_$fP+<*I$J2ukyDZw)Utt9fR_XoQ`*NxiE5WtUduK|)}XR7sWZINQO{i} z0h~rAwlFM4kSCUC2ZzrdhdHcZq=lu8iFGlpY*(nk zZS@JHW{N9AF#WZgtdVXBa#y^0`p%k-iAH4+ip*TVLU#yr*9}igBJ53!Qy!G6-)!Lc zQbbb(+=BDfNu4zg<0xa{DDtudl#NyDFraDw;v(?*`oFp0PEfw+kO;i*=i85=tC9V4 zr=(N257Me2Tdvc6A0WeZFSr`n;5eCH>D)#zqO2ia4KH z81J3Zqt@p9KJGVwd191XI!x6vTU;$UhO0ikJ3?~_s$g_T0%6oFnWCvl2((!2)eX@1J&Yp85|8vVoAPc4Vj z@IG8*fXs;t3Eug2*yxospFJv4PX`MuIw-5Npz7C6d@E+nQ0wf-pHtVJ@7MK=U+s$* zUbgR#X=Ml-uZz`wVsC9Gl9^u(qeVCj6yP_0)naap_lpfXW*m{-7L9(E*IH@@Cf-Mut6Y?G8t#uzhYLVr(?7X@l>cja8H@?$D-B4=@xzT&Z zEoqn*PTNq|<%%>aF0E!Yne($4Q6xNcYDqIO{L}{4!Pej2%hQDT)Sc8@(`}2Y&VCOl z;f*E3l#dy(Q{<6^s$ojfzu^~dJT&vdGkKd*eJYs0T6K8O22t$wKv>5vZnyPUFcI!{ zH%EDLD1OHKa~LR&mPYmq4(vo;YaCuO92cr6n7Fw_()?rGGDxj#H^$` zU-$DDwPp1*?Zd7$LT6Lt6c1p;NCP^Rkhvav)-X^vis@Y1sYHm9Lm{PG&Py(km((|s zjagPi~Z{oERm=sCJoJO4@1fvBV> z8C|rl;$yH?mr6y_ou~o9=@$zTNgM6_S+Lw!w+PEc`a_+@CZf}57Y#91uKFr_$=*>! zm3*ohCM)bL!qGM8PSoUqG?mR-bhf;D$x&~D3M6tDfak10_T8V!M^^`lS4*|(8cIZ> zSBgjCGup6f2d*`D5T+ZOp~xOKRz%>+ba-BTA8^}aVP2Qljr9pU%1fYtzT=%CdS_nU z%IStjzech6*Szofl-cTfbSL~7GYqDRPg(C@y#g$}HZ0(}#p?QdWr^+m`wqc?(Je@P z|291xeaY3-dWnz90NozfW>=sgKUCZOSHz6sNov_enFNp~Wl4NQ%OUa!XRE(TxQ-E6 zV~UlUzmhNrrg2PYFDY5iUIi)5sO?|qtCdKbc;w+U*!fFIHF9ee-BI!9`PuSZWmz-Y zUn{j={5G++t2QYL!LS2oC&I%>`%9rjoZ@V)P>2i-0hPJoVzgW>5f_o!yYA$jK8C~+ zuaOy)9FIvF=w=o?J_qVmMWp~zW&lAnlpP*5Mrfgw=3ceth9Y=SCg82vU3%L%t=^2{|ae4kriTd#as8+k=`g5f930>{K_W*1S6Jeuo!3AS>~9c+6&x z-OJ7Q@j^iSMtROZjtY!=l#EV-@_W~|!i)^dxthSB-txS6RZ#vnD>kJi&9v8OCZ9Iv zxfI}<0Sb9cQq{F{|5t&WZ32O|kuLa(FHBk4JbAS8I)`DOkZjQ*amAPy*P>XJ-W~6V zDoKL(`9wsp51N!rELDWKzH(P(HRvg94EN3CsOA)S6ZI2&VqPaOQ1j z+&2>TjQYod_fwL+>mgAlm2>Z7nIWj8m*S#@iJ40~wI$4cCbT+dU>LE{mQs9Cd7BTU zju#oSS^9nU`!XYmRjk?Bf()>Z5MHR{wIq}i`%eLrl+J9VtD-s=N(<$W|8PC4Aa_1i z#hriXbn(?n^uH_7LUMUCuYo{Lyzs#TrFFe~ab*+0A+2>J*0*a>2@6iFR|f26)J)^j zXQ#^}E(EB;VT4SNCl<6`z@o|@W_F9DkJcNdbXlsnLdjJwxgw0}gbJ%%mwD%;JE9G% zr{0^ZOBh6Z8P|(+TP&e_FXG)jf!SMb&C)u#o`rYy3$Zu_R`LCXMAyGk;MP0O5Wvkj zw=5tmolWmr{2S0~?ARXfYd~Ns{2a7BF@US|I)1E^Z8B$9MI#w8Hk#@O7x=Dv$O@mM zrshxNyHC>a=dS^Kazvd(pZ5E0Z*^}?8@_VqmTO7>B`8%fEZ!b&v5CXiM@FuXi_L`v z6W7mbCy7VXg(&3pKn8}1I4e4Qr0WyYENav<>&D8$Vtcfr2yFwi3uKw-K(sNY5Jh+O zNkqiDiF+}k;QqOQyk)p=VyDh?n)`3_3V~4aDUQd3ctyH-Fc=MhwnN|_6!6V4zTnVJ zv>M^+h8sbFPK|I`HWnWVNALH0fQhTK>-*Q)=c1&2<6mJ11CCf5E$_QeeK$N*m(7}j zI@+c=3hGSlM^nm)267PEvt}AxZYCeba;f-44^a({4ieIgJehzq=)!IJj9GxDL@QJv64ukAS#y)X~&Wf=52m!j_&HT*rK^CDV~?(l{3EVeDq<5g0$a{Zl}nyERvVFhL6^{8{1ctZg@cT4jXhQPgs`hch?0^~QJ(V=q1HEhc&>&x%9L zoePfzjrbRe^U8+fFL#I$FQfIK& zID(?6EL$*j>Fy~t1P^A*px+gdQyLx|7c>7Ao zlSwkORF=wlT-CUFYW&0kEwJxJfTduk$!gUeyhQ@i%B3tKVGXpeZn#}4;~FB3%c_n) zH_zAX2vCzXf;vM0oubfhZNTasoyIH8V<*g^b7Z?QF`qF$vsH&``3Kq(pW$h-Imqg)rzO8Lzz&4RNP+_+YrkR7+jv7Ez1j&)F*!(jgb)m*I?mC|A zuU@oSq;M-~6Qs`GRKrqhW|S;1q|aHkiqH_|oDXe1q(G-Ej~#mi&Qc6I|1rA|CY~~I zb92r&D<3!COC!uETVe8{&G#Oj?YxM0pRz=!r2O`<(}*9R#7X4ms}0F2QFi z_>s=_G#4TwS6`XrLL_yz#VB=Z^%ICCb2dsti)&61?sA0J-!~;v9r$qpBafs`kzU*| zoW;vC-bfB3Rar0e?qY|GVvB}~JK?HWsknb{2mA;A?F!QJ)m|S#bmdQs zeS=hGOqa$!9D|7hsY&3pr|`E=y*=in`Ok=h_93mwt+(GW3seIi^5TX*F3#jGXxiUD zJKsLp04z27x2$3JS@+MjfNM(?Uf6|zs_W$h+anxr;WO&XAVW@h49;q1lPBQn{m^Q1 zoY|xpYIL}Zmx9%*XfWVT*rYa*h70-|C#)IFr}65V#sJfNM-?mQnjCnd;FLQ_QiZxG zOBMN4Suwb(2L)0e$3LY$*Wtgdxj0#8-;rp7KYcTSAwtWOWfjiZin-wJgLb>tQ+ z4wO1qS6geYA06MfdRB1`jzT$W>Cj4AmYuNKsF42F+L;oc@vw#paBK&(!S zfW?i|@e2HC>ZAaVIITEiF|N^&jyef57J$uvE$;qqK1V+OBWd82F@Y>ixHNr zIv~O0hG>3^)i*E{9#~ULfjFkpV(X%f-b|t%s!r4dGIPU4o2@Aof>whm&LoS%)6tOS zSSqq$iw5b6&G2ySa82%h^Q!J^k#wxVF$Scat&`{t(aECPb6NNmC)Zb7iN816Q)0v3 z5f&!&Y5Mg=b=r;&4Aob-HzyzXsi*RT2!r9l)e+!(yvkt+q4)U@8-I{XTb;t)px*k( zn~J1vFc7Um4vBvV-N1HlTR$WyyKVd-$a@~tV7!#Graklr-$+JP8(W#VT;i1{U>$6n zzq$o&kq%VG^ufkP8uAGiHMqeO$E3hrnw7~>?dlgvPJW!khE-*IjcnbhWEF8~(hjxV zHYvNO2PJ)nq~^m2W?r$ZQS9LVI81CMx?-kfvL%F9!QEZRg;K;cV6Po(ZENdkdMZV) zysc~2sPSeLC;K0QZGj3aF3UXBy8QDOLIUfY(gt3)&mnHkLc%Psgug58*bhC!lg>G+{I?repC?57mSJJ5uZr6E zn#rn5i|$DJI0iFHZdlqsf}@`UHsC8#d6*n(#Q1i>5EbC@PA7m{vh4YFtnN*D+uf$a zCz+M1iz?E{7leNuw@n%);3mZnDz33$-QTafp!iRS+Yy;#ZQiwA2{aE+OF6LT0hGWnqySLrg3cNy|L$U z%&qRLD;>`p77H4ILVk$Vy?$E8nHanWp2i-f=};RO%wll<)F7ovEwTpo(unzq9{YK)0>u0rlZdk)2gAWP=_fs z=LdC@P5$azJc7u^=oTZC&Ng~QAAD_nydLskW1-r&qM-6jek+oL?El=%jK7s9QfB#t z;LNSX7`7{vNjov#Qc@JiSey_PS+lHZCxuWjS-C8*Ma_rrs`4 zlo7hPG~E4*;qDgkVl29T*LWOgnqbzGnyqNonu9AMD(kr6ARGj5bKK0eO-yv#ix=kY zHZNZI(x0K&usG`zZNJR~z8g;icsQ4D0(J&sqFMHW7wQOR=EES`9|Znbv5AHKC=5M3 ze4jM!Zd^P(KOgVnLS^^RXy1Hj)Q6rZD$dnRv}}lyVDgaNq`;oA3959|a?#7SvMv$c ziNrgJ8$zf-iz-cvmlZIE(B4YPxz1qBN{)_ln;Julakc6pm37Y7TTlHfpw_KN5kQ#m zuXyXD&e#1Wv(uueC2Pj;bk}B(fZ~p zmcl~;c(n4$F#VvmB7P$Hp{&^68WRPPk^l^~nSu~cC>D)(R7{v@*HSY+p@GqO`DG1m zmtwitfaDybc&Y>CK3!;F+p6b;M2}c329uTMo+Hb?>>BfwJuv79SL|T-DpjVGHY-0o zYiNPie^KBhd9gS<2aAYpb<|^Q6ZP3Ax&%HxiU2;pfDj6bpD1MinqKhGbDf}=b!OZQ zOQQ%syhA8bR^4CLxz4U=e_o%-^gj59+u11DkOCa{=Uj?O=;`&~>Yx7M;NIKW-B6A? z6BVS@c+r$E*jjRoh9rBC5)ysM_muuqJV1}zc9nV`WfJh04?bybn_Q0_%|3E@)-fxD zW7w`7KDIL|pTjO-ftayr3@MYRqG3@`viwX*X_@VvKc~&Cp~0M7)a=>wDIA+cv zaaRwGs+72smiej+=oMk{)ib49?PqL6h0+VC3##vtLx}>XdI;SipDzg&r2a3Fmv%d| z>5$v+rcAxKK!sVEO#b|YnyybI%YyFx=FD^&TVXyd2Df=cgIw4FuTquDVi7#za)u<8 z5<=g`J8;s_8XchqHuo3v)0q&pWg3S(@xEuN2 zcN6d;wSC`Qw!V;ICi9o?Qw!nazDgq|BQ)6)4=yx|^Q<4~;l5Y%_*r zjBL%8^eqZFsHgOMA%A~_l7U8NPCYTaBeApn0T6G$uaz@UIOhL?RSD|4pdp%hZoC^m7*OPr0GX#{fdO5 zk=ulPnEm$n9E60onWCX(e&WzKCykdW=oz>KC#|`+`I=3&r9CqSt`v)MQXJ(mInTo?@UJ3=<`zFeJu;y}B| zZIQC)E5a=i+V4>o$tpI$Ty2KWP@g?aOpcvnMXzdBsOV5gD>p54T)JxN=>6)C6FY7u zqm?ZZSV`+IPEQmw6If*obl51&%MD2hQ0%5AvucH6&sM4j-9U1a)L2z@WYKd^MFkhu z(Kjx}HCDx?++E*A!wbQxk6foOFfVCGnShT}wTsD8%7`K!mXGa^I}I%6)tQuz)M>1^ zEp^-xXqQ+xRb?PC*Lo}0WDK7Q%-fa_O1AK$@ad0pfd`|-oCTr)BCu?Q#1TKl0q}WBt$6Shjlg+a^*KI2YwG7Z zGQf!7yt^;#ETQL+TOzk`gff&NiYUSD$#D!eati6d7`^eKBVkE~fT)Uf%@q{ffixJ| z(M4knA9aF%^_LoV`0nS8uh6q~yh|BAR;#XGVbBJwy39a7-JY|y`RDMKd zST&P^p5%WVTS$9_5iIOU%oIGjA?BL-uQQ@MFo=RmMHdXpDLFf;5Bw%_P+lg_{hM>Y zJ;i;=y%R>K8=er3_ecV-df*QUjfn@g2RCM$N()c3h??e1^DaCF&41Q4zho!@v!>P7 zH1@L`HI4G|@&@YZ>;4Vce?W>p0Xqrb@UT`8j!4Z$f9NfpsPh=Mb0sh>JGv>gZ@@-Q$8aOItr%l%QzHAinI@?A1wJHlbJ7JFmc`zZ>9|^rhexxS?c~B_Zbjds%yB zQFr2w!q8!=%W8(R2yc(fpMKqBe`P;SJgEl&Y#L(i3g0jk6g@mNJ>--rIa4hPE7Met z>-68)t~}{6{y8QgfbMZ-GqGUsB#hQh*xjk8hMqhb2a2l}W|Pu>>_CHEkTPiVU~j!R zX2Xeg82h_a4bgjx{0g!QrYnZ9$o`524V-W1;{TF4Wp!LnKyeuO6_#*7xz;2ujvTHS zBBsxSpx2D_r_52=_C6E#7irvHXkan~e_(-QLWOE7(L&tykF5I#^3_Rtc`*HD&gw9vx#yPd@qWhx}Qn#N~_@)I{_4G#r#n_~Z$Dhd~we3djZIu)*-v0E|8# z?-c8vt%9J+ecXTFheIK%enz2j%;s>jhQ$oWTx}aYOI70mgb@TOWHN_XLusV@wm@zc zd@@AAYh2)c1w+I&h3dxH_dlWFWCyw$>lc?TgpOh%fy0!KbK~SRY_7ore)ri*Pvr;6 zC;E;4$y0xxf*yr4Duhm=vcEm>oqW3+t%jHxNy*itq%c}qQMWcNSEBMK>G1cm-95?- z{$2o`dsH7H3d1~aF!2%801MJ^7Tv6_i>E{$ySL!5fFp^~H!5W2HyIr=j@E8|=jn&w z1O=Ok^C01DxSNm838(2D7No#Jiomq{>p=YecdQ&_H}Pa?ZlhyDIC<3TKlzvsca3f< z4H6&2R<2li0->WZVC{k8aT2ReFDZNWk>;pChRFp;a#DKzwnM|SgE!}kmBke2N4eN1 zm_+JOQ6S|C6#gMg#YCidc-QJcIdvT-W0Rm(d|5G;&hQm{GtnN`b1&u82ITs#S-8`p zy0{-alo#;8zH0ukS@oK^T>=t0Asqx-Dk5@bs)`AKRe|ul1&oWoekwn}nKig=q^Bh{ zi)@0Gx=6I3GN}x!by95texPY#JsJ+D z^}W5`H4sWf5I8%3TT$J8X?DB_C!!D|-|itne^8)1_@OtWD;mm1KJ6@i1_n=onDF+w6Sxtbm3m3;YaD9XRXx}Ho5AQ z4@WM|CphWJT(DZj{PGq5G+%{3K!##S(*~Me3+{dW1E2pRG$iV`{uLz$1QenQ{7<|O z`s=?hhW~GSi~b*;zyGgghKT>`+P8PKbGG~E+W#*i!xn6-e~$h8E?+RYkCB~r!A0;c znB7wU!}6N0*uPCVyz?~1nx!@^o|v8_Pv~DaY(k2tloLMtv=o`)Oxakou&1Q!tu{0- z(YT8wa-Sq53XPP4qO8_9!+(f5!oU>bAzp6958?C-Gvp8ix`{zgMQhXkp&>p_>XGYv?#vA(oQ1^~elC|l!Xu8t2ZJU+0ZQEIC z+qP|IrES}`ZTr^O`|Nv9pW9#W^S5J+H^%xCE1nf=Js5M&H>STj{%PDyqB)}@f73aB zF`uyby#CbGY1pw-?PdIrbOhe8dcA%Hc&^YOOtWMMf=c`;bjT~J!gA$ zqPRJ_nVKp&sC<-{36|;p!~_x12f_YsX0ht5t`n@%gnMr5gmel=Zp5Lm;^_&5updVK z9rPo!u|46{^~BWnM-b4G7ebOB&gxfV_Hw||Y+G;5k~3uRCm5Dk!It(^+#tDu_SaFa zi9P+dcLDokvw$@C@SLcAIrO_>zFlZ`vEJO03U0c=}lb;PO%w}Xe*DL&kWb@A_=p1Gys?K&}>X2F--u{8^-~MCrOq&+~q_tq%GWeNLV1h(E58P9)sz? z9JD^)ZPh-4T7E^R08Q()CLDfLv1RykZ#}#facQ|1U=2)KTe4?K>FG0Pxz@ir9!;g# zN@H&~@ew~i?o>6~9oKBMLa^#LHLCf42<9IC#OmY^^m)_DPN}h}!g^lavFUi&v2Er0 z)WfPs=&g!4hp#b1IPyHcbJa_8`gA^}?KoV=2t)rnmso8_q(b?Y`GuCu2vG3oh;MTv$5^L{lN&-vS4stTU7( zVqVwo*7$g49<`PGX_(cD=P7rB1Hl zcI+dLTZNbe7eQRvEchOQ;CR&mxmZ^=RmWx3R-gP$P)i6sLlmUjQ{# zYeKp^~{t5lhjI>-}Cd_ z#~+6oZP}+OcBmehujX(e7^B-@EmR_);At*bZ+kIfH#-zYnW8ZXLX8yDkCe#+!7OMO zry5hyI8*QGi^_xiHC{B{=$_f5)?jE9e?t=yzu;)%CThF!YkernIqJvwq29}0p||e` z6e=$C-beiS#zq}o84&*|Zb>GZO&31~Q~iS*E`p3Zs_aEG)2Z;~FZf@mS9-KE%ih|@ zxgeE(G6@nzrGllo%M0IPb-)YJa%mZ(Rg{9TjB##Jh02=jM1;GeAPPrCwzd^>klW;Q zq3{KH)RkpN%c&p1^9;d{NTFQbBN_NC7k;xz?Rpg`&Bg$zg921CN_!*rmx)+J5@|m) z?5tQE- z9|q1u6lKx2_6bnYq|&%lgx6S8H9_Eftt6eeS1mUQKH(jacFtI>gP=6E=20rI&j^3- zm!TGt>MQ3oS#JeHRQ;L{Uu?n=p2+Yx8oU7HY)w^e?^cr&TBdov;_3+ zJ=8<=z=&XT22E=hM0-PlkV>2O_Xy?ylW;8dUcNH zD=Q=s_tqhf_DS4y;qNab?Qp{X#wUS*8hS|aa-GQkj%}!zbCmtt-bs%P<;SI-&r&}v zjQ4p(nVPizt*wkqEOl=!z_7Dwd6T&+g+_#pWS;b5GV+5{S%#^�nWEj(}ahv_hqz z%Bk#``WR_doFA(ru{gM?yk%3#Goai+9{j_txxJpb>%eg6QL6$eUMg}ey0Y~P{os$N z2)|HsW(wDCOdVxX2I}COIWXPYv?T#9yjg*IW2v*ay3c5PB^{1tzZ0Q3;L9l^Bpc(LDC^`3q~Q8@PG z`IkGB+JGpE6IZ(`MUihFa!jdHB^ZLMrk>dX*$Zi6S7J-Fcb5gc7u7}?wZTG=t=xI& zByNNNci%h+BLgb6K=}grjEQ*#$r`h`mn9k_zJX=gTF01wC^LW| zZ5pF&EXXFQEPbS0LC&R;3K5l&YG)lm*K919FoyEQGmmH(-U)J33XIiatt#Xu%fklX zZTp+hl{i2SmouOS^iFZHY@S!Pe@s=)+GYrjMH6m&=|UnKWuv3J!;k#L;o`<;2+dO7 z=L<>`2 zjpUL$J2MChe*|^#Y>bUn*8hR_bjCGF=FEzyrL9Fl5O>D!->T22ZL8L@s)9ibjmB4` zJXB__(CB4(HWOJ3_UR%U&mhuZYI0Itz0~YF()P)WWjOLaZ`Bj8O~^vI_)ro#btlZH z+_oDN_SPVTVhbiSW&z#gqoHEu-3CHcogm6V!OtT4ZT`f~<28u8hW<4%ae2vI4k8EF zVlAWGNyTnSI^+y?q?!ATma+tuzh#(pRMZ{EZrPxG+0m6E$=Zjf zvtih`CZOnh`d~zQg5@$0y9@0<+ik5^LEe~%#ZqQjV=~1&gi8v6z}u-ef2uKg5`z5- z4@|%~e*&)2gdis`;cS-4lY8sTYzp^q@=DN!80;HUQyqsT%fa(C_GI)F7KSfqwBAAR z(MdLBv!E{CAtfnNZ86Sz=|;7hOd-Z>BLalTtlU~_c&a)AK1HJ>w?^?pLZWZ{aNL}z zJq&3X5sgyCG5)dsTvJ*b3b81BApOPLuVjq84;3gk--hy(lFpr$mD31QkPb5dunwA+ zD7G<{8JaFop~;v+%TC2x1OH)0v7rxumn8HguOgyCi(f1}+gD=2p0oL>|L4s!r)(?^ zrDz}pSp0R3!S-N{Yq!c>?`l206z8Tg+EgDDB_?MZ_DBnbKRLFK7Rb~k7by8T3_`S8 zI%!!4`=-nBbz__DxTPqBzrbx@B!b#mk2Kni3)(ddes>H&l`O&bZuN8gN&E2U^W(UM z^D%xve#~VmhKiJ%Yh^BAL%SM4zGmx(6%288AYj8iIBD258iltX&>O7!D*^PY=hxx= zV9In%H#@tfSrLH<3C^MX3_n*BIPE6xZ2J1$aAc}<+Be&sz%u|r+qcWd>u5?&Tqkm8 zd?&>-oAH;on%A5q?%^q?RW!=VpiK4l^PIK?Mb?gTr=+v;lDKx0|UGeW8b`r_ZU#J>aH>;SfeKaB82(Zt$ERhpb($DB2ClbGO{E>hvNiR9}lo zHb6x|cR2P<8n-}_bz>Gp7ZOEYjPTMFE+XtGG*!fk(a`&cCFIJm^H{ajalDCFu`f-b zgPPV~B$_h3jQXW@BYQk>Rc@sQ0!&qSh*U0Yy=`!1TcRA~peUZEM+e_If(rG+SBOeSB)#2|GovO_e$#ZjM$m`WUJEg$TZCUgNriS?aKWqtF-#ZZDF}xUEKC)G;eQK}Tz!cmHJzaX`sm4*tig zu!Zs8*+O9b3tNbP12FtYBPekHji5M~oBnZc{cn>(8RJaVzt$rlGC-B8bj&&{eCN3; z{HU-7`xdS{IrPg6p>!=@I)Jo2BoG~qJZn8-S-cYK*Rz*k9I`RjC~6gLviQ_;23!BE z!_(doXoVBeqJIC+4xh(8G!2Su9*?IIn;{u`mxE{+5udoI-;qpI?||bfxX9Pk=LcJM z4jO2K@B1B`cE^G8QW{UfsYno`(O5^!4-nw$WQp|6?zTa_VdMZTzqJ}fo~0#})6v#iFPm7E$W z&~aBuA72@HL|PF168ZHy*VShm+qD}F?h!xr$|xDb%<{;$I=t@RudYbeN#|`F5Ld|0 zqBY02+-4*?za#Ae%yVzBn(JUg>rM-&g(p$0u)zfaQZ&Q>%+xXX8&USl^`ja9a(}uz z?M28{d-nz~oZm_j7?RWllHfEPCyE|~*k*sK=pz!7|?pyyd2 z{!SMk!^SGG1|i&q-0z{Ht% z7Y9-Df~un7F|%{N)9l3@R14rZ>^1+C;x2tz^u~Gci6~c|oJNOB9m+?^^D*%USQ;(M zlLtV-IyGRA6`g?fW5@ZDWb;vg=F{ywwq#Y3KOo*%rYx~UsYoi#t$mQuNVn)-*>{dG zwAvS>dAxaYnFs!!cxK`y=pG-Cfa1fOLTrp4K4B)k5K`*dDPYwv=K9adP7sb< zQBag|U(cBY069Qxb@~iHtSz$HC{YgnGimOz>WCj5J`7BzoW&ion8hx4&$r~~`)+DF zc{mOEu}c@#(rqVcsYo8dHHzCpq*)8J)N$KvfPt8g+3yLZY^}@%;CJo9i5%sMF@T;> zDP1rm9$fJ2_S~g=`yO1*I9hzCFB)q&fd$1ps*-n%s4^^M!^gah!IOPNoIozqvs_K7&DXs2l>t7l^oB-Ft+9UMx_TfK4>zm8$=2pZqW zo?=U8vevmb@kNI|g%7q%1IX^9NrOgpO214xXM9F0Cz;{+YB4%L>Z)_g7Ky?sH&y|4 zPmF8iBqXm{j3x^j2vx6T0#<%^E^U4f9g%0e1A`_A$`-c;*3}Hy@p>g?$sNW!Dm#fw zlD!c)uw{LoG5|N07n8iMMysYI&c}h^@t+MnM`mOD}x(LB%3P@4E9tru+%p5E?!8 zK`cmtkx09db_0glN)RL}OjMPK~5%K+(~9eOjj z^oLrdr~lgQEDtPF8!*#}CgFQ@k5XzPOdmj}_gUbDe8!JR0CbfZk=e+kgv&73wVnMN zjiI{(sc4x_Qt#9(ww`(pPTKF%`=moyIZ|L{=Zv3^L(?7WbuXae9qLwvyvysGgOAx8 zEfY5EZpqrKS{n~3&x1$0`r#)>=gtt?ZRKN7L?DKoo*c7A`Lj0RkLBU;F0oW!fPZy6 z!B#x$Mo<6%aD;!??J)mExBL5M@E_9bAOZe*@yVcy|Ks3)y+QqVvvV|daCD$kR)Yr6 z>S$AK{-T`lnO(Z`|z!QAFynVp${YJx{<@Q>wE( zq`ue8KsZH8ys{3`_yHwF2>h)3)zit_i;pvVOwYD92wE9B-s!2X9|`lTtBbs5kz0N; zBVh_BLId#FX%6n4OH^?$2HvuQn>-he@Z1R6x%hez^ZYwhV`w}NVy+_kI4Ci3ds^T< zuP)Y@M<T1^FmZTU5`?8^kk!m%LDE%Je?pf z@eY|S)Edc-01HzlAeWMGl3}q{NW$x_+Y1sEg4BtE3j0$gf=3Y+3lI_+s_cYGm83Nt z>_+!jfjWY7$l`Yq7qWKMNYxZmv7cp3{0=tgjVgE4FW6$1EE7T_RL|Es>fEtq)0h6? z4YRpf%($jkzqL$u0vPoot+{_z-$eS{>?Q(|dz#I*8k z!}oT+RSy=a#Viu-R4oA2uJ!3-574^qW$OfYs$PQQf5?p9s&sU zqpVtpuQB>grhO1Elz7o>ED@BTVu=7htPnu1+8-B##o)I08RkQ`wTVVvy@}3aVK>28 zHP*%pq?w}=t+MRWo_*~Ziyj`Tqq!zKJ9FsnP&>?Y{1v(eSXEK&<<$3-D0UXSePy3` z8E}fFMSrA1^q8!U8p>Kx0jyELs{39UL+`9Q(o&iSmwN%uwZL04VqTueDt3SeuN5nY zc8v#FJTHOt2;^Pr}!=uG@~92Y!ziVnmW0qamRCW|Z$+-D+f*-)viTaPf3?ta;G2J<-*fxvyj0O4i$hQ%J~GG2(=k5aS7_x=k`9 zNWkYZB-+8J_7a14^c&>j=sPBmyk@9xgKP*Q)BUPZ3nGg=8zc(+At_EMlM~y@yMRU* zW1(E&|F*q%{J9&#Yq$WX$jrvVyJJ-i#vx&|h_^Qj50}R;_F!wB-HkJ6YOuw9Q}+2& zMscdob2w;E2gB}Xd5Hoa-|enRhsrMa?C}T8{Tr9j*WH_|Gph=tV-{wk3wWSgU;uX? ze^N1m%(8QnQ%KY747WG|ErkN>i*ku`XAjlaR0fh8@M?d>pD2vo7 zjYntDBX8cLG>?LnLs-mEmy4}7NK-k@s#p|f$QyImK?n^MKan8)EZd&NJ7)nAm>b4N zeR|{={y7}OL9IiJ&!S%TOy#=LrQ7k8gx_2YL~s|386m)@zBq(9lEqX`fb7K!KJ_S) zB6MPQqNuJv6YXn1~*&T$*v53j%9T5h;PuOu_s8RtrjJ_){as@arM;X)I-T?P-#vNDs~)um3(9< z!35j51&Dw7i`yXXI|L`Q@?w?=1g%AE_}M9oROyOtsaj^F)Xuqt_|DL0aGvPh5he9W z9@QeQka0%>u(M-6$}PTGWtIC^&WY2Q*8#py?4W>U^+Q} z>%fi!zpc&bWNPQ_^#>O3BT=ANFKKpaPG?QmCQiuCw zznqS+_{!F3dC#e#qsRTdi>-<6ln-~#W~QuejQZ~*J}&Y^VCtYuGQ+0?x(BB|Mcr}7 z1$5AFYIRAs*dmFITa52LV8ssB;xvZeAq+%Awk$VnoSd8*HzWZR04PdT1SO12x*-Lo zLd6YCaH6K)zHHnlA+UR{RoOpeBMmC7eC7iSxic z7lwA&QY0I}%1ktbv`oHBxxeRgFXXK4BRXVZ&OL;fMV@1b(f|W^pquZ`$VMmX*sMmJ zp61|C=tQq?&C|RE2QBtfy*^&zI$KPxYf3H-A461A6j?h>e7fD=lcjZa8GXYd#@98S zALF}Z=xj@7-tLmuw!iFt{c?Nt2M;5OD6*LXz26r;jOrh)7l;qcIy~VHbf&7F=G?Rt z>U1O5i0b%pxg`@9Q|Jm;uI+3VN`l*nn{ErD{dCRwdv%y%Ssk+;MCydsvKLOed_EO& z$VFYbq%3!UhsNbuuc{v%!l{jBu^ok5%eUd>Sz%zOG$|$QSsYSHs{fnfW<9U6E3Vdd z_x;-L!~$#vQjQCqn)!LgrgdA&#ctj>e>igQUh9md9zoMBd)cNdsOxSe8<}EEoyZ)+ z;DM)tl(qmz=Qhb`wYzR!o^G1cXiWm`bouesQnMWe`TdKC?(eehXrs%r9@ZJ-& z-5}ij>ks8)G$9q~fGgrCloFOhF`DeqzY2k3Wjp4tWkn_p49uc1c(DMzSHoseu*}r- zzeel#m{L5h=d}-7Ph8BuTKMX*qcH=oKIjD*n>t0!?i32c5!=t%H(7h~Y<@zO>GR~+N1URz|H25rED?5uCJ>WZ%( zZicwCwlcf@Kw@~&q+~NxqG|y|RvY`Wme)IsXxS57Kr@xDg=s=er-6rp?gPPwq+P#% zmIBG?zueV{MygyU)JA8^zVDe%Omo5Rd}t#y0N93Hpg>{sj*tq2JbT&_Ev>Zt z9Nv*?m{n0$Zcb>*-#kWp`PiR2z*-^CZL~jC=_;VWMx$g15))v_yvtP&;L>+I9&Qn! z&49m;`w8{qIT8;iw1KsowB{PHZTJGnbk}tb5DdJpS+mIP08~A_4{Kn~C0Ak=tXHBW z=_3p=$x$utGbLeLM3gIy!%NRZLa5_|=e~c}XhHy2^%cV@%&tu^c&b*5ai4P)l_vGz zdEmWE{1Z;Ke?HrS-Q@Vh+ezqwiH4vK!g14j9#E6ri_YWp)?7M2mRrypf&mes(52YwMnNN%!pVh-5aEpihzp;HG;9oGcjxOzF8;7^n;INP%$da1`(ytpRYsBFm*Sdi<#IwH8l_kq5LUKn?zJ5qyjB z1GW~`CMxxafmi3&?KwsfD|3zKB__ML76;&OH~K4fhtGjE2Eech!WX~so+(Z zp_1nyJ*^8u1*WZhg{mub4^NUv2r*zS^N$#7f&odqWKfv#Gc#K)w}ZJ-tfg+Cxd?g8 z)GhMzG0=fY4G_!sA=UyCIBIBvh%zj7h_D ztZ$Fl-5fQrpHd9nYqrN#5ja*;B7Ny7IXAP<0E+ggOaB`d#?k5hIf#y>i>Bv^O zz31w=EuuJM$IkVuq^O+}F9R%qwsq7FD?c

    1}HqaHG?PTz`-~mv9-H{Swi$_4d(M zZ#cT59q{zhuuU1RF_=^()xT;Mi=Mz5W}-FTtU+a1az)lFzM%$?l6IrdH7JG4FpsVf z#LaYWI>>kzB=gR_8pc6O2+e3K^kr+H^ZmwbVX;_MR2)}PE5WYRm}Px~-X2FMg#S91 zux{gATEBoMf;Hn@k!6HI3s6B!*v53qLF~=I+g>br)nQ-t*?X7iI)|2bh&uCS32|=% zUHBTZU|*%HYFmUi%>PEBv<$|6VcMKRq%gRwYVn?CF$>-6= z=+%d}K^Bi=pJ0D7Nsj#n=$(>=OcPih0rDS~Vt@3$Y7JFrA#%H~D_{UbjLsx^GPE+I zb>J6V7n%z$6B_k*-P6daz>+>w{5c#aA?yv_^8jgv^|&G=U6K6ZQBu5EPl{d1F5E`#Ykx9Vi+WSLt-s;km!6!=x zqezsp92GZ=z@4*WfdkL}oQ;ga)t7A-J50ODI)*FvVR-fCWL z|8g*aT1H4}u#~B8Gb`FZ{bFvZ`C9S$J^y|nEz&8C-T%CE$_?6|PE)^vP3vg3WOrKe zizr*zpcWsJ+-bNj*nx&n6=s=L8aMi8+<{+o0~3C^sWUU9g`7W za5jXv=g%*zw`l2kZ|d2l!(w&|(CY9F?Rid-QJttcjmo7!2-m&|YM+T$i2(-k{`H9S zBJ0md33t2)o6M@=vu9_Rv3SENuiqFp!HgB zn3?QQ7ekz(h6MLguCk#vlXvmX&jE8?CYnK(aYRN6A6kLD=0u5uUj(|=;|N1m-C;q0 zMD6aLjjn@@tcoW4Jj;sjV}rG8=_TtvbDP~went-me$>U=3wdmnwaarZbM@DSQ_XU2 z9(rx$koQ45X>LF}pgZ#mpSKL{eiBMRrL$KvR~wqW=!mMGa0;JGA{BHqr z8hmkjk8iD=GWvSSKvCU^EGTs%e)XdL@pfWLHo^=6<-~@V{eV*GwNjR-6XYndTCUyZ zRQbb;egbD>>F{BwAh{E&;}5tlMHWuNYJb8kSEf1zd@`(Q> zigI6uNV@^zBs;2kc|cnDd3Hd^;~;|9?)Y42=n<&0wmk>J)iG%rKc(O;8az87HiopetnaJ zdHthoX9jRRvC%Xtw*;?{m)LJM>hcCBoPm_Ji8n8geQilMNl{f>BG4gueqO^ z&Z4D2h%Ky4z4?2)e@g7*r-i@ZM&ZIM8zsoa>X%8f$<*FjAg*N4s?>nMw22Z<sLjDsartZQmM$q&}#{t$#Yut2<306p1zM`{0a9V#q#-mpCK+qwV!KvEvuw0 z2whR-8Q#wQvS|?ka&@WJZIRv`j3p*JN$cARfTlj~Z${DM3MT~03g(noH)u{5yEeGeyfo0#02PAxaZ!gMcrYOgn`J0YmmDIy(HgDMTEM{y)xrPT!$HP|FQU{|*zQ`dduw{|Y4fk4D9?{}~lCG5@2mGI#%bL~K!Y+j5;A-uqdX zU;`}!6e*)G7hY|2DTJSGrL9Ch&?-l>q%KL~k5qN}8u>f%yivTJ_Kz4-+@W9jyC4sEiL}3>g>|oY@zBj2bKlvo3ws>yh!Jvki zbUCA}5M<&pCh@+7h*~8`4fBjf|1ZqRihFMiTM4z*dnZfz**?b&WS zFFHpE*RaH7ri6^2ngv1&LG%|LU0$STQflmH_vOf9pQlX*9e_w1s^$BhuJ^Zk^}_Jd z_Gwb_4Shg-3(g+s^^KId;8<1smn~ME!HiC(np9oJ^;O@wOSes=o*APg{sP4ZY$JRE zX?zdF66ZVAJ)#Ft#JR``F2uos%8;QWVsq@wQH4*X{`e=(+9YaCnTEXR?HKXcGI1R+ zKcH$qYPZOOFA+95I(N6HhE4!}k$0BKtUrTyv)R>nW$wNsaH9#gkcC%oXnUsZ_g#Xl z4Dp~|sndH5GjAhYDRs$m_?%PJ*hU`vtt9mwn{H6{^ai~_R?nKHcrd%a5PCf@dgLdl zobD{Iom;1eSS8u!^LGt4x%Qa6_-N?a{M49011~c28zQHQxgk1jHQ>39A&p@_N79}3IK`VahWZ1levtp8oixv3`U z@JGzqb+4+o0hOfhb~eiNlelS)Qkq(J4|@&@j4*}KI$jT8B>c;Z4}cgDK0Zl8nrkuA zm>acy`=_6BuUQnMhH`9JJS%pRlVlnnhvz1r5veA=EOdBjPILP)JqmMW9!qV zauangwl$TQMZim|zOBvR0f;fYnn)8z# z)@Q}e!jz9|SF8FRQB}(i&-WVsnJ;{p=Ylp)fR*M@(Y9?jo-VJe^JktPPr^5`N5E}~Rv?4I5X#vv zpiJ2$qP4!km6Sa!3yGI@(wKF@v`C_p;{td#NxoYm@j`RW4ORoj>>E};iY|vD#sKGa zn!>u~=3Xy5D{WlH2QPVay4={hyM`k&I`h^doX&kSaHlT#MmfvTT~pFj>t$-dR_$6} z^O&z*l=ZmnrJ#v-C(=PdinQQK_slh*k@?%<K<{V zZ8b4VYRFld*cq3!w&bFgg57!cmYtjB@M7uX!p27LUC=h#XR?u>E}bCx+JI!*)wWO> zy7i&<>eO19>D&|~M9b;g%N9G|y$n{9)c$N~v~l@$@aQ7Xt_4B)$*EnAYVEt}n@pwg z;3E=tp=?T#<^%VBqxMtxjGd>-yJhoX`rFoH^7^ltjoPZZ=lxG9?L+x@GaJd@meT*B zy8K5MQk;J-r2qKe1busZeYd|OFr$0OByWUm*?BW$5Cm6*g6t-w@FV`sRY~T@& zW059>h+|k_%{bb2nV{l>pkW)@whZpgKCU&5(!1?p8tnExx@A1LwfbpzfrI>%dCKw) zu$C=v;J#01Cjc6zkIN9lDcK4?+=q%8v?HEwx^^c(@_ykfK#IyDhGhR@&Zw@2jN)I5 zX`X<*nq^Fb$w_T|zkY|Py#q54HmAM2RO(Z_He9~I&sluB8$`uIob$^)vPcBvE0{Tj z9DiE4x}FCUiGJt_F>?31F?ij@;7vMvq?0G0S5G?}`(^}Y4X@Q!4)@%kOr}PDiY>w zY^Ks?hGk<>#Q0JAKt12eQQ#e=rUvD=A()J<%!EciV7Gc!sp26s-;mbw2L@9;P;l2CDqGZ?JJk zH%TJS4$WlAJM?o}Uon49;YH3z?DQ^=O4i@o!MX2r=$u~x>ys$<8eO||JdAU!a|>eU zJXgi3{xx60wx(RoOVk{5%2oJIXpg~9vZ2U`B!Q}4!B1@t%5U{w{}6n(a5bV}0RaGD zg8jQ(Bmdi6`=1}c|LAo4KO_VHxLQMfD=PziL(9L*xJwP|Khq21SGBI+$g^L1Ii|Qx zHPfkiaa|?x)L<={!ukd=F07CMs#Uxm|6J4BsZaMHl(=u9&c%v!4Tm&o^2p%c>ZU0g zRP3d5JIgLe$$%)ezi~?I1UqazZC^wnsJes}$*ckq4{Ff?EzAiaURg~byT5r$ zJY+q75rZkNI3lL@QU39ngygqWT~M-o!5j+uA=aIP8~^sf3K7(W_A#5GOu+()i%tbI zfwXu$QPlFb_9Hjy5c_0^9kw>&{uN4B?-j|*EozMeMoG}lJ=j%;0$baOCk$;Az{loW zW^HZ+DLXhwBE~LTV2)w5Adrn@o4u8Q!ZMj8^iMk;np1;mqF-ml%*51^TwvgU9JRSE zd+|}nAH3V0zqTWNaNCdfU(abJZsJ=;DJ8v}v3jR{V_PCRQ2fg9@eytn#`0w<7T18I z=C5Ibr-SIh^NB%^e06`J|DX?Ho;~f5D@H*$htQr9byN0S>+^L+YwUhBcv0-t3>Qym zyeni=)*bqVEe8iRatVbcIQ)>MUn*({S4n0U>tOcA2Jl#yh(E%3^x4U`YOW(m(jzq5K32`XgzXk#^ zTwB4ZRK>ESw|*>?8h5jm@yH$%#l$?;C% zD^u3gAvSxe@Z+Ln)QFQOIs61sm|8gZOBfK1Pb(fd3X_MpNDN?K4;nK{j>{mH;WDZp zi>bU?_*QLBtM78n_uiSy~_ zSCxL)?nM2N9NSd19b$gEyhj4V?GlAhnrc3+bz{pSp2o&9;y$8`UkI>#1O7Zr`B)E&lSvSv{l;^B^gv%7qpQDRjpc7V6M~! zB|i34Wnn?oig7gsPCx)}hr4>KckePy*X%Lwq-JM&@CikP8^%tpmY4SEw2vZK2y(j6 z1Z}0bSjrH=MvfK5I@s^Vb@9r42$B^vaztqEG-PptU#NRRmY1?9mEmRJ%ixkTe~qxN z)Qv5+QkhPK;kz@{uK&8TR+iOD+Yw4^?YlRpRsPi-tz&#_^ji&7hfz^cW@zNpWi}Ts zC)y+lx{#_vYlHVfOHE&;yf9QjgChRUpq7za&#bF_+pzF~r6ul0Np5jAeZ{H7?ndgU z8J+4OH<`i>2x*0E!p@_m18HkyuXCL~ zN5SN{dbCNG|N4z=xyRC0r`alz_Y3mp*7yE%FWx1tE89S<3qZkr&t9<*UeExT}Dz*jDvQ#@h$c?3BWZ*7!#(h#k@|1ZZlKNaQg%T9G63 zpsH5B{54`W+Gt9Vj4Ap|nPr7MuUr)LYIF?vsDqZGQwvKpoo#hTEB}+;&JWjqsZL;K zge+_cRrVwkqL^QH4w#iiwFAqwP0P?pVe`67*zEnZkYWeuCx_9Dr;e;Kzm#-FDo^s) zW@0`gtON8#lB&c=EDOSWh12Si=7XcGSVrG06b_HB#7eUa20wp@nH@9ze*n$;MXi01g*O<r4W?i&H z8L@~vKS~&fFBhOpRFPMGT+mXi2nzT*eRKT0P<#+<*G#CE=G?GUnS9@ni@Bl3Z=Ulc!%7C z##b<#QI(Jw*eN+()_OqG%eDgx*^1BU7kFy809KCc+|-gV0&MHQP@b8aBuAw!0p5T@ z90-#i^DxR@%cWmr&Ci`lB9THJ4DrbS8E))X!zE=VM#m%fWJzi`%A)xG~+5;XF1ceDmVnkyC?-;GLJbR9!q$kOoliA)KDSNbZM_-?t$Uc5&?rv9u z*;eKKQ{-|(hwZs0t^K^nxQ+iXO6;wCN^QRhHN&8Wc>R-7m2!Md6(<2lda|=E;hW1D zX%!1I;B&Nc&u)53&rDqyW@ zwZ+(iOxr|Ru#d<(?UAMdrdnCco3ER6)3X1E zyLW63wd>M$W81cE+qRvo*tTukwr$(CZLK(2$zD(Qmv?vV=k8tge(L@Q*HtwK=9uF+ z&kv3A(#o^=&+SKvbOEo_F;Apma~oFsvL4);5xvonmPbPks)ihzfwHR%L6fp?yIn=K0G)0|AX^Jv^m!O>- z)g;+Wd7erB`xH@mRBNGrb34ivSc}sy#x7{3SH|BW#fF>=h2MBf!6dExktbDM8TFO- zbCB%ReKQd_;VmG?e?+Qd1acb-jUMl(0$RLv`RsR;7GdJpd8b(M;3vXB4lylj9y9Dg z@u}m2tZJ<<-C&=$=4^i~rhK{E&GWNow;KLdv3l>Tr05yxt1%m-|GCOg-;zgqsLaJ?xQ;t6+5tbodTd9jci$+y9?DzW*8Vyn8&aWL30HBHCpW5Mn|6K6@ zt3mu9-4XwbJO4i>dH?al#uop6e^x_V{+bQJcdo8}P`a>IZ3mzSCjvZDpeM(PDrYGS zig1b~X?TCQ;^w;Xr?;0xI@>s)Fr=3SJ{)t-+X)}raezA5VYgBZU^H9}79FaERn`M6 zT@f^pp+n$U&2g2gUycaEv1r3u{x;RkSk_(;#d^FIJRGQ)-|LVA`l$@$fw}&)TlFhY z9Tr&B0~g*OYXDWA^Gfi0J@?C#5(FI>6I4lZj)aY7kB$c+C{A+805&sl8PWOB0YRtp z2wJSTyRe@rVm5yHGG25IjbIwq>)ECqBau2z=4;8=jS5wfWh50FjGZ_LUogen@&Q6O z10yUEuBP#k+g_ZE&*M6YoX_JjY47IC7+?k!RmrCW)w_11m77z@8#d4(QIKu1 zIX)B=0`RPS;GS>|EO3RlJkBqTvAH(4_{yB<8nZfOBu%~Zi%W~Qt`*`{<;bpi&(71x zQ|&x1vuZPI2-PLBPotpB)|Ml$UH2WXEaWIW<9K3b3yvZnCu}_wzxqe)%E^w5t2sBO z9SAW-A|%wfFk91>wh<-rD`e|Vjly+g3uir*WzpiD@O{GAGZE9%8T2LZ`7I-mklES! z=u7W=x?hhgoCKb*T+vN+g(2#t#(o62Q%P)l+jJjbvg(JqnZGn2V}?;HT$UPX*?>VY z$vS!|qSw7%hx@9I6_g`x!X#by4Kp0#HRdow!F!S!Y8a=r*zyQefVkTD&ZnmJ)D+LF z33Dn^O(cX15~~3x*kl2p)L@PjlPOt(wrnp|m-hb3?*#KDdIeDcR3Itn!yQisY#&=L zVH;nCY@W()o^owx)!6T?t}h9;#XJ20d570J{}FS)6O)K}44ATJw5hpUU0Li!!qu#fqUzZouvEk&~bUtaw#e%^xX&S(6t{| z>+J`xc53cbnZ?GT-VoEysn%!Xcu#UmS_cVmnDQJ_SnFtmE8(x~B({T3Zxw7`dwlgd zP793*YGTbE395*o1u?d%)_kk3c!+P{|?u;RpjdR%4Dkc;b~4^BaEN$yT&FPw9+6L*t<;5G}FD0VRT z{SdnWM!u1c{S~Iv6kmY4{r+Qiu#Tmnz5aJj$Bgt($&&1!X9xd~JN`%K$$zPw{?|Nd zX6S6<_U}h^H@sF(TVjcO&(vh|s-Q9y!NDY3iaD)n2_+hu+DarF52+_c0mt!yus|$$ zi5XR&f}h+kak_4E0OnaA8pc~DA5y^m*c=xHByymgcG%Zi?srX%V^Vl)p&Zh2tDcq08FEoPJ-NNDjz z3Y}4+`5rlFo834#xcJ)78-FML@q?MApM`tTb1}(iHcf^(S}5>IFEjt$Mlhbo)un;1OIXoLt)^G^+RtaZhzZ=-ov!nuh+eNW1*by$^Ov1|zGzbHB~wS$^PmBl}|T zDM9(!*WnJL9AAlPns4}k-Tk7{bH13O=EHUrFrGXb7(Ut6@XmJVccQ6JIC`h!27cJ6(ay1&|X)Q1}^pM%Xmx;XWh$JX1CtVk)MG z00^Ilk8|7O>%qvu%h`ocj}XR~$3dVLlmNnHFGkE42fm&m2qaPL4&F;o^1U_;Nh&Kr zY{>tZO`)U->xcrLPs+6?hpyd+8*B)WJs2lZ_4u4V#xD4#$Orf=AUpk;Ltx)`pZs8c zJpJB(#-888H=prVBEA6A>_O*)=|s8?fg05`#m}gZ!{N-X9hxoxO8sOl`BJdCZ_uJ`Lq^ zC-`lZ@gf39@i<}i-Sr?PG;YFj=~2;*+3AkcZ(x?7Vrw2}?V)Nr9DY47{rEQ>T_Qg9 z%t=)s)rJsQCNH(U-O)jXcu~AFrzBtHfSp_uyH_smVAl4RGoW+|alVKVz;#%&^MNwz zS-IHL4WK!t1JwOehcNR>zl2%>vu?Qz7B&hao%$%`m$;%d z`i+q&U{_DqMn8Ms*;u#>>P!v_AA7lzIsLmkd>kHh4&(&f51z2tB*GBNVvDxma2ksR zXifS4bM37|t8v2S#|@pblP2b@ARnE<1{oly)t(%m-P^4uEw)qD@SMhiaV`Y>ms6)X zT+Z{p$gdazzG>r|s{Z@F<07ZwYaxPF0e|j3-$&I0A7{@$#ejP#s%CtC3<0(1Mltfw zjx>C6wH)b1ifEtdu#GiS3+RNklF4^TDv9m%LWUvL>WHngvmx(&4&vFc@8g12enT4^ z2EGW;Mw)Wg#wl+FzH|%)!&Dh0FsGW=Jzh9(I*y<|4yoeTl@r3MBag%Z!Z>uUOsYC% z`FWhy+0&-bWREZ}gb_whHq2+l;4APEhY3ez!H3*W{(U_jDS}D~O?*0Y)xSkSH1k-c z1`&`@oB8T==A4KppvDjSSM1?>h!?1&AXidGX;Z#yBIi_RU#|dv03+21k2ks< zZtOt^VGavZ&hh)#3hllR!Q2(hbh6Ag;8x0D;bg)}Y$z)9jr0kE<{e=h? zT8O1FE+zb}TS`%(7c55Os=2qjy{LbFUl9H+Kz}V(5d~neZbDJQc{r$P$O+oVX|F(w zG@X{b|BbPkF1Qv)3VzI961hvPHaUhL&|3|Ifo@n;(Rg2sxT(36uKRH$t$$)9M&RsT zeJ`{+J1NPmKGKe_$UJR1?WoC>@=HQvRt6k~w(>-zLRW-uUt`{BISia8`B9Y1_ny*$XUTnlm%crkeuzJC z9+*#04+|Aor89!Mw;E{i!ns$W3JZG?kLyecfV*={b=jxQ9199WWlX%0cOk>f`<4@7 zwnS8%C3!Lw+EO~=l!{QBV=zQzo(BH!&odOxif`Uf8Iy96Kd ztua8Fe>jk@oTopn-Kt(|f$=nV!iwZ1IP-fvZD~AR( zPyid)68?-|APJ{F{#O+!86C%nHWDC{i5)-F(U-WbeC0|}>-!^|?%_w0YGr9e(g)W} z($s@si{#qmfqWH-Kag)GeUW{iKhcklVO`w2?1r(~cnU7rRP?o(ML4^%@U|upfTuu} zn!fK`7O0{Y^izQetH|KA)TSwTOIj$#=E}OIKW+R#G|t621qQFSmenDsoHZ7K*yv*< z>iZcj(mav|4g&+MJ%e!EoCZUoh^6&0skG!I;&8@N{3u!4Xet70H(kg9k+k_m)9S_u z>J!>mkq5*i2IYGJOoHWsG!SUUV7&16P?I^>m?RLx+Fr%?6XKw5&T@1RC!At+eFMtZ zpOZ4X80Q~oBAcHxkDz$*-~Dk6hjk2J)SR&>Du}pU;qb?Jgz*qusBL32y}*vHw5~3x zoMXKc4D8qjye4vlUnbN%PUhaPnBuCvyepMS6U<{V?#5Kq{yb)EINaQ{Lp5PYikL{( z#6#LJETRt%`E_knUWV1_UhWVO{gShb9vJuMkTnwNpldj5)Q5|}RM+6qow@-ahJxe# zq_*W`W*=`7pjDJl$Y4T4-k7^yg+48+A}C{o5l9VR^y0-K>n7A(i{}*RJ)NpnmggGT z^%i7oURkpsB{`pw!&kmSJ)43~8?UcOm#UJ!&*K`(JnmEVa7&Rr_FBQKyH%+$or&$q zPG|>sDKaIDHH2$u8RxJC@50T>KFCUOGbXIoq`V$fo9#1C>J^BMG@x+3XjS3Te;8>T z5z74RaYIiyzn8h<3JNahCyWE3Ig%~WB(P)avC!xj)QRE`y(bP8M}R(7+1` zn>aGHnals68qq#ycK!gad=#))!+X;l=tZbPH+hQ7@9L#U?*Ma&5IBVR^sSW|=+jKF zefqe}8c{gsMy~fzRS-g&C!;79lHj7EE)F0p=wBYKOjNfUmR)9miO|!qmj@+tb27s&QnZ=A zR$!}%B2~I1*Giws{ASyVuu>td4M&WPxH`P2juRDAO!%0%$3OJQMukOhZyrn-^f`&; zq%%SmEN27QLOE+o3-ruUZzG1>o!BraDgsmKIP%Iar=pq}M+dmH1FUb2nj8+{>bmSL zRf~dh&hPqA7uDOQ64Y9#*!ErQAbR2nMZk zoL1#6U<4Rh5*!Z=bZRJ;No@F^Cbui|hqj2;R09D8?n{MZ8VK|{^O{>2-0>P2w~<~G ztK)zNTx0F*qQm@iVtDsr`7&Z@+f@2E=G&15=1wibO1c=BtrpN*KI@X%8&&SX$rP9v zMN_V`0v0DJuMO57x}bUI5+J?FSTQ2#FTYv4IiD8grLc~3SfV#(0zqH|yw@43fSIX1 zwyt)YwiwbA%UVVn*~M1tb-h)lWQ=jNtmVS_2UKWIZH7b}6*u8D)UnG;5ss*ymse6> z{WO09U%Fc4FgH_i{PMn%`(~v4E(l3(!3v{=Y1}y8-9&LJf{wRdEIpezojL1YFun~4ZWNv;UwnXx#F9~^2Eq8?|ZyTswihB{n=_KOA)A)ACxC3@@ed#UX zcH{#l^08+|fci_+^2|ZHtwr;7`^s#6CA7&1I(?@E-~2-@c$OhU$V z6OH81I;xlfF7CcU*}!%usvV$Ut5y3-mNQi!^ydA}*_}R`G5ypdJR0#k+tgns4K=J# z7qML<*>_8FOZrI4GXdwG~eWed^v^C8;d!C)o*1soKn_gVl z8m0K#>O`2!EUjQ!E3Q*}VZ54bl%*rp21?Dcs0*K(lw_~=^|vc{0B`US4Y6eNm$@kCWce0K|t)CDiy%q!io7B_Eq} zbh&B)0u}evv<*R!Wq~ZJ>2U)eGrH`yZ$UU$JmboD_Ps4`murqIV^v%}bpjRu;E5QXUfTwhzkH)B%9uKY5 z{50b1z*ReJwhmDx!bz}E$hj~^C;(Mbs2-^nu2y3~uLlPLdJT6&Q)`kxOG zEk0u=ojWcO6B`^ay#>O;-vE3r{hqiQ$~1)Ym)orv@$?R-2HI5l5R2|S8P-bNF-h|S z%naj&nO3ta9PS!$RBL0fszJ%MlmZ;_j4K&Ixc=Zi@vHJeCp?AbYbu$@A@9a4 zZ+xm5=SBmNbOXI1omC}LBX`fLcNjFcL(GEm*K_sM z^^MKjil0!NP1nN4)1tvG3&4qU-2?<=8mIHTJnf9{)#Xdp6;Uf}%MEEu9_#9^b+zo5 zMS0JP4mReNf>&3A>*$>dpW2B_g`O2(nkCzBcVR=v3QOi7_R9v+UCYSC+)KqeiYx2z zTTZxR&7IN%81cO z#^ET#K5v+o47Tmz;R0R#1v*aFW0=^(oveg`8!AU#SxW;?P~zWu4H7;zz>AZc3cMV? zq-_l&5m&4iv-1m{%eDUJ#~A>Md6DUvFzYL#$vObqZUG#BjU`%&^#OToZlRZ&C?6iWUTKC?$`N0GiDS#@kJSneEjM%U zrI{d5g!X8)GXUiAhhvx&t+*RgI%codrq~=}<{e@>JHz1FqKTrce)i;bIg0>;8D5`8 zemZYshpyrc60e3=GgazwWyeVy#s=O*L6TmPIiH%&qG7oRsBbC)by}-VwBf8M`gb`4 zdq$KwHcxko{o%|=HyRh#&CJ1#%xnqlz}6??DH)d5ymQOmmx~ovY4E=5R^-dhiSnX) z4C2jHcWvFF(H-Eqf6V8Zpnx0fdU)WE_B+Z?)TOgEXN(?07q)B{c@fBN;<1+|=NL?~ z6#a1NOC%fQChcTMryI2`o{e(JEaj%Fex$L6I$v(0&=&RQQHN)bN*)0fhAVZx3>N0` zRGM9IwD}PQ3W=d6OqH(i*snyju7oxS6i;+JAWVGaKzhN z=-n>LA;V!b-p75`SZ+Mnkj8cMYzJ|-UpsOJ0_wcycXM1)5olsC7u%=%gWEcsdV|r% zEWNn*^K!b3556P-6ljd<`9EVGru zk=a=RNVCY$+hoD-^H@zypjj6i3vTzK*UNXljr=*h3Ng_~5JAfoB^L@*0($POJclT& zoGL+!c|gSVx+^)=1qKhRkvq}Gxl`r5D@i$+XxjKukLQsp(~y)5eRUkc{j5X50YMYJ zUI=^8&UVYvzs2%obrfmd>+w8#!Y+F3?Hd#qwSKU$*nc{}s>Wyv{Z_qBay{lWw<+H0#VQ3DxF3e=W6hWL!@YQ{0{MvQKCDm#nSsQLZgY#(lph0L z#<+wna=c&$>L(Q?auAC$_$$Y}g4$}lW`yG_XKkYSmPW8AIcgPElzO*EXawwWP)}0U zVy?1eg=$~KB>$2)L=&s#=;U{Be6)Gqq3QUB!FL!?Lr7|`6gMa z9XUbCW!}5(}0{|>V-mKk3aB2 z&rwf88e4y2Jn~AIN*>MwV9!4)a8eAsECDBOH4!kXVcTtzyFi#jf#CVIDgF7v?UVQI zLddZuag9>7WlFUHZ$<9M^@*ku#`w3bz^s}3bE=TxORb2W6fW}dLY~$*DHQ6qMr8I^ zJ;?Z}55R?08afrmD$icE$1eSCul#@t-!2Uw9akud`MrZCW|wG&ELBn_NQDi3&M?02 zL;c7?*NtkWWpV*mU~MzNejQ?k$`&?;it{U~#TVr+l(G62`oN=M>Kld57z&euka{fbp1eF)*ZI>P|2subVr+ZoSz)^Yp$GtctLZvy zl%WK6uGZDxvcN4B?pR%N^N_oU4=2V)RR<$?kn)qj-h6VFUddN+XeN0(YZ8`5 z6%h`9p1#-W$yfkQg~I&#Ma2xFd!Ayk0u(_3M7jX-&C$vc`Lf2MLYW}MQ!5gx&c2o| z9a&4INIR(bzNKsN1ZqzQzHA*;0Te2?k&`UfHL{G>VOthOhYD>%riQZy>TvR28i&ka z=(yU2&`I2Z{Hr&_kN<2BbDKcBZi28hcHVc$?qstiGK*Nkp-iN9O}8t4CKuDu5^Kks zGz!H`0jSLbd{J5kyeY2&KZv7;n*gbc*r3LkS8kP~mPDeynkZxhzU<4lriZ@hKYSr5 z*DU{?#4Ac2^Te;9YQjXRY_yOVDzY6vi9y(G1o~iS!dt?Tm02X37 zNwt-n>PSha(dCZf*V_tBJ0`7G(8X`nC#KT|6gkCIHNQ z;?C^xXfi3C&(`I!z$5Rz%aC}az121}EwOWb46oVL_AVT-zIStEHO-CsAe<@ODI5_myjDVgw@yF^DZr5l@vmQZ!`e$Fx5NT|nTz;VY1XKr|BR_A&*luRb; z4`2k8Ov(GaTl&rEfUVjTqj@U!EV$)78Z7MBl60h>&6q+r*=UFn$qd&3krg~Zvm*}} zizv$8*73+Gg=@<)s;o9_YvHlU(bbL=g}8wN|2wXZ06s4;50mUT1a%};eRgD4JXf%A z1%96Fc1UcZ#SxG*a~zvYcXK~mri+q}oj~5mkj`5WT+~e$QcI>AGRBr?Fe?FP%=s%*Zg#L1>i(R@fl6Cqt$NOMd>=d8Ai+qb zC2*yTwaOhXBz_`;`IK7*HX^+C5o`G;TX#aEG%R| z7JUN>2!@R{F|XuX4tGtxIPcOZPSi+09{Wt)G5Ao-MH9L3Zb96xq0=`KdUxBN28BUa zDi5zSzRJwC96*;+`XObWLu69*T9q2d^!ACWlxq{pmxv>T%(g^1COHtKkPbyMv;9jI zDFXUBoFGsF1>Qx&+#4WreJ`98lvtJoayl%YL)~PJ%9YYi+nHDN!z3$>(54~j?-v*= zhz&d8bPhX$r-`Y$#*nDw7OYJ^MYMn=W>91H%Yi~!>L{hNx0pjz?e5kv!^E$9_9Hj)rR;Z#pxVUSap7I&v2z9HKMh|jyIXRH60S8r648ZAL7IxFH}SHC@%>P;9?u)g3h zT8^WDKBjzn+Ry>>#h8n-R|)|gOg4e?D3_O2V<|~b)i+k4p)R{jvIXh@!m@~>ZjGX3 zZH*=zqXA40H*bqSDB9@{Jk)amA~<@Jz!?-ctqfe~SeeW(un17m8!j^H2ZXmq_F+xV zMaFi5(qjuUCU{oTf>qx1Ui8YH$?s zf!8P|w2F9k8<2)ny2IX^0UvNXN$)Ow$q`y9`5cR((GXrdX43J|*!LA)VMRC7wh#F- z@NdExHDCxDw`O1fS+)j2%M%LuPn1)805dcg9{@nC{Gaw3sQ!7c;lFw7{~x{E@L!Pe z|JVUDH?g+=cSOWW>VWMQ15D_h2NY2cg{VQkXaEqXBSt7%k9F5I9Ru4DMGCE8Vp6-j zzND)Xw`?m;>*$K?qI~3+jFcXRD!^IPq-rP!xT3iUj!KDys+bOxI9N#z&}>a&3h^$5 zSkVGe`sIgh3P}Ct)JQ_4@MA51ac1mK)uDlpDgkvvBMMq{G zg^Tqg0}wykAZ=X6&C6@%3k(}zb9|p&sGxSbVD9m;@8W5L!zdZAWa(l#Bq77593irr zbhVlM-)%0d1+7QTn7ke|708Mw>iLg9c;tm3F&EIM%K(;Ym|TTtd}j_*ndJFAUk8rql;y+C~p5e&~^mkzX=)qIf51? z&AGQNU~S$HB4z1c&Wu(`S6sct{UNBqMJ~z87W~Nl6PY%^xtX+rL-!?p=zHe<^!)zl z{fl=D?z7ChXKZaP+`WY^Q7?{cM~+W-e_=@dk31I2y!ppZl^aCf0qnmpKDJl&z{kIG zq_!~s6h{A^i2iSn)&IH}vbJz?{&z7nqor%N#fIYZT4%6=-r)kG8LK^=9`;OprG=tf z@9G5!6VOoitD~`8NyX^rGa=bn8Kt7#f^1E~AmJ77@%~;fvhajM?hY1o+8{t8aCZ;~ zRv%7b|FiGOv+pRo1|lclN@fGfX#Ls~4xEN^BcDO%l%@Y&ySu8J@4nMbbOj&< zJuUqZ)*dVaNRHCbSnd$Yy@UIQpdURMi7Q^zqe1|ruX@#rqOuZg8}BeL+QCF<{ZJN5 zIDCoan0JQo=-1pm(x7!p6KIFDJ0qCHXOEs8ptLh^bX78=o7`jYGkGH|l+G-OZi=v5 z5uX-PBoPW}_~9{_)E%asG2Ri4aGj&rZu~JC#phr}M@n@rMb7|cGHnzcv%ycXE!ab2 zb@G$Pg7m^NA6%QhTP>MErRCkN&v2O%^^W{-Gb1Zq91YSe^z9NoOBml;>ER^6QkzZ- zn6B#LU^(oFb`pgmC-lmW))J<&@sgK+4T3Oc`B>Q{^0qF@Bn535cP>^$wkX_&{Q3Dl zp>SFAkQ&RA;_rCehX|D4W2E-45b(VxJOMf-NxeP@U(~-?@|cC5AH4CWHv8AwurdGW z%@(eEiBj%Lep8i=;Z2Ya>fK?C&qg+%Kxe@t`Ad7B-969Ja zYnjGij94s{bSa?sL_X-HUhTmTe~zVMe5|9LAYC{l5NNjlo%Pv^Ne4{430+;@YLy^) zpc5@`V<4#yegc!!l2+~k?RPYRhg;z)*cbHw-M0bDG5zy1R}M%)F}W4A?4Xdyy^I45 zhqb@&ey`bd#k}!~_k;2$;!J;mb=n~p(5Q++_zRx611O;c!!~)__yvyLy=zTOXRm68 zjH{f!mRi6n7vz;3iX{3m3rAk)CBHbKRe&`YKTGinsxo~JHN|9!*p~joPwz!_4EnRP zx$er<3nXqC0S^~-h?q58+=qL{%-r}_d1faW`(e+NP*OHSja8{K<=IxsZ+Rg&-aM=# zxn+D@8~#qc%Z?Z8%CK}bvofS|oxq}T5UQ+xnKNti{UQ}t0SSGeJ-|;KaVPiTn!$D# zL+q-_@!G2XUMxAWCgA4Ank=>^IV6P(6_nehia?1Rs_>Xx{)d_;%U0zDZfLEyicvU- zONSSyw`KZ01B$fsRJhVfuaFoJ7haBJ@(VP}+ML4tLM{L|R&yp5@ww)@- z0&`dEFePj4Hy=qZ{Vn?0QcnhE|C{Ne(w&zu`_xmpTy)=nd| zr&!JnlXEd*qj0f#&_?&QnyK?G^|0k#QlZ5)PVz`s|F%}BxorAm>A~1%1|@0jrmCh&6Yy~&LX!!j&x!42wZNjXC^6g?z<>cg zw5FgHY(h>dsoIJ}Cj~nE2r^3dB+!^)eKJ>o*$A}Fv^sA($j9gbwXI*fuVk4W-<6Bf z_hISPM`(o~KXzyo9gvp1yp_K)IWRMHVXJ;P9LWpOR3KjU-1G9*3uA`fb2chay; zA)mRq`!ojJy7R)Qdg=2+$8v7}WE~rsZvzRjbdTJnSLC$n#@jiF-kY$SSUcn;Zv}df z;+ZT2g^AoFyeqaf3i1{|Un93vmK+732Qxp9qNOqNE`ure_6lJipW--}akAi@V-8bI zY8jYc5Ink~7;t`7Kl&3&mupPZr@MJrZ*9HV=%M_TD}&AbT!yx$kDa?pw&c0iidHt} z2{y)#mA^+dw+VBbXN1oe5^909>qbSwv!eXss94b>`^sx3549Vb5|!!hL9SKEAagI) zZ9iV(I%pqML;{lo)a;ibzUvJ1tcWA{L3sGw@*9CWQ~rT)W9}>W;Z0YyKdNI zCHB*}&ORUh`j9#uam3QGKmcTO3*{p*OnEz}xM3{CN8E$V%`;%IE^L_II_OM~alzWZ^BHc1dy{D@J_F zA-KEQ*gc~3ME3?NueG#%aSmur-=={liZkUrbh=T1EZf>UUNbYU@&lo;EOAo`2~$$9 z^uDNsOy1EX;{sZ);K#1uM>av{@%7=JLVY>6gzMrj6V(g)gyena|~LEhub{to`DiwOJFtKa`=T+)ZR8 zyy4#FDEF;8YopCGLhFWa@t3GRw@Dp>2WRsFm^}OY0aJ02K&P`(rrsRLc~`!4m(&MA zkV3g<0Xm`Ad(X`5uNm+9T<^z6h$JT%+1aq1Y}ii>$pH@rcI&y*liyeyf+wt9$^Nwk zqM{juTT(p^uOKF{%LzaxNu?EL4mBr2G?iXvqL-#K^{OwdG@06Fo8va-j$jkqS#L=i z`ZKYaLIpN&4OxYn48p&W^!13crxCtG-r;aUCTTzeMfB0nt2@k}7l|xKvg|lf)s3;?mY-wfOay_a?)#F^^lk z%8|e2=6zO2QG44U?$>#Sx^>z{nCnfZ7>W%O_h(9mo36U<39 z@L#$aaRGrBb=vn|y-w1@%~JtxPZhanoC5^evgyMo=Y!3s}1wG-R!c zrIA`SJsS;*u{eKbj8==g?5b9}W2%l@pUdyxW4)tqb7GQ+>L!Np;}t>^W+vN8H{U68 zaV2^^N%(Fj@hscXb;-*!f--|u3N+kXc8gHj$O zcMN>7m^?)rL&5ib&_{9;FrS90?9RqSJZc8ep-XU+`ppe(Y<%{!0L1wgN}PM3F-y+1 z929?#f{?o{KK7WM#rWGP;LCaJ!R>%+BGT>vRR8%6jxQ&WS|nDIFnDYF@BvC$6Z1*_ zUxj}OOP<2!Z}2`t|6c^}KOE@)b;0Xw?ey>N@Fi;6|GLBbuGJANhmEI~k_vVKfwIuJ zG_eY#aw4li6DO^|}DWzI)Bfk|Q` zv$k!i#+er*GPgxbaV})Z3~CXg?Yv4dB{%1{VWoF-L)hc2x1}~TX*Hol+d_ zM1kv42%y`M-mD?-CeDkM+qy$2{AZP}XhOe^{^Dc<-^|&O?Gpoe$}!(nvqPp1)mO^j z5d~zgpNvaRPHr#X+COlZ%M>R>TrC-?JSlZ^bK3l=meKC&V)#fj?@IRvn)ir-?BKO~ zxQ{nb-l!#}1}H0=J?i4=vYMu3e;Tr8kbZ8-qQ%WyD$#WUq2L-w=OY`k=u}84PkcuG zrp(m_Kv^XLN!4khn$%1)H1z5ns@-bELoTD2j?s*FVLWj97m-n;C8(!Bptrc1V9(b6 zJB?=|Qm!80q$bP3?_6yx*v(-5K!IaEMdarGLMbYh+6j5ei2A49RxSPaFS+@MUo|*9 zMzs`X&T0-cDh1=lK_wOA7LToCrlFiJ0gF+q$+bpEehj8)o@j=IpSscMn5tv%xlsrs~RrElTHdD(O!PZC709`?gb?Exb_QZt&Qb z&-p{mvQnq16H?HqHxTdoZkdop^@D}C*j{N*QQ@l`$+2P&`;O3T-lGSTNk;X#6O%J9l5%3b#)Y(f9#n&0f)lWkFm83l&v2|&)AtgoxVSBMrPi_ z(rSN`8ko?xxa=B;q-Q|sb&{}7nOA9CmcCdIXS~X8a(c~um8O2@y$eT&LV?rh^crI< z%6Z91$*i`pX?iaIm4GK+WNk%j+iq1Ye$i>eBa@Aj<1%iy8lWC*oHke*0Q@WNU=T|m zQ5J^^kgbX~jui_N&qe2J?^4jf&7m8#SHP1FCtO?e@-!YM>v>&dNc0MeCFX~qo%vWcG{+$qewbg(+_e+Hbz=huv14xRlQAcR!;S_&+aC*9H)p3fG$5P%z% z_zl5fT@xxs@OM%IM-Mnjy&x3X$lg9azgm$DWyVwsgo=gU2V{&r;~jb`|HMhlw+eO#ig4h zQHBBSkl}+vThe2&GFC4E%aPNKA{Ee9Ss9*A7Lr z7`V(Qto@+Lx?eRj>e#@0jTjUE(Za0h>Rqk{2j_9XP*SJQxX9;R^cR_!7L@6Kip)l{v#P~&^ zEY=dkqw^c&!Mj58|MX)0D-ooW{ntE-;U6cq|4pOz5BcwZbgus|Mr<}FHg=94`qn0f z|1Tk^|5L{DM#si!vlHn%SHFNzf139%n{2T(ZG3vFv{*_{hB)^ZMzO!Vjw2~I z`TlH1`YzgP@(an_+{#a)KUJ-2<(4mwBue+`m|s5rCJxumF`O?}E*IyXn|p0IZ@?z* z;BMNbHl8`_Gg#EkG}TO&VFU6`&RWwKt$jKJd2cj1u)BP*qZ>J;bYO+Yhx{IAl);2s zY1+HNZK=1xme$FL(c6rcNN^M`@QRklgBHJ8;q73XXcdN8#vJ;YX0503+D=%;mQgV` ze6#L@d<=gtZuN_p_e1)2$=^xS8Ka3t4E5W$WLjgBb4Xrm=S|2ri7dXD5$w zrMQ~|Bqs9`MY>OM;(yd>ogF=0zrLU7zaNBuhV~De-c1?Y*#?A-c+I zT36`s=gw+>MZvN4rkYoFF8n&Lz-ksbYZfFY90&FS@v6SMn(7_45yhg-Oac;mW+7gy-yM>z?@_v-QeNkKS1PvMI0 z;t>~D^kmH62WcuQ;afy$X^?)XbX-PERyws&oQ~fUVF}G^YQWkhO9vemc4Bzw3&jNRDJ`l$dU^ujd9Uc|Wrn1D03Co84mX9r|o*`qhP2BBgbKHF2iEyDN=6 zeaW!ILg89RC=Zg$*|5R~@oTWj6$C*1eEQY?BcmED?l|n&kRc2}IoUkoamIDqnwc(3 zT2YAfT&ntV8+2%!)7o-(d-ixqM11OGR1L*Om@r{w3(#r%g^_UekheLX#iGLo1MoT~ zCUhRr=P`oYmIU;x3Oh#yar-AuI^v!J>Um%V);(uozoWar3Tsm#hQJ%C5<9i(DMt&s zz>q^Gc5GUE3icimKMIW=-a@Y<1-Co;CK1d-gPhns{c=eM0Aic%5ef~~y4rk{tDx6H zbqG#M{e&a6`R@L0AFY@HK;M`PwkWh!UKBZ|&ag3f6GMnx#Vyo~PZMK%cRusCyfL(I z5pC4V4;;;4u9m=RX8^(Mx~@M!TZash_)E`KxTA2c4uMEa!= z_-x2gAjGLGgQJW?667wM&)O+oR!KaiHe!5knbuz2(>e0Jl5VdiUm*uuGG6V`cccZhxb7|Y zYQmsKn#1l=g5eM{1YTJ>%EU!#*O-@l~m`JN^O+-$LSZ&N>eJ!E( zk}CKZUD*_%sLETl)JPF`r8YF~6gm|qG9=ki;bD>HrX%3XXV5gEPh>JZYem&1gl-?QUJs{_`!U!7bs=0Mz{jf0mUzE!j4 zMBiCSe|v{-Q+Giln=^S+t260^Lo`(_$}MQgrkplPv!%?7-hr~VF0VIgT6<`Q(|T1E zO*I{N@cEoN{!FPpoSYmTNS7%CFi15j2x?jt(tHrO7%G`PLXp`PG3Rlzp*fLtrVCaQ zMOD_XT&VTe)TzLQtKYe^!aAJcnxH zj_%<&vJBST<$zNprEQ^3zs<0PWUHzabA@&nELQZ$>M~~m9U=-Nn#5TX?G1BLeOH?a zJZcsF>c_S^*7>Zx+It}Kek1xp2c%vz>rmkl6RO8af+BNoNZ$La3=i;YJTt%+=6%gF zZ)B1VpLv6yl5 zt02OOFNES26or@@hvUFF8m8eYYcwU2HxQnhq^B1>0MCrH^DKkC}Sm7pBqVZ=F zX0Q+Q&Av%Bcb8Uiqs>H*7hP-j8ZYjZ9CuN!v;2E)yyUwtN`FGCLoYpn4db5evRV7dY_o2*pVK8eY{s%8P4I` z(O!$T^rSj3>PZnHB`@08m?UYuZx@4j>(l%{)m?R1l-<)_q)|dxx~02IKnW?4PU-HJ z?v)a05EP`llu$|<2>}5?Qc7tF=|=b#-}vpS@WS_>%eB|VA8^k-=b1Az&zW=Pv`ldL zU^Nid$vGNS0f)%6{OnMhmacr!-a^hO5q-g+R>w6H(je}lbS(xM6Vs&`@d|Z~L3d|P zKx*`o37ib0gRzX>BH1KB1h3y8;>>UT(-Tn=Jle;5>K+_vVQR>Z(Z(uok|{*WWV@(7&ZSNr^DTu}cs!UC=u1zNjNN zZrSZ?(Oo>ANG?#7lgx6`j{YqR&3#FBH4nag9w{YX<^7X>{q|RpTehlr1AB5iK}?}s zHQgFF5zt)=xTxlYgE%RR=*oHV1dNvq1i9VJL|5Yr^jrMa?TPP>U8NlAw;NC*K9pLN zJpqOu$OM!vaQTc^>j(fd4hoV<$8@n*cX$F^@~+b&)P~(>hV@smxAGCpL6P>IPR`-j zBKbZ%f7Qn%lMcU>z|mG-=^<5tWZy)n0q{WbZ%Ftj@Dn3}5)H`930PztO0J z0dt*S@7?CU1GaB~xcpIweyr@AIi0`wV&mIl9Td8j!}La-iD19wuT3ItQ%q^Mzm3c9 zA`)jGMrobgYF?cc51!PCU8lRR;aXKkVMaw%vKV4L+F8k6Ih3@x_1LCjaChKo5$0OK z#Qp>ZKKR1;7t;QD&ORYx=w(_Knp^s#9~KwwXx=W|#BW$InU;UPAJuyu?(qq`MJD`H zY|^%z1t;|N6y>4ZCd<22ad&z{REBipQoc?uA4tQ{hz6d#h;N-*8$|I<1Zb%9$n?mw z^8)d;SNc_gx)3dIrmMm}Kz+}}I!XGhVt3`WL3HsVg9VpcZo8-qi8sTBU4)?}SIn>m zwZeMKPQu+I8|8EzRivU6Uwk3~1P<37ig;ly_)=tN8@4bX8K&d!uQxv2Qjg^)>L1hR zxQfnYpBpSHL*%VJC0`_b)O2rJVUWxsTY)2t=;I#l>GD^5Nq1-ic+r(Vhk!V zsm2$j-LcQOSME>A(^Wqig-s*-hVp8s`g{n6^t_p5ZZaOWV!You7lBYFwGt;ILLkz(Ku_oQfqPNnKUu`1n(GQH(bc z5#>w-qtm67n#xr~yRr{GI zO1-K`a`C2qhO!>x@-%geq# z(ujslYDJ#pz2H*$V%qoluEl<+D{a8{z9;D-BcSHIq#@?rgKkfc$K_mwO9>JdkEFG(=omNqHeSYi-I1f-&QZ@B zN9ZCg(kHrh0@>FMsc^9k6(X$<5=q4xe=J_Nk7k*ggBPzIQO=vaXLq;$Z*%^mCoctb zkE+vlRl5Inw>Eawx3;q~{>=sRvGOv~Z-p>wSBvl*?y~uFtDpq4w3nc{K48A&n>=PUYlbUp^jhG>NLU|-oDI#Nu zMf^OPLmzhkiqmzRDayT|>RTZ=MkRKNOO8zgH{ z6iO{&HZ1jol%c`rv~TX)3>IK7eQngkou5)usV0!W56cEVv5UM%ceCH$PgiDw->@4v z-;p&-G=(S7%|MPlKatAigrhq++hf~T!fkUf;VOuWdoq|?uf8?!&iN&C-Ud;eOJxd& zn9Unq8_mM)MOG;}a_rYdS$g3WU~@_{0w`fD{dUKZ_9pgMpY)g~e4N}L=etjS1e!9# z*)#?4ZpmD*xt?7lWj=G$@kJlrA8oLd7cf}%-&m}Se6`J6)@x(i*#0yg>-~AYIoHJa zjp$2(@Qu8V7Ha5z=`jbZ6k>D+X#RFH6T&x^va4-~3G?nr~%pTnLIuuD%>g$wWg@E0K1tPl}nR$YVNe6gn1rvW(0}N`{b# zJymv}??tc^jJgyt!fq#W<=Yyz!eDr9i1@$sL+R~ZxL$B$FC}ACJU1hncE@NX>OeP? z5jXCe5{}KcLQGms$|(+0{5TB6Qt9Md*FHHG74io4S{4vFZL;m(&wZ6?UwhrGT%vs4 zv{$L}z{v5(RYa7iCDDrj03ZTvPG_4E%jHbzcfwsg8v$=C*b3EJ6S&`F`w z;w>vi%^^eiO+F0A#k|j=v{_kCFbNj2fN8u-cV3c+N5?F#GTbkZK}e5LH!(3aC9niK zeyPi1y6sRtPa-GaE1y(CqBFhB7ZJDiE^BP#z|?Q&k#iT5+CuM}bXtq{BtNG$#EeP* z&D`ENlTm{eTy!y*^_60nR)3u2lD;&nI9BEmBs|@OkL{vdFx^)^x%T=;k!-W%CSiYi zIwMF9_f3xwVb{n$K(f${lR0%op;zWn9X3~O4y@v0bWQ3LgoH$eo8OohyR1U$DhS@8 z+}c=;=_HXtn30SQc-8hS|4zWCYS%Gbe1kUKbM~J7X_7^zQJsGgc7N6?@c`j(S7kGkywD~ffXTA=` zhE5I-%$<$TvELSD8Ot%&(KZ+ZPzNL^QeF>4JKc-tNZcJ>Xs3%fmW>yu zM^F_Hbc+KSkv)D$x{N!*6!TamH7ztcSYIv5%_=Po)_ovc16@Qvc|fB~ajtkUuVA%= zh5jhR>&83F)RzkU>RDec3&_%NiZ_Z+!9tTh22 z-iDQG_G|kH&`}9ZtgptOrzuZvjNA3eW*y0Pv@dUbX%u#S+i1Sc4QtH;-RbVdqDS7S*`Q%mLth)_WXMW3&-J3W99GH`^s&do zM)`afC9*9igRhTgM0SBA41AAsR#UZ^?W@cuE_&9P4TB=1S2PIUNm$yy5SqfZK2daM zb31GV_lFu!xYy>S84@vW;&5Ds-w{78yxP&QyOm}Bt-W&TJ2EBmE%AM>Cbk{DGElY@}ixt?-b2jR-v6v)b@7?P3aBM_mB-MDBqmCOe)~1i!o_7NRY&;OP*7% zXR8CXIv7q0bYRPUGR)5cayv9?2^O@uM6rDr9i+>6mPseeL88!$sbDK*6k`$Wx*gBn zT0|!AyWH(`wXjYV#6d6{|A^G=ieqFmP z;-82*P#1XuQQi(?Jz=_%DFh=cUo5EP zNEjnosPu<#C2;O=4CHLg##<1eX2bR^Rkf{nzIMLvb}*l*wMNasl*%IBCB{;9f7}k} ziSS6L_mRlt`)YD5|&FJ}WPMh|2%;LMNQK(?xAq z=Y>iNvwOAW%eiPO^0~m-o?jp85&G4t7n6f}Ls(jv5O@31*h-_Bl82;3YcF%aVZh{-6 z6XBm1a?d_Ud=c03N4rvd=p^=mv6acGCq2Cs*4SD_{Te{4=8f8^Bi)7p0&3$%&fj@(Nd@@;p`*UD17W^nK&D$%*uS$_HB?twdV7xt8sQ7Ux)u zy(uxQ=@b;B2(O*zKG!B2qfRhnCc>t{-b!!MPrl!HG+~=ghFQN(bP_?>h*cwb zl~E$3DK?c{GHMm6WVH}C%1)-Xc;z6GzW!SZLul?;jeDGx(_0 zOT^P%a=zklR5`IVQb&y=XvCRCSDbvW$Voaef=&<#k7#Juok&CyyAso_LKR`>VLSFB zH<4rPC*(SN{!eqeh1#85>W&+Jm9aL_zHO$%Z^fvO6LYB$V!jo8*-Bewi0g70l3io- zJ=*Jt5Jw*#1csWsVr^N_T1O2kQ?Ay&Z>eKPjA}x7kv&z`8%~UU!0v$k$X8}$0Ce}> zeN5F`9WNDkH)K_e)!El_5^@%^r=Lm4_<9MNEhjAxFTu3k;T<6@w5HHHx42j_MAG!L0C#85 zk|#B1S~xP=N0Y{b%wF6+G`O?!MA$AX&riPUd%ZyXv}m1q1IJ$3x;Oyci>L%(!vuN39RVdE0$)Ec=14zifL>4r#T& za2t2&tG;ReynM79$E%ZowPd{{dWJp=MeQ)IYN@vU$RJ_uumZ;ZEB0;K;0CLioMY76 z^o+H;WW^I$R7q`qG&kIYQ{0@_x?jDIS4vx*bi?OiLt8;_&hxnSxweh$-60xDY`K1% z^LR1r#A+oM|6U?7C#IJno^SSZk_DV9cCTWRjfq?E1|msWi)Cho@u8%Pn1RUl+sj(6 zOq4+>BBECOJNAR68Af+FL*Dcd4D}U83-x4`9Sp%WdbR|9l3}B6o3AO(=rtL^WN3mr zLZY~~q*{`X9-?$VOhFQv_8A%iaI}-b_Pw26WtXRTj4))kjbhl@zPj z__{Dgap*x=Pxc+1R}xso@(w=Bq5VMKFHEx~gkSS0aDY1cbFnm;jK^tnG6@rBH?8Rgh>b1ot! zN$oqQXpy-_^7Khwur4Ma%bPj}rM)84_CHqw#)3MbfgZ)WhmfM?ajI-qmZsJ7LSX^ z!MFLzwa%^jX9ZR__S)g>Epvc81}oCUH9ItO?Txs>hCq`4*;X9D+@G)v(CTgS+(fC~NsQ0hmT(;F(10B<>dOeEa>v(dY z`+4T%c`%pVu~A@;#38|S1zprI2X3A0?h_e11MI$oppB#`WTL2PBC;o0W`4JC*|{HE z`Yhq_JU9LF-G3>+lIdf%v-#ckX&wfynQI?tXt$z1KhWgh&CZX!^Rf38%8t}_uWO-{ z?x&g{be31i-?~2Fyspusc|ZVbaH|7I7O5uS*Yxl<7jC2Wwq9BEmnv%^AR#vC_p<8~ zL?_za=(eaB_->yJlG}SE=gnEuR~%~6wAY*SpQnWM^ho>gT-PLac*((;oNdykc?kdg zj${lecsq=YAB~Hl;Oj`bX=^6oeb6)S%^g8qjy}$s-95pwej(dlS7YrR?SWhEn%VId z({`Q$gmTF)B~5vs?Xl*LhH2hEbUjQpzrjTBo~i&NNO}!vD3$uX5waw=2~(@5Zv=VL zm7rA9$KeW6xKZ!?Y+XUkxnzFTHl>?koVaDY%s94vi?Yp~9)u|z%NcS7(KqzpRg9_w z5rn8F4keSLF`IUa)-?fSLf*KMZwdAE;;mFaat*(k9ogV$tuA_&AeXdGq<)=kI^5*A zFjz{uit~Q3_rto4d_P}vDg&8ZW4na6?jarSuJTu|#vt2IN9a_sRE&&M7AsEzZ_1{n z#;{1F-#}=AmrDp*jn~js_!74FytCcuWgx9q9iJ>J#_l_5-RsIg!}oRp)53>)OVf`! zzi!>GMSoiE?y)ED)G3Ob;)N2&<~65iO+C`;l1*V#MAFW6)1ELxg>aV~h;27#zBo|s zZDbJ76VwbBS&KRJRPT-w2a#WK>QG;^{9G;hlIn8Ss~WED2^~L;+L&utO5T-q*hy+5 zG`^s_f<^KH9|e=(QM36c-u1%mli94+_Oep~Ph3JyBEpt+Pm@rWFosVY;FEX4sOKf2 zr1Lv z(_N$K``p~oE1eSO#FfTP=iqz1?j#?)-dFllrgz}fM&6yF1foU0(9^%s+sRn?UTH)s zSOmwLaQoyj())_(w^PRS3rd+HKx&t-1;+QiCH>#&mZfO(Mm&v!UqrS;yfW)$b@iHy z1@?wSy{s2zAYKCs2{TExbIs~)WFO2=sU>n0onKbZ0fkeS~4Af5fis!K+l{`F1j=_$< ze1lU7vyDDF2{EtKBynuc&D7a>?_ngf6^CuaMm#*71j=W7p-F&;fF3G5pXnF_peuq2 zhenyK&}Sb{y5ok&*6ap>&5+ts(d1wS1%uoR8~yfJEDFa|%_q}0r%qoXjmXFUtJSCe=96_NVlUKM>O~6_)eK_84B@)? zyQV=cWMWq0I6sCGCa1>J z8yNwUXe^#*aU?etxQ>~Q%_+%9`5h{EW(Tc zbXXS-&`Q*JXW@Esh&+r}<7aaBR={1{cJ$7ju(j@FTJTk838Xik;ba`7e(g{nCp*Xq z5_Itu7$b2z`tcs>jmcS84z8-Wo_nt2?A5&gm>7SQ4-k^YkpeGl{`*pV(Wt#M{z+o; zKhj`eu3*E0H%Ea#KWhjN0DFtSK0+(Pr@#D23slsReyqxzP78VI(|-p7kib8u7DuTE zJ1hhKLtF3{1M=TN9RK{guCBR_xudS`4+H!Q>2!VPFQmVSNdGiLks*-23wMK&{vG&J zGTmSzr{D7+(4ZjT!9|UKA^ueg8s~pX`9667hye?o0)Gi1a56!{zu++Io~z&gKnFVr zBI@7)05pQ1M~VsTr+;uV&^NTSvvqL%?MVF(>1=v@NI$E|tb0LF_Z^}P3LpUBq6Po} zA#`2=QGqkbAM^~Z%#Cdv9~e7;C-@~Ol0u+mf)a>-rFaS8^}}#iMDY5m0hkozBVGYL zCi#b^t+k#0WyTO6ir_f;ZwOulQt2SW|09@M9|iz$9RhL%)OO=qmHJn97e!qydFrkLekHgf4glampss)hY5peo*JZ}GR#wJ_j^LQ_;1Z9S0s=7; z1mgOY=Ow{!G+4of$N>NWsxUuUq!TwJ-M`yzUV)g>rGyPaoGf2DXqP|+$<>oTz0LArjL5HVGS{mt&5 zBIeJY9@6U&KmcWe0+WAbcOK^NgU?k6%oSn#l)uUS^-g+T2*UQ=gU11TYOwPH;6Q?a z@}WV_b=1EPM3A2FyD%{IXGQ~CpP1nuQSoeXW zgSn0A|JrK&oH+bG9MVDHW`cV2&Ln*~5o7l&6l%Z|=(REcKn5XV1=N`jMg6Zx23|lp zjggln&j5j(37V}ri~Oa9JQLXgV1Xl(8dwF$aIymWQVT`@-$-&<>NHT)WgE|;eklQr zRxcY4!L*yO!HuLK1gwC>o1kc$+kmab$r_xToR$&YPMyU?yciV(W+o`U^(@ktfUo2) z!zKi$a|8$g05Jso3Mirtit>+GsPEtac{TLKOq>=1HWLKjaTe>#Nw|Jz)u$4iQ@vvS z*}{*(5`LtAfBCwA`hxh7M^>RzD`ada(!V{Mda-B;omx?mpT+x z&I%}u_P>?*>wwV3v|xci&jgLoT}q4p5(IgUgb_l}3TTA>95sG>VB})z@IXk(1Vu7l zOr1*#gxtT+1R-z*^o;2owSK=*|6+RaLWs!(t+HH9uggk=+-nY<+3&Ibsu<)R{EKM; zo!JwypR2{i;2}45LMLmO9KTZjrF?cV9}AtV8StFT`w}uBHv~YgcCLW5d7;SvT=~D) za6@{IOwc@dksNxh{hyTQ;^>g&OK1Zb6M-iD^Dg|04K%cYEP~HGoWb~F@TY|o$n`O7 zs9E$`{Ac)o>y3Yn@9zpLkmWb%b-PxHzefPF&UUc}1v!+NAV$ewdHpx$&$>dA%LL&{ eL16w;vSJ@$f#yuDzFf7e&r7MV+=%bWhUq_X^r zH#f`P&42VV$1|9dGMLYINZV=CIs65z8arM2I~)(ioa!ao@ltTF#5lo^S4-PblK zy-)^8!w`v0o|QoxZ99dFDMc75zp?k3fo2|d1Z5ZRo8&x)3LBju_h)PcAkPz;!weMI z{)}eE=T9UMfkS~x(jv?~6%`-Wl>*`w0Zcx<@i_P*VX4?uGh!yOxFp444%{297-k?m z73R}T0KhRE(Gd<_T1K7%!Q$a?xK0?PJ!+t3Os~N@G}+=JO!)L>@SVh~N)E#5XjaVA zRRDFHHyv^zAeZ2SfNR`pgV!L@{#q3~20qP;IFg(Lug1yL$V^9<3?>ECLu##>*(a?(5%JZX?j+I~RO3kc(>$>z<^^d3qMZcn#>Yyk>3K3x@ z41k6#Wfu>{KQP9Ch!`waWhjP4OnHeDZz?j5zfmZ*-&VpVG=IBJk}bN_0U->h(1&=Y zp~`^s;&XijeW46KegJUco0?Fqo%t~6*a47k(LUcZ?bpCjm|U>zb{nnFC_C=*zfTp~ z{PT<6x^kEuR+R|~$-&ES&H*Z5d-0<0iUrH`F1V~G9oquImm??3f(8A$X4u&kMt+q_ zM3ZcJW_!RqWM%)WpjL8jM{N$}ufkEtuO{oyk}uXA%2x4V&5?AxV8#2I!e8miKmH@m z?=@RFwUHG-LGB1rUTKaoJ^I&ya=SIlLYGdM^Y4Ah{#Dxc8KFh3uJhJMq&_YnLD@VdMa{Ec{;W{F%n!HV&j7gCSgph=&y;u>)}hhC|R z7tx;^DWaz6-v*>G65LUz(D0J_hWhps8Wk@_6) z9jV8Ps*QfoVZ)cr>*8}*4hem=l%?0QDO~1rkQ+kLzWVKlhV1*h0 z-SccbgI_x%J&EHeXW-iq0Xa{m%l3p)( zg(;{&Kw2i{#Ud&wy`v|b8$9xax-=MFPvZNXggoBAovLZJkK-%pv>f=jQgf3?&T7=+S9>@qnQ zRmjEhp3~G66(z%1Q*ip6#&qJ5z$ur6fMC1>*D0cEnph38HZ}}sekyF0RZ0eSp+3lr zG=`V;jB&z;OBL;T8F0{4cO@D?D$c@8Q8$N9gkA(YLMb-g6MbZo4^$dsg#gubMz7Sj z?9mGRIwrZpb>e@WkswsdjMJbsttq8VbF0)T7+oz&Y-4JmJ2oyP0{Bod)5?A!W zN6+KN@>p+TiPB1m6j&4lm=f0{@_bgsMv3m_$n*1lpBP~P0J*y|_V%v*tE=qHRu$5I z>Arrx=zhM=-`|K|TCIHCS$@2*=K4BvwY`Cvn5E{6v$gayrR+{+Ly}nR-QlJw_H0Is z3k50*n`oXXqn;O{WWQ`h)jR8nKVGl>X^RD8(E)yX{X-MckeW(0{ceMy2Dm!;QV-8K zRcu>*utsrGu)q+hp>c{@(enlj0>;Rp?gdq5a4B?UQ-mn=jOS6}Y(p51I4NZ1ax{bP zOU#)HX$G6S=<{Zvx(eKG_bsMq^YktLX99?OzMG)vatO}` z=LmeO)`zKY3PMpisuG*WtjWn)^FkJRWF+>akWin*E`W=!ZEus_05F=5NBubT*qXjp z!i3)XwB5RTy@5E?(hQ_3OV`)K*oe0XV=Av|hn;VFLx}sndTN?YW<)!NqgNlefECuK z&e)1IyPH9zZXZr!HFV@&JSs@sV2ngllwiY!Mi)JJcwYT(_@`htK0DaRm>^P-UIe5# zDwN7f=lYn7N8DxK5h}jo8!)7j zqA?>t6;gG}u^s5J(K_Az8DZc3pPwO)K>_hFA`hmhwY)0i!&|OTd6yj7DSA=l#Ho1} zOJOvsywZXd@ViS1YO3eM48DK32G8KPwYHJreW*YyT1BPF6veW09BO0;p0&(5Ln(1KwEj|rg_3tHSjY=5Pw#6?6U0-UxuaSKE+$Dzny%=wb1>#>hla@`@$lPdb z;l6+++M)?hkm_5S7igBrBVjS$>|T^^A)2KJ+yBIRh9^-)pqjN|B4I#c#gx(V(iN23GQ&d{yID$&I*SXv+uZHy%@ zZwj#hjuzu!opRqD2?frtnQuc)D2JL|n+(wobld!^V@h11N4tzz2qNuO7#ImZx^M;r zJE;$IC8f=}SJVCUhN)m5?5U6P#05(jy+jORX*PZ)rap;)NaIx;DqS48a`5;su(33qP z(C-Ndau(t96nS4zW16r5$8(~+B^KwUGgM1`HL6E^(ikk)X%C9B4b1c@)2)Pv%j~3{ zQxe=XMuJwFhdTVXpd&-&Vf2uvrf}4bb0aJJRgEO?je(Zzs$rGiq>g^9>TZ|*%2coV zMS@6NLHXCMMtvSgM~)l%;?ov13z1@oJXE(bk3d~ zjVD5q%4SS4-yY#!$P1)m#lizf!L2AwMEsg)t+?rIZmof9}g+jCpRqSTQI27ks>hF21Y$50fb&D0p*5q$>)lMrlVy5nXva^{R= zA2R7>%cNS;#AtA=y^D+YD^SCH_>c_2n<*0Th9{j33SHzkda@@GaAS|iD`0O0BV-%3 z>nu|T+EgH;nn8cv9I4NAwUNBZOIl95l^5!PAB!u$+yt$?nj<_ZY#-cQt0}hwCcU!i za&YB9GUxJ>_^oUJu@Tu(e4AZ#b^qR)!2C1%?)u2R$TQUfw?LH|XB4|{OoH}{!wGDC z%j|FuNwFD`;uf;(gdlxz=2>8{#e>Dw)PmZD$L&KSxDoLaE` z)rzSEffj;y3Y&y5y_yg5D;(J<=h>S^&fvDdj>M_EiT!cCg%Ely+^VdJ$|eTvbX#Bh zx;On6DE%SC>4C=*pv7v_z}1JUvz+W+286fuVR6$%pbiG%zx%^#FG#(xVYbF#FbDSF znxIQhZXLx_?_F5VQBb3x22$8+!8I>Q-?Z!N64B=i34EpYAT+>lzbK%d14{6~Up#N~ zMkOKbCfzjlO1uPaXyJf;75(vezt#SnzwCXS8a5n&g8&pUxzB&j-3|V-vt?KWXRdZz z_vz|j!wZOg*n(v}i#|}QJ70??w-7$96SeZ4{~N{dPUZ4yvq9XrfjW)dK<*b-plz_kN75`RWl39lW6qTS~CUa7W0 z55eDBgJ3Ryb@m80E0mb^sgH`-TasMw);Itf*FN4$-a>Dv`+er}dAfx>9dl?$skzHL z!G#Sl%1C7?CMVaPV9um{_!!2xrbp<1%_~W|dHC#8t#d5PW*s}lZwA3;sC%@ve{Wa2 zq<-IF3NwpE$<5&N@>L@=NaLo3q8tMnleoKHk+#Z?hxBF;Jc^_ zaZ*~q!NP?%`jWFK;%mDap(_iQlH^*JtUKXvxL#TE+=FBD`AP{ZTujVq(YA9mKTN`P zs?U`$dFRV_#BPn3tBhJrA*tETuhg&hjw=rj&85vfHxwf$t~2A&5umj^;7W_v{0c>m zm{OCOWDa9E_CecHo1TuuP)3q;r!(6W=Dk`f+~|8VcU-UH!_iTA&pNU(Z8}~}EPKS< z*Q?be=XWaR!i!Hv)Fvpb=yZA44k_6WFV8d1`0VT#s7)KK_r9gFb534`VG)|cj3c6WB*=cB)OJLr66UAbhYgf zLEEs>`vc~@*MrNn>n~uX7j((JMFvc?Wv#NoPHQRl!W2U zFE(O3P8S_B9ca#wI?F2K* zbe^lV6!uWP4AS6ago29*mlp>%+~JX{i73lIEN!+?*|*X;P7a|(QD@ThVM}I606O^X1xFLIGx>3 zrz2D8F*14rJF+mqvU)tHA{uY0?%GYizW}7WV3Tje^B29#m6R7op{Kohny#Lc^3W5MAWr=z<0Lq`!puV%Z)ZYxW;Noa ze68c`cnd(E>q|G2bNL$Q?V8Enc}UV$j4h zqq3p4NxTto^`}@!o|U7gqZHgmkeRamwHRdW(;JgngzRA0b+r80oH2I;t!4KjI8*~T zF|u)g9+s<#r;+k;cfW%r!x^=;r30p8QKVYwyebFbeR?%Fe#ag@-j~P6s;>%??Pq5C z#)LQb5YbDGw`<~1csP5<379~S`H(|&b{}#)gm)DR-o{2VRq2uvO;b6O!M(?(7zeAE z8VD(cwRiu7A9w<*|82aNhRqZj?Kk@2zQ50fisE{E5s|<;Uw?|WFNJ_unB(dbhRAn! z;0kwvSjO!eH<)TqWx(U4fvU!rRqv!Pg>fq6 zR5fSr;FLq1wlLJ_Ir4`@w%j&$Y3oXp4-t;Ze02B?GXPpvTODb8#>RtWzWBVqyQ{rB zQ=q?dZ-Eua*h`u3uMKQ=NXIk&MA7H^@ zthG>VS0%h}YqI@}-BC?)Jha(&cBjkMGaZE6ie16Y+M~=sK*)$Rpi*mxHG)7axd9aw zHk)W5<5@$xxs2*b3mwD!WBiWt_$0CpkHypeA zyhjDze{7R$bL?jgLbIr2tJ2B0A1wEyuy1kILaq8%Y3qP!q%P-iDKV=>UfG*N&o(W# z#eXSPp`UyL#k_qf zW^`@;QI0sP(HgU%kC*&XjOCezb8jowfgs)6rl3~Lz*%>*_<^4>SXd(u-{Sr0?P>S* z4tD;Y<5bz*ya6ppbcc-q^YL5Sil4c&LNgHgr4b=Rx|&9xypEpE5b zm?CtdzM4yj*m_w3G-ZzQX&ku+>7t%0_x9Giqo^6aG=m+`LI*@67QA z`~Ro$?muHazCnP1jG+HhW&S@J&)(L-@PG7}Zq@17MFymvQ!24-=>}@#3*r!JFAbu- zfDnN{jslfqk`&eM%G}o6n_3WQih2@yD3W8+5ubxAy$P(p#V_nW?Jgt zNOLg3!6Tux^o$q9)s0gY?a-;4a@qH7V8OcpsYNjqIwxovTqHyysq8_dw|1SaJ}dBj zBZS&QQ=lhD^ofsU+w6v`nG*ZTF9Ek#c=$eWD9>3CS{4MOmm{@w{sF*>pC_pO{se`_ zCPy$gevK{ev^bHhTp!m<4JDS-vS&}`5EM`P9YZmp(WPZZR3!Yu6~c@~=WCD9!kD@p zHn&oVBJOZ?r#A<2^Qm0S2-k!nu8CB=mtf3IKH@g@KsO_Fk=KfJF9DMi+>7(#Ka!;+ z!T2OF^Yk%>C1+ercu;AI-%93tLM&(_tcB}?+hr~*{Y67DC&K|39J=MqJ!2+Ux2WtF z9;3H&o*8TOujT=mzk7c*-dSJTl73N_v>j{*|bt~-C5@f!)lqjYzLnz0w zD(k{aCKkmj3U^7$Ju7#k6U<%qbzb7lWz4f@3;Y)fN%)i#N}0as154zWN`g-V2K1FH z?Ie`P0k}6T{SuR}sDF16o38X`v`aGb%`&yaW37Wnh$Blv%^S;9y+?IJn7VU;{G zP3Os@N6SCz+WoCX6o1_>5(g_-%loSgw1RuvzFjdtim7BUZ{E~i^&A}QbBA3O$GUIm z9*{#x;bbh23 zES1x}kki`J=PQtlb>Wlv12xB?5R&Im=Jp_>N^cBaeN+f@)IN@vWQ zZ<504AzqZS{OCRzDIgaxXdtCAN%SLCy-g>TXJv{eqlBi<2!n)eb3eVwAsyE+w=QP( z7-*_iG;oHVUBe+e94S??TowSR=UlU#aNd=uZU|9urUbO_p`}AN0<|znxnMjRBzVYE zhk|q}d5^Nwt2iRlZ%%&ZE()>!ZTx2KXC;?RR5<6p{Z3VFJ2og$%dM|_qy_$<92~tn z4_K$zi|uGupdlx=&mjV-+IRrRHNiv(lcQduggt14WMDX?Dp5&e$IY;5t3Gcs5USE} zWRyn@7i?D=LW=m*P^BSXD;-jk_{}$S%cD6{ta`>|7#*sZ7->&q zDKi36Fw=QKTy2R!qPa+GGDRk5WP85t(fsZ^oEz59T(Ggxv(Ed-d+X0X*>gmxB7$Vo z9`JN@I|7gPyW=AjITdS_t1L=j<`seD6jFY5o3Dmc%au9Jlg}B;fKhndGHT;oH!KZ!92oZ*{B?XYe@a^xnuJ!> z;k6>Pixz&*@~VB*S^2TREj&pLN|2eUytb${LFk*D-m-EdLwSV83#|qz*?-WLkHbsr zfJ*jeowK{$Bb}sUrzN2co8cuL`;N&&Z{Fn`f&VXFp9fC*i0$)TX8HoIodaXJlH~p| z`^%#|Phq~bmUlZzGvgT+C0~cKbw#7L|nursAQUdyP zK+G;$Z0lLQ9HuONI;c+3s3oFSJK$LU|2qBuh+MbzX_VjEaky{ti&!pd|U z^_Wzz{R9^zp)$<~Pd@1ELZFM_=jlP)%CdR$ zdc~4QGen70Y5GtCoWut(0v%C^n^9zd)ce8C*At$PS1O1X>3H^^0xV(dST9pMZ^W%# z(nu4w8i>OD7ccjK)AchS>#q{>$d8EJkOdXF(*^xN)23B(EXsMQ#Xk~@n4wB_{Cg{V z|Jr4)JIuL6ya>WC((cs!?7zctKPfR^0{2KIl02vYcHN?A5ibCg*&~PKFfj~+$$6+C zH7dUf{(hVa9-Y#;WmM`((@JH`vL*CaHk}gT56APV^IJX%#6xXp%t%Ah^AWs=9j-F5 z>~<=0N$P|}CVj8xF8*j^du}RgLg*WN@V|sE(BJ0TFBVxn*S*eprWC)#H)*xtFi!7p zYb$lj1!NNadC0XhjHfQ~wJw^rrxkr1TfB+KdGDB(!Cd)c8&aZusgfNTp$m~#LPfOd zsu*9ukR_ZuwEp=lxT~hVrvc`qz27y3q>YQg6*GUPhqH)MD=CU#bQdnaPG4aj$F`a@1p%(V%)Z>=P<^2*q6SbRc)!ZPEDC!F4>FdQltnIC z@@D5>GXg#7`x?sMoOEL61MMj=7t2V_q-XRI8kC}n5Iu}1yZ8}j3?a@}If;k3w(eA9 z=mwOLba2I(aTMM0A(lzH5dkHO6DUMK7^PX~WVllNgb(O;76ByLt!_uGv50Vczbg3- zT<6m%k+o+^;qI|q$$3=UkA)fN9<)J?7?2naxDux>JmZH^WjGoIQT9g;D*o!YNJqdJ z?ir0lCyuq7CR`XkFDuR`tnd=m@Q9XCL!%UwI*OEI?%X-O9jbo~=tpekuIV+fLb*j9 zQ`k|4V>b3!rp{y;k_6CH2r5zfL0Y`KF{e50ztKZcV!ZY43pXcE^Lqovo*U4!y?=gU zmu~Sn#Y^>0d7`IlBB$X$fh}5N1suZ~GL3AC{Uk4?ThAo}ziLk~efvi-@YoCcVRE1u z(;SO)$?ld7O>7%TyT%nsune&c7AlD)yy|X$Ju-$CA9&!h_>vtU6b{gUnj3Uj1ZU#< z)71w|H2ZMl(|x$$gsW0UVDfCLo-*NPbcVo)Lvkg8v~ozvZ{*?N@=&@5w2a<1Ei5}l z59b6bb!+y6byiMlQ_k}`hiWda@cU^ZQVQ1Vr8rtzNlBCEwa(5e_UzA-HOQH&E~=?z zj%a%lZ0^+rmcd7Xgk3wa`;t`_mC`;0i`X1A#XhohR(R8PjKjjgJ>^)y*|ybA58+`! z^uz*RM!WqHhDN{rH*~K^%>6EqS|5S$h6(L-S|w()rV*i#6vpj>o2Xljh)w)Mp8TuM z*cvK7`4EW1*S8+TfDm+91?RU|IOk2x$$U4B)TrCdl)UtQpN!tZ+0PIym%*wZPpUV~ zLi8%=S{Pvx8E;GOv$|UkxbK_e^Aj@{Y^||}uHlK!jvUrzhxos#b;l2RxG zU)(}5Q0g9aC8v2=z5Zifna*m65d?|c*$l$Pzg4%=!K|(*tlygoa7yj$4B$$(RI~u< z)r#qhU4HAu|AITMfbAKAzf)(AIvR6y1t0v%Hj(z%9=sXk5N}b1wp9ASk5@W|9)_w6 zm3`BSi0)@h?>Z+zZaQwPg8#ZD?_I}R4S@8mgEYh0V`nq+4&p+Y_e+C@bFK!)qs4L)#T;VuK1^H^O$=VXOa z`AxQfK34HAhP7m_bIL^<*NzKYTV$@ha3P+toYn-lXivT+(9IN(Q0XDeh|cPdcSqCJ2wv9=?`H{Lyr4XFS&}HW+Tyh*I>hpgN~O&8S}nnGcs)) zU3e$SenoagtkZ3q3qw$yQ*?$hIY_UaN=j+rH+mRja1CBDfJ-6GV2~rPNjBh zHRUV9)Q2Jd!xV($lzNeilCQ5@R0e)5L2~=l6;cTv$;ru3^u7hLG96bdEbtz-j zvXPVCq&)%$9?ev>JO*8N^JkEktgW}T>wA*G-Cj9Sn~+WRXK!%uzgTTH5%5F-RZ+*q z24EW#8#`Sr3YM`+1x0c&b0Zcl0M7V7aw@U$Sh>l!g3%B!8P~Qq{A>t%0mWa-Pbxgl zRdYBFp|+?iHBP`qcd~5fMLCS4VHEM2lh91FOz|NGQ{CVRPFGd14yp(nQ2etd-?=;J z0Um`7wQcBd6ZY4Vf{pa9_-1PM(<}jgJb=&vje03bpqej`S+e&=tHuYnby^T_?2^Lq zpp>Kk>VCec*4h&yAA&WFe(Krmcki;brVT#NbZT`z$gby9T6v8L+2#IJ-=BXu^so;K z=4{6zpW~aM4`?UI4kuhIW)0nLRHZK7I_iuuE(9`5o2;Lq2SO;GfU5*1BFyH_@_lN; zrA1kNe35Eva>)@z2fWRk!qDc1xeY5XtLHlQwG279;(YJea$)p?cd zf%jhIj|L48^(U#Z*>!>I<%faqKY{M!&b0^EOTi>sxY@A=A!prwMh$y{>F*D z?8bE+jHU^_8-w6nS=|oBA6u2&12Y0{Ks6)?NpqdIPvx*RY9;1q9B9n)adQ8K;Oe2g zVy$4Qtj@hJebNrgH!i4Jm1qU4oTlXLOv-mFycN!A;eMC?JdIkD|6EMOCd{nbF=R?7 zZfe(SS~qS=PgYuQXG&k3DnuIoS3BNL72eLz*w2v%!&V*(Ig__Rue0UXd)LpS=eIu_ zRb9>RXl(5g%I5Wr`c369EQw1a{7WzhwCyZ=L^GuP*up1(nOKb`fL-p_uT{B@7p5hh?NtgK!@!Xqg=|I!i8s7$?IS*xsl0$#Gaw`p31q$<)&l zGjf_{Hx`S$sJ*92<4|Ivaht=Mma3#cTuPk-n;kcnj4~p9)cNv!M-wmQTc+COSpfgv zP4no|LGc=WyTxX_3aV8SGql;xUcFoickT+I);~-|tt0=UcORFZkE{7pX56-GA3vcn z)%uKyP0nVOo*ey~i;>a6&H>cbCx$MczBL@wlZIdKU#B|d;*au?j+65TdQpBQ57yKh zR9_ujsSf;QcF49?5t3kn#$)*%rAk06IAg3|?AyVbEsz(+JlOLIUMtV?E%-Gt#czwB zR!ra~W%dP9GBY}Sv7xEVT>Z^k#q_}2$C5;sGvbi$i-+IN=1)i*;Ih>k2Y_DNi!*o| zXme)-r)R}Ev6JO$^ypS4s>MH{SIIh_oS=e+QAltamy79Wnv(8alL4u+v4I)yiq#R# z0@d03c7d~fJyL0gkCr06*II7XO0pLK`;S%2GbNWDJ*?&8B%9Km;^x`?=``)&Oe^+1 z;VG;&eA%@N>bVMU9t{1;)7ePMK2TfpgwvDanSTsw164Q)wknp=e~~B7h`UqLz7rb@ zO<#!JWlLR%m$<4s8=$By`P@`rTg%Vm{k9D;RKV!Z6y(iJxg^B&5l)IMViPZ#H?bI@ z2ekb+8mBNaC!Hyw${k|{VH44vZ(0^EZ~VFBlW%lZBZ#QR13$Qq;C3M%udtW;0G`Vc z*I43Rdsv*)vU2(96e1f%qQj1vNM<{xE(SGZ^q>E9pXIvl==pvq``2N?ZolY*4aZTUE{MvR3;>RYU_a8#TrP0BkESL5<`lC`_Hy%i;wo(WYIMLJbcT)9<=oldSvD(e1p`}BqPbRt}L&bF-SD}&O{jWg|o?7(lu#u{e@*=gS zN@Wop`SP`GeLBd)wm@doCKyXioe(hn>2ar#A3qo#;w7;@`XX2!A3gc_tN8dUfbZL~ zwl%FbU;SC{fhwSz>SXxZ_kpLE4#RVOuk5+(QIk!(cEvTpRMNAv>H@uw&M-OJM(;pl zCQC6c0cmvPM*LF#I(!dEXgpYijdXPt2DcnvcJmGSgwC`%2l! zGC}s&^_P!y^Ly2vadwbH%zT@xDkbOF+-56G>#@kpN6p1cf-oB7Z@`0>;bX886UzN0 z5yi&er7afGdTAcqT^TR2tNiay(Zc*PYGtYg=>vPY^mQ-`kI3yGmMeaCDBCBpKei~sXG8h#UiMU-VUZdv1B;H`x~XdOgWbsK&q^N& zF4zT(PJC0xgK00Hz1pZVk4$)Te{tfoF`oq@=u&nEc?UeTrV@Z>Gx@PE=l~p4Kw9SOVVKjE0Ky~JI-?h*-cjKJZ>L8 zf9h60pPtQg@pH^ENBwz%wD>(f773pLguZTfAIGL{cVF-C^TXggH~gPFaf_vQz2?53 zFIP`rD0;rP;k&;-uh090bpF0xi8(pBKR<|ek%W|UgM)>QlUEOK%k$^fWp@vIcW0}m z^tpsN#@tF$+MGNd?+=T?gS%#!;-~*DLN4b|AbOkozxCTUMDG&hnueFs>l5JX5#$1p z7OLq#bbcPbSYKMH)PmW+KMT&|5`TDq8Vmd{T<86{L)Pu0aL5TGt@?c^tkssCpsVcl z&&(J^Mn-mlcRLyH;B!mG9D#Od(vmOZga$w$lUpd1W@C1DcIgn!=JqFTVaKj8=0#<7_&*&NY4~Z> z^83iS5^sL=O*x;(RSBXl1Rt<%b44(R9d>WzSZpxf%D98(LeYO` zda1Ei#%;2OfS|p)x>Z1aa!ONBel&~`59qn+s$ZJE+#ulCLo^Hv7Z-{))`-TGVvzTt zl|(KLm|?FZCx?oHi&}lA8kFz`xq5<{2I?uq+oOI=P_EDu1+&2ajQ4YYK!UVgikT7`vTVl49^`)kWolJwaf}DVxkzqM3 zg$cv7jtM!MHn>eOjtiQN`2|5)*j4TS@Nw{D>gMS6wplotz_(Bnn`kaQc6i8 zG2}3s0i(Np)JHG;R6APkz|%f=X<4 zx|mOODVnip3QFx83y|;$sf07da`m-Pd{KJR@dD-PA3cMVSOh$qR7jQ^**e~xzP7G^ zHnibJexwH))xaL71Mb5b!L-1&ML-6KXi}yd(vure#Evfbu&tjNpYW`akOFf5e1Uat z0>ub%Z0w*=@iAWClGP2b>jMFsE{wx75EibUY+bp7$0g@mG=d2#;G&(oauvjuT&i-) z>jy3HiYzr#1nA$Ls?t&(PBpB=-PX zI((lm_KrN=obE0t-(!2GV)gUsGIhItSSSlair`a~%Qk-(3lIqdeU0{H%BhMNiuy?c z?6f$!In32SaL4g4js@kF(F9}qjr($i%4)hnFG9)VfratFfyBopn@jML4GgpGosnD{>Nb+9pUbvcAoB^t7FABwcSEQ&hC>aKhIh|%E1eAe zWhQYj({4&nz@gZkajIhe2sa9^{abkI{PGtYV3;}^3D#1WM9Pg+J49HzFNC0;(QFq3znCATg8MJ&S`!l!A zB?!DhB+-YAUC{91WcI1%1N~Kvz2`n^X1yRqoW&N_-6`B$ATVk3390@8$v-2{a4wa-N6OSw_xVJXCyrZ^C1N8z(-O>?fU#;wIOw4K!mLg+ zo@)+GC&$yhIvrhXAPz|i;_zM<{wErE1uu3~IA}^bR!piq=CxvmGeMZyiung?^ zo<*sIpOEzeH9IIwTs@^^p{2JZRA=K-;=8 zm@s{bLt&f>l(5`hHciQu=MKt`J=13`Xa3*I5)@q*oy(jD#b&7}rD;jdfdI1L7D4t?2}J9m9pOfH@qqWkd*KM}pJ<&7tmb@OgRO63M#-O!m1Cz+ zx(1&2Xw#H-)6_ZGqnI(6^iAa9#0@)vacT?N#edw@S(A|NI0imBIpkrZ{@XdRuX5z9 z8~mP{{V*D!oyckS#!b4~%By9nVF?xg0^1RDMa z=|-x|0Av0ajod|w;{iw!^a>f)VC&IaV-&p(RuJfCwgeK&Fu=X3R6lT)P-j7+zoUel zGxjv5n}lQZF`I&8z#}A#QZLv-WQdqlqpdMdX&gXA^+NY(QGk7?-OKs*_2={Xyt43C zFD3fc2t1p@w@QgnvnBwoM1UrBU?OF}Xna&OPZvf=)SApA<0aM z_76&l(>X2~JPFfM<)`Vs1Z?#4wW{F|X9}mbY^-$5_SllmcO1Z2qH!R=a*Zb-F>vBa zt&yc1{e2$(#f%VJv))60Wf{cxA-IQ3gi)S}wAv`2l2mX&+1JK%&L?2*^DZzBtPJa< zBT&-vW{lmiIK^4K0hrmfpCj>8=FO*E8kr1Q4Qbk0W69aIRCz)W(Pf*V$w--PiI85A3}8Ec&^b} zF1qshS-^!}V^i#iXns53E=(J+zMq<-(E^^hPz>3jyf&SwajRDsdH4nz1-ji~C~*5R zBBL{tS8!hg50YN6oNqdhyDh8g;Y>JU;oy*}3wr+vl=$~<`Uv5L^glXc>dr2i{>Gk> ziEg19o&yICTXfxyJ(d_VQjC#yjx!si&YQjfgs4|~b&?8=@akwXS?rDsfg*?z%$PLF z7tiEFk?yDy3-_&XxZ-Y{f=(#)4p1244~tuW1uwBZQ__&#}h<41(bV+@$Z-845xB6dYrE)6A@W!6vC!K}qYL|3QBEXlj8B&*- zOA3!RhSQ8cl{TEp)sx{$Cm<#=Q63WopM@skE-nX|^Dv?A@1GdO2lBI{&73_>auxvj z=pq8lzRv_Mp7cRs2!W|ZtP1y{CvZaCAFd?jQ0aJklgyarxAY-%40yZ<D8xrSqx!$D^d09x%re6VYHK4l^y28_z<`rCQZDl>TndyYBvZQ!^ z;uEP)yD{RG(MOGkg=w?KkdZxJjv_w^a<36DMAK_USDaF4i{c-mT@Y?giDLzIe`M6< zw+Yx#J&*sJ@&);$y*v*31ld^-h|G*y_oj(T;lV0U)`>$Z#fY8#)5qZ07K6y*!P zM;rHs(YfY}K(4G@7Xk}jU!52kFJVXzUtij(nyinQSW0K{Ry=?e38&Q=TGdoXEZ`K?Y1SA{R1v&~dR_j4T-f1Xv16XrDu{Ri z?>3d_wm%KS`(upeK1xu<32t!iv;|GcuNFwwN?E)&Q_h^v zV zF_KXz2ZKuI$8)Bj;sE!}*kmd8p?~&dM@g#oe<=TvxY864usFle8uhUZRsPu4(!<>z z{0^_y^ueOQ{*$py;SYRC*~xB^6LVy;Mx^Gn9uP<`cG;2|zC-$1(PVT0fO2{=67$#< z)C@_KH|!U$5F4zZ2ud+3P8d!ith4 zqNXIvLa3{6AG*;G+P-HE6G)k^cX@PSyW>8Z@0=7y?-$%8<|sYUBprA?$8(uXR)C0d z_s)axI8vyCt_#EhM#dF-koF{%G6S`&jkZDTaxJ#y2wN$ON5IFe*mpt2s!m%r+K5j^ zE!Z8%=K1eKibgi!SiIn>!Q9v}L!bY>PEqX2?Ok;nsYN5BXoz|IJgkbMlhST~L_GUi zcER1qph{>gV50-Ai-G*?uzsB{2;73^!ws+A;x2L$FKE?1jf4090bW3%zlnQ=|6(bPzya*SMt%SA!A6Fq@FiQOqLI5^Ct0*pH z=uU=>=pS%2vE}yhv|!1GV)miKRQf?P62RX}E))bzi@~FFW&!|T(*y-bq`~l|7r%y6 zaOcmzIRCJzL$K%=^oKj1NmcxiLlhv+w#L$9IQ+}wlNU$lB({-1>nVOkK^nlHUt9vC z!+J7CAkyw)T zqUjZVJNF|R)i|pE5G$}FQde-I;JPcQsc&>Qs zLK|9DO-T+A=LXx3WZo1Af&ivhw7xlsQ_PWEv zHQXms4-S)Yc&tLRWD(7}IZpv+YM(xR#!H{K&!#YBqI|@EtAJqm-w(d$F%Een@O@UN z=TD5}@a2-?EWY6#N~RxlI(#necVX`ig4*L0)BWVC-w#ALWo?5#4y2M}>!kG!@fvLd z1IPxUKKnjLQ#Nw(dciS{$Uf>7={Q}njnlGO&hvYEtu+6fpCWByQ!nSEbH8*6g>=JR2B1ks41X3)ZH7p==GbVfF*LWho zVC~q@3;502B(>DI0RqxOAH>#AKC7xea!kkm@ai5Oj_{lF`6*7s)moxP@|vE?I-t=o zC}8kPzM9+sB5+;}cSk}4h6uE6tC-^sTya6N5E2|$6Si9YhyykF#|gX~eA4%aLJ2}+ zGzciK*%Er}DcBlx#;u|hj10*zCw^ciz)%xR0792I-4$CJ1aZ{^-FCX*c-WiAL3oK$ zXxW$|<~X6vOHztu01Hw_%XwZKYs?ryjE&h%O{qaIOd~x<*ukl5ta=qnUp70%fEH&z z15#jp2u%uoS#WLk$6p`4Jd2NM=$~8c^wZxQ9d=kF^$##e{q_gm@eTcpt*t_^!>t9C zzHHeY{2z}y8&b^U0Y(-4V-4%!%Y%!HokkTJ{jVEIvCBd1#t_$g_!<#&2O0oK*J6Kc zr-4{%Xbdb>V%-LsVc|%(01fA{ZSB(p7&AJsyPh*K^49hw&{O=~)WdYs*6Qu5`gPQ6 z+g9cC;WixZK1NPkLqp>3kcqcNCU$^qd^u3j^#>i9f*BnM`kti<^(EWD28W(4BeDya zO6x25nQ{6e!-pa6R2RVPc4M2c**%+Ota1O@m`1*ev zS%)ng^Ca*Xou2<8V6fZEKR3W^=-BW(6T!n|Et*t)efU zz~1oZTOK~ly73q^{N_7WqH4n%>{*A_q+xa7SwilpDV$YXk?@6$7z>AL$`rU|L^)%y zg<=xFPG-X;$!KVv;lG?J`o$V zQm(R$GJt8E{Rk!I0(}IZVE7w+d?JBxwr3dB1~~XAK|O#oOfA^3x~Wa&Qq!Ch=I9q- zEZ{YN>DjmK0h$t-sYu36W$3&Gfb=lK(AA92U~Gj90h|wpOm(P6sBkFUKA98y4}?6M z>Kr<~kZGiod3)_+o!aw#3c+H#8w7g>Fal(d+XV{sI|NX#7gSek5fE_Lt|XFXKf$j9 zhdgYIB&hX$-CW2dfsa9B41jDK>wzO<2#R(Qn$W@rtU@_0VZt-xLyd4CB0jycb+J-R zqxG(UCAC>?U$27wBNwa$+2V&o);5ZAM<8+2WBJlB>1ccluorkRdYe24h~W6MTX zddSj0Tjgq6JuBz1q3_u;$ELrH^U9atvg zJmVxm3}pZGs}EQ=9ules$eR@P`i0jJeG%gGiRKJ9FpruR4#VRP ziPzL~NeopsoGvtOOR0+w?h<{Bkz^(&r83FPxCqRVa~i2O+XbFn>#tC|qeIPA^=`f7 z=&**4BHtThtAGT*r39c}c|&9bR}DrV!c^O1#RAd*qdve?m00VZnkBEL{;ij|vamgP ze^a8nOsv-!+~G&8iHq``mWeO}$26Txp20-pY{a9gt1>zcty5*Q&_VB=xwWv`qM?{0k4 zz>Q&5!vnAkAIuO4#HZd&qo`yENcy;H?8voYUh5CPkn$@iq(8z! z%1Nl0{`3p#!h&M@!!M+iiNN$%0IP$puujLT0Dv>8SjqiqgvTHO>AzHl^qc-_#KVwST4=ti3tm@5@r2jU^)wsVsFZsWbr1`bVXm&#O93hCnOU8ymiX`Tg ziIr-IsscuC0=W7b)yzZ>xafK=N z2h_#G9I~Y?F^-qF!s0{rc!Z*R_O}~XJuw@;g=Q2IVpJA&cN(ImWl2AH&*Zfd(vfYG zN2_)3RBW)3qLGvdE~s$S#nY?FxkpGJGEgIe&yg_LL?;8_g)-H(`a{#DJ(i6~GxEaX zTu9IyZ3e6ST>tdbeYGW`7(bAl`#V2yM^Tx3~9niH{Z2+HK0=)i0(LV1P%=#5utG1knEe zv(5^VA+sH-wv9}*&a$n=zEra}#-f=+zwqNKsAfAIIr3htA=S{90T_uy0-v^CXzkSm z!2Bh6s}-Sy{@9_{5}AQd-GR)FH|3@TyW*2)QH`i!*Do@_TEiU&6WqK7a#JsIh~9=L z6XF5E({?2UdSHFmWgQ~7Dd~@tjeM`sq*8;C&T&raB3V6%Et8^yJX#?l(+EWd$Mvla z>|9TN<1jvH@)_G&6c&p+6MN0rlW9T8ZxovDa1=@OM2~>0Wsi(nHg~@(m~on6JkW9N zCke%u#+g(QMq)!D^PuX;>$1z{*ABAF3>Yp0TU#G>O3>{sbsGQ3 zdk%HzbKvgx;1~+G2Z zzNFT+0g*&uZeNFipwupoX}x%i;!>M`ua98~NHdt|HT11fDkAV1UWt>?am*PI>1W~+oa{(^V3YG$Q0V^y=s09K_K z!1Yi4F{ZR{LFir;;qu>UU8d1^4IB#e`YYF7}#4nL?vt;A=lIbKPo3_ocL3bsb3wq`{&$iZ_8MPq56 zh6?GZM!zW3wZ}@Oupba$JE$X&q+gUNQshC9~F45r56 zsBN$zPfhoL$3#?<; zuYD!g1XBm~k^#J8i9R%L-Bs*051MboCmK1}WX@Z&3DhQ-9zeIH zgwTm16I0BPhv|CF$)Q`6Mveh*Dtpe)qlz(4K9-Wn{|fk}bJ|%eY&gcH_!-)?kNvIQ zLLPNVHW`jgyj91Bm52|v&aDe*8>gmAwXGRQyQCo6x&=CBv=y`mTm$Tsf>~@z0sqK< zD{L#Z2Y-6k@WM8-}@Ir;4eH&}yY8 zyA|=rE|YK2q*+z5b*rcrbzlR=_8isu`RRGD*Fn^}?{?C4EWn1qV&iUmMQzNRUMbi< z$z)22n#4E6i;+OddbIELH-or=3pn(tEoxM+0@I^9Bv@LrogBhxcOjAp!KozijYtv3 zJ#7t>SY|;N99)4L9b@k3cK;O~xxZE!LG#C`R2w&WFrP=3PMjRLpp0+MQQ?+p@SXKN0*omFM=BcwBQ;{k%##o+0PDhzixO1GOf z+7ekEG0hz{*X@pq$gVAM4@Tbgws~`eDTkRDM#6z!jHFR&WKL7{v4ovmB>EJHaBO3GFW!VSWs zy~=HRq+l^=DE!}VkE<&j{ul^+C;1OLI4|K2eOyy~vV-@%ed;lCEohp(vTw^!2~ z5@LDNp{8`D3{`I=AIn@(3RixDt7Ghf zb5(+~q%)yCGl~ zbV8)A3K;XkRm< z_9){o7Kl%lv>JaO?aJ|eI2Hp_PpyACs<)&7#(ZyQdI*^#!iPbQHdEHDDDmX;(=Sdk zPOjmnu7H4dI-c8KuVznv(ic9uGv)2J|8{vk;*53Q&_WGgOUEn}@8qE%j9oHX7^b_) z+zrVjcFkw8gNgg=Fl2*Kw%w5JWD|0L{X5yOzjDK8w6tS#IrMwC*Kt)=yh`;*c3nzX zGb`^izEaJkJuOXoQa(bDNnKzwyT^!6JgV&jV=Fv`Ci(l`!0vh5*S}yLF6jpeFfvF; z{^B(9ZCH`O{{ zHcn8yg;`$VGT|($J}ha($TBL3x##+DPSVc))g>5n&|qg$eeN*vGNiPcHF6tpXB_*( zmK)cl#g9U@Je6M&-7@el$~`|iJUagTNW(6bEw{v6n$y|np`$X%piGO@+`BRK8;_C*`h`ux!W&p?*e*YfZ@W>< z<&h5tU4y3bE?4&WwdL3&?D`!rv_KF@ZOToniFa?|uS{z?dsrrpr~W4L7xPk5fU?-Z z!p1QR*xx<7e_p)*Zn2ucj%t(CDiqFOs)?P88X1wf*275rRLYUX<4*Vy)EoNRk@;`~ zA+Q!bGIxT!$Z& z>9}^o1!$=Rz-}QU1rRiBi-3c1k;yM?zopHv=g>v1?o=JMAv4?xt})S8=0RM=8RQ4n zxB|K19lhP|7i?<*zVG5vY^@g3+`yC?Y2RJ1+||0APhHCUgh?-2Njkr7o;BqA>J-v4 zLp(mx^Kc1vr1ij5t_5EED~>{ z0r%Q+da6UtvSm2hvC=_Dz1>@!*47U8ZT35VdxM6AthX(0!)C!}x1Sl5yWCzM?rT8i zM_0eh1iFO8o$7UF8Du}e=fx^!WW}$aG!LuC{ne0KDOy;F#*`|cp!eN^-tpMV0{|H{`ZIa9c zOaQ)?pgR&W2n$)ybH`G=qtb$Z?~40YYfsjZ_d4JNJJdv7*;ebAlU^uVZNG@MGh>n0 z4lTzn|8eX??(ic|Ci`tjjbKIW2R(^I(yk=tC?$5i8%!n$+qjQ6WHQHjanhK>Y?`Zw zl@|!l3N+Eqw%dCebkhOvUl7rLemJcDf2CP#d~B^5=L7TkdWIuUd7&p{c{CD9KQzXl z$&lm=o0|=xuv`H!4n6G>`CeDv@SBH29COws$&VXON==#lAO8 z4idb%k}a4_=hzinVFn+Vncp=hXi`Dc1=B|XZ#oS^qoHy}t zdAmn!&lrSTREs%bP&WK-cRfL_#e=(gGuf-?@av1~ijrkaP?s$ES`0{odo|}P&?yhJmt2xZI!N8&&%zt* z*V|;@zdShj!!o?cBc>`3;tPVk-ZK@HdNFgv9Acb$E{84SG!I45450k&g^X+UDA$WtsN1xjdoL4pnwo6#iLJQZK7U zWAOM*${iZil%MMU>4{F*Td~br;ZQr!^}tzBVFri*E2CEr?3iN zeR^bSEba@0%uez)BMetgog(v>L!t2&dLF8Xn@e$YywujP1)YKn^X4 z^ujeDHY?0kX)Y<198ZFpMl7TrSmOkxum(EMO7}>!362y%zCb!hzfR4WocQlGht7>$ za3%o4sp)uaN^+v|Wy}@dfGWUjl5|!VP4r=KXlwlE_qSs&ZE9|GZipdCuTyK*8O!4o zYNUF_K2NS*qts{BC!$dgQj76aHa@<*qdt z80iKcd`fv9hV5|K&^v24{BgpJS6|jE?a)jz8klQKBR+Q}W z4`)YRnM$^5=r|s7cdZ-J=*3~%|GN0N*WmX5H7Lsm;6FR8FwJ>ut{MC!Cw!9ALB9N2 zyr7gVk3i9HHbs#YuJ|%XaD31`$9>}SvbZy+I^V;0hW?9R{35B6eR`5gsNOL7rdXYz zrd3zut*uqVY*e>WRn1CkxaclDr*}S}@Gar$UQ1$;EK!?r1K}i3!MgCPq<95^&1ff=RZns)<9`(0kIO**}%BG|G}TB$C*L0 zyrmQ{^o+lo8t$`!gEYw3K-Q_u73+h~`a@Gq^SST4U6pc%#so(kBO!%}r6jUp(g`l@ zKhkv1yL8Z?UND`6;&A4FoMU z4RjLOzs`CPY5$Rx0NObGEt~*vqZf}0O82^zRPCQD)OH?0R}4sd2l{@P(k|uz*8uR@ z?_icLVa)d4wxjtF)pXkDizo-n){|#sqNGw8B2E9R?fV(x>p?t*r0{& z;%fV-?%I#wvfYJxsc_s~nEjD_#q=x5cQ#6p;q+~670l>w_P$P`b1-}qsk#kplj0} z)mZfr-LNp~yosF;aPK|hLV5{e!jBKX58e!!BTTl*s`{GlSl}!HkkN*(?8|9%03-MX za_8!xO_>ESgkCOyR!dJYR680{4TKuUE?1^J*j+$1N z(9~c!Cc4!q7Xw90I^djICQ^oOkFLQ-@uZr~43V>!Z&BiQiqQDJfNI5ePU73yx{+J$ zA8^;C&o4z~Qg*P<(bE2oMr#d&7r(sE{3|K{0ABqQ@V}q_{tqqpQ__Zz83riCNi9g5 z#=B*n>1ELJWXpYof_~6KP@uIprl>P9kIXfHu<))g9Y+@=j*YJDqTwf@Pu`8@B!El=NgkH<;GANR z=Uv%2bJ^hSN_~W;9)M^Zx+inb*ePn1NShUL0H1w<%oJUyPT2(O^1ZQ9Yvux8X-XO& zcoXH0S8x*SN8iJ#t6IMX3(x%#1FwSHf6Z(faN>;-w~X1EcQW+wrU2AH*ff{BlDs4LA9|ytlh}PJ zta}r+mVdy-ur5J7pc$(1pc`oIpzPQiTmB)vY2AUk+KVUDL!cEF?aI-V$61vv#vwH+ z;twH99SuERrrq{8`5mXxtD_Uy5!%x>^qZ6~A9{Grs{eKna+nS-PjEWkMRwe1jNeXloW~A(?*EM8g}f|jzj`C2QhS_3X~gSdJ7u+*qxJ2; zH9kCmUSp{3D{HU?q+@_>CnIBz?IOY8{{L;FX2*Di#;~B7ZY0gjg=of97up~P|XL(1u+dL8c-={DE2?v5X1PE0WRPJ&s{kQ}uaTB+-NRwbY zg$nWR%(XjvIdRGd5&{8@?cLdHc6N5ozMFl#^8I|6*HF?As@U$F*&nv?CsrW#WQsSB zi!YzdzIyLdkj{Y)JzSEgSzVNP3nukUVO;yJO@zB*o2%Q}6L^>?DN;oxkz4FcQ$wk! zdw8aB3J(>ApahOAe5~tqgHIF}mJqvi!9J=Uy+2ZG z=ok0SC{=a&c)kyZ54xMv=Y;I%U~?w>#pQ=nnB${c1T;J|Nj?`^d-LvIg;aK75ZBYs#RhT${GZGc0t!gd*cJe ze@BLSTwAGf*f9^7wws7HJEghgButWUM_^!RS?ZpZCx~$;s_XD-7YP;Wj_Q;vR0%%V zy17J%gpshG{f@~n$Sh#4O)(czIGpi5vK-^Up2z@?y@>C-G#}yzPY&{{E~gz~?7)8X zB+TexYFR=}4M2jd{KvJ>$T2J&Qi|v*UfVkgVwNa6+caIZDn(3lL&%?(pCm1!>)A z`d~9-T?(3tb=gwA25Jb)CkDwaFkD)oTDKriRZ)(KS}lcvze8ap?R=`BN=X#Pf~j-I zZDYJMWa_0 z6t6eUI3n>hai`f=Ted={M-dlWzO~3cFr!*msU%Ms$XS2IKq#E=12!8v*LU@L(GdW$ zJfj|~5d<{k!^lZzxTbNF6*3(ST&q-g^)i)!{yegb%5rLk)tIF){c4MB%Z-P5FNREpLf+`*PL+^&{)0{ zevx)O==@e`8xtQ{9eLr+D$ zN?4GxCH$qTwjtbv;a-}0%$%*Vhh{(}mFQ{>6tL){O`_gZAY=k_PhSkH`KYp-9E-C$ z%p%IRqjPoEkEI;R;htu-5|jiUh@*+y8O^AV26G)bUL!uPc5Qvnfe((Gy&~x7;TmqQ zxirqR-)cW$i}1FeJrn{G%9v>?pORS)HBqDhstwCN0KI!nZg|J;OZ12sumUk`sngFu zI_Y30uDr$HH(!*$RIOS%$|#|-dTNr|g6n!ZWl{r&w!8_>ubT?QZ@#CZ*HD`HyNzi`7R3OJPs;0$ z*0~c0}+7jA70PKr-Hl6=5lfV>jSY)gSSt~R^Q#%0-{ZJW$pG%MKr zsprDUQYxvaF_2IiApt40DW3rl+HC{DF5}i@oTnM;9bb|sYqES@ui%-cS9}NhB~~L4 zEd2DsN1M$EaO1&|Hhl2_FDLlR`TJhUaV;VeepNPHq9BwoJwjwIyTvW^tGp=@JE&JE zr)w%$_M3GH?djU{zT0pmGR(m8R(@WZtwFlY5kGthAM>AH7{#-s>rZSh;eY~dYn%*M zHm_RHNPI|M7cx8YLYB!O6)Y1r!lyk!qpMxzPudU@jG1q}b&vmbw+ z_~&w6%{FUb@oVF?VclnDZMOsp68l_uCE#);^lDiG?OsDrnVI>{bc?AVcLje22e@c4 zfr&<)A~JTfn`X1yrkKdjX(j>>FXq6R6f*=uca8WYp_op>i>h)Bp0wI%g!D7Wsu9cN zymwtcYL}K?Rv>S@h8~{Ib;S>o*O6kgd-B3UZ8*qVSrI@-CaV z4{fo9lfC|((hW1a0O*SkY#87-9jG(`-J?MhHrZk8BTdnsRR{suO?nD5u9~On_pT|8 z*A9Z)mM#8eSb3-|(&mNFx~iP@+-bNbTX3!ihqp6{v!9zz&5Lo+Uzc`-_a^R~Ah+yF zJ|wUfF?4QIgxRu)wlu;e_5(is0ySfN3rW=}0&b9fvtKkJDwXh$>Wga*(U=Q`@NNiF zBdY_4!fewP%j$j&$77+MPUHi*MR9-&19=&YE7vZXdP`BYN;pL;I>;NMOz>1>-{HKV z9)&P*Rj(_?_^$T#;j)`rlkhdVDu$H#sZR*N{BKQ2bSM$3D*r)_r#m2R66V2hGF~$Z zWr-&X6%nho>K@37n)Ui99K}f?oQ54}Z%XBw?snm{v{}xRJ&|SFd;Ug8_Bgw%Z*Gar zg^`F;!Rmppv9gN-Q)4Q-Ygu|h=O~zp-!Zpbqqv!Zwy-D_*Pl`$1xpc)0@Zuw)LNA& z&16O_N%>|>LInjUM2poD@HqfKgeE9NJy?z-hgtmEF1ji$e!hlx>BR$JP=0w7d z^fDvLwA%estLdQOLEP?G9H9EL;9@#NhYjB!RyMc2Vd3FP&?pH`9HEh2_^oWjDO397 zdp6Ss3J=5gyJq#$`!8Q!d<^!<=#%+;$6R)^+ja`)LT(fuIwR?BC&@doN8)i_0Et$> zwNLo6NSCv#N)d~=op$5zW*bQbCup;~xh+m&Q-ktR?3>$+I?9JG@^g`nrCEP*Guq3J zV-d@TJ50E8;1l=Q^d>W6yo2XS#CHeJ9<&rY9=~##bNg!`f8O_D9Qhucc`+jr&N#96 zD_v=5#>ajBAAlUsW|(3BJ&1^sU4H%H`xhTwyf0BI0z<|qPz7A`{UUO%wOcq3Y6(vY z&x?w@huJC^3p6h_IW^%84NP=)W6?peH)vk?2ReHse5JL^#QhjR`g_LbshAZ~! za(EpXJAf@W!>j5s5dkC{xh;PF!3R@_5ObvUc|e4zU&Kq(FK1C$=Fk9KSP~(U=Hw@~ zn6`AkknZngy@w$&C~5IH)A)bUjXyHs?-%eN=LYjHyN2)88TfNX+3krp6O`;#7CbGN z-J?9o%|4O8vdM_L1yhXg4%^#PF~1^%9$>3)N(M^Bc7^(kuKImQ^G@l30f&@yVHdxv zN|-fN3aHBa^9^8Z-CC5d+YK_A88DL+jtyf1LUj%(sNnc~5-WU-k9db=mK7Npiy@=h z1+=VDXh_0{JQrT|SH$f4nY$k3Hh^$|A7KnzR^^IKxC?)bD;&XDHpP*In%%mc8D$d` zAuc#BAQm1ow^ZQmqnhi;c_%v2JbjFc18D!*02&68kUJt$aFkGl#vRzk6fVsuuV-Y9 zdM4%$1xU>iok*|mqTxib88dmpzy4?ir5K8W<3G7HGcm6(zk2^O2owgEm0!5i+Rf-F zok^Q{40+sdArTiY;9{}NdxsH53Vi(~aNrg`q2AuVImur0-@NWsz$wbkyk>O{@fV7N zlkOsD&ReY!FLaM1G@jJ{(%5TL^wpnxjXqH39awDKwKq_4bS|bdY%bH-xZj&Zu{{MI95MFBC|P9clT*o0&2o$Zr^^@!cSIdm}LVHf%JV7W==$LIK>e7LHFUn189 zl%>(GK&PMGHrs9A^>G!;a$E-bkyEBnF<6H!;z9pFQV7(o?(q;(!EE>dGHdwtG#GJi z^85`8y_`(fvr;jsMSyGZ9P1Z zMAU{)dM?VXMA%dn_w?ETupSv1-W#7tJ5|kM=s*GmHvN|Jh0C`K3JFR{; zq@QU^Y2V0b;gb*8(k_cAfOvnSCJxY=$Ej>IQk?5VHdqVOpt`DEkc^Z|@lR|@s)f{B&@_3XNVtE*Rlkjll9xS*uo^(inZ zr{u9&&`tN4oakfS_CTqAuNFIO3zvq19ng;W%?Rr{YK4=N_JC+AjEq#mofSvQI{V+M zbYTW;V>T=(S}F9MF3Yrf*i>uG#R;M=vG&<1()E z&ErS|RsFm4!QY^f3r0-*3zTFvc|e(JilT8kc)fbXSDYAjoca{b*cPN9gZHf}Zlgnv z1#e!Kfw@nx)~sLThM98b0CLv zVf#8BsGc#`4f^YU;^D+skw+>LZQ|v-%{_(50WBRUD#~BbK1UZKbcADbh1J8;QAj>k zBbn^HgoL~m!iv4Us$>vt_0+;sABfW(57;mN2&uC}m-^+B{b#zOWyaCcx{cYQz(}+h z&C=1ia)eAJJUu5hpR06A==(TwhbQ4u3@1SWoYN@Q`tkH7qZQGAP`5v;{}zP9uC7y_ zgr)5-X0EZ83|S2?URCK`Qi3NnC(_^XIKj=(H_cs|Z5oQ9!fy+7a@gi$6Gvruws!z5 zt!r)~w89}H5kF4K=^n;mCMcjX3cwqV-7H;{yGly_5Ys0*RscfoLocc|E#PR^UBC3! zC^1jfDm9>w&!_J}k|h|;=a?gE@Ys@wk}SEbR=K6gKk|rw9zM*zn}3ijYWm@mZ>C?) zzJT8_oFTKJckuMJnIr3bv;JjGw=TutQQ_le09{Gwch&PvWsDQa2+;VtTYK886>1_D zH_XdYc(q;znga+Uz2&jPF^?#J)qqpsx_DX>`r&l?)pYQ5icCeCh{d8x?ue2dA?ts_ zu!syM@6D*2a-|rf7)EKx?`YyX&JT?Y@T`;QW{r!!a)Kci#Sz0H(aS$Q7*8nkBB;OT9=+AW8|3AaWk!a~amj!F!W7 z8C{f=eD!{h0Z1@ZE<#7AmNJJ=nx*tVY2)%G;L?JIPEYoV34Q6@$q2gLzZ@bXO0@t9Ti$_K%1OD#|z#d1$Cy@fjy_oNY(j zO2v&FJwWzLs-2WXk7FopQEs=mh3Sb4{EhB&Zv?cFIfMxnQfUo?BJ+&%C$t&Tgu2o+ zb!v#*6CvQ7syy)VDXWd$5D@6sO=%fS5K8vXxGom}B7||vm}tVJ3LI42FF8NnoaLaU zxYF|{17)sl&Dmy#J0$@m*#$gs`);$G{iDae>9O{^T56A5H`{m>&!Rutc6S8_q`-wM z|Dsb!r?%5XX@JViC)pgiX!AVA7d8V0%@4m)nHW42 z;QewUxg-xa5ykPsw?F>$U#~&k%vaJ^bi3mSVgB>$;Pc1M=T3=t=(Zw_xXU2XW zpiguXu=~ypkmC2^`|QQ=;;$=IPRy*_H zLWfO0n;p|vfeH0=0>cf#3mxHqc>UAa@6aWAK2mqKQL`mc-L8{0P z006i$000mG003}#G-@w!a&L5RV{dFOaCx;FYjYYm@caD=m41Pm-o$C9?a;b27+)Mx z3>0uuPm(#rAD`T}iE0%UWy&+a+-pdvU?x9-h6_Ol%8y`YD6|ILpMx z?E4i5JPijHJVU71Y<$lbQN}iL!fr(*5*~cN+b6p?6;>#+ZMi-|@uft3X)znRc1TJR9vWNCeM-GLF~;M4dkCvZqf@pIBCp?}1g)W9K{p zO#%@f*aZjoa6h#$Vm3(}GS}Vgvwb35m-*o?PBONSd@o)LH}ErbB#7dxG!FKea3zK% zDbW)@K2uaM9WVn=*6!-ZXCgud-O|`v;FuzX4(_Y9=1>7B;y|=wM-0>_&Xj=^a z1u?`AiIukN=BmQrArm}F_~B_6QSlp*y5Nk$pBf84b_cLd^n(H$vMh$c=&C=75PoDa zes?U(df9)~cdz*FIfea?&Tskv70_QLr7C5CX3ufcYobkX=CM~N!oAqz>Kn8TqLVs>So#EVk2QBNWKh>YSWsR># zMrzcurv1V1{YxWZ2hJJw=VsC#uy7VHCKq}n1TGWz%DFms$oFx}dNmmu_#K#O;mrDr ziLuiyNV)+o z;5WboxXib(SiOCg<^(QIPU7S&PFQim8j+@4LNi8C_IUos*o82g(XvBD-NEGQst>=X zau9Vtft3&y%#-7kSZgRMJ&3~)wyD$_^hcu{T}Pyc+@24e(M1ik0T8Yu2%GBxn7W4G zBf|W}_2OdkdhDtdQ^QIDi(;k*2>^m2IFbvs22T(U6Pe8FTtkGta7NP_WD5`(!!-~` z5Uj^nL+5o3_#T1Ip95#wsLT$*7sI+0)(8L_IsK~|Yz{E=K_53|mhM-ca)wgy5?0py z`n*PWYvwRNf|3?cM#F*Ah!hy4kuz>Y5e0L>92U)tJT;M-*aX7M{5UGkhMY)~xgP&M zp1f|BvCXl)jxX047~z%k5UWI~I)Dm##k~(K-9haLm3cY2 znKERj5}8{|T9ttlMtCxT=#2LWGKHBFQW}wJl(g41)$APT05rmF{RrYMP4dD|ivFP8 zl|pcSEU*cMiiKvJFvpSQT4NaJC@K#$;oQ+gC)bPVH7#k_8x~ZFe4;8^S2#V^4tTg) zbM~>z4m~!GBhitSwyM!8Xm>o=3q+NF*#Y!oNxQ?&u+U2OqNS*8y;GDRQMaX=wr$(0 zv~AnA?MmCWZQHhO+h%3w=^m%g`0wo)@emL3xc1z!=3X(s`U5UeNgKgTUM?t|WzSKH zhku3AMNrd)!tw^GkY&m|5pM~9$a3k@V5$4-Wap}zVd3p+c!tOw_XbhwLkh2v9heN&3 z`HMGC;x1=AW`^kY@G(RHp@eXOVC?>?m|Q>0fSxAW+mubcTI|XY3~(!s+zqrA*En>> z(6G^#G@#T|6d@&SjNT|ZcHoYJDV0luWIE3WG?N?@OC*7KMc`GABT$UFpx?3}O|yus z&k`q+^qG*G;aTR`pN({Zxp$rH8DsMxV7owE@3VfyK!de0Og7uKlFeY7RO0JRefhjK z){oq>dlUhood5ymxVLKdK}}Sf8s{lq#GqCv@hFAEXd^=@yYGDtz&}FxxkFs>AmCpx z9w`<^2mX*0q_Gt%&S?k!!^i&O;}-8hV`iwSC_JL}*Zbh!s^44**AMZMKY4E=NgXr+0 z&y?Xn=te1E!t%vqv2su-OIz20e`wU%yMV#AHUrZ8UGTC-^KO(l{V&T$8Xu|CuWO} z&*^caji59?+UP^=fwJi|DDL9Anz*nGEPu4RLuFB%GFnl0z&o1+=&WJ9`3}sB|I_qA~6-(&=Aq zplpM5jKE(gk+~c$ZL^!i88M^o|IH<~o)lv)H=)(4p{W;_fAMl?_%QIJ7Xc{eT1Llc z`tclvN)NPy#_S9F%qIMY4S3Xf`ZM}(IW;REZkZ2W+5{tPS>;FS0V59&LB2c8U&V=Q zG2^rA&AwPC`YqI~B_yLnzYW^CAJSsmiUXb%2aRFu@I*~5mDJ~B?qz}^Us`*`UCHb1 zo-1Odj)@@9*DPf)ylD_kU!73owJ2W;fAwjJRBW;48tv&U6p+p|lB0j;AD65}cVOYl z2k6W)koj>oB+4lXHh9IaXI)xtj@_r=tI&6n7_8o19ikRVjvdY>UM!!&(hUX`HoDAR zuSMP1HYv=dP0G_MuvFEW`!iH?Qt#0KuF}M$M46^jGg6} z=*X!|TrLUC?damvEo5jL!j3RI^l^emg4Topk}ioSm4#=DNE_Bjpx|Z10-Go~h-I2# zOUl6rXRZFBzzq1$P5kTml^&qHyrdBCHA?cUK|mIg@qVH#96Fw#zL#E`0n$r4MIe)) zM~=OT&ZWEiADR{LUetT)McW##`gF?j_kDC{0s&T3Wdrw@Dg+j>$^^A+N3J9ZB(h5b z=UpBTHS(`Eb*{<;G$?YDULYx)bql0l)90N?2&=?`m8EfZaThvTh&q}v9|^O6uzm8q{AfE~SwEfzPyZHp)pRUvpM9%v z^6Bcy-%%$~H+Seo3I8qBHj z&n;o@Ul6P8YrZel1{ za1^6SZy5QkCmn##L{QV0Cnee)zAK?*4@yR>m!|PWUB|tf;>U|VkZO(aK{zJSHS7WZ zs}(Lo8&FcpCM0uEz*_0NcisDoz|b*-<|v_1$w4bKUcY2}k+cvK5wj5>sP2=D8U^v$ zo9_H4@4F0V_JiKrfx^|E!%_4ypQAuvc@(5+pkkDn9}mh52$>kUTra|m<>;_*9mLvK zGoULYpljp@5iq8!Lg1e#wY-CS{RFVPaeK=17SayONIUk7eXgdfKsO$4@-qb|9W?4u zt!NpPOsC(&CQ=$0plWQH=;W?(bGc(~^#HGXu7X?h>5q-}AFwrVzMb(e`plh!!p_MZP7rMUk>OsHt&tDi>m=Wm7~1bf}6!1 zRRPqUBH{d>Z2yy2USjl5T&Nx2CKD6@0OFVE_S^ge*n6;8(mPq0{Zifj3+q&^Ha_!z z@P{;QBxN9QIP%4K{4|1w@#?Y=pls<)CE>W%p}9Rg%(@3flwq`?tl>Pn*1J>et?S!d zZW`5Cgz#D~``?Y7@aHnFmn+q36_{N5FJrXWO&?@f3{lL)ANjyvT;e;JE8*QIAKIqt zpGKZs_ErK?WH6s#AJT#ahZ!2d?tX$J0U$nd*RbYWh@c2yf6S^?d5{-iV&Xo(`gWJO1hQ!98)d~MkI-V|3V#HQLR+PV2tOkkU;PRYxJ6I%*X! zOfLU}?!Zm!#4=z2c%JX?XNy}2 zy2BPXG&xzp$1DjEz%;jq&!C(|heXq7Wj4_UPMlyeN*2YmU4y+-vhM|jZ6E_IOv{_9 zXS@&<1IZY8V{k4wQKgWeE{wvPFX2^x#&NRlY%~fW;Gyf5>(OUY3Cp$N5j@Ky;_xmp!0~LxGD9C`ZR?L$@4_;}W z=fh)12w*ymhg{cUXuaFij0hye)B9sKS1Go)4ewt%Q`fGzvc)#jlLBmD#H2c1oEYQG zMwmN~t167L&u~1}oFPP*R!k6ue9D8I_C&TZ{t+RVbcko?ffe?PB0fVI$1pk2Kd7wh zd0haapo5G~rd;;ZpAybe3NeJwRC5tEk=WvTQ&XTJlV1D~ug1623QMEGT8MKnk_l3z zUh_n%9XL*gvv)2LG9-qM*L#$fF%-x*!93xfGesyOM7`M_?NJqVgx*exzKD`9ru;4M z?j)9Suh(lXs25xBniX*Rz^78cSGK9@cKT0W0{HYh!by93lh6;##mM# zDi6HFy8~NFJ0KQk`1eON7TPL!UOAkqU&zAjVLDmC85SA`vXYK=0{O$GxPv0OoNl*l zWy`lCG}IJIO8!ot9|X)uP&s&*I5HWGAEOVEH8J6|;>di}t9O7j6ITEz=6Bfb^^ltG zBDzy$^?}V{bnmzcx)we`#sWOY_wd(YN>OhxCm>fJK7O>N;Fy)+m!A(_H2~7h&i8J= zKXeVq%R)u0vwUxd%MC2{#p=aDbZg(i^Vzsn-s9<2HDIIDHBuWkek^yb{Zo(@$t37H z@>D)%CZ&v8{}O&wn$KfGl&|8E@!07%PFkocAE|9e6IKK*YUcQ$cycB239 z7a;)t3+Pv4H}1O+0svt7tM>mF==XnKG&Hd_GPf~sv|{{K{$e#PyF~^R-`m={0Vyb* zjhM_f6pKE60?v|Vq&!9u+k=*!EcS+M8`OhdC-3oM63Xl`A^Oo%@6$)KTq83|_ka`+ zEXtX5VV3bWd5S?ng(=UQTPiAF%8a;!)~zw!*&(w`a6EjpQaK( z&rnRQy)#eIAy`B+^}QepB7mY-3tw+vmY=6*XEfh8Pdr}^-fwokcSPCPQ*yoCLP$>- zUc^Dg0ypk)0>}Qbbqq*CF{%2Pm2|8m0NcmgVPN=2&xYqXvqJ_O@*u;;pg z_5u9nvzJB|p$i z$|`df(mWUMiRprO6jj(N?qpyRt3z2TsIkJFV{)gGiQZPL2?&^8QPcp~axOc8k7HxP zn|Y9EFV!o`9y0Nh0KX=6%zA@{3V{3t3n4=ijs`8z;Vkha$6!M$Vj4r-lRNjC25pV| zDX4aLaCPJ(C8)23Fm>PEGR@hQy5-D?xrzD((Ucwr1-le8;XHb79YUkWDqkoffCVF%3N!+mMv%oO zG9>{jK+wSnn02eaA?DlxCkkxc^JD`g<(ROF^Zew`JLjn#A~m5Q3oP>t)SYKVmE9k< zObT=$OW!yk>53y1zfu37YHSkn%>nD8R0$x_09?x=ZdpLVYIaT&hD2Fs?33bHmle7o zFRelxTi7L6GXXRPv`wl#i`xPm#y@qtrq3%uj5p7hHh33;NA@S-g)BJz?GCkRKzUk=>lp3Bg2)0{q#bU|ypdlSK?xlbO^L0mOuNC(y}#jVC?HBUad$ zt(d1owb|rRMZBiprq8tg)pov8Oh8vzyM#S)5nbC#BaXP+dZCmY-w%1Auv}wTh|z^M;)M zws3#Kr};f#B;a|1e4XEl{<-`hnQuOu-fMyA+j zSVB)^c+%~FzGmk?^|0KTf@9eSBme33W#$ zPkG+{P=S)7a>E0xo6rnPba}S$ER3ziSoFYP#w1w5PO;VP)3_F1)8@}%p)GjrY=F;2 zK4RlbjLykaUxB2TDu-k^dqME@*?=mjx%@EVm=I3WTT$gk-hxZ@5RXuS7J*J)fzDvN zSJNUz1XE%7p3dZ3c6U%cTC2M`HDu#!hJD9bloO3#-0fxB910*euR+FA-Pg~`{gZHy z|8osV96lGnR07MLY~xox`#Z3w##J>#C@h*RMtUO}a|%*m)sb4D9=fL>Sbfat;$%X| zS?b|Zagq{S)W+X9iW+RrQ)kUCOV;(y1zkxE`wlrE5OnzIYCS?Esg8wWRBWnv2TW38d6GvAQ$Nw&vYaA=5P2s!Fzr*_sfv`1X zvIhE41{8#}{w?DyCx80fXCBs7sn4TiV;Et6vbVdrvWpUJIAq*m(^>0p>RUToXFHQ) z*8<(}>*Qq%o&SaTFnd1i5nG>6F{Sf6Wkl*dvS-v$XiU(w(n89;Ya8P(a>75q8Ka)g zGbQHBL+!0gsC5j-s8)_QY$VV#^&nSN?+K_sYYx&R)KeiSxwg;!qo)ExQodH}vB3K=6&5e74K9%UOed=R} z>TNzhCEPx_{7f`zxE3c4Df!QJj6XLF2BB{@uv%0bj2zMj4&A<7v@Tn47$bD~9#&lBaM;nyy(6X=p_r}n z-x|YmG$h|s1)`&i+=$0b9kzV#BP<7eizTDD1*_&FW;}bjr4Tuk&}pCj_(iW?_tq&>#OLqIra0Yr!Z4P>+jEhI^6|h z3m-2Lqj-Bu?pJrSe!)n81^fyyrA%&aK@ZQBU*FQWsVFoWl&rG77;Nbt7P~K&OV^2F zOk0DtH7TzoC7b*rnr^{cV9cFkbDGWs1gyrVq64CO=+G5DT(A1*G7+Z!I|N z&tGprX}n^ypR_Di2sgpDc+DH_=wA({k`3uWrSbZ8pRDt+_7*Y8i;F$Pfn2?3N?AHe zg}DP~1S2|i?CUDkvCm;b8DKJ`?*<(u{H^uL*IbwIhuI@dI7<;}5a$?S*USmaQm2dM zF%&14y&3frQAbt>2K`2sOuMSpQk7db!o()|$z6S|%F`&Y1~NHC z&Z08vmnIwMtGS*x59Nnaqk_G&v%l}P-I62opqPFGyPrgU!{w6z%R*W4u~P9?8{&6$ zNCZ?obFJdf7}gwjyf_UWq6RP?0ymJ!6-X=SyQPn0?tqVu{^Lej0nmuSdH4%8kLs7a zZV%~!X!8QoXH<|z(k5fYzYAqFXA_cAlK|Nl&rG}xnHwQ?^?JzH(JyTZIqM}X)*Dfp z=i}DCP-sf!#|U(=W=-n(Hraotvi-gER)bdQO|_~IR59_bJi)W!Q)goBpq$i5xdvopAP5MVn*KgX1*JEdIoOOd()UA= zY56B7EUp&inyXsZkD43opxQ565pqFGkjhcD78g!}Mem3m9I<}#f|Qn}8w|)?5a=JK z3Tw!M3t%3lH58&S!k^(pC>kaM01<{baWG<7VlRFqEJ9ing>;=zbvDgS8=iC|GY%h5 zAXkmuNI;(2MhbLLxH<7R|AHFqJd~i2bbwW!ZX@81QEg7{B(t*!V90G8>0x^6d5*b- zj5n<2p6GcVL$fw$F`?Q~wt-dbanaXMy45}oS%jR2RnBCdhyA(r`LG<1#zLs<_diQ;4Z7Og1Ryc7^?vRQ_MN6S3m8e8c;z}UminX{VT^9 zoWy|9X32zn!Yg8Wn*=ws^R8)Tfrcn115HNUEJ40xc;6W&+9Bf(}lF*%Km^ z9z0oqV#8ptIP}Nt9ol>@>_BK8AUeb(`MDSw*>}&d3#|eL!P6x3x<>R9HYinW0BNo$ zLb(J4Yf-CzFu45-*KeP&3d)^PuXs*|B3*!AT!oLqE)mlndMT9z=<`ym$l)Ej0sCj{ zOgAR9#j~vQMQ(FDaR+CX5f%fF+$Q=k^xO4Zp>Wk-p7&P8g2Fb*Mu@|&)ZCkz3g36m zF8y>1m&p!qE7$rtpCbG7@RK>PV5H6y!3o@;n**=cZ}6)fE^QMxf%rl5zDd&JxzZ~N zGzBOMbp@zyQPF_E_AlQCj7>2qJRFxy*LsXoaTmUwHU$DXPPPYeAbrV7o6WR|oHL!4 zV74K{n0f2$d3PM8JHgj=nX8h3rtp3aTp#B0^(@ZCJR1LJ67DN}9PDJ%C62fBL#=-Lt>c~%7yNapReYcFMbHH3B%4$`d@Z;_=mf7{NtQ7AUu^ySz zVMb$joKe4i@h0tY_lpW+4WI|yR+$U8TP#>lr}aMOmY>(WZM>#NcJVwrC!hQ zP^uLsuj+Xvqz9|8(tzH1qGm{Fh~UA1z=+%s*0d|CjMw#SItQ44^HHrGz<~j9(&hD` z{D8BA-g2mSqkP4Kho_mRnXSH(d^eu{{=H`sl`1kUddJD#xFU!l@tOLRCRf48vcMGC zbC5bu*jJEkCb$N}NRH9rvb6kVkpPuYW7A8eIy=5uTazJKSBKMsY zhXaj^C8y)+yXd{VYrH2A8BS7AyPbc#11k7G($%11TRIk&n!DY}fAy(X5BpHvY z3>N-6x4IJJlFvI!BE(&6k76aS5=mnRuo*bk#=_@{DIbWE*}gtEAUo((vt7L-y}N^5 z={u9>gZoGHvD~Q>gs#*CY%IW%Mo9;!t?=8En-loE$|$)ip()X}%8GOC@JN6rbV@k{ zXMzgB*Gw{Cdr_ypsJWMg0~CKi4;7Kbd%5J((a0HtaXB4FO;ZI&Y*`PSx{u(E4r6lZ zq^1>6^Np5xV8?^QrW8h8al^o9U)+JKVR1}6eB0NF4n-c2Ev&20nsasPZV9L9)Cda6 z%(vaHnDMyfg|-NV*U+uDEJ3LfUeTT74Cbo^Hu-r;G(f5wqzvi_d*KyRG7zs>tY%yL zSWQGOV1BP2tSA22#_h#VpRma^!R|FNas4De~-wWS+GYQNOdWL>!nnp0H-7P4m`fan& z7fXp|EheYj-SN_Ebd!Vr#k>!GKZbsT@q+S4=CD=tT`tdArEytvw06*OSa9WJcNERx zfyBB%W6(43^4`YA>}?~k7XcNDwhCNkI`u+RG5 zw3&_pPl>O)1i3fdpeiDSTOoTdf?H2@3@@`9%&%YAYApQjgtsZ(4iW^~)@iV)2;e6y zK{uxAsIF*Y2R?b3bJ*Ls+)15#1NhA}HfL|OsrTh!kS@(2NBmN%`-Hl=5K_7(AgY2I z)FRUPU|M!m1Qh_%T0WlHi4@N*LrS#7*u~qZgR;p)$j(EcbVa@g`(>8FgO4X8Qc^_8 zn?tobMwomk!^#Rfh!h{lfY(Z~PX-W`MbA`#Zpm#4I~D_w$(GrLX!C}FW(hC?^!e)$ zNRnbCL#aa5Mcwk++dqL-M~dZywY}5WLc?v*Tte>KCi^5zztZCH z9dzW4XB#+5Ah!mbbB4rAzVy(hU*=Jz{qkYIT>$gOc9@Py;k}nh!EI4@6_Y(i>edNY z_;kMeS-Sl{cihr#Z}JAN7P1}ieI&iBmTS?xBnp5SZhx>D)(PMs4`Qhjd*~s4{K6D`BEwuRUOx7ED*?)TdtAC-r03bQ*uQA)GT;0A6 z^aC`<<|ER!$GCWZ7QAv8rb$+r!^>=-birgOl3puSiwag7%FfAs%(M~&5^%nCbxXBb zNe~ces(u`H-8_?zsz04h<^I#6NfUelKmi5-Q2VuMaQ@SxF}3(T-nQ`kAA4q5ecLvZ z0j1}ZlExEQGYF`Hr5tIY!n!+cNoJ#?(xL%}y{TwKHdb?wAIp#kwB0nKD>q*MW|sC6nEEMO8Qpk1sRtNtz0l+dnUq=3Lh(!xu$W|e zMbSbrj3t3lKF3l%>XdmPrTr)vOw&SLwb}z)BRhH}?sDQ3Y^^(>$RVs%l~kQCr?!g( zHLCP4|DT6UE@|$6Ge~Wg!psYu(d-N&T8&!`RT$l)>X0dybre($0nD@tY|-1$k$cUm z<#TYnDRMlx@VDcgac^g22Q;(eCgjdXj-c^1K4}z5CFo26+@*fR;rs(?BS`&-3-WWJfktsV==o-n#za zJ?Dy+N4N$w4FD4C$KkciX_~|lrX8-=uY1B0wU`y7yX`fm|E*nckVL;h7zZT_7B9Dp z^8-Q@xFZs;JOG2tji*Nt$YFjKPeG_B0w+R7S_TZaBs$4CZB}hXjG;Xp1D`=A_@o^> zbhJkpj(twTnnK&|?RGgbV?zMnzcSp2A=ZA~eR2ryjooY3ctdV9AKf0ochcU7?we~H z{0awVZZCPyIlsAhB|Ww4K6W?k>NxxqeEW!bUBJXCBe#zN`^+lre;Z17PwQqF8^^IK zmC8^$J-P5x_Goh8AQ9*h(ahRd0#4YU_vFm$Tk2}s!w!4zB^EYIGJR_iDAys}O~}^} z8TtYG&(%a@UDJ2*yP5`20RXW6b2VAm{+i=1HovB)iSe&Deyk?t^j{+*pK9iNhRh@6 zvf1N+l74rR;1hs!Si?I-w%>w^SP!@HG~*4%Y_LP1rgzdx9?c&w-77T~It5L1P~slC zYB0wpN=h+IZ#}{My;pu)^6KCx-RpdvvRImrnpwg@`L4L)G|j1K9LiaM-Jq5Tg8<>D zUEA+lH(9GH#X}=-Jfe2vvlo6;Bb|2=os76sw zg86Q>=uNhM4?Z-2P2vKd^}S`619%NkZh|z`wK(7-8bqn=%TZ~99mO=gpfG&m>nM56 ztOpO4$RM%v=wK6{VXNk(zO@u_SCqt75WG`-G<-h8b}5r?x%1NX!3y^OEdG5t{%a8P zXnuS9`aZ}Mf)0RL2^%0Y!=aD`ty1QyQd^zZuxeE|7%68b_SN!~TDgO?z$$BycPj7h z5e*S2l_L75%+KPw)s0f5Z31$AhPjZj1Bfrtycoa`f|i_TD~0%HtZG!S6GgPitSA}d zRAq({4in`Pqgq0DN;f{hze$vr}r4 zJjgcf?e)l%Tn$*oN4cd(-11Vg8S`2wb^}>|`Yt^R*LK$))}eW2o5;f0P2MZ#^x)q)u;z4E zwGiIDdMMrf^W)&@E%snHnR}mOrH)`r*T^MK1kDS!DaNpv0KbST=pNEU#*KpnRRf?# z_{xKKH5^+tz#(xz(OV{G?OQsrISao;;~x!+TrYj2X9=kje0mYLPIWq7-Gk%E*2hPw zF86bAQ-YNje*+J8J0e!{KfvHqO@WY|0vkSCU&;+!zUXJHF_MMuT@3|@0+5G65B7UL zlLlYBImV+T%M7fb&j2Dm68Pv zc-BSmafeC%xGI(9Uiaev_^Tba^RfHCbe{D^>_@#zwcy5Y{wxepR&$zcC7u36YCMwv zr$XR%`2X&l003nW006lEsSuV1t_FGrj*bQ%|9u{ktSV!d$&b+eNUfU1FE}>h02fd5 zM`ocSc|&Sb$+(*AWo<+gt)p}3x%bxh&VV5;1)KS{JDm*%y64tQ4oF}>>XDbf9IoA+0AHXCm}zR}2=ir(FgeHIMmPsi(yf>ZR8exFYAl$DRF3T(jZz8~ zN}q}7j?77wst9i1168pY)00+;GKtf|B=T{ak#~S=uKVT&AG*k30xl;* zUIvQ0fWEv5G2$-)HHIPulxCQBpkheQsSSp>1YWS;queyig|DI0D+tmcICaA?ZhV<9 zs=sgxbN>ZXtAl{*p-I56D{)$H945jLnQEX`UMbxMDP`SYy#& zXHNrPykYq-I%Gq0qbFOkSKTD8s$*j;nSGH+u07(ltmp827?3`~e?HU7wAjFpFG>7; zNA-d@k?sN74fc8a{4`Xs`-{#WG%}3g$e$Wz2Vh4B02vlX>?3k`Ahu#hFYy_6-UL)i z_w-V=q1cj-7vRmML?CnOyRC-s+X(t~kk+1C0edz0;%m^5rzWxOoux~wF-xr4SEogX z$ZZCuCSeEFa<}&%6Yg%KI{9XTtAuF%3qTm=ozKp(DNFwp_bBWl+OWQ(j2z~8-ZKKPY7t*Z9`$$Oc&`QY&_|*f?+mbWY(zCrz!MPGO zi38KoO}t5a759sH{QeV+uOSBvz9v^9W~-IT5I9&R>jk*P{B(pe2agWoIx3|lZck#df z&jyDmeET;E5&)o);s1A{_rJ^6$iUj#(7?#*zbp7!(`@#4oBOw;X8?tdq)-4C0-{`y z)F37@iIg`cv(@S$aj-|iFYx6yL+l`$30L9=wZnS<{*d_aU%pfEI7 z40qw?wXJ#yCYy+AqBbuxm4JFLQ777Z2MJk^-ugB~S_s&1ZIV$GwZ{4o5Y_-voPR2x zy4I755c1s?ct=#?n}MLE*mNWv8Up#A^tK~7J#_=Z9%PbM1f|G#Pb7+qG^*>K5HEY2 zozfAt7+etFn1Bj#jCOW<5Q6o~M}@xXngk<02@aRx<)Eb%Oo(F_!m6gv(S(B~$}(2M zfRht9ctZKx_X2r#Py9RbA`0Dq${V#64~)yA6DTPJ@MFPKymXt#_~HX+BDw}XP?EOq z=eKbXlZKr*nb0WIuxU3aj+0esQnQnjC)iCK?4c~8J_6Q!lG5Zf@dC8vwW#{FvK5r0K#0Oh9k^d zMJUWNRS6W*k~ZvtS-W5J)zl7f4QHumPxMx?w`?qEJj~rf_Ap8j;XT`r5FYGPyffti{KPy3mE_?+kh>(6k)3v z$s3hBIeSSgbt7|P`5S<@<@JUzeJeZnbe?!`%qsM18#(!I%nE)nE0R+8GVaGFINELiz#LT~9t=~={UM!;L zHRNXC@XzZ!crV;ie!>xI0h=qk)b7i?c0KPX+Tr_~(+jil?c@Px`0hY(cl`JBMr)nu_VIicvCQviT5N~Q8em#Mk zPALf~tR7U`^7?aVk8RCQj_>;0|JoCys-nqin&SJMI{h)gXm7v?t%+`nuX4_uKz81be zieH;;{9S=z9Kac-h!ZnW+T)fh$O&&II&r;(OthXKFBo%vmPV~bvyz5)z!Q?GhlRTT zK{PdjkaL?DV$^xDwrhT9#==~x8Q{sxJuF#!Jz~eP=$bo z1@FC`kyegSb@?{Pe^iZ`l)s2lKFt&`)*j`l1R)c$HsxZJaon#8svl5kwXzBy9Qj1} z6z&S}LPi65Iuu07IDO&>je!0@hnKTHyS&;A8mlXaP$%?v#EaZk!>uc}8g#a%WIfNQ zzn=i6#F2|MYUVwR1+DkMZ;Q-bicRcDm@>{4e$L`+7!CmXx;N2VHm6W~evcDbrG3iu zi_x+(aFgLLxAW$fD?vyvzAHrA{W1PC?y_{S5X2PkSYpbYn_+p;UbpJg|$D@1J`_#3Z!VrzuWoP+m zm0ktcJz*K!|J79j`qWWFJIKzv!es4kfZ3PrE)B-Yi zTDP3Xjrq!s?ehJBm03-SgfyPhF{XnVP$1cJBgLd1srR%@hUCgNF|v;t+60q)Oja(s zo&O%{-7(mA39|eOO+_Q`WUZ|Nk`r>pRVPrapBW}JJxR^fFiO-+Tooq`wHs)M@Z|%+J%Q?RUj0WtU?ziw>sQuum`2%qf_H zEj89M&40ZzZ<1OO8HE*zKM|nAH`?WY%b4+1FvGq-L#^s|N!Io-`azxn|McTM^Xo!e zU&395Bk72JOZsgUoDYy6XG_&tUO1zrUi?HfWY@M1;i-Gi_L!*%e94JCh=Ig-+@yuf z-H79w8{SM*kO#-GLim*0YvdIafd~E}H}{UxNb^;FnwBT%Lrs4`D}wH}3PWm{$y^nY z{zDd%Qr%bnm%ZzOYSVd2!>iyV?o@jliA+Anqy_JDhnIYTuQTtEC(Ym$J=Fy~6|g}L zD8wi)Jz`nvHj-lgediDD4CUPclA5}`V`l-+Ac~t0JdzFs{lS`dj%gz{Q01qjZB2(f zuK%R<_imm6>2?*#RAJ3&luy7Q%ZEn&%8y9@3)f&()%XRK^05R<75D45`E%(V7;d=K z$K0lxS><5G5FdF}+Iz8_vVT&GUsX<662e%#qO$PK4{sf*P9%6lr((v++lX1a=`rDG zKrAZ>Fru&t+B($qbB$?lp_^B`v{(LX%Y`GSt=RixGI6e^aV=pqlpLMy3jPK^7>)`` zH9t^LA%41Xrk)<-|K`Bb1RsUP{YMPKitH2 z&f)|&?|3Y&0s=o>AbGAys^zUXA$!2j8htYN8Z&HY*y z%t!zL*#D1)#Mt70#ej2S~8*C+?j z3SXZxN860N9R9P8kc9w>y50it59WL{OMM2%Cy5{jkh(^482_r@B&W({Y zPgkKFtf*WvcP?gLs!J{4WIP$cm}`%BHhB;V(x~GU1D{cqYq^29T|Uka!yXUvAFAVT zkHo6)7G;3D>)Re8@4{4)Q!OiL{Gl$VCN ztjs}6s|PXtVr4u_{Ye9av4R{{Oj+M{;4v^%w5Nb7JEQ%jE7?M94jv(%;vD{Z%F-?| zSat=wEJOXP50TPYH)&V6X!~y83qd+udMcB;Zlm6n6?HSib&efM`6j3{Pge&G7VyB zUfFO{>ynm^2oC9zWAHBfV$PyQk8_eMmOvS(qr#_*p4my`J*PRuxkM6;5H?t`0gz;p z0c1{V9@@14CLd$V)uEFlry%$r+rX}EkUVv64@c>wHG@obE!+BlH#KXXBjeM1={`0=W?(pV zGbHIUeuL}2fsL(D{=+5T$}wXVM^jpEX>TApeXh6WtZQ#Z^p5u5mbTnilm6K5VueGkkVZ&Y^&1H&jzDu5>l?) z);VFf^cJ`mj(8y#Rd7prW+|*T)&Q%J&Xh-L~@6S2K- zfd3Q%t$xgq+25gF(J}x4{Qrj#m>D>mxc!eeagBTJcqn}JH+P_IN?*tlsg);B2vf2) zIIy#&>DcNldQ^h7oMRNOWl>k(oggO=U1;IP^V^2pV*iaoEORYEbwN_|`n9Fy@g{~9 zYE+kjXRfE6cl#Y7*ZRDp#|4Lx&A^Bjrm57r2=;?#`sS5yW8-5B?)-XQ*RlxG@uB)?ZYZIG}90=ezZhgGtR zA&m4Q?Ym3=!6vS5HT)KKSnA#Q1neeB1Oya<*zV*5>4@^DgS1Ch>|t?A#+GBmdY^oU z8%m5OMgS7&F@P`r&R{D4e=+t>?V*LymS$|*wryv}wr$&Xc5K@=cWm3XZ6}qgr|Wbd zT=doY0dszuYm9ds*gr;)BAuQtdbqkeZY|VI$M!hZ)X>Yt=sto(4Mw34c;u+|n?C$K zfo@OGl7HY(x@YAT59mqzYNLzeLVo<+$;@$p|$O>iV#BQ>> zI=@Tq7({ttq06I9HW^Li_E+y%s%%joJqPe_S!pr-RnBo(Uz1py9{p-ydO8r9?(X|G zJGYt{cbZ=oJ4a6^ndaa?n#^Vhyg*?1L5!$%4u^XB9?@B`cbH%= z?e{|nwnMtO=pkN&{Q*a%WnCc&2>sZ_m0|nKkpxPJLM&fFReG%%O*?#{+WuHtJb8)A zgp{%1Uit8TJhz!_?{%LQaGVoj65bH=t#6nTOKVF@jVxUcZ##*Md00kQPe(_Gi;tsA zwvVBm&S%|Av6F*`3~7+8#uILz&Ih$4$>lPL7q?$;$fK+tCU6 z%KB6L^HSrZc{cDT6b4Zqs(=Zmu?XjEOHbr-?&L(g93A@Qs_6+`1 zUS+g8>bB*`=fi1=AO}`)du)8`l_&SRSMWr`ZCXLW&=)U~;TSH+#&V7OYtM_QGqA zxr9XGH;S_rZJW-tsT8amGjo30eUOco^pA*u3o(+rQP|+T*BH%S`W_#_q=-ok{;jF1 zN)wnaKC7m1A~g9&kG;2UbZC{jmEXvY`zIQ!9YzZKy;y~X26HK|X(Zm|YaRMI?en6$ zW7eP->w>e3xfna~t&OQJBXR^>{;H{bnKc5~KpMab)?v7=ex7uEkNhEcjS=>PU-9{7`mu2Ed+DzCl6|r|^YW58Dn=Noc zQNGD`ezx#s2)dA8edyS&oW1dW*;0FXm8fH-RritX?EjSd(h64PF95(;f}8{F`$`ufj)l0iNCluThmt-tD+BMK_X0ga*q1O7HouO zl-3p^NGkO~W||mME7KpI1)OU@6*G#VATSBhwu5;~PEmH?&JcIVAv*afA?KdL46Go4 z9`?=z^%P}A5J#%$OOWI+&jaxcvisG$y*1G@x&R*c8N^eEK0y{34Tw_BVv?l>=#af< z2ZuEo82o?+ffkh?g{l0dZZ6=hL=l2jUxc><{HBOu7nn>C?;(X{l*RhECd1GTS%l^U zBosm}iH@Yz54ojGbXDA$J{vA(`+Z%(ff zal9QAihv;E<#=P*8Y{*}wWDYgA2CR&QtWcLA1s=7Dl^yJ7MuIoY40Qa6Zq!qp1ApE zCnO%;v$K7U8m#P|u;#WQA>1OvPpO=fbrkC#kT4BJ(6-s;R~-fx6p-?<7(i0_q!inW zj`#`5X?Uoj!ENgJxUo}m5-pBJ94wp|>phPt6tLGcv2X$b7rSWUu^79cBQnKqCct;V z{R8L`5BUOwR;L3dMo`W$Hmd?O;1b|$fLNS*zul?n%0qmu#%~+$fCRqE?DqW#=fPJE zWpz{Rl7%BM0HlJy2fol$+d4+2+tp!7wHp&3ZyoW7HA%`8;x3KJFse+LII_y44NG4_ zUf9tJBh2grkB4@PlX2spo1*i`&u64vq>NTd6iQl>*29tk8NeRvv&fT8=bD4ne&{e? zL3u!uF7i$L`IEOJ5RRyV%kr3`XmOT_f@U_ntPJX?IrL??U!;iHHj`cdGvV9GlkC)Q zfU3CUC{L}(m4@dIenhb?mIWaD^zM$D{{o7yN!Y6=>6GaYL5l4tbqNu7*J zG0;fa-HOS7W~Tqsv(k||D!d{jELb0EJ^6uQwtyZ37C+1lW`(*x4ig+sD7u1Icr3Ed z$^cO<=`0I%SnOPp|5N3*m#sV2{cTM#KmjZ4kHp|T|ByI%Vn}6yYYhcq09~9PSXe|-l(g?#+UWQ$U zZ>WVDBnr!yHDmfNPz+24=yE7=)w>MHK^CnSoGYK~kFd45sMP`UQ`b&yH8Fik zB`QozijJ;b3d(IN_dF#F>4inUprXnIr-fSo){Q^4DDFqP4U|88qI7}tWh;sjrf0J3 zLQp5(o)rE2gxc zn(i>zLiC-{l(3L%3xmSU_<{^ASQ#l$bWgb?xo+ru{7ySwsAnh?3c5rzrz5AX3~a;( zYDiS_hl?+!289Cphngq)Qu&fn3R8F(aP}seqH-Y|+P8&NFuDeMg zX?QdVE9s(PHbnfN<`*LjDnB>z0GxHkX5&msobc8R>CEXnY}w*RKP1z2D`G?mq2K++ zmsh$gl%M3&bC3t{!(d>=UuhUBU{w2RH#*c?*N27f4O0BOO#57I#-RH5NS7_@4uq!R zA{JdMD)UR9xHIfCym0ZIzPA~G1;2T;$>DAeb3$RGOA`TY3`w?NiX}yu?O--$xzELs zelv`%PHmde$XPcn>o?ZTypg1`!yWVbS{R~lKEs>iIns}Ky{T3MYww%;TnbJh5iZF` zilzuHknslWz-(x3;0PoMUj5&y4>d&Ml?$O_lGGaW?&=UTt zXuCY!l^gyeYgy!GHtbI?)(+pRw44~DQ|%a%32btbv;_p}C@fbj{$;$J=DEyX|HKyt z<$TBDQ2&YfHjOV7c5#M*ava%moQ|;|H~!SW(AeqB&pTlVfw+)a(MyuYL!P3A=DvzLkbCC4jA=|H@>#@%G?Q&CX4Q9G?fPW8+^P`)JgsOtTf2^+@kuH` zrFvzyqmmWy>g7s7amg3D7~2MK4>=F&p1%ffiztGw&+e;fZv80|p!o2O-A*;;8Ctp@ zSA!0WaOw?|aFV)X(H3a564b(F{$88S1x4Nj?+9kgKT!11NO=P&?s?H40=NE$2L1MB z%T0aTNQ|L4l$!1sl#Dq)v&r1$D+&<%81&UAuofqhR?PZm;I{c?Rxnl@ulH@;N1XMU z$w1)l3NIELN(ZscY#Nuw(&sx*`|>L3uf6#KLPFOO$zu0uLdcp)wGLTn{; zt|j(~f-}0=>=LsU-^YR5Qxe<-u(_?TI`YmxK6fD=h6e^$&?pvgf5F+;cG&H22l|kanel;|Q=V|Y%%E~I9)HZ6${~n!N>JMB3#3pR zO$#Ox!*w|RMKJ$*Bk{m+JOL;_G1HL8Hc!|GMl#`RFh@4BTOsI|C9uiZYANxWpcku#a5@6ga<#0!gCW_sGv5Gy zJ`j=B&o6r`yO6vsSHr_4^oi|}2w4jqg#?3{llhTX%2qe6aBj2V(3@wVY=$>% zhC#kq^)DIq;Bvnoy$qel=;!*F8bmu@UgBdYu5Ch6>P1wPPNiejvzF`6R8O(B<BGdF+N=FjAmB&p#cDW%;J|C^vn9Fj0NuF04$wkI;h0CS*jgj;I9jPQp*Kw`T8b zuCcq&Teu<&YK5+#BJ5X?_GiU^O}K$wS;Nnv<F5b$|31_TcV#f!0siJX;3LR$L-^`dX0?#*K6^HF-|aXS3z6J711L z+ju?eH%$(Pe@}6RKj{KccQeefWxFtXo2YZ+*3z4eQ+jG>dDFQ~EjMuQMU@6pt{J!H zQE7CQB8md7>0zHUL;+bi_4tavu)|TMoXS2Nz~tldWMdhsZm;nRmukFpFGj}MlV~RP z@zo|pKGA~G!H_;NVh1Q%w!qqMs~c~uqs;R)tJf6q=bFK@MXMYFy~PkZTZ|69X34fO zwfqrD=VbycQtsGmRXj-#ic?!n_8TKJoOfp~O;9y(V~Pw?0YL`mt&M5R5v@spJ(wtg zQ#?de0y8y@6`=*sA)d1@4U-aphGXxWg@jlQ{&9>8Eg{NG(Is;OjwIAHTnp&dRo}nj z#RK}bvpHjiRk6Uqnk*uEn6BlwS)X;Ca6?H^%ZX6qKs=141_XoU74;v#HaKWv0s&u( zw&p8&y2>Q^6oj>c%7eCI%cZ%?^z3{?0wCm<+uvB#q3ngxT}~j5eQ?MK<%5yLW~$d> z*mv?M#yaM1$|$wwl(xYy@J&RwJ+f2;t|Fe}zSIyWM(I_94#Sq6Xqby8m?}UNA9$Uv z`RYrA_sU9m+Fwh5xSD&glbr-t#IO01dIC?CS9eCkg+CDy9FpOkG0#wp(9|NQz8utQ zOjxf3gM&R~kX$%&uI=|A5BcTcG`!d4q~)MNP3x$i*vG|4sdF+oMluHPt+z!Y*e#6e zVXGNGY@qLWUUrY-WD!QkMsD^QfiGTe5=QW8^_w9^E_!?+xn*$S)<>(v7PB_)6kM3)Q}9aT@O7@c91Rq)X}$N|)5U7=oX^ds|z2 znp<-@#-UgL;`wPkuA;zx3S(El2Hgg`7ocwq_op}C;O`-Ts;xn3Er~L)UUW-sGVST> zVg!^fH~QczGr^C2Y_rezEP!h)xsXU;W^nEKVubT6O)z|)K)~b9iubbFEcWAGDR}lpmAfhA5?vt?-nZnB&`VzW=(Xve8huYj z@_cVKG&#vqM48tG0S!s@ep$w&Fjt}uctK+VI(ixh^#||P<2V1wQt6WSJ32q@9R~4& zy-GHW4&X_;-lFOV=gE_6B7efbGFG;#eMOZ*7>Ads_!K#5&V^-R*T^X!5q3PqeAMSv z^vDuhSlxW@HqDnx4gyK7v%J5^ZkE1yA(3zI+DEakBJ0i$Nl_!yC}T#~&3q&3o*X+x z%f3h`aDGM9wz-#6kEvFTPGh^nZsz2*1I+5v9ehft<)E=xUSF|%4DbxcK7cz$9>mz@ z&eKGzx(JY9t8MLW$(SC6Mpj?FvvZk`q5;!)G2uNkY}*(*-d*~U_W*hf$7<;ZUFP=5 z9`E~Rc`$Yy3FUYLJaOKeln3UD22?bc==j1OVe01eFDMb*gG^RWocMJa$!4_S`bQM^@)cfm2{;CUi27U!Y$yS=tBK- z7<%FZDkrLZ=$A?W#?g(7{w>c2?}pm{tETGz&3ISK5`A4x)h?q|5|N5un20dNDjsund-KJLsn5|mw9r%{o=jf)g8yELU(x4X%7t-fXe;qwynp}X>gNVIEpf>9=e2}yntgJn z6<79Ol4~UXMR+*<09m_(O1t~VjP*{E>_#xcf7KD@+WRG^-~}alWm#JTQ)i}t z@=^KdX%#iEpYC3F;_4zW;M*g6QFh z34%T@jR!NEjDlu18k@xYjHs)a(dEyf!$)yO>=| zCGKf{j?iDXlugV7pw0ci)`}CP7u3Y0(ak*R^|0s^D@tGwHbi!5PXO4%65g>1-c@A1 zG-$oLfc}n}U>dOPV6a5lPg4>#P~K@H$D8Oh4JAngS4uE7tzXm4B9dnQU!w$gh z&hn*cPS1N-tKDmLqv>cQg6X)+=y&oy=OT>ld^yqvmoFbK6KPV_j+V|aK#64)2y7`2 zN^5adoyU3|;A};hD%JY1=2KKz!i3h2$1!UGDKdY>>6{uv2H;OS+l8O|>xSsno+Z48 zuCP8}YS2Pnc$KL!7zP*9_)5RILmtMGo}9F+>eFtZ?~zaDP&xb^S5=u|{&JjDQaiUg z8TuzMvCB)8at{FimtPx8MNkBaYG(EV<^5F?IDax3C{LqLC?hrL#Yvqdc&VxU0He8u zca%?udQ5_dyf8`FTfYq4Laa)=X-e)fXp+|^ID}5&fRnKcvR2Lza*{v}p3wr>K0soKW2DhA!`VeFmFf-Tz2;_-mFlUqCa18E1CU4 zUzj@7%vZ1-`vj^WOF5n+Z>N2*Wwkf7fqZlI5CIo88I(yx?bkATN`>cn<{5;9WmjN@ z9asWHJwdeUZE*yXGCXN)oeKMtCX1#RN`m!J5M$WpeHxz`5Z%!eLegf0@BCx%o~J>C z{m6^YTf4lDY&DjHwZo{k4f~pEwtjVI60}MrNeQ0n81uVPxnD%vzF|X#no5lz7WxfZ z5W$y|+pLHY#Lql9Gf6>5NyPPO&Y)^gQK66X9Z1ywNx3Fw57-Qh5+-{`oGl(spgc;W z^(SLmp8x|vWJieGse}VD#)FqbNWv&rY>LpNWOWbM@fa6_#EOVE;buUgB}Y3()6`(C zBmw1;-AT_TS{n9waxqjMBUHPXSP88UgEY(>sjEoB3>%MPNvAiT8YeS~)8Q2-sg~53 zsMravWs&f4_jSV$*-3V~9}!p4dWpS&i049kdVv}Mg3qe375tgVN4<{;YQ*?=FiF7d zgp;i#Bc{JcN%RT%pZ|rky{HA={&KEtVg8#d?*GV~%}uQB|F6V(E2YnNg8?S=#si9| zV^vr`Z^$1A*byTXt;4$Gl8%AxfFhYzFd?x`j$gCo!c-yxXL+#NHo;f%&`j}u{v0S+ zUA4P`1Z1?|{34&qq=Feq(zBs#6UKh8seiZI7qi@g3quC$Eyr3-h zu)b%8FrKlopMoo&3z~=*l62Xi5=g>=Ef}V=sD97y-m!(86nOhV4H>^WG-t^2ebm!W zUwCAVz`jRBCrdyV%r?p!7Bk!@wK%=A%VU@+n_3Tv*iy9+v5r5%1cT=04zutH%7t8q zDwgz$%24PLbTX40o^_a)IVuBn6u5n@adC@BUCry?PMVU#1G($+tJXjT+i!#kEkXiZ zyPx?p{I?FS%T=R-kRPyF{Q#`8CDQ(ixKLpu2 zBvSafg70+iTiRe|7ILc0ap#HC$k1W$@E^{o>1W{nM7+BbB8A6xpcR? zdtKk;(NN|LJ|E|}17@%PB@d`{_1~f+7y!T;#sAzX))r3A|F=`7G;I2II(CBnam9&i+}|Uyng_?T@QTBLL#({(Myq#yCwHF-UHj zurZHg$cusi4~_`lWD^nNE8YR@-M&r#$|F;-t$;WSUbLuHgCIX-7{$=0_8iIKK}gdO zL@@K89io86_YBT6Mnql2RA-@K5kvAb4D=8zxyz6R|mm~C)xI6bG zrz4QY6NF|e9o%_sb%&vK1^&e>mBp!qvo!R6)$9Q6e3LN+=#+EbcJF*5r8*p3eXC$c zRKNuKvP#xJ=S7Zn)G%EOSQ*STzh+uO2B%;VU-G(u$$pXpSMpuzV;T*IaYeuHc$2Za zts!00o@9Iex*Ud3_yoVc!)UMjocmI6*+XMgHOQynIp^pn|<`QUZ7 zX>YzziAEh@-7NPGnACGa)xDdM$IlMY?jG(0;hS>}T|rF*5~La#Bhw@Ym%qb0Sy9?(oE^N5&l02g9TS?% z#Rp%FSN~2-Txz#8(3EJfFOs!n<0BU$qm+yfSQgT!S#tY#cdvbdx!ZyG~BgoT;c=u%w`^)#JPFB4A|!;j7v+|T{2#e z`UN3XA1*l|WmCjHwiJBzv%(HrU8U+rahsop+u0 zgOF-<(3;CmOzuR1AacD~v%w70PlE?HhdKk)$+!sX*9?7WOdk?Or*yNLPuAAX!oN0_ z4Z)jJEBUr7hua2igT8uM`0Ol@s^-x25J*HvHGeci-&9HVPqE>Uh$hl3b6WPJUY&xM zuZx<#Zy?ZEGCZD}Rgrd3E5o6Dv#q@ed%UrpWrxR) z-ndu~e%TTChJvw<{#sQ3!U@Y77y^aHcBdy5*r}JeE^Jmr|7g z8lRM}ruFa@ZBk48&nYGew7%T8EE9&G!nbA5D5q^$ry4b7l%zt-pk3SEw!Q)JimDw_ z8#1Xo^0e>Lv>o=;Sgkt(8FY3Q-fbq0ZA&Y&o>8iBNo+kL8Rk}c*Vm}vq#0njxZrF8 zS(+pnY6+V1Ler*;1}S9}lVLC|lEHi_2ugV4ZB-k3=xjwhgm~!lFZNX*oxvPSnLK_? z)bt_4_7kq2=gF4=;z)E_=ZhHnW15!Fh|$st)hpRQAxq9=9%t8n1Cx}9PN7G|R%oux zh6?zTGqDkte3~mZ%Gt20pyhh1H>od*I?J9^D*03U>&V@_T?EY*6Ak(7b`!OYYoB5x zGPhEvn-x3X-HOz0#7V zUA45Imz>|RU1o`kH8lD@)h>JJ4F;H|{#6JGEo7;|p?A?o?O-Ad*Jq5$~)7lohnKC0otulEq^*TD5ZNqub${^L?Jq@riF z!GPfRQaam-jve(k`-*h3ZmtG9ND%*DDN`^`E;e^Vv4mrz^<&44G52?35v@rbq~y`Z zN5@AtUCjj-I#gOfRCtEApyW0fiS8fh5sw+HMnp>y#ga)d4rQ;A^%F_u;Q3C*Xdp=m z1&MqZi*rL-ro2vuJd71UgL1S^MgeZ<5lq@>!014>q|F}6frKq)&oB;GYrpVUP?KCP z1%dYoR;4PS$vpCDwzQm(JXPoL15S`wHg&L&+^Pf|io^s-(&aqFGE+|ZE(>WgS5{v~ zfHUrxL_yi>-q{F=8NFr&6#XUEw<{($kpllV%J0|#(CuY+C%<<1)3#%2tNVP$eWAsd zyh+&pJ2;oH=Zip#Q=@ejtV#AWi`{nA=Mmq~1d>>zm(TXQ{GhQWkCyK{94?YDSu*mj zCk&y|3pg*wLabXKvtCR8a;eKj#?AWZp!>3VB^d^6B7u2EF4Gr#BP3s znDQU`xu4LY7UvHN=L`Hes{S7LgrZT%u=Pr!ToMVdtHpz}KmzZLnRFub2x|!iqs8rm zMQE2+EfNHJMqSZHMn76DVbREXU$rH3S{&cnWI-@5B z>_&&;&mryBVPv8Prn9j)>$-5HY8(md)+FCQd7x}+}q1VUd6pHoLX zD;`kpczV4Y1CJWvi3f|+g{B#{v%9VU=E%96)^M@Jl|w~;Tks5rD(S>*`Hkmna|OUY ziRA*g0sDQ2>ObR(-L1OIo7UmBMPIBm=iC<&5nGc-I0u-Sd?7@pQ=9`d7%dfy<(F@eKtfi&r@p zb&b_1jGuu2`NqG1B}ZZQyEdMFvEcvH+Hf-Yk89kPs!n940E*9xn)YR+P#mk)dmTa3 z7A9e-uEeC|WaVedKK?-Ml6fjI~tx)P+gYAZi3Cv*tlb^=o>4EVubDM-q=UWe``I zehrlwoHRISohtzR6Jx>?kDY2B<84jL zy{anpylcNoP>S!NYpXd$=Ng;11zhMQJ1Pd%F`PACYcvW;tP({3!0vK*(M>?&9B;LJ zGdl^UItqajdL!Q z{+y$V?Fsr48-0PCc>{RE7u*EI0?TdbH$bG<=(4fwCtj?-ksl7?i3Jn(iE!`OW8?7* z3j@`q>6d_6&RtLD%pc_X&3puGcDhg>`_ISGmZcb{{jJ$q{;I_S^3sPs8B&TeDD41y zkSRwTKD_G)L0ucIn7oasT=BUg=Z*gA4LtH!Il24XoK*gon__9XqRv1`O#yZBbfD7Z z;s#E=my1nP`<{Bt+Sec>2;st+n zvi2^4P1>&)Is~;VIl4G9Hohq9x(Uw3prke_^VEZdk^9w5UhlsNKRXAn+U>f0Cq1`N zQZMj1B)iZGhPse@H`Dvy=)?Im@Bg0!lK%~)2i1^NO#G!TGyJ|t|EK(X#)%}VBkqX*(M@-eIZ#SM@R1)0|XwE=1hx+v`+NT1n(S3@PJhMC_2bukt<8! zBqRmZbq+=9J{LIUf-Hl1vS692lxE;{Nsww&4JmS}LQZw+#U?fU7H+{8iwlXRtE#(f z;0skG)n}E=#;xs%I?IZDq~{fTXP6ok*NzQoikK_M{dama&IVg@z+(8)Ta zS;$P50yLbdeErePd)Wl0>3qkZRB(Yk`pCGYFS2-f*8YKmT&6g`Cq)&BJUMN1bf&^*TYX22 z^Z8vtZQI7T&%p7neS-E@&nH&?6 z!5Qy))%N`h_YeVGljAM&O@yQ93G1&<*tpG^xoRcxaSkk#al3qlg7-@MjGF8T9N5%g zfSAU3Q=|mvg*M&X=n<$sQ>tI_CnQ)^Bf6#a!zCJ*|B@rvukmsF!5`G#5B zRUd8t{igmxA7DszyK&4+cxfyuK-Lo4=&uwkj*Q;d{;9lPi$f=9r(jbq47=@lxsf=R z?W{gF0ULLH=37_SB;Js{tNiGi`f-7211Fw#D|2h|xm6SVo34KCNvz|q=T#h4Qc6@E zHKzGZpZJ4RZzrJ(=xRAe{OVU-V-i+kXd>dzDK0?(tRe%YZ-C#YdNfnAam{Q=lUwy7 zbnpRlew|M(%L?-`hmXBC4!BnIG$8YggD(EL%SAvp5=UHZ%g+$_xcARzDrp^?S}9dHgquyp4&Couh}IwTZ$1 zixAK`bNt`i*-q(fFh3j!yu<`&BBrExK8{Q&Bzx3|OVaS}Y9IJ-Y<%TFiucgC2s8(< zjkQ!qWwL9}Ov2H&K&#!hXbOAFR;1YJ)amInu%oHnz85pcRp_hP>FUX7&J-_y)Xb#x zN@wpk9>14C{e9yo8P1_y+IuJ&0{_FpORxi=mkGRG^1LBY0>!AW#*y{Fn^_=PqG zVQ&jML&0^tz!~Va`zRfA!tbNS$0di*B&8Rsmgg?F~WZ}>|oHS?>|ufhTLTG8RN zUF$_8F?`V(aPr}^wcRgp&vMFiLryl=0sjc8%=tpKv;s6%X|_XWL|I<7tay`zW3X~j ztV3nZ&$F-~^x?>X*0ABTy;Bcp4JB(I<~(?9>yVa{7WrL$o|y?!J=n;PzT0}#w}3BT z?AKDxOm2?ldmQc*4Lvo!I@8%g@v9Wzd~BgFELTkjW0)%k%TFdtl(~qp)xb?H%3) zUiDk}If9Wix$P`ppT@HVLN2>znZw7|<;j+-nP`ue(`&LrYnjj&k=4=>8UFN1jLjSv zTQ^O85^P~*qmrw6`xIu_u<$U$iBnK!1uE)g%qPTy>XyLGPLG%z_^C02kXTTo*%u-f z?CivWTJ7wq2fkq1)B6A+7hgeG;kP|V@rSUc?`z4*#Y?WDpliKCYy#Pg0=q0d7D{KY zL91gFi-pZ4h`S1>13VO^Q#RVdDPpAPGcS7OXH)FJsM{Ql_BThiv<13#-`i}`lxG4a z);siGPDpe%W5&Y4->ZJ{BlL9mZcnx*_zYuCQv`8~;zH6LqX%(nSo=y`tWY$4GawHl z$7pD33*etPE3Q_U4CnoJ4xeb1AgRm9HjF94UJ_!>CT}IVMYe;6H;iwc z?`Cl-*y|a2NpDGEANy}q(d=b{_D+Q&Ea%$=LunzGDd(aV@Z&(u(SMLlA;a(sf9+Er zMIw{JSm zU|6+QMLlF_FQR}$wiqPM*AB^~-W;d3`VEbb$Ii2l2O-*dMN zg($$)Xt5KCITKcsu8Y-yCSDksjL*3l(04+!w>kdukH?rWJL?CY2Js2-Ug+vw@5Ug> zQ*GrL2^jwkNezzaTZG7{7`l`v7vp6H*ok1oGF3{!qU$ zh?)=X%xueiKbjZ+G-N`@`-6>rjTgy}*WIbk)h#9Dm3S*V!W4Oz6u_qZOz4F@s7Pz0 zYAp}$;)FtP8_p=#X3Z|v*_`zi%;IX&?Q>zPq@D>ReMk9ZSgMguKK^GI=}{jm#E40g zWCTHls!i*X?l$GH%&?2gIRk$Bq4y_>zHV+y0)56gR@2F8fP&Rr*GSxZ855ARP9RLZ z=@zli?gL$f8*~RzA4EYIr{LOh{T>+0Tm^oRl$gW)FFFB7p+i5+W{&Lke&@3C9`-4K zM^$g3Gn(WzlNV$rvRitTN-I;r{oFV-gu8U{EpWi9?qFcoC7 zi7izpsu`RFT!6|3LiE9(g?qq6Axj(m;R2LF6 zOI0PZ84U_zsgD$8^~J*}Q_dBJ69bCYoa(qb&xscW)AyS{_A1^U?k=zMwy6M7Ni-z# zbj}KJK5^m)A#`dE^>B#@`8EcgGbl8s%rFB&s4eTbpquBMy8b`>w}!R{-xcwf74ee= zg>m034YX*;8GzY!!1&32-2n+zAexoyX^V04{+n5y7r+A!9*C2x;7wWe!}Tt?e_AC^ zs-vcIp}8$M4Np2u)kdphwl~CXljXAp7P6D`c<2d%ujQ*@rc?WEKBk$Z(T}g~qDnRd zlzlJCA}UUC!cs9hB!+1bO#dB_{3&iKqHeS0X9vw>Zx(ljz9L5}b~y%(D%n*x24_Gw z8-Ssjj73)=m12bs+$gybxo#7f9t^TLk)jalfR`!qFMeoLrlOfg>g$w4(YEhN*kQa9 z_GdJn?rlNvKyIHR@r&a-;4)bDib{SOM_!b=R*GOM#_Fw3LMOA@>_E9)mEOg*#&Jg# zbVFB}S3mNfSj6r%2TeT)BnWrn4=yti1r$x?zzeLWXX>AXe55S!2C_(1+Qhr@D4=9ic7)6ibUMZi zKqHZOjV5K{(2G#5(t3@X!f1jYw=TGJ)i$hZF(#*uI;ryGd?FPBP{7&zM~9;1>u3hy zs$ad6OubHqt!g;9bs{j8VRtgTo1CFqsctl&$(@o9Z*Q;ICjM@PPLp74_QW*?vlV(5 zdb*}5DGqz(MKv3`Lv1YCl?~lh&!(C*yNOyNSfE^$H6+}mwp-Du2gysWQwML`Y|kd(7WNUYupT$<1kTHHv@jr=u9PEqLq~QK@eJZH-Y;z%El?sR5e?-0 z)5-l3@s8Xd0gO4Hcz0M-F4N>pd+$6L;M_1(xf;E*rKFJzDjC(Xpw6YB*KS^}lklbY zSwnaV3-1x^UuMzu zW2L^LnBw|TywYnGI;@DU-_ci)o7&K?p|YR59X!GQBPd4-IRE}RsspNn8HLSsw0dXy z8b)Iu)pqVRERJl;gmrq6!|BT5Sbxcy(qyoSf;0A;)oUJ=@ksY?s(a?qN>lY+lS_SN zrGl`F+zvl9Zh5sfYfPSseXz~Y1gqIrmk0NH9Qj7)myU-Y7`$79lpQuhK2Q?pk8NPu z5Sj6%vhdVXoYE2i>!WrAtVa=znAGa03IvekM|A`lNo0$bOxGx(mX$oYm3hY>kANg{ z1#vesP(15xplCp2kYz=Vj6Z-rBWg@O@C!d4nB~4me?umpgvY_D=}T@SFYTH4dM@J6 zr~o263GRj&inBdJPznUjgQn_q0dHPZzOt*KI$XFExTZ zjlaN+>3gu~^!#aq45?%hitetK-BL!OrW4-TS!QbD;oh8P_~#17z;*VIG#%f<28F3> zOv+I(WiFqs~DDT z*^MBE_#~3X65x*LigatdGKofAkTMw@{V^YTO&yy_k=zaV+} zdAUgs3(NWk+o-uv=%fV%+X@vogc4ZB33ziN z`TsO|)~}p9yXMKE%mR+b9Nm^_d9|kl|7Ct@2#xJh`PCTJ#w-_XgJ$$;@jh? zeON7Nd@Ri$gWX<(d5ilceqX7Z_WHA_rFK%svA(Lm)6bX3d#1L!qje8J+|nG@o!Psb zzOe1%%1>*+cmQ+QH*0C+U6c~IzIE#_tYI^L4RIdm8b8qM?HnU({1BKcEdPLl183Ol zLX~sM6<9Bw*f*hl{+GO}W}~Bg`Cr*b*>8^$|7Qfk|4xe3`wzOEiMxS~y|u|N$2D1A zMrz;}(Rfpf;V8+N)It>^)Vm(ED+t>4muA7(kX^=g;If4EXM3s?55+m=VY}1g22_y1 zRr-_yw9r6{v1%Ey3T+|;fnB{LXb<)>ek%Zq3VROE*jlnIqtZL)0P?7#htCMOq6&t) z?WLo`7qt&A(lB=HF8%e-XK)lLry+H)I8+l_i$z5i6Bo-z783gg#BHJ?Sh+`4cs(#J zss;#E;BQder#?&bpn)=&m^|9LOUD-$nOY+lD0-Y60P3^Z`jKbzKiw+&RH?bX>^{2b zKX%OTSmNB!jGbUQkdFaZ%hO&xp+W3~os^W#a@3@Qj3A0BWlc$fwXCjaeznwD+a$s^ z^R8WEmQ-}#)pPc27t>Sv{ZevU&*3!_UF4qnuY50@ZSlzp>r*x#a%{QoEj({Dv&V!Lby7+`|k zctQ8ZGv@M?2$c;lAVy`CGjFl zAw-(Uz7%<>{GUNJ!!*DD^Z7iVnd|d$zH`sH_dfUDbDwh@xANEMYv)+$Y^2p$p6Tzz zzw=zyX7da17dtT{bB+A$8#YZd!aCUS`&HsaRf|BNu`` zw!X)=-_`)s^cZNn{Mb~5JdtAO21ND#-H3g3dz{+OQ#9}L)k#ZHcrTPrD{>bV-KC=c zR78^}xv2DrpH5Y4p|ygkE)zdL*%%77x2G?V`SfBpweDNjSdNVBI;GNx^M~oz{W(Jp z#PMd`0u?v@a)@(yGqFa-NN-A`EYMi{m}yc0E@8#UD^~Qb&bX;VQ8(mPZ_n;9F5?)fOaVC#GPFI#cx`q3j>BlpiL)dw;xX{o@k5g;SlejLTZTXApf9uTX zR>_;DXqW@6@`(@x62N8^FFP9_uWOFJb|_})HPA%HTWSH~ElF|c*;R%QsmXa5;}gA# z1`CGdvt+wG@LKBfZNC(`7q?uvC^lbp`eGA;>bmS|%YWWH&uF*k!*v=7?v(C4Z6AcD zkd^D?|?F{EX`Fzh%4?sYAb$WjWe1y zCs0EXTKI(0xS)n*y>Q-QN%GyTdio!ng*gpg`~faZ6QP@r>f%k{Txz>)se!H~*-&`~ zILqgor5yLrzw_1nk zpZM9jWX6U>aB()$Ix#SSq_5tc%wOr)_ptI-&TZEpHJcf1 z(E|~V*%tpgl!m7dfnI(VD9+qN*Fhfw*0}=2Bs)iUEELhc4z4-c1E~pqIlPvQr;rY> z;C_fzx4KAGq^fMLDy;0p$u=>^kHjoJ>XUzEF+_}-jNRy$>LZs?HAy-#+FmB-y$|HTbE8YQ4_v2Ao#vi$>iKkfe!v8F%FIG z$O)1@5!K2%4=;8}+nC!TGnL$Vd`~#(lHuHUwU#y2c-3?^wpRoj*vRqSBMSpJj4zrK zwq1Vpo))R-EH_v-oJcHfCF ze9KJOsNZD|YO8T0(fm0#r|PZbyAcL8atmz4cM7FtErRo7*3~@_1v7{pDMN_!3!YQe z21?{L_~W`8U+thnEP3r`uh!z`bVNxveCigvNV@;fxH&q=-$mY#5Rxt2>V^pa4q2j6`> zedU5)?~4I={SC?SgOAj5en|0%`X`$R=I$k4J>4>ExNy64T0QbpYAm)PY)YwVgfL%*Z<@JCT%6jJQ`IY7m<8T0Uv~Z{p%CF5^Jg`l?)Rz?sLQ= zFDKD*{}huqR&QDQzBxHRfre44aKm*~s+l^Uy*G<*V=%G_+A?QT5D&T9zO$_*QVx?0 z4O9ppE!)up5qDT<4@0k*R*+lTLLDaCI^j`^DMDOtL`4hdPA2Y#>bVQcdbeDThPmh6 zk5<2b$^o}{deC%QrUN!%@+Hkr{cU_qS#dcRB*0wU;f=&&7`4VnMqZW=9dpV7ri0FK z$I@Ion(|=lXR41B*{;+=T4ES`t_^!UT!z;Z&eF>aR;RlaGwu=cJdb>OzCy{91s%2#4kz7sJ}s+{8S>GRRxV>E-1wIaX* zGYj~(9_i1=A7uSA#4y^fTAOy%mlRAT_OuF^^7sRO@gu|Kkx`-r2&N94({9-{qL}Y^ zIgzX7MN5KsRm=0(Icr6a=C-wHsq_mkj_ao9ms+)>zqyXY0()Gvsb5s&*AtWar-Qu7 zI{!Bvbp2wp2Vt3r_Zz`s3F*C$P!X4tyl*{iJaX_ zJ@xVvGA8w45bB%rUd>r8F$9xJ-O48#r^p3*HWXQvq=$Wdp|Z_2{5skp&J^6t)~1#X zhSdr30g_v)wAP?&{>}bQrt44l*n`-_a_h9%SX2=O@He9i1TaVPamr*7)`)v;0v|$@ zbPyMUn8Uk9L=5dX|Fh0EPJT$74m% z;TSni&vMx@sKMLJ0fb*f_#@wk z>+*AJGx9|XSI(SKR=Z#RcV^g&;OLU7b2+%ubR*6)d!tX{U!YtCIJ96_xkq$MHgC2_XAuzsQ%iTUW~;wOBf zcHK*7zCTWqk^dAZa7EH3b)n;ZQQ2w9Touy*9fzKoPQRKk+eP|}p#fX@J<>fJyz@FQ zXS_gmB5vd0q&cut0$j)wAg4pN321=t$omX%BcFsq$i0$lLJ)|fo1-@b0<3=6z&h8( z)W8P7-@!USAa*NzI}+r>^*|1-sE+EnaUh}`@c+o0{HGis3G+2acQ-%)1akNt0s+f` zxy}m;pBEJ4+LwVO6M?v6YgE`R@kR}Rk~?ry{FH(6D>Uy2lZ2e7z~2q*xC!Gm{o68y z`JApGl~|4$RofKbKNp@%j%DL4i}pE8K52?=gDbIbaGbScJ-JTR`ze+E@=J}{ku1l> z-ppI~KuR~h{ab1^r0Wm(ka2MT-mg8Nk4#-v0Q>7A*k2tYi^yMty!N$20s`+Ba6q)6 zwa;6=Fz!K|pP!yEv6q2B`M21Es=eu^kIyCmfyjW-gyNT2q?}(n0>}XO9`0Vq6z8Y} zP_$odLe&5ct_|rtQu5C-DgkN8fUR={Dbyg4pApiS$Ym|^0t5F7z}^4h2idU?l4BpG zcpDFNGY;agR~}Lh%SS2eyK(;Cfq~H)IP6J=MyUm6RRn@e+T4|+L!+;fz;6|94hM49(F~oR;C5)0OLCDB@kdhR7DK|+S*4P zBpq;faj~=U2HNknqol*iIB1?cFv%a3#rn(zKT-n(3OB?#ViG(-RMKG#%s)`Bdt;{{ zcooXJfG*b{aCcQLJI>pEfl8?6m?ks-KvgvI?`VNWTy*EI0PIFNvIv;RnbA$`}t{g=fbhB zhwK=S++PO22hO-T0Xq6^?+`$EA-ZsDJ2x8#S1T`P;iJ_!wC`H@VKx3>?*ayHjnBXW z%K)G;{i+6%)Rcl&4eRUPb`T(OlcSsc&*OEdH3J>Wry9o2n=q-@wF7KY0}PsAG~PlQ z8t;G~%yLij5}5N`|%npy$0prHz-P(T-%n!lX+> z59}=T3_7T+?~l<~`(p!IJ{?Rr`!wu0^o%E{xaR8PxL-TOhfWMkME|GQiRclzQHgOa z$B8KMxiP^qPm*J`9D3AWR9ITuaoCX)rg zojr8kF()M^9P~YQ9J-eY6_<&NgW5`v4F;3X2@}+hh!U#-(LE`sP(J+s4MqA@Fi|6g zuv5{U5~x%|0xVPvE(%QaG8lF``sO<-edz=iI;O37O!%fk?0EE@TvR+0@p1g`eO*kf z$qno*^c_=F7Awhd){&hNOu!X7Dy;1ZecuHYs7Zzf{C}`m;xV!QwDLt|Nl>D(P*$Lr z>NI76Y0mVgmGMsw7??BN_{sUbPDTTKdxW^mHHgzEaR@jp|DHW#S~jBs+J6H6RE5JO d)4I&jB@+Sg@;TBu$__dWOq-RdfLSf*{{W)JaJB#d literal 0 HcmV?d00001 diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index b7284487c511d..f2d9e6b568a9b 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" - export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.3-src.zip:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH}" export PYSPARK_PYTHONPATH_SET=1 fi diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index ea4e1160b7672..6e4f68c74c365 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1179,7 +1179,7 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.10.3-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.10.4-src.zip") require(py4jFile.exists(), s"$py4jFile not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index d245acf49aa91..99fb58a28934a 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -242,7 +242,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.10.3-src.zip", + s"$sparkHome/python/lib/py4j-0.10.4-src.zip", s"$sparkHome/python") val extraEnvVars = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), From a8ea4da8d04c1ed621a96668118f20739145edd2 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 21 Oct 2016 09:49:37 +0100 Subject: [PATCH 151/177] [SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays ## What changes were proposed in this pull request? `Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays. Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates. cc srowen ## How was this patch tested? existing tests Author: Zheng RuiFeng Closes #15564 from zhengruifeng/avoid_0_length_array. --- .../org/apache/spark/CheckpointSuite.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../spark/deploy/SparkSubmitSuite.scala | 2 +- .../history/HistoryServerArgumentsSuite.scala | 2 +- .../spark/io/ChunkedByteBufferSuite.scala | 4 ++-- .../serializer/KryoSerializerSuite.scala | 2 +- .../spark/ml/linalg/MatricesSuite.scala | 4 ++-- .../spark/ml/util/TestingUtilsSuite.scala | 24 +++++++++---------- .../classification/LogisticRegression.scala | 2 +- .../stat/test/KolmogorovSmirnovTest.scala | 3 ++- .../MultilayerPerceptronClassifierSuite.scala | 2 +- .../apache/spark/ml/python/MLSerDeSuite.scala | 2 +- .../ml/tree/impl/RandomForestSuite.scala | 4 ++-- .../api/python/PythonMLLibAPISuite.scala | 2 +- .../evaluation/RankingMetricsSuite.scala | 4 ++-- .../spark/mllib/linalg/MatricesSuite.scala | 4 ++-- .../spark/mllib/util/TestingUtilsSuite.scala | 24 +++++++++---------- .../expressions/StringExpressionsSuite.scala | 10 ++++---- .../spark/sql/DataFrameFunctionsSuite.scala | 2 +- 19 files changed, 51 insertions(+), 50 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 9f94e36324536..b117c7709b46f 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -500,7 +500,7 @@ class CheckpointSuite extends SparkFunSuite with RDDCheckpointTester with LocalS } runTest("CheckpointRDD with zero partitions") { reliableCheckpoint: Boolean => - val rdd = new BlockRDD[Int](sc, Array[BlockId]()) + val rdd = new BlockRDD[Int](sc, Array.empty[BlockId]) assert(rdd.partitions.size === 0) assert(rdd.isCheckpointed === false) assert(rdd.isCheckpointedAndMaterialized === false) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 2d48e75cfbd96..7093dad05c5f6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -65,7 +65,7 @@ class JsonProtocolSuite extends SparkFunSuite with JsonTestUtils { test("writeMasterState") { val workers = Array(createWorkerInfo(), createWorkerInfo()) val activeApps = Array(createAppInfo()) - val completedApps = Array[ApplicationInfo]() + val completedApps = Array.empty[ApplicationInfo] val activeDrivers = Array(createDriverInfo()) val completedDrivers = Array(createDriverInfo()) val stateResponse = new MasterStateResponse( diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 732cbfaaeea46..7c649e305a37e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -91,7 +91,7 @@ class SparkSubmitSuite // scalastyle:off println test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") + testPrematureExit(Array.empty[String], "Usage: spark-submit") } test("prints usage with only --help") { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index 34f27ecaa07a3..de321db845a66 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -33,7 +33,7 @@ class HistoryServerArgumentsSuite extends SparkFunSuite { .set("spark.testing", "true") test("No Arguments Parsing") { - val argStrings = Array[String]() + val argStrings = Array.empty[String] val hsa = new HistoryServerArguments(conf, argStrings) assert(conf.get("spark.history.fs.logDirectory") === logDir.getAbsolutePath) assert(conf.get("spark.history.fs.updateInterval") === "1") diff --git a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala index 38b48a4c9e654..3b798e36b0499 100644 --- a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala @@ -57,7 +57,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite { } test("toArray()") { - val empty = ByteBuffer.wrap(Array[Byte]()) + val empty = ByteBuffer.wrap(Array.empty[Byte]) val bytes = ByteBuffer.wrap(Array.tabulate(8)(_.toByte)) val chunkedByteBuffer = new ChunkedByteBuffer(Array(bytes, bytes, empty)) assert(chunkedByteBuffer.toArray === bytes.array() ++ bytes.array()) @@ -74,7 +74,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite { } test("toInputStream()") { - val empty = ByteBuffer.wrap(Array[Byte]()) + val empty = ByteBuffer.wrap(Array.empty[Byte]) val bytes1 = ByteBuffer.wrap(Array.tabulate(256)(_.toByte)) val bytes2 = ByteBuffer.wrap(Array.tabulate(128)(_.toByte)) val chunkedByteBuffer = new ChunkedByteBuffer(Array(empty, bytes1, bytes2)) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 57a82312008e9..bc6e98365daef 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -100,7 +100,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { check(Array("aaa", "bbb", null)) check(Array(true, false, true)) check(Array('a', 'b', 'c')) - check(Array[Int]()) + check(Array.empty[Int]) check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) } diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala index 2796fcf2cbc22..9c0aa73938478 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala @@ -287,7 +287,7 @@ class MatricesSuite extends SparkMLFunSuite { val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) - val deHorz2 = Matrices.horzcat(Array[Matrix]()) + val deHorz2 = Matrices.horzcat(Array.empty[Matrix]) assert(deHorz1.numRows === 3) assert(spHorz2.numRows === 3) @@ -341,7 +341,7 @@ class MatricesSuite extends SparkMLFunSuite { val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) - val deVert2 = Matrices.vertcat(Array[Matrix]()) + val deVert2 = Matrices.vertcat(Array.empty[Matrix]) assert(deVert1.numRows === 5) assert(spVert2.numRows === 5) diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala index 5cbf2f04e6269..2dc0ee32d5762 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala @@ -110,9 +110,9 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) // Should throw exception with message when test fails. intercept[TestFailedException]( @@ -125,7 +125,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01) intercept[TestFailedException]( - Vectors.dense(Array[Double]()) ~== Vectors.dense(Array(3.135)) relTol 0.01) + Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01) // Comparing against zero should fail the test and throw exception with message // saying that the relative error is meaningless in this situation. @@ -145,7 +145,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(Vectors.dense(Array(3.1)) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) } @@ -176,14 +176,14 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(!(Vectors.dense(Array(3.1)) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) !~= + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) - assert(!(Vectors.dense(Array[Double]()) ~= + assert(!(Vectors.dense(Array.empty[Double]) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) ~= - Vectors.dense(Array[Double]()) absTol 1E-5) + assert(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array.empty[Double]) absTol 1E-5) // Should throw exception with message when test fails. intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== @@ -195,7 +195,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { intercept[TestFailedException](Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) - intercept[TestFailedException](Vectors.dense(Array[Double]()) ~== + intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) // Comparisons of two sparse vectors @@ -214,7 +214,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) - assert(Vectors.sparse(0, Array[Int](), Array[Double]()) !~== + assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) // Comparisons of a dense vector and a sparse vector @@ -230,14 +230,14 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== Vectors.dense(Array(3.1)) absTol 1E-6) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6) assert(Vectors.sparse(1, Array(0), Array(3.1)) !~== Vectors.dense(Array(3.1, 3.2)) absTol 1E-6) assert(Vectors.dense(Array(3.1)) !~== - Vectors.sparse(0, Array[Int](), Array[Double]()) absTol 1E-6) + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6) } test("Comparing Matrices using absolute error.") { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 862a468745fbd..8fdaae04c42ec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -622,7 +622,7 @@ class LogisticRegression @Since("1.2.0") ( rawCoefficients(coefIndex) } } else { - Array[Double]() + Array.empty[Double] } val interceptVector = if (interceptsArray.nonEmpty && isMultinomial) { // The intercepts are never regularized, so we always center the mean. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala index c3de5d75f4f7d..a8b5955a7285d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala @@ -124,7 +124,8 @@ private[stat] object KolmogorovSmirnovTest extends Logging { val pResults = partDiffs.foldLeft(initAcc) { case ((pMin, pMax, pCt), (dl, dp)) => (math.min(pMin, dl), math.max(pMax, dp), pCt + 1) } - val results = if (pResults == initAcc) Array[(Double, Double, Double)]() else Array(pResults) + val results = + if (pResults == initAcc) Array.empty[(Double, Double, Double)] else Array(pResults) results.iterator } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index c08cb695806d0..41684d92be33a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -51,7 +51,7 @@ class MultilayerPerceptronClassifierSuite test("Input Validation") { val mlpc = new MultilayerPerceptronClassifier() intercept[IllegalArgumentException] { - mlpc.setLayers(Array[Int]()) + mlpc.setLayers(Array.empty[Int]) } intercept[IllegalArgumentException] { mlpc.setLayers(Array[Int](1)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala index 5eaef9aabda50..3bb760f2ecc1d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala @@ -54,7 +54,7 @@ class MLSerDeSuite extends SparkFunSuite { assert(matrix === nm) // Test conversion for empty matrix - val empty = Array[Double]() + val empty = Array.empty[Double] val emptyMatrix = Matrices.dense(0, 0, empty) val ne = MLSerDe.loads(MLSerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] assert(emptyMatrix == ne) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 499d386e66413..3bded9c01760a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -154,10 +154,10 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val featureSamples = Array(0, 0, 0).map(_.toDouble) val featureSamplesEmpty = Array.empty[Double] val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits === Array[Double]()) + assert(splits === Array.empty[Double]) val splitsEmpty = RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, fakeMetadata, 0) - assert(splitsEmpty === Array[Double]()) + assert(splitsEmpty === Array.empty[Double]) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 0eb839f20c003..5f85c0d65ff2d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -72,7 +72,7 @@ class PythonMLLibAPISuite extends SparkFunSuite { assert(matrix === nm) // Test conversion for empty matrix - val empty = Array[Double]() + val empty = Array.empty[Double] val emptyMatrix = Matrices.dense(0, 0, empty) val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] assert(emptyMatrix == ne) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala index 8e9d910e646c9..f334be2c2ba83 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala @@ -28,7 +28,7 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { Seq( (Array(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Array(1, 2, 3, 4, 5)), (Array(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Array(1, 2, 3)), - (Array(1, 2, 3, 4, 5), Array[Int]()) + (Array(1, 2, 3, 4, 5), Array.empty[Int]) ), 2) val eps = 1.0E-5 @@ -55,7 +55,7 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val predictionAndLabels = sc.parallelize( Seq( (Array(1, 6, 2), Array(1, 2, 3, 4, 5)), - (Array[Int](), Array(1, 2, 3)) + (Array.empty[Int], Array(1, 2, 3)) ), 2) val eps = 1.0E-5 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index d0c4dd28e14ee..563756907d201 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -289,7 +289,7 @@ class MatricesSuite extends SparkFunSuite { val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) - val deHorz2 = Matrices.horzcat(Array[Matrix]()) + val deHorz2 = Matrices.horzcat(Array.empty[Matrix]) assert(deHorz1.numRows === 3) assert(spHorz2.numRows === 3) @@ -343,7 +343,7 @@ class MatricesSuite extends SparkFunSuite { val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) - val deVert2 = Matrices.vertcat(Array[Matrix]()) + val deVert2 = Matrices.vertcat(Array.empty[Matrix]) assert(deVert1.numRows === 5) assert(spVert2.numRows === 5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala index 1aff44480aac9..3fcf1cf2c2635 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -110,9 +110,9 @@ class TestingUtilsSuite extends SparkFunSuite { assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) // Should throw exception with message when test fails. intercept[TestFailedException]( @@ -125,7 +125,7 @@ class TestingUtilsSuite extends SparkFunSuite { Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01) intercept[TestFailedException]( - Vectors.dense(Array[Double]()) ~== Vectors.dense(Array(3.135)) relTol 0.01) + Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01) // Comparing against zero should fail the test and throw exception with message // saying that the relative error is meaningless in this situation. @@ -145,7 +145,7 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.dense(Array(3.1)) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) } @@ -176,14 +176,14 @@ class TestingUtilsSuite extends SparkFunSuite { assert(!(Vectors.dense(Array(3.1)) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) !~= + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) - assert(!(Vectors.dense(Array[Double]()) ~= + assert(!(Vectors.dense(Array.empty[Double]) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) ~= - Vectors.dense(Array[Double]()) absTol 1E-5) + assert(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array.empty[Double]) absTol 1E-5) // Should throw exception with message when test fails. intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== @@ -195,7 +195,7 @@ class TestingUtilsSuite extends SparkFunSuite { intercept[TestFailedException](Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) - intercept[TestFailedException](Vectors.dense(Array[Double]()) ~== + intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) // Comparisons of two sparse vectors @@ -214,7 +214,7 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) - assert(Vectors.sparse(0, Array[Int](), Array[Double]()) !~== + assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) // Comparisons of a dense vector and a sparse vector @@ -230,14 +230,14 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== Vectors.dense(Array(3.1)) absTol 1E-6) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6) assert(Vectors.sparse(1, Array(0), Array(3.1)) !~== Vectors.dense(Array(3.1, 3.2)) absTol 1E-6) assert(Vectors.dense(Array(3.1)) !~== - Vectors.sparse(0, Array[Int](), Array[Double]()) absTol 1E-6) + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6) } test("Comparing Matrices using absolute error.") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index fdb9fa31f09c8..26978a0482fc7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -215,13 +215,13 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Substring(bytes, 2, 2), Array[Byte](2, 3)) checkEvaluation(Substring(bytes, 3, 2), Array[Byte](3, 4)) checkEvaluation(Substring(bytes, 4, 2), Array[Byte](4)) - checkEvaluation(Substring(bytes, 8, 2), Array[Byte]()) + checkEvaluation(Substring(bytes, 8, 2), Array.empty[Byte]) checkEvaluation(Substring(bytes, -1, 2), Array[Byte](4)) checkEvaluation(Substring(bytes, -2, 2), Array[Byte](3, 4)) checkEvaluation(Substring(bytes, -3, 2), Array[Byte](2, 3)) checkEvaluation(Substring(bytes, -4, 2), Array[Byte](1, 2)) checkEvaluation(Substring(bytes, -5, 2), Array[Byte](1)) - checkEvaluation(Substring(bytes, -8, 2), Array[Byte]()) + checkEvaluation(Substring(bytes, -8, 2), Array.empty[Byte]) } test("string substring_index function") { @@ -275,7 +275,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Base64(UnBase64(a)), "AQIDBA==", create_row("AQIDBA==")) checkEvaluation(Base64(b), "AQIDBA==", create_row(bytes)) - checkEvaluation(Base64(b), "", create_row(Array[Byte]())) + checkEvaluation(Base64(b), "", create_row(Array.empty[Byte])) checkEvaluation(Base64(b), null, create_row(null)) checkEvaluation(Base64(Literal.create(null, BinaryType)), null, create_row("abdef")) @@ -526,13 +526,13 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // non ascii characters are not allowed in the source code, so we disable the scalastyle. checkEvaluation(Length(Literal("a花花c")), 4, create_row(string)) // scalastyle:on - checkEvaluation(Length(Literal(bytes)), 5, create_row(Array[Byte]())) + checkEvaluation(Length(Literal(bytes)), 5, create_row(Array.empty[Byte])) checkEvaluation(Length(a), 5, create_row(string)) checkEvaluation(Length(b), 5, create_row(bytes)) checkEvaluation(Length(a), 0, create_row("")) - checkEvaluation(Length(b), 0, create_row(Array[Byte]())) + checkEvaluation(Length(b), 0, create_row(Array.empty[Byte])) checkEvaluation(Length(a), null, create_row(null)) checkEvaluation(Length(b), null, create_row(null)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 45db61515e9b6..586a0fffeb7a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -273,7 +273,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { test("sort_array function") { val df = Seq( (Array[Int](2, 1, 3), Array("b", "c", "a")), - (Array[Int](), Array[String]()), + (Array.empty[Int], Array.empty[String]), (null, null) ).toDF("a", "b") checkAnswer( From 3a237512b162d192b5503c08d121134a2dac6ff1 Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Fri, 21 Oct 2016 11:39:32 +0100 Subject: [PATCH 152/177] [SPARK-13275][WEB UI] Visually clarified executors start time in timeline ## What changes were proposed in this pull request? Updated the Executors added/removed bubble in the time line so it's clearer where it starts. Now the bubble is left justified on the start time (still also denoted by the line) rather than center justified. ## How was this patch tested? Manually tested UI screen shot 2016-10-17 at 6 04 36 pm screen shot 2016-10-17 at 5 54 09 pm Author: Alex Bozarth Closes #15536 from ajbozarth/spark13275. --- .../main/resources/org/apache/spark/ui/static/timeline-view.js | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index a6153ceda75e2..705a08f0293d3 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -24,6 +24,7 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { return a.value - b.value }, editable: false, + align: 'left', showCurrentTime: false, min: startTime, zoomable: false, @@ -99,6 +100,7 @@ function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { return a.value - b.value; }, editable: false, + align: 'left', showCurrentTime: false, min: startTime, zoomable: false, From b3b4b9542223de3495a7a7e0dd27634ddb9f929d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 21 Oct 2016 11:25:01 -0700 Subject: [PATCH 153/177] [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness We should upgrade to the latest release of MiMa (0.1.11) in order to include a fix for a bug which led to flakiness in the MiMa checks (https://github.com/typesafehub/migration-manager/issues/115). Author: Josh Rosen Closes #15571 from JoshRosen/SPARK-18034. --- project/MimaExcludes.scala | 7 ++++++- project/plugins.sbt | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index facf034ea7e7d..350b144f8294b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -81,7 +81,12 @@ object MimaExcludes { // [SPARK-17338][SQL] add global temp view ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), + + // [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness. + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.aggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.getAggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.org$apache$spark$ml$param$shared$HasAggregationDepth$_setter_$aggregationDepth_=") ) } diff --git a/project/plugins.sbt b/project/plugins.sbt index 8bebd7bcac58c..76597d27292ea 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,7 +6,7 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.8.2") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") -addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.9") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.11") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") From 4efdc764edfbc4971f0e863947258482ca2017df Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 21 Oct 2016 12:34:14 -0700 Subject: [PATCH 154/177] [SPARK-17674][SPARKR] check for warning in test output ## What changes were proposed in this pull request? testthat library we are using for testing R is redirecting warning (and disabling `options("warn" = 2)`), we need to have a way to detect any new warning and fail ## How was this patch tested? manual testing, Jenkins Author: Felix Cheung Closes #15576 from felixcheung/rtestwarning. --- R/run-tests.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/R/run-tests.sh b/R/run-tests.sh index 1a1e8ab9ffe18..5e4dafaf76f3d 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -26,6 +26,8 @@ rm -f $LOGFILE SPARK_TESTING=1 $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.default.name="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE FAILED=$((PIPESTATUS[0]||$FAILED)) +NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" + # Also run the documentation tests for CRAN CRAN_CHECK_LOG_FILE=$FWDIR/cran-check.out rm -f $CRAN_CHECK_LOG_FILE @@ -37,10 +39,10 @@ NUM_CRAN_WARNING="$(grep -c WARNING$ $CRAN_CHECK_LOG_FILE)" NUM_CRAN_ERROR="$(grep -c ERROR$ $CRAN_CHECK_LOG_FILE)" NUM_CRAN_NOTES="$(grep -c NOTE$ $CRAN_CHECK_LOG_FILE)" -if [[ $FAILED != 0 ]]; then +if [[ $FAILED != 0 || $NUM_TEST_WARNING != 0 ]]; then cat $LOGFILE echo -en "\033[31m" # Red - echo "Had test failures; see logs." + echo "Had test warnings or failures; see logs." echo -en "\033[0m" # No color exit -1 else From e21e1c946c4b7448fb150cfa2d9419864ae6f9b5 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 21 Oct 2016 12:35:37 -0700 Subject: [PATCH 155/177] [SPARK-18013][SPARKR] add crossJoin API ## What changes were proposed in this pull request? Add crossJoin and do not default to cross join if joinExpr is left out ## How was this patch tested? unit test Author: Felix Cheung Closes #15559 from felixcheung/rcrossjoin. --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 59 ++++++++++++++++++----- R/pkg/R/generics.R | 4 ++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 11 ++++- docs/sparkr.md | 4 ++ 5 files changed, 64 insertions(+), 15 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 5960c6206a6f1..87181851714e0 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -71,6 +71,7 @@ exportMethods("arrange", "covar_samp", "covar_pop", "createOrReplaceTempView", + "crossJoin", "crosstab", "dapply", "dapplyCollect", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 801d2ed4e7500..8910a4b138a37 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2271,12 +2271,13 @@ setMethod("dropDuplicates", #' Join #' -#' Join two SparkDataFrames based on the given join expression. +#' Joins two SparkDataFrames based on the given join expression. #' #' @param x A SparkDataFrame #' @param y A SparkDataFrame #' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a -#' Column expression. If joinExpr is omitted, join() will perform a Cartesian join +#' Column expression. If joinExpr is omitted, the default, inner join is attempted and an error is +#' thrown if it would be a Cartesian Product. For Cartesian join, use crossJoin instead. #' @param joinType The type of join to perform. The following join types are available: #' 'inner', 'outer', 'full', 'fullouter', leftouter', 'left_outer', 'left', #' 'right_outer', 'rightouter', 'right', and 'leftsemi'. The default joinType is "inner". @@ -2285,23 +2286,24 @@ setMethod("dropDuplicates", #' @aliases join,SparkDataFrame,SparkDataFrame-method #' @rdname join #' @name join -#' @seealso \link{merge} +#' @seealso \link{merge} \link{crossJoin} #' @export #' @examples #'\dontrun{ #' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) -#' join(df1, df2) # Performs a Cartesian #' join(df1, df2, df1$col1 == df2$col2) # Performs an inner join based on expression #' join(df1, df2, df1$col1 == df2$col2, "right_outer") +#' join(df1, df2) # Attempts an inner join #' } #' @note join since 1.4.0 setMethod("join", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y, joinExpr = NULL, joinType = NULL) { if (is.null(joinExpr)) { - sdf <- callJMethod(x@sdf, "crossJoin", y@sdf) + # this may not fail until the planner checks for Cartesian join later on. + sdf <- callJMethod(x@sdf, "join", y@sdf) } else { if (class(joinExpr) != "Column") stop("joinExpr must be a Column") if (is.null(joinType)) { @@ -2322,22 +2324,52 @@ setMethod("join", dataFrame(sdf) }) +#' CrossJoin +#' +#' Returns Cartesian Product on two SparkDataFrames. +#' +#' @param x A SparkDataFrame +#' @param y A SparkDataFrame +#' @return A SparkDataFrame containing the result of the join operation. +#' @family SparkDataFrame functions +#' @aliases crossJoin,SparkDataFrame,SparkDataFrame-method +#' @rdname crossJoin +#' @name crossJoin +#' @seealso \link{merge} \link{join} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' crossJoin(df1, df2) # Performs a Cartesian +#' } +#' @note crossJoin since 2.1.0 +setMethod("crossJoin", + signature(x = "SparkDataFrame", y = "SparkDataFrame"), + function(x, y) { + sdf <- callJMethod(x@sdf, "crossJoin", y@sdf) + dataFrame(sdf) + }) + #' Merges two data frames #' #' @name merge -#' @param x the first data frame to be joined -#' @param y the second data frame to be joined +#' @param x the first data frame to be joined. +#' @param y the second data frame to be joined. #' @param by a character vector specifying the join columns. If by is not #' specified, the common column names in \code{x} and \code{y} will be used. +#' If by or both by.x and by.y are explicitly set to NULL or of length 0, the Cartesian +#' Product of x and y will be returned. #' @param by.x a character vector specifying the joining columns for x. #' @param by.y a character vector specifying the joining columns for y. #' @param all a boolean value setting \code{all.x} and \code{all.y} #' if any of them are unset. #' @param all.x a boolean value indicating whether all the rows in x should -#' be including in the join +#' be including in the join. #' @param all.y a boolean value indicating whether all the rows in y should -#' be including in the join -#' @param sort a logical argument indicating whether the resulting columns should be sorted +#' be including in the join. +#' @param sort a logical argument indicating whether the resulting columns should be sorted. #' @param suffixes a string vector of length 2 used to make colnames of #' \code{x} and \code{y} unique. #' The first element is appended to each colname of \code{x}. @@ -2351,20 +2383,21 @@ setMethod("join", #' @family SparkDataFrame functions #' @aliases merge,SparkDataFrame,SparkDataFrame-method #' @rdname merge -#' @seealso \link{join} +#' @seealso \link{join} \link{crossJoin} #' @export #' @examples #'\dontrun{ #' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) -#' merge(df1, df2) # Performs a Cartesian +#' merge(df1, df2) # Performs an inner join by common columns #' merge(df1, df2, by = "col1") # Performs an inner join based on expression #' merge(df1, df2, by.x = "col1", by.y = "col2", all.y = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE, all.y = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all = TRUE, sort = FALSE) #' merge(df1, df2, by = "col1", all = TRUE, suffixes = c("-X", "-Y")) +#' merge(df1, df2, by = NULL) # Performs a Cartesian join #' } #' @note merge since 1.5.0 setMethod("merge", @@ -2401,7 +2434,7 @@ setMethod("merge", joinY <- by } else { # if by or both by.x and by.y have length 0, use Cartesian Product - joinRes <- join(x, y) + joinRes <- crossJoin(x, y) return (joinRes) } diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 810aea9017743..5549cd7cac516 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -468,6 +468,10 @@ setGeneric("createOrReplaceTempView", standardGeneric("createOrReplaceTempView") }) +# @rdname crossJoin +# @export +setGeneric("crossJoin", function(x, y) { standardGeneric("crossJoin") }) + #' @rdname dapply #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 1c806869e9fbe..3a987cd86213f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1572,7 +1572,7 @@ test_that("filter() on a DataFrame", { #expect_true(is.ts(filter(1:100, rep(1, 3)))) # nolint }) -test_that("join() and merge() on a DataFrame", { +test_that("join(), crossJoin() and merge() on a DataFrame", { df <- read.json(jsonPath) mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", @@ -1583,7 +1583,14 @@ test_that("join() and merge() on a DataFrame", { writeLines(mockLines2, jsonPath2) df2 <- read.json(jsonPath2) - joined <- join(df, df2) + # inner join, not cartesian join + expect_equal(count(where(join(df, df2), df$name == df2$name)), 3) + # cartesian join + expect_error(tryCatch(count(join(df, df2)), error = function(e) { stop(e) }), + paste0(".*(org.apache.spark.sql.AnalysisException: Detected cartesian product for", + " INNER join between logical plans).*")) + + joined <- crossJoin(df, df2) expect_equal(names(joined), c("age", "name", "name", "test")) expect_equal(count(joined), 12) expect_equal(names(collect(joined)), c("age", "name", "name", "test")) diff --git a/docs/sparkr.md b/docs/sparkr.md index 340e7f7cb1a0b..c1829efd18f44 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -591,3 +591,7 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` + +## Upgrading to SparkR 2.1.0 + + - `join` no longer performs Cartesian Product by default, use `crossJoin` instead. From e371040a0150e4ed748a7c25465965840b61ca63 Mon Sep 17 00:00:00 2001 From: Hossein Date: Fri, 21 Oct 2016 12:38:52 -0700 Subject: [PATCH 156/177] [SPARK-17811] SparkR cannot parallelize data.frame with NA or NULL in Date columns ## What changes were proposed in this pull request? NA date values are serialized as "NA" and NA time values are serialized as NaN from R. In the backend we did not have proper logic to deal with them. As a result we got an IllegalArgumentException for Date and wrong value for time. This PR adds support for deserializing NA as Date and Time. ## How was this patch tested? * [x] TODO Author: Hossein Closes #15421 from falaki/SPARK-17811. --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 13 ++++++++ .../scala/org/apache/spark/api/r/SerDe.scala | 31 +++++++++++++++---- 2 files changed, 38 insertions(+), 6 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 3a987cd86213f..b4b43fdba42ce 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -390,6 +390,19 @@ test_that("create DataFrame with different data types", { expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE)) }) +test_that("SPARK-17811: can create DataFrame containing NA as date and time", { + df <- data.frame( + id = 1:2, + time = c(as.POSIXlt("2016-01-10"), NA), + date = c(as.Date("2016-10-01"), NA)) + + DF <- collect(createDataFrame(df)) + expect_true(is.na(DF$date[2])) + expect_equal(DF$date[1], as.Date("2016-10-01")) + expect_true(is.na(DF$time[2])) + expect_equal(DF$time[1], as.POSIXlt("2016-01-10")) +}) + test_that("create DataFrame with complex types", { e <- new.env() assign("n", 3L, envir = e) diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index e4932a4192d39..550e075a95129 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -125,15 +125,34 @@ private[spark] object SerDe { } def readDate(in: DataInputStream): Date = { - Date.valueOf(readString(in)) + try { + val inStr = readString(in) + if (inStr == "NA") { + null + } else { + Date.valueOf(inStr) + } + } catch { + // TODO: SPARK-18011 with some versions of R deserializing NA from R results in NASE + case _: NegativeArraySizeException => null + } } def readTime(in: DataInputStream): Timestamp = { - val seconds = in.readDouble() - val sec = Math.floor(seconds).toLong - val t = new Timestamp(sec * 1000L) - t.setNanos(((seconds - sec) * 1e9).toInt) - t + try { + val seconds = in.readDouble() + if (java.lang.Double.isNaN(seconds)) { + null + } else { + val sec = Math.floor(seconds).toLong + val t = new Timestamp(sec * 1000L) + t.setNanos(((seconds - sec) * 1e9).toInt) + t + } + } catch { + // TODO: SPARK-18011 with some versions of R deserializing NA from R results in NASE + case _: NegativeArraySizeException => null + } } def readBytesArr(in: DataInputStream): Array[Array[Byte]] = { From 7a531e3054f8d4820216ed379433559f57f571b8 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 21 Oct 2016 13:07:29 -0700 Subject: [PATCH 157/177] [SPARK-17926][SQL][STREAMING] Added json for statuses ## What changes were proposed in this pull request? StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`. ## How was this patch tested? New unit tests Author: Tathagata Das Closes #15476 from tdas/SPARK-17926. --- python/pyspark/sql/streaming.py | 11 +- .../spark/sql/streaming/SinkStatus.scala | 18 ++- .../spark/sql/streaming/SourceStatus.scala | 23 +++- .../sql/streaming/StreamingQueryStatus.scala | 55 ++++++--- .../streaming/StreamingQueryStatusSuite.scala | 105 ++++++++++++++++++ 5 files changed, 187 insertions(+), 25 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index ce47bd1640fb1..35fc469291684 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -205,8 +205,7 @@ def __str__(self): Pretty string of this query status. >>> print(sqs) - StreamingQueryStatus: - Query name: query + Status of query 'query' Query id: 1 Status timestamp: 123 Input rate: 15.5 rows/sec @@ -220,7 +219,7 @@ def __str__(self): numRows.input.total: 100 triggerId: 5 Source statuses [1 source]: - Source 1: MySource1 + Source 1 - MySource1 Available offset: #0 Input rate: 15.5 rows/sec Processing rate: 23.5 rows/sec @@ -228,7 +227,7 @@ def __str__(self): numRows.input.source: 100 latency.getOffset.source: 10 latency.getBatch.source: 20 - Sink status: MySink + Sink status - MySink Committed offsets: [#1, -] """ return self._jsqs.toString() @@ -366,7 +365,7 @@ def __str__(self): Pretty string of this source status. >>> print(sqs.sourceStatuses[0]) - SourceStatus: MySource1 + Status of source MySource1 Available offset: #0 Input rate: 15.5 rows/sec Processing rate: 23.5 rows/sec @@ -457,7 +456,7 @@ def __str__(self): Pretty string of this source status. >>> print(sqs.sinkStatus) - SinkStatus: MySink + Status of sink MySink Committed offsets: [#1, -] """ return self._jss.toString() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index c9911665f7d72..ab19602207ad8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -17,6 +17,11 @@ package org.apache.spark.sql.streaming +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.streaming.StreamingQueryStatus.indent @@ -34,8 +39,19 @@ class SinkStatus private( val description: String, val offsetDesc: String) { + /** The compact JSON representation of this status. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this status. */ + def prettyJson: String = pretty(render(jsonValue)) + override def toString: String = - "SinkStatus:" + indent(prettyString) + "Status of sink " + indent(prettyString).trim + + private[sql] def jsonValue: JValue = { + ("description" -> JString(description)) ~ + ("offsetDesc" -> JString(offsetDesc)) + } private[sql] def prettyString: String = { s"""$description diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 6ace4833be22f..cfdf11370e06d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -21,8 +21,14 @@ import java.{util => ju} import scala.collection.JavaConverters._ +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.streaming.StreamingQueryStatus.indent +import org.apache.spark.util.JsonProtocol /** * :: Experimental :: @@ -47,8 +53,22 @@ class SourceStatus private( val processingRate: Double, val triggerDetails: ju.Map[String, String]) { + /** The compact JSON representation of this status. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this status. */ + def prettyJson: String = pretty(render(jsonValue)) + override def toString: String = - "SourceStatus:" + indent(prettyString) + "Status of source " + indent(prettyString).trim + + private[sql] def jsonValue: JValue = { + ("description" -> JString(description)) ~ + ("offsetDesc" -> JString(offsetDesc)) ~ + ("inputRate" -> JDouble(inputRate)) ~ + ("processingRate" -> JDouble(processingRate)) ~ + ("triggerDetails" -> JsonProtocol.mapToJson(triggerDetails.asScala)) + } private[sql] def prettyString: String = { val triggerDetailsLines = @@ -59,7 +79,6 @@ class SourceStatus private( |Processing rate: $processingRate rows/sec |Trigger details: |""".stripMargin + indent(triggerDetailsLines) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index 47689928730d0..a50b0d96c13f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -21,8 +21,14 @@ import java.{util => ju} import scala.collection.JavaConverters._ +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset} +import org.apache.spark.util.JsonProtocol /** * :: Experimental :: @@ -59,29 +65,46 @@ class StreamingQueryStatus private( import StreamingQueryStatus._ + /** The compact JSON representation of this status. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this status. */ + def prettyJson: String = pretty(render(jsonValue)) + override def toString: String = { val sourceStatusLines = sourceStatuses.zipWithIndex.map { case (s, i) => - s"Source ${i + 1}:" + indent(s.prettyString) + s"Source ${i + 1} - " + indent(s.prettyString).trim } - val sinkStatusLines = sinkStatus.prettyString + val sinkStatusLines = sinkStatus.prettyString.trim val triggerDetailsLines = triggerDetails.asScala.map { case (k, v) => s"$k: $v" }.toSeq.sorted val numSources = sourceStatuses.length val numSourcesString = s"$numSources source" + { if (numSources > 1) "s" else "" } - val allLines = s""" - |Query name: $name - |Query id: $id - |Status timestamp: $timestamp - |Input rate: $inputRate rows/sec - |Processing rate $processingRate rows/sec - |Latency: ${latency.getOrElse("-")} ms - |Trigger details: - |${indent(triggerDetailsLines)} - |Source statuses [$numSourcesString]: - |${indent(sourceStatusLines)} - |Sink status: ${indent(sinkStatusLines)}""".stripMargin - - s"StreamingQueryStatus:${indent(allLines)}" + val allLines = + s"""|Query id: $id + |Status timestamp: $timestamp + |Input rate: $inputRate rows/sec + |Processing rate $processingRate rows/sec + |Latency: ${latency.getOrElse("-")} ms + |Trigger details: + |${indent(triggerDetailsLines)} + |Source statuses [$numSourcesString]: + |${indent(sourceStatusLines)} + |Sink status - ${indent(sinkStatusLines).trim}""".stripMargin + + s"Status of query '$name'\n${indent(allLines)}" + } + + private[sql] def jsonValue: JValue = { + ("name" -> JString(name)) ~ + ("id" -> JInt(id)) ~ + ("timestamp" -> JInt(timestamp)) ~ + ("inputRate" -> JDouble(inputRate)) ~ + ("processingRate" -> JDouble(processingRate)) ~ + ("latency" -> latency.map(JDouble).getOrElse(JNothing)) ~ + ("triggerDetails" -> JsonProtocol.mapToJson(triggerDetails.asScala)) + ("sourceStatuses" -> JArray(sourceStatuses.map(_.jsonValue).toList)) ~ + ("sinkStatus" -> sinkStatus.jsonValue) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala new file mode 100644 index 0000000000000..1a98cf2ba74e6 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala @@ -0,0 +1,105 @@ +/* + * 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.streaming + +import org.apache.spark.SparkFunSuite + +class StreamingQueryStatusSuite extends SparkFunSuite { + test("toString") { + assert(StreamingQueryStatus.testStatus.sourceStatuses(0).toString === + """ + |Status of source MySource1 + | Available offset: #0 + | Input rate: 15.5 rows/sec + | Processing rate: 23.5 rows/sec + | Trigger details: + | numRows.input.source: 100 + | latency.getOffset.source: 10 + | latency.getBatch.source: 20 + """.stripMargin.trim, "SourceStatus.toString does not match") + + assert(StreamingQueryStatus.testStatus.sinkStatus.toString === + """ + |Status of sink MySink + | Committed offsets: [#1, -] + """.stripMargin.trim, "SinkStatus.toString does not match") + + assert(StreamingQueryStatus.testStatus.toString === + """ + |Status of query 'query' + | Query id: 1 + | Status timestamp: 123 + | Input rate: 15.5 rows/sec + | Processing rate 23.5 rows/sec + | Latency: 345.0 ms + | Trigger details: + | isDataPresentInTrigger: true + | isTriggerActive: true + | latency.getBatch.total: 20 + | latency.getOffset.total: 10 + | numRows.input.total: 100 + | triggerId: 5 + | Source statuses [1 source]: + | Source 1 - MySource1 + | Available offset: #0 + | Input rate: 15.5 rows/sec + | Processing rate: 23.5 rows/sec + | Trigger details: + | numRows.input.source: 100 + | latency.getOffset.source: 10 + | latency.getBatch.source: 20 + | Sink status - MySink + | Committed offsets: [#1, -] + """.stripMargin.trim, "StreamingQueryStatus.toString does not match") + + } + + test("json") { + assert(StreamingQueryStatus.testStatus.json === + """ + |{"sourceStatuses":[{"description":"MySource1","offsetDesc":"#0","inputRate":15.5, + |"processingRate":23.5,"triggerDetails":{"numRows.input.source":"100", + |"latency.getOffset.source":"10","latency.getBatch.source":"20"}}], + |"sinkStatus":{"description":"MySink","offsetDesc":"[#1, -]"}} + """.stripMargin.replace("\n", "").trim) + } + + test("prettyJson") { + assert( + StreamingQueryStatus.testStatus.prettyJson === + """ + |{ + | "sourceStatuses" : [ { + | "description" : "MySource1", + | "offsetDesc" : "#0", + | "inputRate" : 15.5, + | "processingRate" : 23.5, + | "triggerDetails" : { + | "numRows.input.source" : "100", + | "latency.getOffset.source" : "10", + | "latency.getBatch.source" : "20" + | } + | } ], + | "sinkStatus" : { + | "description" : "MySink", + | "offsetDesc" : "[#1, -]" + | } + |} + """.stripMargin.trim) + } +} From c1f344f1a09b8834bec70c1ece30b9bff63e55ea Mon Sep 17 00:00:00 2001 From: w00228970 Date: Fri, 21 Oct 2016 14:43:55 -0700 Subject: [PATCH 158/177] [SPARK-17929][CORE] Fix deadlock when CoarseGrainedSchedulerBackend reset ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-17929 Now `CoarseGrainedSchedulerBackend` reset will get the lock, ``` protected def reset(): Unit = synchronized { numPendingExecutors = 0 executorsPendingToRemove.clear() // Remove all the lingering executors that should be removed but not yet. The reason might be // because (1) disconnected event is not yet received; (2) executors die silently. executorDataMap.toMap.foreach { case (eid, _) => driverEndpoint.askWithRetry[Boolean]( RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered."))) } } ``` but on removeExecutor also need the lock "CoarseGrainedSchedulerBackend.this.synchronized", this will cause deadlock. ``` private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { logDebug(s"Asked to remove executor $executorId with reason $reason") executorDataMap.get(executorId) match { case Some(executorInfo) => // This must be synchronized because variables mutated // in this block are read when requesting executors val killed = CoarseGrainedSchedulerBackend.this.synchronized { addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingLossReason -= executorId executorsPendingToRemove.remove(executorId).getOrElse(false) } ... ## How was this patch tested? manual test. Author: w00228970 Closes #15481 from scwf/spark-17929. --- .../cluster/CoarseGrainedSchedulerBackend.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 0dae0e614e17d..10d55c87fb8de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -386,15 +386,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * Reset the state of CoarseGrainedSchedulerBackend to the initial state. Currently it will only * be called in the yarn-client mode when AM re-registers after a failure. * */ - protected def reset(): Unit = synchronized { - numPendingExecutors = 0 - executorsPendingToRemove.clear() + protected def reset(): Unit = { + val executors = synchronized { + numPendingExecutors = 0 + executorsPendingToRemove.clear() + Set() ++ executorDataMap.keys + } // Remove all the lingering executors that should be removed but not yet. The reason might be // because (1) disconnected event is not yet received; (2) executors die silently. - executorDataMap.toMap.foreach { case (eid, _) => - driverEndpoint.askWithRetry[Boolean]( - RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered."))) + executors.foreach { eid => + removeExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered.")) } } From 140570252fd3739d6bdcadd6d4d5a180e480d3e0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 21 Oct 2016 15:28:16 -0700 Subject: [PATCH 159/177] [SPARK-18044][STREAMING] FileStreamSource should not infer partitions in every batch ## What changes were proposed in this pull request? In `FileStreamSource.getBatch`, we will create a `DataSource` with specified schema, to avoid inferring the schema again and again. However, we don't pass the partition columns, and will infer the partition again and again. This PR fixes it by keeping the partition columns in `FileStreamSource`, like schema. ## How was this patch tested? N/A Author: Wenchen Fan Closes #15581 from cloud-fan/stream. --- .../execution/datasources/DataSource.scala | 26 +++++++++++++------ .../streaming/FileStreamSource.scala | 2 ++ .../streaming/FileStreamSourceSuite.scala | 2 +- 3 files changed, 21 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 92b1fff7d8127..17da606580eea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -75,7 +75,7 @@ case class DataSource( bucketSpec: Option[BucketSpec] = None, options: Map[String, String] = Map.empty) extends Logging { - case class SourceInfo(name: String, schema: StructType) + case class SourceInfo(name: String, schema: StructType, partitionColumns: Seq[String]) lazy val providingClass: Class[_] = lookupDataSource(className) lazy val sourceInfo = sourceSchema() @@ -186,8 +186,11 @@ case class DataSource( } } - private def inferFileFormatSchema(format: FileFormat): StructType = { - userSpecifiedSchema.orElse { + /** + * Infer the schema of the given FileFormat, returns a pair of schema and partition column names. + */ + private def inferFileFormatSchema(format: FileFormat): (StructType, Seq[String]) = { + userSpecifiedSchema.map(_ -> partitionColumns).orElse { val caseInsensitiveOptions = new CaseInsensitiveMap(options) val allPaths = caseInsensitiveOptions.get("path") val globbedPaths = allPaths.toSeq.flatMap { path => @@ -197,14 +200,14 @@ case class DataSource( SparkHadoopUtil.get.globPathIfNecessary(qualified) }.toArray val fileCatalog = new ListingFileCatalog(sparkSession, globbedPaths, options, None) - val partitionCols = fileCatalog.partitionSpec().partitionColumns.fields + val partitionSchema = fileCatalog.partitionSpec().partitionColumns val inferred = format.inferSchema( sparkSession, caseInsensitiveOptions, fileCatalog.allFiles()) inferred.map { inferredSchema => - StructType(inferredSchema ++ partitionCols) + StructType(inferredSchema ++ partitionSchema) -> partitionSchema.map(_.name) } }.getOrElse { throw new AnalysisException("Unable to infer schema. It must be specified manually.") @@ -217,7 +220,7 @@ case class DataSource( case s: StreamSourceProvider => val (name, schema) = s.sourceSchema( sparkSession.sqlContext, userSpecifiedSchema, className, options) - SourceInfo(name, schema) + SourceInfo(name, schema, Nil) case format: FileFormat => val caseInsensitiveOptions = new CaseInsensitiveMap(options) @@ -246,7 +249,8 @@ case class DataSource( "you may be able to create a static DataFrame on that directory with " + "'spark.read.load(directory)' and infer schema from it.") } - SourceInfo(s"FileSource[$path]", inferFileFormatSchema(format)) + val (schema, partCols) = inferFileFormatSchema(format) + SourceInfo(s"FileSource[$path]", schema, partCols) case _ => throw new UnsupportedOperationException( @@ -266,7 +270,13 @@ case class DataSource( throw new IllegalArgumentException("'path' is not specified") }) new FileStreamSource( - sparkSession, path, className, sourceInfo.schema, metadataPath, options) + sparkSession = sparkSession, + path = path, + fileFormatClassName = className, + schema = sourceInfo.schema, + partitionColumns = sourceInfo.partitionColumns, + metadataPath = metadataPath, + options = options) case _ => throw new UnsupportedOperationException( s"Data source $className does not support streamed reading") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 614a6261e7c28..115edf7ab2b61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -35,6 +35,7 @@ class FileStreamSource( path: String, fileFormatClassName: String, override val schema: StructType, + partitionColumns: Seq[String], metadataPath: String, options: Map[String, String]) extends Source with Logging { @@ -142,6 +143,7 @@ class FileStreamSource( sparkSession, paths = files.map(_.path), userSpecifiedSchema = Some(schema), + partitionColumns = partitionColumns, className = fileFormatClassName, options = optionsWithPartitionBasePath) Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala index 3e1e1126f9e6b..4a47c04d3f084 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala @@ -94,7 +94,7 @@ class FileStreamSourceSuite extends SparkFunSuite with SharedSQLContext { new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath) assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0)))) - val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), + val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil, dir.getAbsolutePath, Map.empty) // this method should throw an exception if `fs.exists` is called during resolveRelation newSource.getBatch(None, LongOffset(1)) From 268ccb9a48dfefc4d7bc85155e7e20a2dfe89307 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 21 Oct 2016 15:55:04 -0700 Subject: [PATCH 160/177] [SPARK-17812][SQL][KAFKA] Assign and specific startingOffsets for structured stream ## What changes were proposed in this pull request? startingOffsets takes specific per-topicpartition offsets as a json argument, usable with any consumer strategy assign with specific topicpartitions as a consumer strategy ## How was this patch tested? Unit tests Author: cody koeninger Closes #15504 from koeninger/SPARK-17812. --- .../structured-streaming-kafka-integration.md | 38 ++++-- .../apache/spark/sql/kafka010/JsonUtils.scala | 93 ++++++++++++++ .../spark/sql/kafka010/KafkaSource.scala | 64 ++++++++-- .../sql/kafka010/KafkaSourceProvider.scala | 52 ++++---- .../spark/sql/kafka010/StartingOffsets.scala | 32 +++++ .../spark/sql/kafka010/JsonUtilsSuite.scala | 45 +++++++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 114 ++++++++++++++++-- .../spark/sql/kafka010/KafkaTestUtils.scala | 14 ++- 8 files changed, 395 insertions(+), 57 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 668489addf82c..e851f210c92c4 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -150,16 +150,25 @@ The following options must be set for the Kafka source. + + + + + - + - @@ -174,16 +183,21 @@ The following configurations are optional:
    Optionvaluemeaning
    assignjson string {"topicA":[0,1],"topicB":[2,4]}Specific TopicPartitions to consume. + Only one of "assign", "subscribe" or "subscribePattern" + options can be specified for Kafka source.
    subscribe A comma-separated list of topicsThe topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be - specified for Kafka source.The topic list to subscribe. + Only one of "assign", "subscribe" or "subscribePattern" + options can be specified for Kafka source.
    subscribePattern Java regex stringThe pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" + The pattern used to subscribe to topic(s). + Only one of "assign, "subscribe" or "subscribePattern" options can be specified for Kafka source.
    - - - - + + + + - + + + + + + From eff4aed1ac1e500d4aa40665dd06b527dffbc111 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Sat, 22 Oct 2016 20:43:43 -0700 Subject: [PATCH 171/177] [SPARK-18035][SQL] Introduce performant and memory efficient APIs to create ArrayBasedMapData ## What changes were proposed in this pull request? Jira: https://issues.apache.org/jira/browse/SPARK-18035 In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658 The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323 This copy is not needed as we get rid of it once we extract the key and value arrays. Here is the call trace: ``` org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664) scala.collection.AbstractMap.toSeq(Map.scala:59) scala.collection.MapLike$class.toSeq(MapLike.scala:323) scala.collection.AbstractMap.toBuffer(Map.scala:59) scala.collection.MapLike$class.toBuffer(MapLike.scala:326) scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104) scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275) scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) scala.collection.AbstractIterable.foreach(Iterable.scala:54) scala.collection.IterableLike$class.foreach(IterableLike.scala:72) scala.collection.AbstractIterator.foreach(Iterator.scala:1336) scala.collection.Iterator$class.foreach(Iterator.scala:893) scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59) scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59) ``` Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration. EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient ## Performance gains The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query. ## How was this patch tested? This does not change the end result produced so relying on existing tests. Author: Tejas Patil Closes #15573 from tejasapatil/SPARK-18035_avoid_toSeq. --- .../sql/catalyst/CatalystTypeConverters.scala | 53 +++--------- .../expressions/complexTypeCreator.scala | 32 +++++--- .../sql/catalyst/util/ArrayBasedMapData.scala | 81 ++++++++++++++++++- .../sql/execution/python/EvaluatePython.scala | 10 +-- .../spark/sql/hive/HiveInspectors.scala | 11 +-- 5 files changed, 119 insertions(+), 68 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index f542f5cf40506..5b9161551a7af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -199,34 +199,14 @@ object CatalystTypeConverters { private[this] val keyConverter = getConverterForType(keyType) private[this] val valueConverter = getConverterForType(valueType) - override def toCatalystImpl(scalaValue: Any): MapData = scalaValue match { - case m: Map[_, _] => - val length = m.size - val convertedKeys = new Array[Any](length) - val convertedValues = new Array[Any](length) - - var i = 0 - for ((key, value) <- m) { - convertedKeys(i) = keyConverter.toCatalyst(key) - convertedValues(i) = valueConverter.toCatalyst(value) - i += 1 - } - ArrayBasedMapData(convertedKeys, convertedValues) - - case jmap: JavaMap[_, _] => - val length = jmap.size() - val convertedKeys = new Array[Any](length) - val convertedValues = new Array[Any](length) - - var i = 0 - val iter = jmap.entrySet.iterator - while (iter.hasNext) { - val entry = iter.next() - convertedKeys(i) = keyConverter.toCatalyst(entry.getKey) - convertedValues(i) = valueConverter.toCatalyst(entry.getValue) - i += 1 - } - ArrayBasedMapData(convertedKeys, convertedValues) + override def toCatalystImpl(scalaValue: Any): MapData = { + val keyFunction = (k: Any) => keyConverter.toCatalyst(k) + val valueFunction = (k: Any) => valueConverter.toCatalyst(k) + + scalaValue match { + case map: Map[_, _] => ArrayBasedMapData(map, keyFunction, valueFunction) + case javaMap: JavaMap[_, _] => ArrayBasedMapData(javaMap, keyFunction, valueFunction) + } } override def toScala(catalystValue: MapData): Map[Any, Any] = { @@ -433,18 +413,11 @@ object CatalystTypeConverters { case seq: Seq[Any] => new GenericArrayData(seq.map(convertToCatalyst).toArray) case r: Row => InternalRow(r.toSeq.map(convertToCatalyst): _*) case arr: Array[Any] => new GenericArrayData(arr.map(convertToCatalyst)) - case m: Map[_, _] => - val length = m.size - val convertedKeys = new Array[Any](length) - val convertedValues = new Array[Any](length) - - var i = 0 - for ((key, value) <- m) { - convertedKeys(i) = convertToCatalyst(key) - convertedValues(i) = convertToCatalyst(value) - i += 1 - } - ArrayBasedMapData(convertedKeys, convertedValues) + case map: Map[_, _] => + ArrayBasedMapData( + map, + (key: Any) => convertToCatalyst(key), + (value: Any) => convertToCatalyst(value)) case other => other } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 09e22aaf3e3d8..917aa0873130b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -427,18 +427,28 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E } } - override def nullSafeEval(str: Any, delim1: Any, delim2: Any): Any = { - val array = str.asInstanceOf[UTF8String] - .split(delim1.asInstanceOf[UTF8String], -1) - .map { kv => - val arr = kv.split(delim2.asInstanceOf[UTF8String], 2) - if (arr.length < 2) { - Array(arr(0), null) - } else { - arr - } + override def nullSafeEval( + inputString: Any, + stringDelimiter: Any, + keyValueDelimiter: Any): Any = { + val keyValues = + inputString.asInstanceOf[UTF8String].split(stringDelimiter.asInstanceOf[UTF8String], -1) + + val iterator = new Iterator[(UTF8String, UTF8String)] { + var index = 0 + val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String] + + override def hasNext: Boolean = { + keyValues.length > index } - ArrayBasedMapData(array.map(_ (0)), array.map(_ (1))) + + override def next(): (UTF8String, UTF8String) = { + val keyValueArray = keyValues(index).split(keyValueDelimiterUTF8String, 2) + index += 1 + (keyValueArray(0), if (keyValueArray.length < 2) null else keyValueArray(1)) + } + } + ArrayBasedMapData(iterator, keyValues.size, identity, identity) } override def prettyName: String = "str_to_map" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala index 4449da13c083c..91b3139443696 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.{Map => JavaMap} + class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) extends MapData { require(keyArray.numElements() == valueArray.numElements()) @@ -30,12 +32,83 @@ class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) exte } object ArrayBasedMapData { - def apply(map: Map[Any, Any]): ArrayBasedMapData = { - val array = map.toArray - ArrayBasedMapData(array.map(_._1), array.map(_._2)) + /** + * Creates a [[ArrayBasedMapData]] by applying the given converters over + * each (key -> value) pair of the input [[java.util.Map]] + * + * @param javaMap Input map + * @param keyConverter This function is applied over all the keys of the input map to + * obtain the output map's keys + * @param valueConverter This function is applied over all the values of the input map to + * obtain the output map's values + */ + def apply( + javaMap: JavaMap[_, _], + keyConverter: (Any) => Any, + valueConverter: (Any) => Any): ArrayBasedMapData = { + import scala.language.existentials + + val keys: Array[Any] = new Array[Any](javaMap.size()) + val values: Array[Any] = new Array[Any](javaMap.size()) + + var i: Int = 0 + val iterator = javaMap.entrySet().iterator() + while (iterator.hasNext) { + val entry = iterator.next() + keys(i) = keyConverter(entry.getKey) + values(i) = valueConverter(entry.getValue) + i += 1 + } + ArrayBasedMapData(keys, values) + } + + /** + * Creates a [[ArrayBasedMapData]] by applying the given converters over + * each (key -> value) pair of the input map + * + * @param map Input map + * @param keyConverter This function is applied over all the keys of the input map to + * obtain the output map's keys + * @param valueConverter This function is applied over all the values of the input map to + * obtain the output map's values + */ + def apply( + map: scala.collection.Map[_, _], + keyConverter: (Any) => Any = identity, + valueConverter: (Any) => Any = identity): ArrayBasedMapData = { + ArrayBasedMapData(map.iterator, map.size, keyConverter, valueConverter) + } + + /** + * Creates a [[ArrayBasedMapData]] by applying the given converters over + * each (key -> value) pair from the given iterator + * + * @param iterator Input iterator + * @param size Number of elements + * @param keyConverter This function is applied over all the keys extracted from the + * given iterator to obtain the output map's keys + * @param valueConverter This function is applied over all the values extracted from the + * given iterator to obtain the output map's values + */ + def apply( + iterator: Iterator[(_, _)], + size: Int, + keyConverter: (Any) => Any, + valueConverter: (Any) => Any): ArrayBasedMapData = { + + val keys: Array[Any] = new Array[Any](size) + val values: Array[Any] = new Array[Any](size) + + var i = 0 + for ((key, value) <- iterator) { + keys(i) = keyConverter(key) + values(i) = valueConverter(value) + i += 1 + } + ArrayBasedMapData(keys, values) } - def apply(keys: Array[Any], values: Array[Any]): ArrayBasedMapData = { + def apply(keys: Array[_], values: Array[_]): ArrayBasedMapData = { new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 724025b4647f4..46fd54e5c7420 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -124,11 +124,11 @@ object EvaluatePython { case (c, ArrayType(elementType, _)) if c.getClass.isArray => new GenericArrayData(c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType))) - case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => - val keyValues = c.asScala.toSeq - val keys = keyValues.map(kv => fromJava(kv._1, keyType)).toArray - val values = keyValues.map(kv => fromJava(kv._2, valueType)).toArray - ArrayBasedMapData(keys, values) + case (javaMap: java.util.Map[_, _], MapType(keyType, valueType, _)) => + ArrayBasedMapData( + javaMap, + (key: Any) => fromJava(key, keyType), + (value: Any) => fromJava(value, valueType)) case (c, StructType(fields)) if c.getClass.isArray => val array = c.asInstanceOf[Array[_]] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 1625116803505..e303065127c3b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -473,10 +473,8 @@ private[hive] trait HiveInspectors { case mi: StandardConstantMapObjectInspector => val keyUnwrapper = unwrapperFor(mi.getMapKeyObjectInspector) val valueUnwrapper = unwrapperFor(mi.getMapValueObjectInspector) - val keyValues = mi.getWritableConstantValue.asScala.toSeq - val keys = keyValues.map(kv => keyUnwrapper(kv._1)).toArray - val values = keyValues.map(kv => valueUnwrapper(kv._2)).toArray - val constant = ArrayBasedMapData(keys, values) + val keyValues = mi.getWritableConstantValue + val constant = ArrayBasedMapData(keyValues, keyUnwrapper, valueUnwrapper) _ => constant case li: StandardConstantListObjectInspector => val unwrapper = unwrapperFor(li.getListElementObjectInspector) @@ -655,10 +653,7 @@ private[hive] trait HiveInspectors { if (map == null) { null } else { - val keyValues = map.asScala.toSeq - val keys = keyValues.map(kv => keyUnwrapper(kv._1)).toArray - val values = keyValues.map(kv => valueUnwrapper(kv._2)).toArray - ArrayBasedMapData(keys, values) + ArrayBasedMapData(map, keyUnwrapper, valueUnwrapper) } } else { null From 21c7539a5274a7e77686d17a6261d56592b85c2d Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Sun, 23 Oct 2016 13:25:47 +0200 Subject: [PATCH 172/177] [SPARK-18038][SQL] Move output partitioning definition from UnaryNodeExec to its children ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-18038 This was a suggestion by rxin over one of the dev list discussion : http://apache-spark-developers-list.1001551.n3.nabble.com/Project-not-preserving-child-partitioning-td19417.html His words: >> It would be better (safer) to move the output partitioning definition into each of the operator and remove it from UnaryExecNode. With this PR, following is the output partitioning and ordering for all the impls of `UnaryExecNode`. UnaryExecNode's impl | outputPartitioning | outputOrdering | comment ------------ | ------------- | ------------ | ------------ AppendColumnsExec | child's | Nil | child's ordering can be used AppendColumnsWithObjectExec | child's | Nil | child's ordering can be used BroadcastExchangeExec | BroadcastPartitioning | Nil | - CoalesceExec | UnknownPartitioning | Nil | - CollectLimitExec | SinglePartition | Nil | - DebugExec | child's | Nil | child's ordering can be used DeserializeToObjectExec | child's | Nil | child's ordering can be used ExpandExec | UnknownPartitioning | Nil | - FilterExec | child's | child's | - FlatMapGroupsInRExec | child's | Nil | child's ordering can be used GenerateExec | child's | Nil | need to dig more GlobalLimitExec | child's | child's | - HashAggregateExec | child's | Nil | - InputAdapter | child's | child's | - InsertIntoHiveTable | child's | Nil | terminal node, doesn't need partitioning LocalLimitExec | child's | child's | - MapElementsExec | child's | child's | - MapGroupsExec | child's | Nil | child's ordering can be used MapPartitionsExec | child's | Nil | child's ordering can be used ProjectExec | child's | child's | - SampleExec | child's | Nil | child's ordering can be used ScriptTransformation | child's | Nil | child's ordering can be used SerializeFromObjectExec | child's | Nil | child's ordering can be used ShuffleExchange | custom | Nil | - SortAggregateExec | child's | sort over grouped exprs | - SortExec | child's | custom | - StateStoreRestoreExec | child's | Nil | child's ordering can be used StateStoreSaveExec | child's | Nil | child's ordering can be used SubqueryExec | child's | child's | - TakeOrderedAndProjectExec | SinglePartition | custom | - WholeStageCodegenExec | child's | child's | - WindowExec | child's | child's | - ## How was this patch tested? This does NOT change any existing functionality so relying on existing tests Author: Tejas Patil Closes #15575 from tejasapatil/SPARK-18038_UnaryNodeExec_output_partitioning. --- .../spark/sql/execution/GenerateExec.scala | 3 +++ .../apache/spark/sql/execution/SortExec.scala | 6 +++++- .../apache/spark/sql/execution/SparkPlan.scala | 2 -- .../sql/execution/WholeStageCodegenExec.scala | 4 ++++ .../execution/aggregate/HashAggregateExec.scala | 2 ++ .../execution/aggregate/SortAggregateExec.scala | 4 +++- .../sql/execution/basicPhysicalOperators.scala | 8 ++++++++ .../spark/sql/execution/debug/package.scala | 4 +++- .../org/apache/spark/sql/execution/limit.scala | 16 +++++++++++----- .../apache/spark/sql/execution/objects.scala | 17 +++++++++++++++++ .../execution/streaming/StatefulAggregate.scala | 6 ++++++ .../spark/sql/execution/window/WindowExec.scala | 2 ++ .../spark/sql/execution/ReferenceSort.scala | 2 ++ .../hive/execution/InsertIntoHiveTable.scala | 4 +++- .../hive/execution/ScriptTransformation.scala | 3 +++ .../execution/ScriptTransformationSuite.scala | 4 ++++ 16 files changed, 76 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 39189a2b0c72c..2663129562660 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -60,6 +61,8 @@ case class GenerateExec( override def producedAttributes: AttributeSet = AttributeSet(output) + override def outputPartitioning: Partitioning = child.outputPartitioning + val boundGenerator = BindReferences.bindReference(generator, child.output) protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index d8e0675e3eb65..cc576bbc4c802 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -23,7 +23,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -45,6 +45,10 @@ case class SortExec( override def outputOrdering: Seq[SortOrder] = sortOrder + // sort performed is local within a given partition so will retain + // child operator's partitioning + override def outputPartitioning: Partitioning = child.outputPartitioning + override def requiredChildDistribution: Seq[Distribution] = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 48d6ef6dcd44a..24d0cffef82a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -395,8 +395,6 @@ trait UnaryExecNode extends SparkPlan { def child: SparkPlan override final def children: Seq[SparkPlan] = child :: Nil - - override def outputPartitioning: Partitioning = child.outputPartitioning } trait BinaryExecNode extends SparkPlan { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 62bf6f4a81eec..6303483f22fd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -218,7 +218,9 @@ trait CodegenSupport extends SparkPlan { case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override def doExecute(): RDD[InternalRow] = { @@ -292,7 +294,9 @@ object WholeStageCodegenExec { case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override lazy val metrics = Map( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 06199ef3e8243..4529ed067e565 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -63,6 +63,8 @@ case class HashAggregateExec( override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + override def producedAttributes: AttributeSet = AttributeSet(aggregateAttributes) ++ AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 2a81a823c44b3..be3198b8e7d82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.Utils @@ -66,6 +66,8 @@ case class SortAggregateExec( groupingExpressions.map(SortOrder(_, Ascending)) :: Nil } + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = { groupingExpressions.map(SortOrder(_, Ascending)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index dd78a784915d2..37d750e621c25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -78,6 +78,8 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) } override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } @@ -214,6 +216,8 @@ case class FilterExec(condition: Expression, child: SparkPlan) } override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** @@ -234,6 +238,8 @@ case class SampleExec( child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -517,7 +523,9 @@ case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode { "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)")) override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override def sameResult(o: SparkPlan): Boolean = o match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index dd9d83767e221..0395c43ba2cbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -27,8 +27,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{AccumulatorV2, LongAccumulator} /** @@ -162,6 +162,8 @@ package object debug { } } + override def outputPartitioning: Partitioning = child.outputPartitioning + override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 86a8770715600..9918ac327f2dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.util.Utils - /** * Take the first `limit` elements and collect them to a single partition. * @@ -54,8 +53,7 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode trait BaseLimitExec extends UnaryExecNode with CodegenSupport { val limit: Int override def output: Seq[Attribute] = child.output - override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override def outputPartitioning: Partitioning = child.outputPartitioning + protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => iter.take(limit) } @@ -95,14 +93,22 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport { * Take the first `limit` elements of each child partition, but do not collect or shuffle them. */ case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** * Take the first `limit` elements of the child's single output partition. */ case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { + override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -122,8 +128,6 @@ case class TakeOrderedAndProjectExec( projectList.map(_.toAttribute) } - override def outputPartitioning: Partitioning = SinglePartition - override def executeCollect(): Array[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) val data = child.execute().map(_.copy()).takeOrdered(limit)(ord) @@ -160,6 +164,8 @@ case class TakeOrderedAndProjectExec( override def outputOrdering: Seq[SortOrder] = sortOrder + override def outputPartitioning: Partitioning = SinglePartition + override def simpleString: String = { val orderByString = Utils.truncatedString(sortOrder, "[", ",", "]") val outputString = Utils.truncatedString(output, "[", ",", "]") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 2acc5110e8950..9df56bbf1ef87 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -68,6 +68,8 @@ case class DeserializeToObjectExec( outputObjAttr: Attribute, child: SparkPlan) extends UnaryExecNode with ObjectProducerExec with CodegenSupport { + override def outputPartitioning: Partitioning = child.outputPartitioning + override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() } @@ -102,6 +104,8 @@ case class SerializeFromObjectExec( override def output: Seq[Attribute] = serializer.map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() } @@ -171,6 +175,8 @@ case class MapPartitionsExec( child: SparkPlan) extends ObjectConsumerExec with ObjectProducerExec { + override def outputPartitioning: Partitioning = child.outputPartitioning + override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) @@ -231,6 +237,8 @@ case class MapElementsExec( } override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** @@ -244,6 +252,8 @@ case class AppendColumnsExec( override def output: Seq[Attribute] = child.output ++ serializer.map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + private def newColumnSchema = serializer.map(_.toAttribute).toStructType override protected def doExecute(): RDD[InternalRow] = { @@ -272,6 +282,8 @@ case class AppendColumnsWithObjectExec( override def output: Seq[Attribute] = (inputSerializer ++ newColumnsSerializer).map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + private def inputSchema = inputSerializer.map(_.toAttribute).toStructType private def newColumnSchema = newColumnsSerializer.map(_.toAttribute).toStructType @@ -304,6 +316,8 @@ case class MapGroupsExec( outputObjAttr: Attribute, child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { + override def outputPartitioning: Partitioning = child.outputPartitioning + override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(groupingAttributes) :: Nil @@ -347,6 +361,9 @@ case class FlatMapGroupsInRExec( child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { override def output: Seq[Attribute] = outputObjAttr :: Nil + + override def outputPartitioning: Partitioning = child.outputPartitioning + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) override def requiredChildDistribution: Seq[Distribution] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index 587ea7d02acab..ad8238f189c64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.streaming.state._ @@ -80,7 +81,10 @@ case class StateStoreRestoreExec( } } } + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** @@ -116,6 +120,8 @@ case class StateStoreSaveExec( override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + /** * Save all the rows to the state store, and return all the rows in the state store. * Note that this returns an iterator that pipelines the saving to store with downstream diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala index 1dd281ebf1034..80b87d5ffa797 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -103,6 +103,8 @@ case class WindowExec( override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def outputPartitioning: Partitioning = child.outputPartitioning + /** * Create a bound ordering object for a given frame type and offset. A bound ordering object is * used to determine which input row lies within the frame boundaries of an output row. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala index a19ea51af7c01..6abcb1f067968 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala @@ -57,4 +57,6 @@ case class ReferenceSort( override def output: Seq[Attribute] = child.output override def outputOrdering: Seq[SortOrder] = sortOrder + + override def outputPartitioning: Partitioning = child.outputPartitioning } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 53bb3b93db738..c3c4e2925b90c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive.execution import java.io.IOException import java.net.URI import java.text.SimpleDateFormat -import java.util import java.util.{Date, Random} import scala.collection.JavaConverters._ @@ -36,6 +35,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} @@ -291,6 +291,8 @@ case class InsertIntoHiveTable( Seq.empty[InternalRow] } + override def outputPartitioning: Partitioning = child.outputPartitioning + override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 1025b8f70d9ff..50855e48bc8fe 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -38,6 +38,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive.HiveInspectors import org.apache.spark.sql.hive.HiveShim._ @@ -61,6 +62,8 @@ case class ScriptTransformation( override def producedAttributes: AttributeSet = outputSet -- inputSet + override def outputPartitioning: Partitioning = child.outputPartitioning + protected override def doExecute(): RDD[InternalRow] = { def processIterator(inputIterator: Iterator[InternalRow], hadoopConf: Configuration) : Iterator[InternalRow] = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index a8e81d7a3c42a..0e837766e2ea4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types.StringType @@ -135,5 +136,8 @@ private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExe throw new IllegalArgumentException("intentional exception") } } + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning } From b158256c2e719edde3dbdfe27a9a65cd3b3039f4 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Sun, 23 Oct 2016 13:28:35 +0200 Subject: [PATCH 173/177] [SPARK-18045][SQL][TESTS] Move `HiveDataFrameAnalyticsSuite` to package `sql` ## What changes were proposed in this pull request? The testsuite `HiveDataFrameAnalyticsSuite` has nothing to do with HIVE, we should move it to package `sql`. The original test cases in that suite are splited into two existing testsuites: `DataFrameAggregateSuite` tests for the functions and ~~`SQLQuerySuite`~~`SQLQueryTestSuite` tests for the SQL statements. ## How was this patch tested? ~~Modified `SQLQuerySuite` in package `sql`.~~ Add query file for `SQLQueryTestSuite`. Author: jiangxingbo Closes #15582 from jiangxb1987/group-analytics-test. --- .../sql-tests/inputs/group-analytics.sql | 13 +++ .../sql-tests/results/group-analytics.sql.out | 87 +++++++++++++++++++ .../hive/HiveDataFrameAnalyticsSuite.scala | 72 --------------- 3 files changed, 100 insertions(+), 72 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql b/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql new file mode 100644 index 0000000000000..2f783495ddf96 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql @@ -0,0 +1,13 @@ +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b); + +-- CUBE on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE; + +-- ROLLUP on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out new file mode 100644 index 0000000000000..8ea7de809d19d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE +-- !query 1 schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query 1 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + + +-- !query 2 +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE +-- !query 2 schema +struct +-- !query 2 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL 1 3 +NULL 2 6 +NULL NULL 9 + + +-- !query 3 +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query 3 schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query 3 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query 4 +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP +-- !query 4 schema +struct +-- !query 4 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL NULL 9 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala deleted file mode 100644 index 6477974fe713a..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.test.TestHiveSingleton - -// TODO ideally we should put the test suite into the package `sql`, as -// `hive` package is optional in compiling, however, `SQLContext.sql` doesn't -// support the `cube` or `rollup` yet. -class HiveDataFrameAnalyticsSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { - import spark.implicits._ - import spark.sql - - private var testData: DataFrame = _ - - override def beforeAll() { - super.beforeAll() - testData = Seq((1, 2), (2, 2), (3, 4)).toDF("a", "b") - testData.createOrReplaceTempView("mytable") - } - - override def afterAll(): Unit = { - try { - spark.catalog.dropTempView("mytable") - } finally { - super.afterAll() - } - } - - test("rollup") { - checkAnswer( - testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with rollup").collect() - ) - - checkAnswer( - testData.rollup("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with rollup").collect() - ) - } - - test("cube") { - checkAnswer( - testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with cube").collect() - ) - - checkAnswer( - testData.cube("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with cube").collect() - ) - } -} From a81fba048fabcd413730548ab65955802508d4e4 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sun, 23 Oct 2016 19:42:11 +0200 Subject: [PATCH 174/177] [SPARK-18058][SQL] Comparing column types ignoring Nullability in Union and SetOperation ## What changes were proposed in this pull request? The PR tries to fix [SPARK-18058](https://issues.apache.org/jira/browse/SPARK-18058) which refers to a bug that the column types are compared with the extra care about Nullability in Union and SetOperation. This PR converts the columns types by setting all fields as nullable before comparison ## How was this patch tested? regular unit test cases Author: CodingCat Closes #15595 from CodingCat/SPARK-18058. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 3 +- .../plans/logical/basicLogicalOperators.scala | 30 +++++++------------ .../sql/catalyst/analysis/AnalysisSuite.scala | 19 ++++++++++++ 3 files changed, 31 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 9c06069f24f76..9a7c2a944b588 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -287,7 +287,8 @@ trait CheckAnalysis extends PredicateHelper { } // Check if the data types match. dataTypes(child).zip(ref).zipWithIndex.foreach { case ((dt1, dt2), ci) => - if (dt1 != dt2) { + // SPARK-18058: we shall not care about the nullability of columns + if (dt1.asNullable != dt2.asNullable) { failAnalysis( s""" |${operator.nodeName} can only be performed on tables with the compatible diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index d2d33e40a8c8f..64a787a7ae351 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -117,6 +117,8 @@ case class Filter(condition: Expression, child: LogicalPlan) abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty + protected def leftConstraints: Set[Expression] = left.constraints protected def rightConstraints: Set[Expression] = { @@ -126,6 +128,13 @@ abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends Binar case a: Attribute => attributeRewrites(a) }) } + + override lazy val resolved: Boolean = + childrenResolved && + left.output.length == right.output.length && + left.output.zip(right.output).forall { case (l, r) => + l.dataType.asNullable == r.dataType.asNullable + } && duplicateResolved } object SetOperation { @@ -134,8 +143,6 @@ object SetOperation { case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) { - def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty - override def output: Seq[Attribute] = left.output.zip(right.output).map { case (leftAttr, rightAttr) => leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) @@ -144,14 +151,6 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation override protected def validConstraints: Set[Expression] = leftConstraints.union(rightConstraints) - // Intersect are only resolved if they don't introduce ambiguous expression ids, - // since the Optimizer will convert Intersect to Join. - override lazy val resolved: Boolean = - childrenResolved && - left.output.length == right.output.length && - left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } && - duplicateResolved - override def maxRows: Option[Long] = { if (children.exists(_.maxRows.isEmpty)) { None @@ -172,19 +171,11 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) { - def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty - /** We don't use right.output because those rows get excluded from the set. */ override def output: Seq[Attribute] = left.output override protected def validConstraints: Set[Expression] = leftConstraints - override lazy val resolved: Boolean = - childrenResolved && - left.output.length == right.output.length && - left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } && - duplicateResolved - override lazy val statistics: Statistics = { left.statistics.copy() } @@ -219,9 +210,8 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { child.output.length == children.head.output.length && // compare the data types with the first child child.output.zip(children.head.output).forall { - case (l, r) => l.dataType == r.dataType } + case (l, r) => l.dataType.asNullable == r.dataType.asNullable } ) - children.length > 1 && childrenResolved && allChildrenCompatible } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 50ebad25cd258..590774c043040 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -377,4 +377,23 @@ class AnalysisSuite extends AnalysisTest { assertExpressionType(sum(Divide(Decimal(1), 2.0)), DoubleType) assertExpressionType(sum(Divide(1.0, Decimal(2.0))), DoubleType) } + + test("SPARK-18058: union and set operations shall not care about the nullability" + + " when comparing column types") { + val firstTable = LocalRelation( + AttributeReference("a", + StructType(Seq(StructField("a", IntegerType, nullable = true))), nullable = false)()) + val secondTable = LocalRelation( + AttributeReference("a", + StructType(Seq(StructField("a", IntegerType, nullable = false))), nullable = false)()) + + val unionPlan = Union(firstTable, secondTable) + assertAnalysisSuccess(unionPlan) + + val r1 = Except(firstTable, secondTable) + val r2 = Intersect(firstTable, secondTable) + + assertAnalysisSuccess(r1) + assertAnalysisSuccess(r2) + } } From 3a423f5a0373de87ddfb4744852b2fda14fcc3cb Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sun, 23 Oct 2016 10:53:27 -0700 Subject: [PATCH 175/177] [SPARKR][BRANCH-2.0] R merge API doc and example fix ## What changes were proposed in this pull request? Fixes for R doc ## How was this patch tested? N/A Author: Felix Cheung Closes #15589 from felixcheung/rdocmergefix. (cherry picked from commit 0e0d83a597885ab1773cb69d6dcc10346d6976a3) Signed-off-by: Felix Cheung --- R/pkg/R/DataFrame.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 8910a4b138a37..b6ce838969a44 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -365,7 +365,7 @@ setMethod("colnames<-", # Check if the column names have . in it if (any(regexec(".", value, fixed = TRUE)[[1]][1] != -1)) { - stop("Colum names cannot contain the '.' symbol.") + stop("Column names cannot contain the '.' symbol.") } sdf <- callJMethod(x@sdf, "toDF", as.list(value)) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index b4b43fdba42ce..e77dbde44ee66 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -845,7 +845,7 @@ test_that("names() colnames() set the column names", { expect_equal(names(df)[1], "col3") expect_error(colnames(df) <- c("sepal.length", "sepal_width"), - "Colum names cannot contain the '.' symbol.") + "Column names cannot contain the '.' symbol.") expect_error(colnames(df) <- c(1, 2), "Invalid column names.") expect_error(colnames(df) <- c("a"), "Column names must have the same length as the number of columns in the dataset.") From c64a8ff39794d60c596c0d34130019c09c9c8012 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Mon, 24 Oct 2016 10:25:24 +0100 Subject: [PATCH 176/177] [SPARK-18049][MLLIB][TEST] Add missing tests for truePositiveRate and weightedTruePositiveRate ## What changes were proposed in this pull request? Add missing tests for `truePositiveRate` and `weightedTruePositiveRate` in `MulticlassMetricsSuite` ## How was this patch tested? added testing Author: Zheng RuiFeng Closes #15585 from zhengruifeng/mc_missing_test. --- .../api/python/WriteInputFormatTestDataGenerator.scala | 2 +- .../main/scala/org/apache/spark/ml/util/ReadWrite.scala | 2 +- .../apache/spark/mllib/evaluation/RegressionMetrics.scala | 2 +- .../spark/mllib/linalg/distributed/BlockMatrix.scala | 4 ++-- .../spark/mllib/evaluation/MulticlassMetricsSuite.scala | 8 ++++++++ .../spark/mllib/evaluation/MultilabelMetricsSuite.scala | 2 +- 6 files changed, 14 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index 34cb7c61d7034..86965dbc2e778 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -144,7 +144,7 @@ object WriteInputFormatTestDataGenerator { // Create test data for ArrayWritable val data = Seq( - (1, Array()), + (1, Array.empty[Double]), (2, Array(3.0, 4.0, 5.0)), (3, Array(4.0, 5.0, 6.0)) ) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index 4413fefdea3ca..bc4f9e6716ee8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -474,7 +474,7 @@ private[ml] object MetaAlgorithmReadWrite { case ovr: OneVsRest => Array(ovr.getClassifier) case ovrModel: OneVsRestModel => Array(ovrModel.getClassifier) ++ ovrModel.models case rformModel: RFormulaModel => Array(rformModel.pipelineModel) - case _: Params => Array() + case _: Params => Array.empty[Params] } val subStageMaps = subStages.flatMap(getUidMapImpl) List((instance.uid, instance)) ++ subStageMaps diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index ce4421515126c..8f777cc35b93f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -73,7 +73,7 @@ class RegressionMetrics @Since("2.0.0") ( /** * Returns the variance explained by regression. - * explainedVariance = $\sum_i (\hat{y_i} - \bar{y})^2 / n$ + * explainedVariance = $\sum_i (\hat{y_i} - \bar{y})^2^ / n$ * @see [[https://en.wikipedia.org/wiki/Fraction_of_variance_unexplained]] */ @Since("1.2.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index ff1068417d94f..377be6bfb9886 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -429,14 +429,14 @@ class BlockMatrix @Since("1.3.0") ( val rightCounterpartsHelper = rightMatrix.groupBy(_._1).mapValues(_.map(_._2)) val leftDestinations = leftMatrix.map { case (rowIndex, colIndex) => - val rightCounterparts = rightCounterpartsHelper.getOrElse(colIndex, Array()) + val rightCounterparts = rightCounterpartsHelper.getOrElse(colIndex, Array.empty[Int]) val partitions = rightCounterparts.map(b => partitioner.getPartition((rowIndex, b))) ((rowIndex, colIndex), partitions.toSet) }.toMap val leftCounterpartsHelper = leftMatrix.groupBy(_._2).mapValues(_.map(_._1)) val rightDestinations = rightMatrix.map { case (rowIndex, colIndex) => - val leftCounterparts = leftCounterpartsHelper.getOrElse(rowIndex, Array()) + val leftCounterparts = leftCounterpartsHelper.getOrElse(rowIndex, Array.empty[Int]) val partitions = leftCounterparts.map(b => partitioner.getPartition((b, colIndex))) ((rowIndex, colIndex), partitions.toSet) }.toMap diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index f316c67234f18..142d1e9812ef1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -36,6 +36,9 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2) val metrics = new MulticlassMetrics(predictionAndLabels) val delta = 0.0000001 + val tpRate0 = 2.0 / (2 + 2) + val tpRate1 = 3.0 / (3 + 1) + val tpRate2 = 1.0 / (1 + 0) val fpRate0 = 1.0 / (9 - 4) val fpRate1 = 1.0 / (9 - 4) val fpRate2 = 1.0 / (9 - 1) @@ -53,6 +56,9 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val f2measure2 = (1 + 2 * 2) * precision2 * recall2 / (2 * 2 * precision2 + recall2) assert(metrics.confusionMatrix.toArray.sameElements(confusionMatrix.toArray)) + assert(math.abs(metrics.truePositiveRate(0.0) - tpRate0) < delta) + assert(math.abs(metrics.truePositiveRate(1.0) - tpRate1) < delta) + assert(math.abs(metrics.truePositiveRate(2.0) - tpRate2) < delta) assert(math.abs(metrics.falsePositiveRate(0.0) - fpRate0) < delta) assert(math.abs(metrics.falsePositiveRate(1.0) - fpRate1) < delta) assert(math.abs(metrics.falsePositiveRate(2.0) - fpRate2) < delta) @@ -75,6 +81,8 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(math.abs(metrics.accuracy - metrics.recall) < delta) assert(math.abs(metrics.accuracy - metrics.fMeasure) < delta) assert(math.abs(metrics.accuracy - metrics.weightedRecall) < delta) + assert(math.abs(metrics.weightedTruePositiveRate - + ((4.0 / 9) * tpRate0 + (4.0 / 9) * tpRate1 + (1.0 / 9) * tpRate2)) < delta) assert(math.abs(metrics.weightedFalsePositiveRate - ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) assert(math.abs(metrics.weightedPrecision - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala index f3b19aeb42f84..a660492c7ae59 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala @@ -47,7 +47,7 @@ class MultilabelMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val scoreAndLabels: RDD[(Array[Double], Array[Double])] = sc.parallelize( Seq((Array(0.0, 1.0), Array(0.0, 2.0)), (Array(0.0, 2.0), Array(0.0, 1.0)), - (Array(), Array(0.0)), + (Array.empty[Double], Array(0.0)), (Array(2.0), Array(2.0)), (Array(2.0, 0.0), Array(2.0, 0.0)), (Array(0.0, 1.0, 2.0), Array(0.0, 1.0)), From 4ecbe1b92f4c4c5b2d734895c09d8ded0ed48d4d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 24 Oct 2016 10:44:45 +0100 Subject: [PATCH 177/177] [SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but can resolve as HDFS path ## What changes were proposed in this pull request? Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir ## How was this patch tested? Existing tests. Author: Sean Owen Closes #15382 from srowen/SPARK-17810. --- docs/sql-programming-guide.md | 33 +++---------------- .../sql/hive/JavaSparkHiveExample.java | 2 +- examples/src/main/python/sql/hive.py | 2 +- .../examples/sql/hive/SparkHiveExample.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 3 +- .../sql/execution/command/DDLSuite.scala | 23 ++++++------- .../spark/sql/internal/SQLConfSuite.scala | 6 ++-- .../sql/hive/execution/HiveQuerySuite.scala | 4 ++- .../spark/sql/sources/BucketedReadSuite.scala | 5 +-- 9 files changed, 29 insertions(+), 51 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d334a86bc73d7..064af41965b70 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -904,50 +904,27 @@ access data stored in Hive. Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. -
    - -
    - When working with Hive, one must instantiate `SparkSession` with Hive support, including connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. Users who do not have an existing Hive deployment can still enable Hive support. When not configured by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that +`spark-warehouse` in the current directory that the Spark application is started. Note that the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. +You may need to grant write privilege to the user who starts the Spark application. +
    + +
    {% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %}
    - -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. - {% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %}
    - -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. - {% include_example spark_hive python/sql/hive.py %}
    diff --git a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java index 76dd160d5568b..052153c9e9736 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java @@ -56,7 +56,7 @@ public void setValue(String value) { public static void main(String[] args) { // $example on:spark_hive$ // warehouseLocation points to the default location for managed databases and tables - String warehouseLocation = "file:" + System.getProperty("user.dir") + "spark-warehouse"; + String warehouseLocation = "spark-warehouse"; SparkSession spark = SparkSession .builder() .appName("Java Spark Hive Example") diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index 98b48908b5a12..ad83fe1cf14b5 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -34,7 +34,7 @@ if __name__ == "__main__": # $example on:spark_hive$ # warehouse_location points to the default location for managed databases and tables - warehouse_location = 'file:${system:user.dir}/spark-warehouse' + warehouse_location = 'spark-warehouse' spark = SparkSession \ .builder \ diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala index 11e84c0e45632..ded18dacf1fe3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala @@ -38,7 +38,7 @@ object SparkHiveExample { // $example on:spark_hive$ // warehouseLocation points to the default location for managed databases and tables - val warehouseLocation = "file:${system:user.dir}/spark-warehouse" + val warehouseLocation = "spark-warehouse" val spark = SparkSession .builder() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a6e2fa26cb5ef..f47ec7f3963a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.catalyst.CatalystConf +import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the configuration options for Spark SQL. @@ -56,7 +57,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("${system:user.dir}/spark-warehouse") + .createWithDefault(Utils.resolveURI("spark-warehouse").toString) val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index a6da8a86c1623..d593bfb4ce19a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -43,8 +43,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { // drop all databases, tables and functions after each test spark.sessionState.catalog.reset() } finally { - val path = System.getProperty("user.dir") + "/spark-warehouse" - Utils.deleteRecursively(new File(path)) + Utils.deleteRecursively(new File("spark-warehouse")) super.afterEach() } } @@ -116,7 +115,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath // The generated temp path is not qualified. assert(!path.startsWith("file:/")) val uri = tmpDir.toURI @@ -148,7 +147,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Create/Drop Database") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -159,7 +158,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -184,9 +183,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { try { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) - val expectedLocation = - makeQualifiedPath(s"${System.getProperty("user.dir")}/spark-warehouse" + - "/" + s"$dbName.db") + val expectedLocation = makeQualifiedPath(s"spark-warehouse/$dbName.db") assert(db1 == CatalogDatabase( dbName, "", @@ -204,7 +201,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => - val path = new Path(tmpDir.toString).toUri.toString + val path = new Path(tmpDir.getCanonicalPath).toUri databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) @@ -227,7 +224,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Create Database - database already exists") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -237,7 +234,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -476,7 +473,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Alter/Describe Database") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -484,7 +481,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val location = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") sql(s"CREATE DATABASE $dbName") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index df640ffab91de..a89a43fa1e777 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.internal import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.util.Utils class SQLConfSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -219,8 +219,8 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { try { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) - assert(spark.sessionState.conf.warehousePath - === new Path(s"${System.getProperty("user.dir")}/spark-warehouse").toString) + assert(new Path(Utils.resolveURI("spark-warehouse")).toString === + spark.sessionState.conf.warehousePath + "/") } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") } 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 2b945dbbe03dd..6fbbed1d47e04 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.net.URI import java.sql.Timestamp import java.util.{Locale, TimeZone} @@ -954,7 +955,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd .mkString("/") // Loads partition data to a temporary table to verify contents - val path = s"${sparkSession.getWarehousePath}/dynamic_part_table/$partFolder/part-00000" + val warehousePathFile = new URI(sparkSession.getWarehousePath()).getPath + val path = s"$warehousePathFile/dynamic_part_table/$partFolder/part-00000" sql("DROP TABLE IF EXISTS dp_verify") sql("CREATE TABLE dp_verify(intcol INT)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 9ed454e578d69..d9ddcbd57ca83 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import java.io.File +import java.net.URI import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -489,8 +490,8 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") - val tableDir = new File(hiveContext - .sparkSession.getWarehousePath, "bucketed_table") + val warehouseFilePath = new URI(hiveContext.sparkSession.getWarehousePath).getPath + val tableDir = new File(warehouseFilePath, "bucketed_table") Utils.deleteRecursively(tableDir) df1.write.parquet(tableDir.getAbsolutePath)
    Optionvaluedefaultmeaning
    startingOffset["earliest", "latest"]"latest"The start point when a query is started, either "earliest" which is from the earliest offset, - or "latest" which is just from the latest offset. Note: This only applies when a new Streaming q - uery is started, and that resuming will always pick up from where the query left off.startingOffsetsearliest, latest, or json string + {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}} + latestThe start point when a query is started, either "earliest" which is from the earliest offsets, + "latest" which is just from the latest offsets, or a json string specifying a starting offset for + each TopicPartition. In the json, -2 as an offset can be used to refer to earliest, -1 to latest. + Note: This only applies when a new Streaming query is started, and that resuming will always pick + up from where the query left off. Newly discovered partitions during a query will start at + earliest.
    failOnDataLoss[true, false]true or false true Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work @@ -215,10 +229,10 @@ Kafka's own configurations can be set via `DataStreamReader.option` with `kafka. Note that the following Kafka params cannot be set and the Kafka source will throw an exception: - **group.id**: Kafka source will create a unique group id for each query automatically. -- **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify +- **auto.offset.reset**: Set the source option `startingOffsets` to specify where to start instead. Structured Streaming manages which offsets are consumed internally, rather than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new - topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new + topics/partitions are dynamically subscribed. Note that `startingOffsets` only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off. - **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations to explicitly deserialize the keys. diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala new file mode 100644 index 0000000000000..40d568a12c25d --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.io.Writer + +import scala.collection.mutable.HashMap +import scala.util.control.NonFatal + +import org.apache.kafka.common.TopicPartition +import org.json4s.NoTypeHints +import org.json4s.jackson.Serialization + +/** + * Utilities for converting Kafka related objects to and from json. + */ +private object JsonUtils { + private implicit val formats = Serialization.formats(NoTypeHints) + + /** + * Read TopicPartitions from json string + */ + def partitions(str: String): Array[TopicPartition] = { + try { + Serialization.read[Map[String, Seq[Int]]](str).flatMap { case (topic, parts) => + parts.map { part => + new TopicPartition(topic, part) + } + }.toArray + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":[0,1],"topicB":[0,1]}, got $str""") + } + } + + /** + * Write TopicPartitions as json string + */ + def partitions(partitions: Iterable[TopicPartition]): String = { + val result = new HashMap[String, List[Int]] + partitions.foreach { tp => + val parts: List[Int] = result.getOrElse(tp.topic, Nil) + result += tp.topic -> (tp.partition::parts) + } + Serialization.write(result) + } + + /** + * Read per-TopicPartition offsets from json string + */ + def partitionOffsets(str: String): Map[TopicPartition, Long] = { + try { + Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partOffsets) => + partOffsets.map { case (part, offset) => + new TopicPartition(topic, part) -> offset + } + }.toMap + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}, got $str""") + } + } + + /** + * Write per-TopicPartition offsets as json string + */ + def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = { + val result = new HashMap[String, HashMap[Int, Long]]() + partitionOffsets.foreach { case (tp, off) => + val parts = result.getOrElse(tp.topic, new HashMap[Int, Long]) + parts += tp.partition -> off + result += tp.topic -> parts + } + Serialization.write(result) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 4b0bb0a0f725c..537b7b0baa1b1 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -22,7 +22,7 @@ import java.{util => ju} import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer} +import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer, OffsetOutOfRangeException} import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition @@ -82,7 +82,7 @@ private[kafka010] case class KafkaSource( executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, - startFromEarliestOffset: Boolean, + startingOffsets: StartingOffsets, failOnDataLoss: Boolean) extends Source with Logging { @@ -110,10 +110,10 @@ private[kafka010] case class KafkaSource( private lazy val initialPartitionOffsets = { val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) metadataLog.get(0).getOrElse { - val offsets = if (startFromEarliestOffset) { - KafkaSourceOffset(fetchEarliestOffsets()) - } else { - KafkaSourceOffset(fetchLatestOffsets()) + val offsets = startingOffsets match { + case EarliestOffsets => KafkaSourceOffset(fetchEarliestOffsets()) + case LatestOffsets => KafkaSourceOffset(fetchLatestOffsets()) + case SpecificOffsets(p) => KafkaSourceOffset(fetchSpecificStartingOffsets(p)) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") @@ -231,6 +231,43 @@ private[kafka010] case class KafkaSource( override def toString(): String = s"KafkaSource[$consumerStrategy]" + /** + * Set consumer position to specified offsets, making sure all assignments are set. + */ + private def fetchSpecificStartingOffsets( + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + val result = withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + assert(partitions.asScala == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + + partitionOffsets.foreach { + case (tp, -1) => consumer.seekToEnd(ju.Arrays.asList(tp)) + case (tp, -2) => consumer.seekToBeginning(ju.Arrays.asList(tp)) + case (tp, off) => consumer.seek(tp, off) + } + partitionOffsets.map { + case (tp, _) => tp -> consumer.position(tp) + } + } + partitionOffsets.foreach { + case (tp, off) if off != -1 && off != -2 => + if (result(tp) != off) { + reportDataLoss( + s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + result + } + /** * Fetch the earliest offsets of partitions. */ @@ -273,7 +310,7 @@ private[kafka010] case class KafkaSource( consumer.poll(0) val partitions = consumer.assignment() consumer.pause(partitions) - logDebug(s"\tPartitioned assigned to consumer: $partitions") + logDebug(s"\tPartitions assigned to consumer: $partitions") // Get the earliest offset of each partition consumer.seekToBeginning(partitions) @@ -317,6 +354,8 @@ private[kafka010] case class KafkaSource( try { result = Some(body) } catch { + case x: OffsetOutOfRangeException => + reportDataLoss(x.getMessage) case NonFatal(e) => lastException = e logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) @@ -373,6 +412,17 @@ private[kafka010] object KafkaSource { def createConsumer(): Consumer[Array[Byte], Array[Byte]] } + case class AssignStrategy(partitions: Array[TopicPartition], kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.assign(ju.Arrays.asList(partitions: _*)) + consumer + } + + override def toString: String = s"Assign[${partitions.mkString(", ")}]" + } + case class SubscribeStrategy(topics: Seq[String], kafkaParams: ju.Map[String, Object]) extends ConsumerStrategy { override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 23b1b60f3bcaa..585ced875caa7 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -77,14 +77,12 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // id. Hence, we should generate a unique id for each query. val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" - val startFromEarliestOffset = - caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => false - case Some("earliest") => true - case Some(pos) => - // This should not happen since we have already checked the options. - throw new IllegalStateException(s"Invalid $STARTING_OFFSET_OPTION_KEY: $pos") - case None => false + val startingOffsets = + caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => LatestOffsets + case Some("earliest") => EarliestOffsets + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => LatestOffsets } val kafkaParamsForStrategy = @@ -95,9 +93,9 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // So that consumers in Kafka source do not mess with any existing group id .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver") - // Set to "latest" to avoid exceptions. However, KafkaSource will fetch the initial offsets - // by itself instead of counting on KafkaConsumer. - .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") + // Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial + // offsets by itself instead of counting on KafkaConsumer. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") // So that consumers in the driver does not commit offsets unnecessarily .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -130,6 +128,10 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider .build() val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + AssignStrategy( + JsonUtils.partitions(value), + kafkaParamsForStrategy) case ("subscribe", value) => SubscribeStrategy( value.split(",").map(_.trim()).filter(_.nonEmpty), @@ -153,7 +155,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider kafkaParamsForExecutors, parameters, metadataPath, - startFromEarliestOffset, + startingOffsets, failOnDataLoss) } @@ -175,6 +177,13 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider } val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + if (!value.trim.startsWith("{")) { + throw new IllegalArgumentException( + "No topicpartitions to assign as specified value for option " + + s"'assign' is '$value'") + } + case ("subscribe", value) => val topics = value.split(",").map(_.trim).filter(_.nonEmpty) if (topics.isEmpty) { @@ -195,14 +204,6 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider throw new IllegalArgumentException("Unknown option") } - caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { - case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) => - throw new IllegalArgumentException( - s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " + - s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}") - case _ => - } - // Validate user-specified Kafka options if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { @@ -215,11 +216,11 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider throw new IllegalArgumentException( s""" |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. - |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to - |specify where to start. Structured Streaming manages which offsets are consumed + |Instead set the source option '$STARTING_OFFSETS_OPTION_KEY' to 'earliest' or 'latest' + |to specify where to start. Structured Streaming manages which offsets are consumed |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no |data is missed when when new topics/partitions are dynamically subscribed. Note that - |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and + |'$STARTING_OFFSETS_OPTION_KEY' only applies when a new Streaming query is started, and |that resuming will always pick up from where the query left off. See the docs for more |details. """.stripMargin) @@ -282,8 +283,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider } private[kafka010] object KafkaSourceProvider { - private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") - private val STARTING_OFFSET_OPTION_KEY = "startingoffset" - private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") + private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala new file mode 100644 index 0000000000000..83959e597171a --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala @@ -0,0 +1,32 @@ +/* + * 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.kafka010 + +import org.apache.kafka.common.TopicPartition + +/* + * Values that can be specified for config startingOffsets + */ +private[kafka010] sealed trait StartingOffsets + +private[kafka010] case object EarliestOffsets extends StartingOffsets + +private[kafka010] case object LatestOffsets extends StartingOffsets + +private[kafka010] case class SpecificOffsets( + partitionOffsets: Map[TopicPartition, Long]) extends StartingOffsets diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala new file mode 100644 index 0000000000000..54b980049d1a2 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkFunSuite + +class JsonUtilsSuite extends SparkFunSuite { + + test("parsing partitions") { + val parsed = JsonUtils.partitions("""{"topicA":[0,1],"topicB":[4,6]}""") + val expected = Array( + new TopicPartition("topicA", 0), + new TopicPartition("topicA", 1), + new TopicPartition("topicB", 4), + new TopicPartition("topicB", 6) + ) + assert(parsed.toSeq === expected.toSeq) + } + + test("parsing partitionOffsets") { + val parsed = JsonUtils.partitionOffsets( + """{"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}""") + + assert(parsed(new TopicPartition("topicA", 0)) === 23) + assert(parsed(new TopicPartition("topicA", 1)) === -1) + assert(parsed(new TopicPartition("topicB", 0)) === -2) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 8b5296ea135c7..b50688ecb7743 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.common.TopicPartition import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ @@ -52,7 +53,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { protected def makeSureGetOffsetCalled = AssertOnQuery { q => // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure // its "getOffset" is called before pushing any data. Otherwise, because of the race contion, - // we don't know which data should be fetched when `startingOffset` is latest. + // we don't know which data should be fetched when `startingOffsets` is latest. q.processAllAvailable() true } @@ -155,26 +156,52 @@ class KafkaSourceSuite extends KafkaSourceTest { ) } + test("assign from latest offsets") { + val topic = newTopic() + testFromLatestOffsets(topic, false, "assign" -> assignString(topic, 0 to 4)) + } + + test("assign from earliest offsets") { + val topic = newTopic() + testFromEarliestOffsets(topic, false, "assign" -> assignString(topic, 0 to 4)) + } + + test("assign from specific offsets") { + val topic = newTopic() + testFromSpecificOffsets(topic, "assign" -> assignString(topic, 0 to 4)) + } + test("subscribing topic by name from latest offsets") { val topic = newTopic() - testFromLatestOffsets(topic, "subscribe" -> topic) + testFromLatestOffsets(topic, true, "subscribe" -> topic) } test("subscribing topic by name from earliest offsets") { val topic = newTopic() - testFromEarliestOffsets(topic, "subscribe" -> topic) + testFromEarliestOffsets(topic, true, "subscribe" -> topic) + } + + test("subscribing topic by name from specific offsets") { + val topic = newTopic() + testFromSpecificOffsets(topic, "subscribe" -> topic) } test("subscribing topic by pattern from latest offsets") { val topicPrefix = newTopic() val topic = topicPrefix + "-suffix" - testFromLatestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + testFromLatestOffsets(topic, true, "subscribePattern" -> s"$topicPrefix-.*") } test("subscribing topic by pattern from earliest offsets") { val topicPrefix = newTopic() val topic = topicPrefix + "-suffix" - testFromEarliestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + testFromEarliestOffsets(topic, true, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("subscribing topic by pattern from specific offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromSpecificOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") } test("subscribing topic by pattern with topic deletions") { @@ -233,6 +260,10 @@ class KafkaSourceSuite extends KafkaSourceTest { testBadOptions("subscribe" -> "t", "subscribePattern" -> "t.*")( "only one", "options can be specified") + testBadOptions("subscribe" -> "t", "assign" -> """{"a":[0]}""")( + "only one", "options can be specified") + + testBadOptions("assign" -> "")("no topicpartitions to assign") testBadOptions("subscribe" -> "")("no topics to subscribe") testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") } @@ -293,7 +324,61 @@ class KafkaSourceSuite extends KafkaSourceTest { private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { + private def assignString(topic: String, partitions: Iterable[Int]): String = { + JsonUtils.partitions(partitions.map(p => new TopicPartition(topic, p))) + } + + private def testFromSpecificOffsets(topic: String, options: (String, String)*): Unit = { + val partitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 0L, + new TopicPartition(topic, 3) -> 1L, + new TopicPartition(topic, 4) -> 2L + ) + val startingOffsets = JsonUtils.partitionOffsets(partitionOffsets) + + testUtils.createTopic(topic, partitions = 5) + // part 0 starts at earliest, these should all be seen + testUtils.sendMessages(topic, Array(-20, -21, -22).map(_.toString), Some(0)) + // part 1 starts at latest, these should all be skipped + testUtils.sendMessages(topic, Array(-10, -11, -12).map(_.toString), Some(1)) + // part 2 starts at 0, these should all be seen + testUtils.sendMessages(topic, Array(0, 1, 2).map(_.toString), Some(2)) + // part 3 starts at 1, first should be skipped + testUtils.sendMessages(topic, Array(10, 11, 12).map(_.toString), Some(3)) + // part 4 starts at 2, first and second should be skipped + testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffsets", startingOffsets) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + testStream(mapped)( + makeSureGetOffsetCalled, + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), + StopStream, + StartStream(), + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), // Should get the data back on recovery + AddKafkaData(Set(topic), 30, 31, 32, 33, 34)(ensureDataInMultiplePartition = true), + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22, 30, 31, 32, 33, 34), + StopStream + ) + } + + private def testFromLatestOffsets( + topic: String, + addPartitions: Boolean, + options: (String, String)*): Unit = { testUtils.createTopic(topic, partitions = 5) testUtils.sendMessages(topic, Array("-1")) require(testUtils.getLatestOffsets(Set(topic)).size === 5) @@ -301,7 +386,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark .readStream .format("kafka") - .option("startingOffset", s"latest") + .option("startingOffsets", s"latest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } @@ -324,7 +409,9 @@ class KafkaSourceSuite extends KafkaSourceTest { AddKafkaData(Set(topic), 7, 8), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => - testUtils.addPartitions(topic, 10) + if (addPartitions) { + testUtils.addPartitions(topic, 10) + } true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), @@ -332,7 +419,10 @@ class KafkaSourceSuite extends KafkaSourceTest { ) } - private def testFromEarliestOffsets(topic: String, options: (String, String)*): Unit = { + private def testFromEarliestOffsets( + topic: String, + addPartitions: Boolean, + options: (String, String)*): Unit = { testUtils.createTopic(topic, partitions = 5) testUtils.sendMessages(topic, (1 to 3).map { _.toString }.toArray) require(testUtils.getLatestOffsets(Set(topic)).size === 5) @@ -340,7 +430,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark.readStream reader .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) - .option("startingOffset", s"earliest") + .option("startingOffsets", s"earliest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } @@ -360,7 +450,9 @@ class KafkaSourceSuite extends KafkaSourceTest { StartStream(), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => - testUtils.addPartitions(topic, 10) + if (addPartitions) { + testUtils.addPartitions(topic, 10) + } true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 3eb8a737ba4c8..9b24ccdd560e8 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -201,11 +201,23 @@ class KafkaTestUtils extends Logging { /** Send the array of messages to the Kafka broker */ def sendMessages(topic: String, messages: Array[String]): Seq[(String, RecordMetadata)] = { + sendMessages(topic, messages, None) + } + + /** Send the array of messages to the Kafka broker using specified partition */ + def sendMessages( + topic: String, + messages: Array[String], + partition: Option[Int]): Seq[(String, RecordMetadata)] = { producer = new KafkaProducer[String, String](producerConfiguration) val offsets = try { messages.map { m => + val record = partition match { + case Some(p) => new ProducerRecord[String, String](topic, p, null, m) + case None => new ProducerRecord[String, String](topic, m) + } val metadata = - producer.send(new ProducerRecord[String, String](topic, m)).get(10, TimeUnit.SECONDS) + producer.send(record).get(10, TimeUnit.SECONDS) logInfo(s"\tSent $m to partition ${metadata.partition}, offset ${metadata.offset}") (m, metadata) } From c9720b2195a465653690b3e221ce789142217b0d Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 21 Oct 2016 16:27:19 -0700 Subject: [PATCH 161/177] [STREAMING][KAFKA][DOC] clarify kafka settings needed for larger batches ## What changes were proposed in this pull request? Minor doc change to mention kafka configuration for larger spark batches. ## How was this patch tested? Doc change only, confirmed via jekyll. The configuration issue was discussed / confirmed with users on the mailing list. Author: cody koeninger Closes #15570 from koeninger/kafka-doc-heartbeat. --- docs/streaming-kafka-0-10-integration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index 456b8453383db..de95ea90137eb 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -48,6 +48,7 @@ Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javad For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). +If your Spark batch duration is larger than the default Kafka heartbeat session timeout (30 seconds), increase heartbeat.interval.ms and session.timeout.ms appropriately. For batches larger than 5 minutes, this will require changing group.max.session.timeout.ms on the broker. Note that the example sets enable.auto.commit to false, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. ### LocationStrategies From 3fbf5a58c236fc5d5fee39cb29e7f5c7e01c0ee7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 21 Oct 2016 17:27:18 -0700 Subject: [PATCH 162/177] [SPARK-18042][SQL] OutputWriter should expose file path written ## What changes were proposed in this pull request? This patch adds a new "path" method on OutputWriter that returns the path of the file written by the OutputWriter. This is part of the necessary work to consolidate structured streaming and batch write paths. The batch write path has a nice feature that each data source can define the extension of the files, and allow Spark to specify the staging directory and the prefix for the files. However, in the streaming path we need to collect the list of files written, and there is no interface right now to do that. ## How was this patch tested? N/A - there is no behavior change and this should be covered by existing tests. Author: Reynold Xin Closes #15580 from rxin/SPARK-18042. --- .../ml/source/libsvm/LibSVMRelation.scala | 8 ++++- .../execution/datasources/OutputWriter.scala | 17 ++++++----- .../datasources/csv/CSVRelation.scala | 8 ++++- .../datasources/json/JsonFileFormat.scala | 8 ++++- .../parquet/ParquetFileFormat.scala | 2 +- .../datasources/parquet/ParquetOptions.scala | 2 +- .../parquet/ParquetOutputWriter.scala | 24 ++++++++------- .../datasources/text/TextFileFormat.scala | 25 ++++++++++++++-- .../spark/sql/hive/orc/OrcFileFormat.scala | 29 +++++++++---------- .../sql/sources/CommitFailureTestSource.scala | 3 ++ .../sql/sources/SimpleTextRelation.scala | 3 ++ 11 files changed, 90 insertions(+), 39 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index fff86686b550c..5e9e6ff1a5690 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.text.TextOutputWriter import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration @@ -46,12 +47,17 @@ private[libsvm] class LibSVMOutputWriter( context: TaskAttemptContext) extends OutputWriter { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".libsvm" + compressionExtension).toString + } + private[this] val buffer = new Text() private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, fileNamePrefix + extension) + new Path(path) } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala index f4cefdab077e9..fbf6e96d3f850 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala @@ -42,11 +42,12 @@ abstract class OutputWriterFactory extends Serializable { * @param fileNamePrefix Prefix of the file name. The returned OutputWriter must make sure this * prefix is used in the actual file name. For example, if the prefix is * "part-1-2-3", then the file name must start with "part_1_2_3" but can - * end in arbitrary extension. + * end in arbitrary extension that is deterministic given the configuration + * (i.e. the suffix extension should not depend on any task id, attempt id, + * or partition id). * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the relation being written is partitioned. * @param context The Hadoop MapReduce task context. - * @since 1.4.0 */ def newInstance( stagingDir: String, @@ -62,7 +63,6 @@ abstract class OutputWriterFactory extends Serializable { * and not modify it (do not add subdirectories, extensions, etc.). All other * file-format-specific information needed to create the writer must be passed * through the [[OutputWriterFactory]] implementation. - * @since 2.0.0 */ def newWriter(path: String): OutputWriter = { throw new UnsupportedOperationException("newInstance with just path not supported") @@ -77,19 +77,22 @@ abstract class OutputWriterFactory extends Serializable { * executor side. This instance is used to persist rows to this single output file. */ abstract class OutputWriter { + + /** + * The path of the file to be written out. This path should include the staging directory and + * the file name prefix passed into the associated createOutputWriter function. + */ + def path: String + /** * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned * tables, dynamic partition columns are not included in rows to be written. - * - * @since 1.4.0 */ def write(row: Row): Unit /** * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before * the task output is committed. - * - * @since 1.4.0 */ def close(): Unit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index eefacbf05ba0d..a35cfdb2c234f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} +import org.apache.spark.sql.execution.datasources.text.TextOutputWriter import org.apache.spark.sql.types._ object CSVRelation extends Logging { @@ -185,6 +186,11 @@ private[csv] class CsvOutputWriter( context: TaskAttemptContext, params: CSVOptions) extends OutputWriter with Logging { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".csv" + compressionExtension).toString + } + // create the Generator without separator inserted between 2 records private[this] val text = new Text() @@ -199,7 +205,7 @@ private[csv] class CsvOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, s"$fileNamePrefix.csv$extension") + new Path(path) } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index cdbb2f7292613..651fa78a4e924 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptions} import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.text.TextOutputWriter import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -160,6 +161,11 @@ private[json] class JsonOutputWriter( context: TaskAttemptContext) extends OutputWriter with Logging { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".json" + compressionExtension).toString + } + private[this] val writer = new CharArrayWriter() // create the Generator without separator inserted between 2 records private[this] val gen = new JacksonGenerator(dataSchema, writer, options) @@ -168,7 +174,7 @@ private[json] class JsonOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, s"$fileNamePrefix.json$extension") + new Path(path) } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 87b944ba523ca..502dd0e8d4cf9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -121,7 +121,7 @@ class ParquetFileFormat sparkSession.sessionState.conf.writeLegacyParquetFormat.toString) // Sets compression scheme - conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) // SPARK-15719: Disables writing Parquet summary files by default. if (conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 615731889dfad..d0fd23605bea8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -35,7 +35,7 @@ private[parquet] class ParquetOptions( * Compression codec to use. By default use the value specified in SQLConf. * Acceptable values are defined in [[shortParquetCompressionCodecNames]]. */ - val compressionCodec: String = { + val compressionCodecClassName: String = { val codecName = parameters.getOrElse("compression", sqlConf.parquetCompressionCodec).toLowerCase if (!shortParquetCompressionCodecNames.contains(codecName)) { val availableCodecs = shortParquetCompressionCodecNames.keys.map(_.toLowerCase) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala index 39c199784cd6d..1300069c42b05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.parquet.hadoop.{ParquetOutputFormat, ParquetRecordWriter} +import org.apache.parquet.hadoop.codec.CodecConfig import org.apache.parquet.hadoop.util.ContextUtil import org.apache.spark.sql.Row @@ -80,7 +81,7 @@ private[parquet] class ParquetOutputWriterFactory( sqlConf.writeLegacyParquetFormat.toString) // Sets compression scheme - conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) new SerializableConfiguration(conf) } @@ -88,7 +89,7 @@ private[parquet] class ParquetOutputWriterFactory( * Returns a [[OutputWriter]] that writes data to the give path without using * [[OutputCommitter]]. */ - override def newWriter(path: String): OutputWriter = new OutputWriter { + override def newWriter(path1: String): OutputWriter = new OutputWriter { // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter private val hadoopTaskAttemptId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) @@ -98,6 +99,8 @@ private[parquet] class ParquetOutputWriterFactory( // Instance of ParquetRecordWriter that does not use OutputCommitter private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) + override def path: String = path1 + override def write(row: Row): Unit = { throw new UnsupportedOperationException("call writeInternal") } @@ -140,16 +143,17 @@ private[parquet] class ParquetOutputWriter( context: TaskAttemptContext) extends OutputWriter { + override val path: String = { + val filename = fileNamePrefix + CodecConfig.from(context).getCodec.getExtension + ".parquet" + new Path(stagingDir, filename).toString + } + private val recordWriter: RecordWriter[Void, InternalRow] = { - val outputFormat = { - new ParquetOutputFormat[InternalRow]() { - override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, fileNamePrefix + extension) - } + new ParquetOutputFormat[InternalRow]() { + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + new Path(path) } - } - - outputFormat.getRecordWriter(context) + }.getRecordWriter(context) } override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index 6cd2351c5749a..d40b5725199a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -20,8 +20,10 @@ package org.apache.spark.sql.execution.datasources.text import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.{NullWritable, Text} +import org.apache.hadoop.io.compress.GzipCodec import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} +import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.TaskContext import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -128,12 +130,17 @@ class TextOutputWriter( context: TaskAttemptContext) extends OutputWriter { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".txt" + compressionExtension).toString + } + private[this] val buffer = new Text() private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, s"$fileNamePrefix.txt$extension") + new Path(path) } }.getRecordWriter(context) } @@ -150,3 +157,17 @@ class TextOutputWriter( recordWriter.close(context) } } + + +object TextOutputWriter { + /** Returns the compression codec extension to be used in a file name, e.g. ".gzip"). */ + def getCompressionExtension(context: TaskAttemptContext): String = { + // Set the compression extension, similar to code in TextOutputFormat.getDefaultWorkFile + if (FileOutputFormat.getCompressOutput(context)) { + val codecClass = FileOutputFormat.getOutputCompressorClass(context, classOf[GzipCodec]) + ReflectionUtils.newInstance(codecClass, context.getConfiguration).getDefaultExtension + } else { + "" + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 1ceacb458ae6e..eba7aa386ade2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -216,9 +216,18 @@ private[orc] class OrcOutputWriter( context: TaskAttemptContext) extends OutputWriter { - private[this] val conf = context.getConfiguration + override val path: String = { + val compressionExtension: String = { + val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION) + OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") + } + // It has the `.orc` extension at the end because (de)compression tools + // such as gunzip would not be able to decompress this as the compression + // is not applied on this whole file but on each "stream" in ORC format. + new Path(stagingDir, fileNamePrefix + compressionExtension + ".orc").toString + } - private[this] val serializer = new OrcSerializer(dataSchema, conf) + private[this] val serializer = new OrcSerializer(dataSchema, context.getConfiguration) // `OrcRecordWriter.close()` creates an empty file if no rows are written at all. We use this // flag to decide whether `OrcRecordWriter.close()` needs to be called. @@ -226,20 +235,10 @@ private[orc] class OrcOutputWriter( private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - - val compressionExtension = { - val name = conf.get(OrcRelation.ORC_COMPRESSION) - OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") - } - // It has the `.orc` extension at the end because (de)compression tools - // such as gunzip would not be able to decompress this as the compression - // is not applied on this whole file but on each "stream" in ORC format. - val filename = s"$fileNamePrefix$compressionExtension.orc" - new OrcOutputFormat().getRecordWriter( - new Path(stagingDir, filename).getFileSystem(conf), - conf.asInstanceOf[JobConf], - new Path(stagingDir, filename).toString, + new Path(path).getFileSystem(context.getConfiguration), + context.getConfiguration.asInstanceOf[JobConf], + path, Reporter.NULL ).asInstanceOf[RecordWriter[NullWritable, Writable]] } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala index d5044684020e2..731540db17eeb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.sources +import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.TaskContext @@ -50,6 +51,8 @@ class CommitFailureTestSource extends SimpleTextSource { SimpleTextRelation.callbackCalled = true } + override val path: String = new Path(stagingDir, fileNamePrefix).toString + override def write(row: Row): Unit = { if (SimpleTextRelation.failWriter) { sys.error("Intentional task writer failure for testing purpose.") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 9e13b217ec305..9896b9bde99c8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -123,6 +123,9 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { class SimpleTextOutputWriter( stagingDir: String, fileNamePrefix: String, context: TaskAttemptContext) extends OutputWriter { + + override val path: String = new Path(stagingDir, fileNamePrefix).toString + private val recordWriter: RecordWriter[NullWritable, Text] = new AppendingTextOutputFormat(new Path(stagingDir), fileNamePrefix).getRecordWriter(context) From 7178c56433cd138dae53db9194c55e3f4fa0fa69 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 21 Oct 2016 22:20:52 -0700 Subject: [PATCH 163/177] [SPARK-16606][MINOR] Tiny follow-up to , to correct more instances of the same log message typo ## What changes were proposed in this pull request? Tiny follow-up to SPARK-16606 / https://github.com/apache/spark/pull/14533 , to correct more instances of the same log message typo ## How was this patch tested? Existing tests (no functional change anyway) Author: Sean Owen Closes #15586 from srowen/SPARK-16606.2. --- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index baae55013787d..3045eb69f427f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -814,7 +814,7 @@ object SparkSession { if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } if (options.nonEmpty) { - logWarning("Use an existing SparkSession, some configuration may not take effect.") + logWarning("Using an existing SparkSession; some configuration may not take effect.") } return session } @@ -826,7 +826,7 @@ object SparkSession { if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } if (options.nonEmpty) { - logWarning("Use an existing SparkSession, some configuration may not take effect.") + logWarning("Using an existing SparkSession; some configuration may not take effect.") } return session } From 625fdddacd58ad54fdbb17409987812176abc812 Mon Sep 17 00:00:00 2001 From: Erik O'Shaughnessy Date: Sat, 22 Oct 2016 09:37:53 +0100 Subject: [PATCH 164/177] [SPARK-17944][DEPLOY] sbin/start-* scripts use of `hostname -f` fail with Solaris ## What changes were proposed in this pull request? Modify sbin/start-master.sh, sbin/start-mesos-dispatcher.sh and sbin/start-slaves.sh to use the output of 'uname' to select which OS-specific command-line is used to determine the host's fully qualified host name. ## How was this patch tested? Tested by hand; starting on Solaris, Linux and macOS. Author: Erik O'Shaughnessy Closes #15557 from JnyJny/SPARK-17944. --- sbin/start-master.sh | 9 ++++++++- sbin/start-mesos-dispatcher.sh | 9 ++++++++- sbin/start-slaves.sh | 9 ++++++++- 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/sbin/start-master.sh b/sbin/start-master.sh index d970fcc45e2c1..97ee32159b6de 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -48,7 +48,14 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST=`hostname -f` + case `uname` in + (SunOS) + SPARK_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + SPARK_MASTER_HOST="`hostname -f`" + ;; + esac fi if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh index ef65fb9539146..ecaad7ad09634 100755 --- a/sbin/start-mesos-dispatcher.sh +++ b/sbin/start-mesos-dispatcher.sh @@ -34,7 +34,14 @@ if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then fi if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then - SPARK_MESOS_DISPATCHER_HOST=`hostname -f` + case `uname` in + (SunOS) + SPARK_MESOS_DISPATCHER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + SPARK_MESOS_DISPATCHER_HOST="`hostname -f`" + ;; + esac fi if [ "$SPARK_MESOS_DISPATCHER_NUM" = "" ]; then diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 7d8871251f81b..f5269df523dac 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -32,7 +32,14 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST="`hostname -f`" + case `uname` in + (SunOS) + SPARK_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + SPARK_MASTER_HOST="`hostname -f`" + ;; + esac fi # Launch the slaves From 01b26a06436b4c8020f22be3e1da4995b44c9b03 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 22 Oct 2016 09:39:07 +0100 Subject: [PATCH 165/177] [SPARK-17898][DOCS] repositories needs username and password ## What changes were proposed in this pull request? Document `user:password` syntax as possible means of specifying credentials for password-protected `--repositories` ## How was this patch tested? Doc build Author: Sean Owen Closes #15584 from srowen/SPARK-17898. --- docs/programming-guide.md | 8 ++++---- docs/submitting-applications.md | 2 ++ 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 20b4bee0f58e1..7516579ec6dbf 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -182,7 +182,7 @@ variable called `sc`. Making your own SparkContext will not work. You can set wh context connects to using the `--master` argument, and you can add JARs to the classpath by passing a comma-separated list to the `--jars` argument. You can also add dependencies (e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates -to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType) +to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. Sonatype) can be passed to the `--repositories` argument. For example, to run `bin/spark-shell` on exactly four cores, use: @@ -214,9 +214,9 @@ variable called `sc`. Making your own SparkContext will not work. You can set wh context connects to using the `--master` argument, and you can add Python .zip, .egg or .py files to the runtime path by passing a comma-separated list to `--py-files`. You can also add dependencies (e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates -to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType) -can be passed to the `--repositories` argument. Any python dependencies a Spark Package has (listed in -the requirements.txt of that package) must be manually installed using pip when necessary. +to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. Sonatype) +can be passed to the `--repositories` argument. Any Python dependencies a Spark package has (listed in +the requirements.txt of that package) must be manually installed using `pip` when necessary. For example, to run `bin/pyspark` on exactly four cores, use: {% highlight bash %} diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 6fe3049995876..b738194eac9aa 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -190,6 +190,8 @@ is handled automatically, and with Spark standalone, automatic cleanup can be co Users may also include any other dependencies by supplying a comma-delimited list of maven coordinates with `--packages`. All transitive dependencies will be handled when using this command. Additional repositories (or resolvers in SBT) can be added in a comma-delimited fashion with the flag `--repositories`. +(Note that credentials for password-protected repositories can be supplied in some cases in the repository URI, +such as in `https://user:password@host/...`. Be careful when supplying credentials this way.) These commands can be used with `pyspark`, `spark-shell`, and `spark-submit` to include Spark Packages. For Python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries From ab3363e9f6b1f7fc26682509fe7382c570f91778 Mon Sep 17 00:00:00 2001 From: Drew Robb Date: Sat, 22 Oct 2016 01:59:36 -0700 Subject: [PATCH 166/177] [SPARK-17986][ML] SQLTransformer should remove temporary tables ## What changes were proposed in this pull request? A call to the method `SQLTransformer.transform` previously would create a temporary table and never delete it. This change adds a call to `dropTempView()` that deletes this temporary table before returning the result so that the table will not remain in spark's table catalog. Because `tableName` is randomized and not exposed, there should be no expected use of this table outside of the `transform` method. ## How was this patch tested? A single new assertion was added to the existing test of the `SQLTransformer.transform` method that all temporary tables are removed. Without the corresponding code change, this new assertion fails. I am not aware of any circumstances in which removing this temporary view would be bad for performance or correctness in other ways, but some expertise here would be helpful. Author: Drew Robb Closes #15526 from drewrobb/SPARK-17986. --- .../scala/org/apache/spark/ml/feature/SQLTransformer.scala | 4 +++- .../org/apache/spark/ml/feature/SQLTransformerSuite.scala | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 259be2679ce19..b25fff973c441 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -67,7 +67,9 @@ class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) val tableName = Identifiable.randomUID(uid) dataset.createOrReplaceTempView(tableName) val realStatement = $(statement).replace(tableIdentifier, tableName) - dataset.sparkSession.sql(realStatement) + val result = dataset.sparkSession.sql(realStatement) + dataset.sparkSession.catalog.dropTempView(tableName) + result } @Since("1.6.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala index 23464073e6edb..753f890c48301 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala @@ -43,6 +43,7 @@ class SQLTransformerSuite assert(result.schema.toString == resultSchema.toString) assert(resultSchema == expected.schema) assert(result.collect().toSeq == expected.collect().toSeq) + assert(original.sparkSession.catalog.listTables().count() == 0) } test("read/write") { From 3eca283aca68ac81c127d60ad5699f854d5f14b7 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sat, 22 Oct 2016 22:08:28 +0800 Subject: [PATCH 167/177] [SPARK-17994][SQL] Add back a file status cache for catalog tables ## What changes were proposed in this pull request? In SPARK-16980, we removed the full in-memory cache of table partitions in favor of loading only needed partitions from the metastore. This greatly improves the initial latency of queries that only read a small fraction of table partitions. However, since the metastore does not store file statistics, we need to discover those from remote storage. With the loss of the in-memory file status cache this has to happen on each query, increasing the latency of repeated queries over the same partitions. The proposal is to add back a per-table cache of partition contents, i.e. Map[Path, Array[FileStatus]]. This cache would be retained per-table, and can be invalidated through refreshTable() and refreshByPath(). Unlike the prior cache, it can be incrementally updated as new partitions are read. ## How was this patch tested? Existing tests and new tests in `HiveTablePerfStatsSuite`. cc mallman Author: Eric Liang Author: Michael Allman Author: Eric Liang Closes #15539 from ericl/meta-cache. --- .../spark/metrics/source/StaticSources.scala | 7 + .../datasources/FileStatusCache.scala | 149 ++++++++++++++++++ .../datasources/ListingFileCatalog.scala | 13 +- .../PartitioningAwareFileCatalog.scala | 115 ++++++++------ .../datasources/TableFileCatalog.scala | 36 ++--- .../apache/spark/sql/internal/SQLConf.scala | 16 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 16 +- ...te.scala => HiveTablePerfStatsSuite.scala} | 127 +++++++++++++-- 9 files changed, 385 insertions(+), 96 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala rename sql/hive/src/test/scala/org/apache/spark/sql/hive/{HiveDataFrameSuite.scala => HiveTablePerfStatsSuite.scala} (50%) diff --git a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala index cf92a10deabd5..b54885b7ff8b0 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala @@ -80,15 +80,22 @@ object HiveCatalogMetrics extends Source { */ val METRIC_FILES_DISCOVERED = metricRegistry.counter(MetricRegistry.name("filesDiscovered")) + /** + * Tracks the total number of files served from the file status cache instead of discovered. + */ + val METRIC_FILE_CACHE_HITS = metricRegistry.counter(MetricRegistry.name("fileCacheHits")) + /** * Resets the values of all metrics to zero. This is useful in tests. */ def reset(): Unit = { METRIC_PARTITIONS_FETCHED.dec(METRIC_PARTITIONS_FETCHED.getCount()) METRIC_FILES_DISCOVERED.dec(METRIC_FILES_DISCOVERED.getCount()) + METRIC_FILE_CACHE_HITS.dec(METRIC_FILE_CACHE_HITS.getCount()) } // clients can use these to avoid classloader issues with the codahale classes def incrementFetchedPartitions(n: Int): Unit = METRIC_PARTITIONS_FETCHED.inc(n) def incrementFilesDiscovered(n: Int): Unit = METRIC_FILES_DISCOVERED.inc(n) + def incrementFileCacheHits(n: Int): Unit = METRIC_FILE_CACHE_HITS.inc(n) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala new file mode 100644 index 0000000000000..e0ec748a0b34d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +import com.google.common.cache._ +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.{SerializableConfiguration, SizeEstimator} + +/** + * A cache of the leaf files of partition directories. We cache these files in order to speed + * up iterated queries over the same set of partitions. Otherwise, each query would have to + * hit remote storage in order to gather file statistics for physical planning. + * + * Each resolved catalog table has its own FileStatusCache. When the backing relation for the + * table is refreshed via refreshTable() or refreshByPath(), this cache will be invalidated. + */ +abstract class FileStatusCache { + /** + * @return the leaf files for the specified path from this cache, or None if not cached. + */ + def getLeafFiles(path: Path): Option[Array[FileStatus]] = None + + /** + * Saves the given set of leaf files for a path in this cache. + */ + def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit + + /** + * Invalidates all data held by this cache. + */ + def invalidateAll(): Unit +} + +object FileStatusCache { + private var sharedCache: SharedInMemoryCache = null + + /** + * @return a new FileStatusCache based on session configuration. Cache memory quota is + * shared across all clients. + */ + def newCache(session: SparkSession): FileStatusCache = { + synchronized { + if (session.sqlContext.conf.filesourcePartitionPruning && + session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) { + if (sharedCache == null) { + sharedCache = new SharedInMemoryCache( + session.sqlContext.conf.filesourcePartitionFileCacheSize) + } + sharedCache.getForNewClient() + } else { + NoopCache + } + } + } + + def resetForTesting(): Unit = synchronized { + sharedCache = null + } +} + +/** + * An implementation that caches partition file statuses in memory. + * + * @param maxSizeInBytes max allowable cache size before entries start getting evicted + */ +private class SharedInMemoryCache(maxSizeInBytes: Long) extends Logging { + import FileStatusCache._ + + // Opaque object that uniquely identifies a shared cache user + private type ClientId = Object + + private val warnedAboutEviction = new AtomicBoolean(false) + + // we use a composite cache key in order to distinguish entries inserted by different clients + private val cache: Cache[(ClientId, Path), Array[FileStatus]] = CacheBuilder.newBuilder() + .weigher(new Weigher[(ClientId, Path), Array[FileStatus]] { + override def weigh(key: (ClientId, Path), value: Array[FileStatus]): Int = { + (SizeEstimator.estimate(key) + SizeEstimator.estimate(value)).toInt + }}) + .removalListener(new RemovalListener[(ClientId, Path), Array[FileStatus]]() { + override def onRemoval(removed: RemovalNotification[(ClientId, Path), Array[FileStatus]]) = { + if (removed.getCause() == RemovalCause.SIZE && + warnedAboutEviction.compareAndSet(false, true)) { + logWarning( + "Evicting cached table partition metadata from memory due to size constraints " + + "(spark.sql.hive.filesourcePartitionFileCacheSize = " + maxSizeInBytes + " bytes). " + + "This may impact query planning performance.") + } + }}) + .maximumWeight(maxSizeInBytes) + .build() + + /** + * @return a FileStatusCache that does not share any entries with any other client, but does + * share memory resources for the purpose of cache eviction. + */ + def getForNewClient(): FileStatusCache = new FileStatusCache { + val clientId = new Object() + + override def getLeafFiles(path: Path): Option[Array[FileStatus]] = { + Option(cache.getIfPresent((clientId, path))) + } + + override def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit = { + cache.put((clientId, path), leafFiles.toArray) + } + + override def invalidateAll(): Unit = { + cache.asMap.asScala.foreach { case (key, value) => + if (key._1 == clientId) { + cache.invalidate(key) + } + } + } + } +} + +/** + * A non-caching implementation used when partition file status caching is disabled. + */ +object NoopCache extends FileStatusCache { + override def getLeafFiles(path: Path): Option[Array[FileStatus]] = None + override def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit = {} + override def invalidateAll(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index 6d10501b7265d..d9d588388aaf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -38,14 +38,16 @@ class ListingFileCatalog( sparkSession: SparkSession, override val rootPaths: Seq[Path], parameters: Map[String, String], - partitionSchema: Option[StructType]) - extends PartitioningAwareFileCatalog(sparkSession, parameters, partitionSchema) { + partitionSchema: Option[StructType], + fileStatusCache: FileStatusCache = NoopCache) + extends PartitioningAwareFileCatalog( + sparkSession, parameters, partitionSchema, fileStatusCache) { @volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _ @volatile private var cachedLeafDirToChildrenFiles: Map[Path, Array[FileStatus]] = _ @volatile private var cachedPartitionSpec: PartitionSpec = _ - refresh() + refresh0() override def partitionSpec(): PartitionSpec = { if (cachedPartitionSpec == null) { @@ -64,6 +66,11 @@ class ListingFileCatalog( } override def refresh(): Unit = { + refresh0() + fileStatusCache.invalidateAll() + } + + private def refresh0(): Unit = { val files = listLeafFiles(rootPaths) cachedLeafFiles = new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index 5c8eff7ec46b4..9b1903c47119e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration - /** * An abstract class that represents [[FileCatalog]]s that are aware of partitioned tables. * It provides the necessary methods to parse partition data based on a set of files. @@ -45,7 +44,8 @@ import org.apache.spark.util.SerializableConfiguration abstract class PartitioningAwareFileCatalog( sparkSession: SparkSession, parameters: Map[String, String], - partitionSchema: Option[StructType]) extends FileCatalog with Logging { + partitionSchema: Option[StructType], + fileStatusCache: FileStatusCache = NoopCache) extends FileCatalog with Logging { import PartitioningAwareFileCatalog.BASE_PATH_PARAM /** Returns the specification of the partitions inferred from the data. */ @@ -238,15 +238,29 @@ abstract class PartitioningAwareFileCatalog( * This is publicly visible for testing. */ def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - val files = - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - PartitioningAwareFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) - } else { - PartitioningAwareFileCatalog.listLeafFilesInSerial(paths, hadoopConf) + val output = mutable.LinkedHashSet[FileStatus]() + val pathsToFetch = mutable.ArrayBuffer[Path]() + for (path <- paths) { + fileStatusCache.getLeafFiles(path) match { + case Some(files) => + HiveCatalogMetrics.incrementFileCacheHits(files.length) + output ++= files + case None => + pathsToFetch += path } - - HiveCatalogMetrics.incrementFilesDiscovered(files.size) - mutable.LinkedHashSet(files: _*) + } + val discovered = if (pathsToFetch.length >= + sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + PartitioningAwareFileCatalog.listLeafFilesInParallel(pathsToFetch, hadoopConf, sparkSession) + } else { + PartitioningAwareFileCatalog.listLeafFilesInSerial(pathsToFetch, hadoopConf) + } + discovered.foreach { case (path, leafFiles) => + HiveCatalogMetrics.incrementFilesDiscovered(leafFiles.size) + fileStatusCache.putLeafFiles(path, leafFiles.toArray) + output ++= leafFiles + } + output } } @@ -276,14 +290,14 @@ object PartitioningAwareFileCatalog extends Logging { */ private def listLeafFilesInSerial( paths: Seq[Path], - hadoopConf: Configuration): Seq[FileStatus] = { + hadoopConf: Configuration): Seq[(Path, Seq[FileStatus])] = { // Dummy jobconf to get to the pathFilter defined in configuration val jobConf = new JobConf(hadoopConf, this.getClass) val filter = FileInputFormat.getInputPathFilter(jobConf) - paths.flatMap { path => + paths.map { path => val fs = path.getFileSystem(hadoopConf) - listLeafFiles0(fs, path, filter) + (path, listLeafFiles0(fs, path, filter)) } } @@ -294,7 +308,7 @@ object PartitioningAwareFileCatalog extends Logging { private def listLeafFilesInParallel( paths: Seq[Path], hadoopConf: Configuration, - sparkSession: SparkSession): Seq[FileStatus] = { + sparkSession: SparkSession): Seq[(Path, Seq[FileStatus])] = { assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") @@ -306,47 +320,54 @@ object PartitioningAwareFileCatalog extends Logging { // in case of large #defaultParallelism. val numParallelism = Math.min(paths.size, 10000) - val statuses = sparkContext + val statusMap = sparkContext .parallelize(serializedPaths, numParallelism) .mapPartitions { paths => val hadoopConf = serializableConfiguration.value listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator - }.map { status => - // Turn FileStatus into SerializableFileStatus so we can send it back to the driver - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - SerializableBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[SerializableBlockLocation] - } + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } - SerializableFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + } + (path.toString, serializableStatuses) }.collect() - // Turn SerializableFileStatus back to Status - statuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + // turn SerializableFileStatus back to Status + statusMap.map { case (path, serializableStatuses) => + val statuses = serializableStatuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, + new Path(f.path)), + blockLocations) } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), - blockLocations) + (new Path(path), statuses) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala index fc08c3798ee06..31a01bc6db082 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.types.StructType * @param table the table's (unqualified) name * @param partitionSchema the schema of a partitioned table's partition columns * @param sizeInBytes the table's data size in bytes + * @param fileStatusCache optional cache implementation to use for file listing */ class TableFileCatalog( sparkSession: SparkSession, @@ -42,24 +43,21 @@ class TableFileCatalog( protected val hadoopConf = sparkSession.sessionState.newHadoopConf + private val fileStatusCache = FileStatusCache.newCache(sparkSession) + private val externalCatalog = sparkSession.sharedState.externalCatalog private val catalogTable = externalCatalog.getTable(db, table) private val baseLocation = catalogTable.storage.locationUri - // Populated on-demand by calls to cachedAllPartitions - private var cachedAllPartitions: ListingFileCatalog = null - override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { filterPartitions(filters).listFiles(Nil) } - override def refresh(): Unit = synchronized { - cachedAllPartitions = null - } + override def refresh(): Unit = fileStatusCache.invalidateAll() /** * Returns a [[ListingFileCatalog]] for this table restricted to the subset of partitions @@ -68,14 +66,6 @@ class TableFileCatalog( * @param filters partition-pruning filters */ def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = { - if (filters.isEmpty) { - allPartitions - } else { - filterPartitions0(filters) - } - } - - private def filterPartitions0(filters: Seq[Expression]): ListingFileCatalog = { val parameters = baseLocation .map(loc => Map(PartitioningAwareFileCatalog.BASE_PATH_PARAM -> loc)) .getOrElse(Map.empty) @@ -87,21 +77,13 @@ class TableFileCatalog( } val partitionSpec = PartitionSpec(schema, partitions) new PrunedTableFileCatalog( - sparkSession, new Path(baseLocation.get), partitionSpec) + sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec) case None => - new ListingFileCatalog(sparkSession, rootPaths, parameters, None) - } - } - - // Not used in the hot path of queries when metastore partition pruning is enabled - def allPartitions: ListingFileCatalog = synchronized { - if (cachedAllPartitions == null) { - cachedAllPartitions = filterPartitions0(Nil) + new ListingFileCatalog(sparkSession, rootPaths, parameters, None, fileStatusCache) } - cachedAllPartitions } - override def inputFiles: Array[String] = allPartitions.inputFiles + override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles } /** @@ -114,9 +96,11 @@ class TableFileCatalog( private class PrunedTableFileCatalog( sparkSession: SparkSession, tableBasePath: Path, + fileStatusCache: FileStatusCache, override val partitionSpec: PartitionSpec) extends ListingFileCatalog( sparkSession, partitionSpec.partitions.map(_.path), Map.empty, - Some(partitionSpec.partitionColumns)) + Some(partitionSpec.partitionColumns), + fileStatusCache) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index ebf4fad5cbcff..a6e2fa26cb5ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -265,17 +265,27 @@ object SQLConf { val HIVE_METASTORE_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.hive.metastorePartitionPruning") .doc("When true, some predicates will be pushed down into the Hive metastore so that " + - "unmatching partitions can be eliminated earlier.") + "unmatching partitions can be eliminated earlier. This only affects Hive tables " + + "not converted to filesource relations (see HiveUtils.CONVERT_METASTORE_PARQUET and " + + "HiveUtils.CONVERT_METASTORE_ORC for more information).") .booleanConf .createWithDefault(true) val HIVE_FILESOURCE_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.hive.filesourcePartitionPruning") - .doc("When true, enable metastore partition pruning for file source tables as well. " + + .doc("When true, enable metastore partition pruning for filesource relations as well. " + "This is currently implemented for converted Hive tables only.") .booleanConf .createWithDefault(true) + val HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE = + SQLConfigBuilder("spark.sql.hive.filesourcePartitionFileCacheSize") + .doc("When nonzero, enable caching of partition file metadata in memory. All table share " + + "a cache that can use up to specified num bytes for file metadata. This conf only " + + "applies if filesource partition pruning is also enabled.") + .longConf + .createWithDefault(250 * 1024 * 1024) + val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + @@ -670,6 +680,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def filesourcePartitionPruning: Boolean = getConf(HIVE_FILESOURCE_PARTITION_PRUNING) + def filesourcePartitionFileCacheSize: Long = getConf(HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE) + def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT) def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index c909eb5d20bcd..44089335e1a1d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -235,7 +235,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log if (lazyPruningEnabled) { catalog } else { - catalog.allPartitions + catalog.filterPartitions(Nil) // materialize all the partitions in memory } } val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 81337493c7f28..d13e29b3029b1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -577,5 +577,19 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(output == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) assert(serde == Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) } - } + } + + test("table name with schema") { + // regression test for SPARK-11778 + spark.sql("create schema usrdb") + spark.sql("create table usrdb.test(c int)") + spark.read.table("usrdb.test") + spark.sql("drop table usrdb.test") + spark.sql("drop schema usrdb") + } + + test("SPARK-15887: hive-site.xml should be loaded") { + val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + assert(hiveClient.getConf("hive.in.test", "") == "true") + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTablePerfStatsSuite.scala similarity index 50% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTablePerfStatsSuite.scala index 15523437a3404..82ee813c6a95f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTablePerfStatsSuite.scala @@ -19,25 +19,26 @@ package org.apache.spark.sql.hive import java.io.File +import org.scalatest.BeforeAndAfterEach + import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.execution.datasources.FileStatusCache import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils -class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { - test("table name with schema") { - // regression test for SPARK-11778 - spark.sql("create schema usrdb") - spark.sql("create table usrdb.test(c int)") - spark.read.table("usrdb.test") - spark.sql("drop table usrdb.test") - spark.sql("drop schema usrdb") +class HiveTablePerfStatsSuite + extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach { + + override def beforeEach(): Unit = { + super.beforeEach() + FileStatusCache.resetForTesting() } - test("SPARK-15887: hive-site.xml should be loaded") { - val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client - assert(hiveClient.getConf("hive.in.test", "") == "true") + override def afterEach(): Unit = { + super.afterEach() + FileStatusCache.resetForTesting() } private def setupPartitionedTable(tableName: String, dir: File): Unit = { @@ -79,7 +80,9 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt } test("lazy partition pruning reads only necessary partition data") { - withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "true") { + withSQLConf( + SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "true", + SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "0") { withTable("test") { withTempDir { dir => setupPartitionedTable("test", dir) @@ -104,11 +107,103 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) - // read all should be cached + // read all should not be cached HiveCatalogMetrics.reset() spark.sql("select * from test").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + // cache should be disabled + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + test("lazy partition pruning with file status caching enabled") { + withSQLConf( + "spark.sql.hive.filesourcePartitionPruning" -> "true", + "spark.sql.hive.filesourcePartitionFileCacheSize" -> "9999999") { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 = 999").count() == 0) assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 2").count() == 2) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 3").count() == 3) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 3) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 1) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 2) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 3) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 5) + } + } + } + } + + test("file status caching respects refresh table and refreshByPath") { + withSQLConf( + "spark.sql.hive.filesourcePartitionPruning" -> "true", + "spark.sql.hive.filesourcePartitionFileCacheSize" -> "9999999") { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + spark.sql("refresh table test") + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + spark.catalog.cacheTable("test") + HiveCatalogMetrics.reset() + spark.catalog.refreshByPath(dir.getAbsolutePath) + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + test("file status cache respects size limit") { + withSQLConf( + "spark.sql.hive.filesourcePartitionPruning" -> "true", + "spark.sql.hive.filesourcePartitionFileCacheSize" -> "1" /* 1 byte */) { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 10) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) } } } @@ -124,18 +219,18 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt // mode. This is kind of terrible, but is needed to preserve the legacy behavior // of doing plan cache validation based on the entire partition set. HiveCatalogMetrics.reset() - spark.sql("select * from test where partCol1 = 999").count() + assert(spark.sql("select * from test where partCol1 = 999").count() == 0) // 5 from table resolution, another 5 from ListingFileCatalog assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 10) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) HiveCatalogMetrics.reset() - spark.sql("select * from test where partCol1 < 2").count() + assert(spark.sql("select * from test where partCol1 < 2").count() == 2) assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) HiveCatalogMetrics.reset() - spark.sql("select * from test").count() + assert(spark.sql("select * from test").count() == 5) assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) } From 5fa9f8795a71e08bcbef5975ba8c072db5be8866 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sat, 22 Oct 2016 20:09:04 +0200 Subject: [PATCH 168/177] [SPARK-17123][SQL] Use type-widened encoder for DataFrame rather than existing encoder to allow type-widening from set operations # What changes were proposed in this pull request? This PR fixes set operations in `DataFrame` to be performed fine without exceptions when the types are non-scala native types. (e.g, `TimestampType`, `DateType` and `DecimalType`). The problem is, it seems set operations such as `union`, `intersect` and `except` uses the encoder belonging to the `Dataset` in caller. So, `Dataset` of the caller holds `ExpressionEncoder[Row]` as it is when the set operations are performed. However, the return types can be actually widen. So, we should use `ExpressionEncoder[Row]` constructed from executed plan rather than using existing one. Otherwise, this will generate some codes wrongly via `StaticInvoke`. Running the codes below: ```scala val dates = Seq( (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)), (new Date(3), BigDecimal.valueOf(4), new Timestamp(5)) ).toDF("date", "timestamp", "decimal") val widenTypedRows = Seq( (new Timestamp(2), 10.5D, "string") ).toDF("date", "timestamp", "decimal") val results = dates.union(widenTypedRows).collect() results.foreach(println) ``` prints below: **Before** ```java 23:08:54.490 ERROR org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 28, Column 107: No applicable constructor/method found for actual parameters "long"; candidates are: "public static java.sql.Date org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(int)" /* 001 */ public java.lang.Object generate(Object[] references) { /* 002 */ return new SpecificSafeProjection(references); /* 003 */ } /* 004 */ /* 005 */ class SpecificSafeProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseProjection { /* 006 */ /* 007 */ private Object[] references; /* 008 */ private MutableRow mutableRow; /* 009 */ private Object[] values; /* 010 */ private org.apache.spark.sql.types.StructType schema; /* 011 */ /* 012 */ /* 013 */ public SpecificSafeProjection(Object[] references) { /* 014 */ this.references = references; /* 015 */ mutableRow = (MutableRow) references[references.length - 1]; /* 016 */ /* 017 */ this.schema = (org.apache.spark.sql.types.StructType) references[0]; /* 018 */ } /* 019 */ /* 020 */ public java.lang.Object apply(java.lang.Object _i) { /* 021 */ InternalRow i = (InternalRow) _i; /* 022 */ /* 023 */ values = new Object[3]; /* 024 */ /* 025 */ boolean isNull2 = i.isNullAt(0); /* 026 */ long value2 = isNull2 ? -1L : (i.getLong(0)); /* 027 */ boolean isNull1 = isNull2; /* 028 */ final java.sql.Date value1 = isNull1 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(value2); /* 029 */ isNull1 = value1 == null; /* 030 */ if (isNull1) { /* 031 */ values[0] = null; /* 032 */ } else { /* 033 */ values[0] = value1; /* 034 */ } /* 035 */ /* 036 */ boolean isNull4 = i.isNullAt(1); /* 037 */ double value4 = isNull4 ? -1.0 : (i.getDouble(1)); /* 038 */ /* 039 */ boolean isNull3 = isNull4; /* 040 */ java.math.BigDecimal value3 = null; /* 041 */ if (!isNull3) { /* 042 */ /* 043 */ Object funcResult = null; /* 044 */ funcResult = value4.toJavaBigDecimal(); /* 045 */ if (funcResult == null) { /* 046 */ isNull3 = true; /* 047 */ } else { /* 048 */ value3 = (java.math.BigDecimal) funcResult; /* 049 */ } /* 050 */ /* 051 */ } /* 052 */ isNull3 = value3 == null; /* 053 */ if (isNull3) { /* 054 */ values[1] = null; /* 055 */ } else { /* 056 */ values[1] = value3; /* 057 */ } /* 058 */ /* 059 */ boolean isNull6 = i.isNullAt(2); /* 060 */ UTF8String value6 = isNull6 ? null : (i.getUTF8String(2)); /* 061 */ boolean isNull5 = isNull6; /* 062 */ final java.sql.Timestamp value5 = isNull5 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaTimestamp(value6); /* 063 */ isNull5 = value5 == null; /* 064 */ if (isNull5) { /* 065 */ values[2] = null; /* 066 */ } else { /* 067 */ values[2] = value5; /* 068 */ } /* 069 */ /* 070 */ final org.apache.spark.sql.Row value = new org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(values, schema); /* 071 */ if (false) { /* 072 */ mutableRow.setNullAt(0); /* 073 */ } else { /* 074 */ /* 075 */ mutableRow.update(0, value); /* 076 */ } /* 077 */ /* 078 */ return mutableRow; /* 079 */ } /* 080 */ } ``` **After** ```bash [1969-12-31 00:00:00.0,1.0,1969-12-31 16:00:00.002] [1969-12-31 00:00:00.0,4.0,1969-12-31 16:00:00.005] [1969-12-31 16:00:00.002,10.5,string] ``` ## How was this patch tested? Unit tests in `DataFrameSuite` Author: hyukjinkwon Closes #15072 from HyukjinKwon/SPARK-17123. --- .../scala/org/apache/spark/sql/Dataset.scala | 18 ++++++++++++++---- .../org/apache/spark/sql/DataFrameSuite.scala | 16 ++++++++++++++++ 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 073d2b1512b95..286d8549bfe27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -556,7 +556,7 @@ class Dataset[T] private[sql]( * 1983 03 0.410516 0.442194 * 1984 04 0.450090 0.483521 * }}} - * + * * @param numRows Number of rows to show * @param truncate If set to more than 0, truncates strings to `truncate` characters and * all cells will be aligned right. @@ -1524,7 +1524,7 @@ class Dataset[T] private[sql]( * @group typedrel * @since 2.0.0 */ - def union(other: Dataset[T]): Dataset[T] = withTypedPlan { + def union(other: Dataset[T]): Dataset[T] = withSetOperator { // This breaks caching, but it's usually ok because it addresses a very specific use case: // using union to union many files or partitions. CombineUnions(Union(logicalPlan, other.logicalPlan)) @@ -1540,7 +1540,7 @@ class Dataset[T] private[sql]( * @group typedrel * @since 1.6.0 */ - def intersect(other: Dataset[T]): Dataset[T] = withTypedPlan { + def intersect(other: Dataset[T]): Dataset[T] = withSetOperator { Intersect(logicalPlan, other.logicalPlan) } @@ -1554,7 +1554,7 @@ class Dataset[T] private[sql]( * @group typedrel * @since 2.0.0 */ - def except(other: Dataset[T]): Dataset[T] = withTypedPlan { + def except(other: Dataset[T]): Dataset[T] = withSetOperator { Except(logicalPlan, other.logicalPlan) } @@ -2725,4 +2725,14 @@ class Dataset[T] private[sql]( @inline private def withTypedPlan[U : Encoder](logicalPlan: => LogicalPlan): Dataset[U] = { Dataset(sparkSession, logicalPlan) } + + /** A convenient function to wrap a set based logical plan and produce a Dataset. */ + @inline private def withSetOperator[U : Encoder](logicalPlan: => LogicalPlan): Dataset[U] = { + if (classTag.runtimeClass.isAssignableFrom(classOf[Row])) { + // Set operators widen types (change the schema), so we cannot reuse the row encoder. + Dataset.ofRows(sparkSession, logicalPlan).asInstanceOf[Dataset[U]] + } else { + Dataset(sparkSession, logicalPlan) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 16cc368208485..e87baa454c8b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import java.io.File import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} import java.util.UUID import scala.util.Random @@ -1615,4 +1616,19 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { qe.assertAnalyzed() } } + + test("SPARK-17123: Performing set operations that combine non-scala native types") { + val dates = Seq( + (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)), + (new Date(3), BigDecimal.valueOf(4), new Timestamp(5)) + ).toDF("date", "timestamp", "decimal") + + val widenTypedRows = Seq( + (new Timestamp(2), 10.5D, "string") + ).toDF("date", "timestamp", "decimal") + + dates.union(widenTypedRows).collect() + dates.except(widenTypedRows).collect() + dates.intersect(widenTypedRows).collect() + } } From 4f1dcd3dce270268b42fbe59409790364fa5c5df Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Sat, 22 Oct 2016 11:59:28 -0700 Subject: [PATCH 169/177] [SPARK-18051][SPARK CORE] fix bug of custom PartitionCoalescer causing serialization exception ## What changes were proposed in this pull request? add a require check in `CoalescedRDD` to make sure the passed in `partitionCoalescer` to be `serializable`. and update the document for api `RDD.coalesce` ## How was this patch tested? Manual.(test code in jira [SPARK-18051]) Author: WeichenXu Closes #15587 from WeichenXu123/fix_coalescer_bug. --- core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala | 4 ++++ core/src/main/scala/org/apache/spark/rdd/RDD.scala | 3 ++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 9c198a61f37af..2cba1febe8759 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -80,6 +80,10 @@ private[spark] class CoalescedRDD[T: ClassTag]( require(maxPartitions > 0 || maxPartitions == prev.partitions.length, s"Number of partitions ($maxPartitions) must be positive.") + if (partitionCoalescer.isDefined) { + require(partitionCoalescer.get.isInstanceOf[Serializable], + "The partition coalescer passed in must be serializable.") + } override def getPartitions: Array[Partition] = { val pc = partitionCoalescer.getOrElse(new DefaultPartitionCoalescer()) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index be119578d2c35..db535de9e9bb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -432,7 +432,8 @@ abstract class RDD[T: ClassTag]( * of partitions. This is useful if you have a small number of partitions, * say 100, potentially with a few partitions being abnormally large. Calling * coalesce(1000, shuffle = true) will result in 1000 partitions with the - * data distributed using a hash partitioner. + * data distributed using a hash partitioner. The optional partition coalescer + * passed in must be serializable. */ def coalesce(numPartitions: Int, shuffle: Boolean = false, partitionCoalescer: Option[PartitionCoalescer] = Option.empty) From bc167a2a53f5a795d089e8a884569b1b3e2cd439 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Sat, 22 Oct 2016 12:03:37 -0700 Subject: [PATCH 170/177] [SPARK-928][CORE] Add support for Unsafe-based serializer in Kryo ## What changes were proposed in this pull request? Now since we have migrated to Kryo-3.0.0 in https://issues.apache.org/jira/browse/SPARK-11416, we can gives users option to use unsafe SerDer. It can turned by setting `spark.kryo.useUnsafe` to `true` ## How was this patch tested? Ran existing tests ``` Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ basicTypes: Int unsafe:true 160 / 178 98.5 10.1 1.0X basicTypes: Long unsafe:true 210 / 218 74.9 13.4 0.8X basicTypes: Float unsafe:true 203 / 213 77.5 12.9 0.8X basicTypes: Double unsafe:true 226 / 235 69.5 14.4 0.7X Array: Int unsafe:true 1087 / 1101 14.5 69.1 0.1X Array: Long unsafe:true 2758 / 2844 5.7 175.4 0.1X Array: Float unsafe:true 1511 / 1552 10.4 96.1 0.1X Array: Double unsafe:true 2942 / 2972 5.3 187.0 0.1X Map of string->Double unsafe:true 2645 / 2739 5.9 168.2 0.1X basicTypes: Int unsafe:false 211 / 218 74.7 13.4 0.8X basicTypes: Long unsafe:false 247 / 253 63.6 15.7 0.6X basicTypes: Float unsafe:false 211 / 216 74.5 13.4 0.8X basicTypes: Double unsafe:false 227 / 233 69.2 14.4 0.7X Array: Int unsafe:false 3012 / 3032 5.2 191.5 0.1X Array: Long unsafe:false 4463 / 4515 3.5 283.8 0.0X Array: Float unsafe:false 2788 / 2868 5.6 177.2 0.1X Array: Double unsafe:false 3558 / 3752 4.4 226.2 0.0X Map of string->Double unsafe:false 2806 / 2933 5.6 178.4 0.1X ``` Author: Sandeep Singh Author: Sandeep Singh Closes #12913 from techaddict/SPARK-928. --- .../spark/serializer/KryoSerializer.scala | 36 +++-- .../spark/serializer/KryoBenchmark.scala | 139 ++++++++++++++++++ .../serializer/KryoSerializerSuite.scala | 1 + .../UnsafeKryoSerializerSuite.scala | 33 +++++ docs/configuration.md | 8 + 5 files changed, 206 insertions(+), 11 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala create mode 100644 core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 1fba552f70501..0d26281fe1076 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -27,6 +27,7 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.{Kryo, KryoException, Serializer => KryoClassSerializer} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} +import com.esotericsoftware.kryo.io.{UnsafeInput => KryoUnsafeInput, UnsafeOutput => KryoUnsafeOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.avro.generic.{GenericData, GenericRecord} @@ -78,8 +79,15 @@ class KryoSerializer(conf: SparkConf) .filter(!_.isEmpty) private val avroSchemas = conf.getAvroSchema + // whether to use unsafe based IO for serialization + private val useUnsafe = conf.getBoolean("spark.kryo.unsafe", false) - def newKryoOutput(): KryoOutput = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + def newKryoOutput(): KryoOutput = + if (useUnsafe) { + new KryoUnsafeOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + } else { + new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + } def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator @@ -172,7 +180,7 @@ class KryoSerializer(conf: SparkConf) } override def newInstance(): SerializerInstance = { - new KryoSerializerInstance(this) + new KryoSerializerInstance(this, useUnsafe) } private[spark] override lazy val supportsRelocationOfSerializedObjects: Boolean = { @@ -186,9 +194,12 @@ class KryoSerializer(conf: SparkConf) private[spark] class KryoSerializationStream( serInstance: KryoSerializerInstance, - outStream: OutputStream) extends SerializationStream { + outStream: OutputStream, + useUnsafe: Boolean) extends SerializationStream { + + private[this] var output: KryoOutput = + if (useUnsafe) new KryoUnsafeOutput(outStream) else new KryoOutput(outStream) - private[this] var output: KryoOutput = new KryoOutput(outStream) private[this] var kryo: Kryo = serInstance.borrowKryo() override def writeObject[T: ClassTag](t: T): SerializationStream = { @@ -219,9 +230,12 @@ class KryoSerializationStream( private[spark] class KryoDeserializationStream( serInstance: KryoSerializerInstance, - inStream: InputStream) extends DeserializationStream { + inStream: InputStream, + useUnsafe: Boolean) extends DeserializationStream { + + private[this] var input: KryoInput = + if (useUnsafe) new KryoUnsafeInput(inStream) else new KryoInput(inStream) - private[this] var input: KryoInput = new KryoInput(inStream) private[this] var kryo: Kryo = serInstance.borrowKryo() override def readObject[T: ClassTag](): T = { @@ -248,8 +262,8 @@ class KryoDeserializationStream( } } -private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - +private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boolean) + extends SerializerInstance { /** * A re-used [[Kryo]] instance. Methods will borrow this instance by calling `borrowKryo()`, do * their work, then release the instance by calling `releaseKryo()`. Logically, this is a caching @@ -288,7 +302,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ // Make these lazy vals to avoid creating a buffer unless we use them. private lazy val output = ks.newKryoOutput() - private lazy val input = new KryoInput() + private lazy val input = if (useUnsafe) new KryoUnsafeInput() else new KryoInput() override def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() @@ -329,11 +343,11 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ } override def serializeStream(s: OutputStream): SerializationStream = { - new KryoSerializationStream(this, s) + new KryoSerializationStream(this, s, useUnsafe) } override def deserializeStream(s: InputStream): DeserializationStream = { - new KryoDeserializationStream(this, s) + new KryoDeserializationStream(this, s, useUnsafe) } /** diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala new file mode 100644 index 0000000000000..64be966276140 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -0,0 +1,139 @@ +/* + * 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.serializer + +import scala.reflect.ClassTag +import scala.util.Random + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer.KryoTest._ +import org.apache.spark.util.Benchmark + +class KryoBenchmark extends SparkFunSuite { + val benchmark = new Benchmark("Benchmark Kryo Unsafe vs safe Serialization", 1024 * 1024 * 15, 10) + + ignore(s"Benchmark Kryo Unsafe vs safe Serialization") { + Seq (true, false).foreach (runBenchmark) + benchmark.run() + + // scalastyle:off + /* + Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + basicTypes: Int with unsafe:true 151 / 170 104.2 9.6 1.0X + basicTypes: Long with unsafe:true 175 / 191 89.8 11.1 0.9X + basicTypes: Float with unsafe:true 177 / 184 88.8 11.3 0.9X + basicTypes: Double with unsafe:true 193 / 216 81.4 12.3 0.8X + Array: Int with unsafe:true 513 / 587 30.7 32.6 0.3X + Array: Long with unsafe:true 1211 / 1358 13.0 77.0 0.1X + Array: Float with unsafe:true 890 / 964 17.7 56.6 0.2X + Array: Double with unsafe:true 1335 / 1428 11.8 84.9 0.1X + Map of string->Double with unsafe:true 931 / 988 16.9 59.2 0.2X + basicTypes: Int with unsafe:false 197 / 217 79.9 12.5 0.8X + basicTypes: Long with unsafe:false 219 / 240 71.8 13.9 0.7X + basicTypes: Float with unsafe:false 208 / 217 75.7 13.2 0.7X + basicTypes: Double with unsafe:false 208 / 225 75.6 13.2 0.7X + Array: Int with unsafe:false 2559 / 2681 6.1 162.7 0.1X + Array: Long with unsafe:false 3425 / 3516 4.6 217.8 0.0X + Array: Float with unsafe:false 2025 / 2134 7.8 128.7 0.1X + Array: Double with unsafe:false 2241 / 2358 7.0 142.5 0.1X + Map of string->Double with unsafe:false 1044 / 1085 15.1 66.4 0.1X + */ + // scalastyle:on + } + + private def runBenchmark(useUnsafe: Boolean): Unit = { + def check[T: ClassTag](t: T, ser: SerializerInstance): Int = { + if (ser.deserialize[T](ser.serialize(t)) === t) 1 else 0 + } + + // Benchmark Primitives + val basicTypeCount = 1000000 + def basicTypes[T: ClassTag](name: String, gen: () => T): Unit = { + lazy val ser = createSerializer(useUnsafe) + val arrayOfBasicType: Array[T] = Array.fill(basicTypeCount)(gen()) + + benchmark.addCase(s"basicTypes: $name with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < basicTypeCount) { + sum += check(arrayOfBasicType(i), ser) + i += 1 + } + sum + } + } + basicTypes("Int", Random.nextInt) + basicTypes("Long", Random.nextLong) + basicTypes("Float", Random.nextFloat) + basicTypes("Double", Random.nextDouble) + + // Benchmark Array of Primitives + val arrayCount = 10000 + def basicTypeArray[T: ClassTag](name: String, gen: () => T): Unit = { + lazy val ser = createSerializer(useUnsafe) + val arrayOfArrays: Array[Array[T]] = + Array.fill(arrayCount)(Array.fill[T](Random.nextInt(arrayCount))(gen())) + + benchmark.addCase(s"Array: $name with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < arrayCount) { + val arr = arrayOfArrays(i) + sum += check(arr, ser) + i += 1 + } + sum + } + } + basicTypeArray("Int", Random.nextInt) + basicTypeArray("Long", Random.nextLong) + basicTypeArray("Float", Random.nextFloat) + basicTypeArray("Double", Random.nextDouble) + + // Benchmark Maps + val mapsCount = 1000 + lazy val ser = createSerializer(useUnsafe) + val arrayOfMaps: Array[Map[String, Double]] = Array.fill(mapsCount) { + Array.fill(Random.nextInt(mapsCount)) { + (Random.nextString(mapsCount / 10), Random.nextDouble()) + }.toMap + } + + benchmark.addCase(s"Map of string->Double with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < mapsCount) { + val map = arrayOfMaps(i) + sum += check(map, ser) + i += 1 + } + sum + } + } + + def createSerializer(useUnsafe: Boolean): SerializerInstance = { + val conf = new SparkConf() + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) + conf.set("spark.kryo.unsafe", useUnsafe.toString) + + new KryoSerializer(conf).newInstance() + } + +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index bc6e98365daef..5040841811054 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.util.Utils class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) + conf.set("spark.kryo.unsafe", "false") test("SPARK-7392 configuration limits") { val kryoBufferProperty = "spark.kryoserializer.buffer" diff --git a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala new file mode 100644 index 0000000000000..d63a45ae4a6a9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala @@ -0,0 +1,33 @@ +/* + * 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.serializer + +class UnsafeKryoSerializerSuite extends KryoSerializerSuite { + + // This test suite should run all tests in KryoSerializerSuite with kryo unsafe. + + override def beforeAll() { + conf.set("spark.kryo.unsafe", "true") + super.beforeAll() + } + + override def afterAll() { + conf.set("spark.kryo.unsafe", "false") + super.afterAll() + } +} diff --git a/docs/configuration.md b/docs/configuration.md index a4a99d6fa4630..b07867d99aa9d 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -799,6 +799,14 @@ Apart from these, the following properties are also available, and may be useful See the tuning guide for more details.
    spark.kryo.unsafefalse + Whether to use unsafe based Kryo serializer. Can be + substantially faster by using Unsafe Based IO. +
    spark.kryoserializer.buffer.max 64m