spark git commit: [SPARK-24940][SQL] Use IntegerLiteral in ResolveCoalesceHints
Repository: spark Updated Branches: refs/heads/master 64ad7b841 -> d063e3a47 [SPARK-24940][SQL] Use IntegerLiteral in ResolveCoalesceHints ## What changes were proposed in this pull request? Follow up to fix an unmerged review comment. ## How was this patch tested? Unit test ResolveHintsSuite. Author: John Zhuge Closes #21998 from jzhuge/SPARK-24940. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d063e3a4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d063e3a4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d063e3a4 Branch: refs/heads/master Commit: d063e3a478221c836a0aa74a69828a526a6207bb Parents: 64ad7b8 Author: John Zhuge Authored: Mon Aug 6 06:41:55 2018 -0400 Committer: Xiao Li Committed: Mon Aug 6 06:41:55 2018 -0400 -- .../org/apache/spark/sql/catalyst/analysis/ResolveHints.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d063e3a4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index 1ef482b..80d5105 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -20,12 +20,11 @@ package org.apache.spark.sql.catalyst.analysis import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.expressions.IntegerLiteral import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.IntegerType /** @@ -119,7 +118,7 @@ object ResolveHints { case "COALESCE" => false } val numPartitions = h.parameters match { - case Seq(Literal(numPartitions: Int, IntegerType)) => + case Seq(IntegerLiteral(numPartitions)) => numPartitions case Seq(numPartitions: Int) => numPartitions - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24940][SQL] Coalesce and Repartition Hint for SQL Queries
Repository: spark Updated Branches: refs/heads/master 41c2227a2 -> 36ea55e97 [SPARK-24940][SQL] Coalesce and Repartition Hint for SQL Queries ## What changes were proposed in this pull request? Many Spark SQL users in my company have asked for a way to control the number of output files in Spark SQL. The users prefer not to use function repartition(n) or coalesce(n, shuffle) that require them to write and deploy Scala/Java/Python code. We propose adding the following Hive-style Coalesce and Repartition Hint to Spark SQL: ``` ... SELECT /*+ COALESCE(numPartitions) */ ... ... SELECT /*+ REPARTITION(numPartitions) */ ... ``` Multiple such hints are allowed. Multiple nodes are inserted into the logical plan, and the optimizer will pick the leftmost hint. ``` INSERT INTO s SELECT /*+ REPARTITION(100), COALESCE(500), COALESCE(10) */ * FROM t == Logical Plan == 'InsertIntoTable 'UnresolvedRelation `s`, false, false +- 'UnresolvedHint REPARTITION, [100] +- 'UnresolvedHint COALESCE, [500] +- 'UnresolvedHint COALESCE, [10] +- 'Project [*] +- 'UnresolvedRelation `t` == Optimized Logical Plan == InsertIntoHadoopFsRelationCommand ... +- Repartition 100, true +- HiveTableRelation ... ``` ## How was this patch tested? All unit tests. Manual tests using explain. Author: John Zhuge Closes #21911 from jzhuge/SPARK-24940. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/36ea55e9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/36ea55e9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/36ea55e9 Branch: refs/heads/master Commit: 36ea55e97e609d25de5d8cd47ce8d2a7ae990d62 Parents: 41c2227 Author: John Zhuge Authored: Sat Aug 4 02:27:15 2018 -0400 Committer: Xiao Li Committed: Sat Aug 4 02:27:15 2018 -0400 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 1 + .../sql/catalyst/analysis/ResolveHints.scala| 28 .../catalyst/analysis/ResolveHintsSuite.scala | 35 .../sql/catalyst/parser/PlanParserSuite.scala | 27 +++ .../apache/spark/sql/DataFrameHintSuite.scala | 10 ++ .../org/apache/spark/sql/SQLQuerySuite.scala| 33 ++ 6 files changed, 134 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/36ea55e9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7f235ac..b5016fd 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 @@ -145,6 +145,7 @@ class Analyzer( lazy val batches: Seq[Batch] = Seq( Batch("Hints", fixedPoint, new ResolveHints.ResolveBroadcastHints(conf), + ResolveHints.ResolveCoalesceHints, ResolveHints.RemoveAllHints), Batch("Simple Sanity Check", Once, LookupFunctions), http://git-wip-us.apache.org/repos/asf/spark/blob/36ea55e9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index bfe5169..1ef482b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -20,10 +20,12 @@ package org.apache.spark.sql.catalyst.analysis import java.util.Locale import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.IntegerType /** @@ -103,6 +105,32 @@ object ResolveHints { } /** + * COALESCE Hint accepts name "COALESCE" and "REPARTITION". + * Its parameter includes a partition number. + */ + object ResolveCoalesceHints extends Rule[LogicalPlan] { +private val COALESCE_HINT_NAMES = Set("COALESCE", "REPARTITION") + +def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { + case h: UnresolvedHint if COALESCE_HINT_NAMES.contains(h.name.toUpperCase(Locale.ROOT)) => +val hintName = h.name.toUpperCase(Locale.ROOT) +val shuffle = hintName mat
spark git commit: [SPARK-24997][SQL] Enable support of MINUS ALL
Repository: spark Updated Branches: refs/heads/master b0d6967d4 -> 19a453191 [SPARK-24997][SQL] Enable support of MINUS ALL ## What changes were proposed in this pull request? Enable support for MINUS ALL which was gated at AstBuilder. ## How was this patch tested? Added tests in SQLQueryTestSuite and modify PlanParserSuite. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Dilip Biswal Closes #21963 from dilipbiswal/minus-all. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/19a45319 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/19a45319 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/19a45319 Branch: refs/heads/master Commit: 19a45319130d618a173f5f3b4dde59356b39089b Parents: b0d6967 Author: Dilip Biswal Authored: Thu Aug 2 22:45:10 2018 -0700 Committer: Xiao Li Committed: Thu Aug 2 22:45:10 2018 -0700 -- .../spark/sql/catalyst/parser/AstBuilder.scala | 11 +- .../sql/catalyst/parser/PlanParserSuite.scala | 4 +- .../resources/sql-tests/inputs/except-all.sql | 22 ++- .../sql-tests/results/except-all.sql.out| 147 +++ 4 files changed, 113 insertions(+), 71 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/19a45319/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 0ceeb53..9906a30 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 @@ -517,11 +517,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Connect two queries by a Set operator. * * Supported Set operators are: - * - UNION [DISTINCT] - * - UNION ALL - * - EXCEPT [DISTINCT] - * - MINUS [DISTINCT] - * - INTERSECT [DISTINCT] + * - UNION [ DISTINCT | ALL ] + * - EXCEPT [ DISTINCT | ALL ] + * - MINUS [ DISTINCT | ALL ] + * - INTERSECT [DISTINCT | ALL] */ override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) { val left = plan(ctx.left) @@ -541,7 +540,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case SqlBaseParser.EXCEPT => Except(left, right) case SqlBaseParser.SETMINUS if all => -throw new ParseException("MINUS ALL is not supported.", ctx) +Except(left, right, isAll = true) case SqlBaseParser.SETMINUS => Except(left, right) } http://git-wip-us.apache.org/repos/asf/spark/blob/19a45319/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 38efd89..9247004 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -67,11 +67,13 @@ class PlanParserSuite extends AnalysisTest { assertEqual("select * from a union all select * from b", a.union(b)) assertEqual("select * from a except select * from b", a.except(b)) assertEqual("select * from a except distinct select * from b", a.except(b)) +assertEqual("select * from a except all select * from b", a.except(b, isAll = true)) assertEqual("select * from a minus select * from b", a.except(b)) -intercept("select * from a minus all select * from b", "MINUS ALL is not supported.") +assertEqual("select * from a minus all select * from b", a.except(b, isAll = true)) assertEqual("select * from a minus distinct select * from b", a.except(b)) assertEqual("select * from a intersect select * from b", a.intersect(b)) assertEqual("select * from a intersect distinct select * from b", a.intersect(b)) +assertEqual("select * from a intersect all select * from b", a.intersect(b, isAll = true)) } test("common table expressions") { http://git-wip-us.apache.org/repos/asf/spark/blob/19a45319/sql/core/src/test/resources/sql-tests/inputs/except-all.sql -- diff --git a/sql/core/src/test/resources/sql-tests/inputs/except-all.sql b/sql/core/src/test/resources/sql-tests/inputs/except-all.sql index 08b9a43..e28f072 100644
spark git commit: [SPARK-24788][SQL] RelationalGroupedDataset.toString with unresolved exprs should not fail
Repository: spark Updated Branches: refs/heads/master f45d60a5a -> b0d6967d4 [SPARK-24788][SQL] RelationalGroupedDataset.toString with unresolved exprs should not fail ## What changes were proposed in this pull request? In the current master, `toString` throws an exception when `RelationalGroupedDataset` has unresolved expressions; ``` scala> spark.range(0, 10).groupBy("id") res4: org.apache.spark.sql.RelationalGroupedDataset = RelationalGroupedDataset: [grouping expressions: [id: bigint], value: [id: bigint], type: GroupBy] scala> spark.range(0, 10).groupBy('id) org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to dataType on unresolved object, tree: 'id at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:105) at org.apache.spark.sql.RelationalGroupedDataset$$anonfun$12.apply(RelationalGroupedDataset.scala:474) at org.apache.spark.sql.RelationalGroupedDataset$$anonfun$12.apply(RelationalGroupedDataset.scala:473) 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.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at org.apache.spark.sql.RelationalGroupedDataset.toString(RelationalGroupedDataset.scala:473) at scala.runtime.ScalaRunTime$.scala$runtime$ScalaRunTime$$inner$1(ScalaRunTime.scala:332) at scala.runtime.ScalaRunTime$.stringOf(ScalaRunTime.scala:337) at scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:345) ``` This pr fixed code to handle the unresolved case in `RelationalGroupedDataset.toString`. Closes #21752 ## How was this patch tested? Added tests in `DataFrameAggregateSuite`. Author: Chris Horn Author: Takeshi Yamamuro Closes #21964 from maropu/SPARK-24788. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b0d6967d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b0d6967d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b0d6967d Branch: refs/heads/master Commit: b0d6967d45f3260ed4ee9b2a49f801d799e81283 Parents: f45d60a Author: Chris Horn Authored: Thu Aug 2 22:40:58 2018 -0700 Committer: Xiao Li Committed: Thu Aug 2 22:40:58 2018 -0700 -- .../org/apache/spark/sql/RelationalGroupedDataset.scala | 7 +-- .../org/apache/spark/sql/DataFrameAggregateSuite.scala| 10 ++ 2 files changed, 15 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b0d6967d/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala -- 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 8412219..4e73b36 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 @@ -469,8 +469,11 @@ class RelationalGroupedDataset protected[sql]( override def toString: String = { val builder = new StringBuilder builder.append("RelationalGroupedDataset: [grouping expressions: [") -val kFields = groupingExprs.map(_.asInstanceOf[NamedExpression]).map { - case f => s"${f.name}: ${f.dataType.simpleString(2)}" +val kFields = groupingExprs.collect { + case expr: NamedExpression if expr.resolved => +s"${expr.name}: ${expr.dataType.simpleString(2)}" + case expr: NamedExpression => expr.name + case o => o.toString } builder.append(kFields.take(2).mkString(", ")) if (kFields.length > 2) { http://git-wip-us.apache.org/repos/asf/spark/blob/b0d6967d/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index f495a94..d0106c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -717,4 +717,14 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { Row(1, 2, 1) :: Row(2, 2, 2) :: Row(3, 2, 3) :: Nil) } + test("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail") { +// Checks if these raise no exc
spark git commit: [SPARK-24966][SQL] Implement precedence rules for set operations.
Repository: spark Updated Branches: refs/heads/master b3f2911ee -> 73dd6cf9b [SPARK-24966][SQL] Implement precedence rules for set operations. ## What changes were proposed in this pull request? Currently the set operations INTERSECT, UNION and EXCEPT are assigned the same precedence. This PR fixes the problem by giving INTERSECT higher precedence than UNION and EXCEPT. UNION and EXCEPT operators are evaluated in the order in which they appear in the query from left to right. This results in change in behavior because of the change in order of evaluations of set operators in a query. The old behavior is still preserved under a newly added config parameter. Query `:` ``` SELECT * FROM t1 UNION SELECT * FROM t2 EXCEPT SELECT * FROM t3 INTERSECT SELECT * FROM t4 ``` Parsed plan before the change `:` ``` == Parsed Logical Plan == 'Intersect false :- 'Except false : :- 'Distinct : : +- 'Union : : :- 'Project [*] : : : +- 'UnresolvedRelation `t1` : : +- 'Project [*] : :+- 'UnresolvedRelation `t2` : +- 'Project [*] : +- 'UnresolvedRelation `t3` +- 'Project [*] +- 'UnresolvedRelation `t4` ``` Parsed plan after the change `:` ``` == Parsed Logical Plan == 'Except false :- 'Distinct : +- 'Union : :- 'Project [*] : : +- 'UnresolvedRelation `t1` : +- 'Project [*] :+- 'UnresolvedRelation `t2` +- 'Intersect false :- 'Project [*] : +- 'UnresolvedRelation `t3` +- 'Project [*] +- 'UnresolvedRelation `t4` ``` ## How was this patch tested? Added tests in PlanParserSuite, SQLQueryTestSuite. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Dilip Biswal Closes #21941 from dilipbiswal/SPARK-24966. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/73dd6cf9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/73dd6cf9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/73dd6cf9 Branch: refs/heads/master Commit: 73dd6cf9b558f9d752e1f3c13584344257ad7863 Parents: b3f2911 Author: Dilip Biswal Authored: Thu Aug 2 22:04:17 2018 -0700 Committer: Xiao Li Committed: Thu Aug 2 22:04:17 2018 -0700 -- docs/sql-programming-guide.md | 1 + .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 15 ++- .../apache/spark/sql/catalyst/dsl/package.scala | 6 +- .../spark/sql/catalyst/parser/ParseDriver.scala | 2 + .../plans/logical/basicLogicalOperators.scala | 6 +- .../org/apache/spark/sql/internal/SQLConf.scala | 12 +++ .../sql/catalyst/parser/PlanParserSuite.scala | 45 .../spark/sql/execution/SparkStrategies.scala | 4 +- .../sql-tests/inputs/intersect-all.sql | 51 +++-- .../sql-tests/results/intersect-all.sql.out | 104 +++ 10 files changed, 211 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/73dd6cf9/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 0900f83..a1e019c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1876,6 +1876,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see ## Upgrading From Spark SQL 2.3 to 2.4 + - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuaration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`
spark git commit: [SPARK-24705][SQL] ExchangeCoordinator broken when duplicate exchanges reused
Repository: spark Updated Branches: refs/heads/master 02f967795 -> efef55388 [SPARK-24705][SQL] ExchangeCoordinator broken when duplicate exchanges reused ## What changes were proposed in this pull request? In the current master, `EnsureRequirements` sets the number of exchanges in `ExchangeCoordinator` before `ReuseExchange`. Then, `ReuseExchange` removes some duplicate exchange and the actual number of registered exchanges changes. Finally, the assertion in `ExchangeCoordinator` fails because the logical number of exchanges and the actual number of registered exchanges become different; https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala#L201 This pr fixed the issue and the code to reproduce this is as follows; ``` scala> sql("SET spark.sql.adaptive.enabled=true") scala> sql("SET spark.sql.autoBroadcastJoinThreshold=-1") scala> val df = spark.range(1).selectExpr("id AS key", "id AS value") scala> val resultDf = df.join(df, "key").join(df, "key") scala> resultDf.show ... at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52) ... 101 more Caused by: java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:156) at org.apache.spark.sql.execution.exchange.ExchangeCoordinator.doEstimationIfNecessary(ExchangeCoordinator.scala:201) at org.apache.spark.sql.execution.exchange.ExchangeCoordinator.postShuffleRDD(ExchangeCoordinator.scala:259) at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:124) at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52) ... ``` ## How was this patch tested? Added tests in `ExchangeCoordinatorSuite`. Author: Takeshi Yamamuro Closes #21754 from maropu/SPARK-24705-2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/efef5538 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/efef5538 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/efef5538 Branch: refs/heads/master Commit: efef55388fedef3f7954a385776e666ad4597a58 Parents: 02f9677 Author: Takeshi Yamamuro Authored: Thu Aug 2 13:05:36 2018 -0700 Committer: Xiao Li Committed: Thu Aug 2 13:05:36 2018 -0700 -- .../execution/exchange/EnsureRequirements.scala | 1 - .../exchange/ExchangeCoordinator.scala | 17 ++-- .../execution/ExchangeCoordinatorSuite.scala| 21 3 files changed, 28 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/efef5538/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index d96ecba..d2d5011 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -82,7 +82,6 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { if (adaptiveExecutionEnabled && supportsCoordinator) { val coordinator = new ExchangeCoordinator( -children.length, targetPostShuffleInputSize, minNumPostShufflePartitions) children.zip(requiredChildDistributions).map { http://git-wip-us.apache.org/repos/asf/spark/blob/efef5538/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index 051e610..f5d93ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -83,7 +83,6 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} * - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MB) */ class ExchangeCoordinator( -numExchanges: Int, advisoryTargetPostShuffleInputSize: Long, minNumPostShufflePartitions: Option[Int] = None) extends Logging { @@ -91,
spark git commit: [SPARK-23908][SQL] Add transform function.
Repository: spark Updated Branches: refs/heads/master 0df6bf882 -> 02f967795 [SPARK-23908][SQL] Add transform function. ## What changes were proposed in this pull request? This pr adds `transform` function which transforms elements in an array using the function. Optionally we can take the index of each element as the second argument. ```sql > SELECT transform(array(1, 2, 3), x -> x + 1); array(2, 3, 4) > SELECT transform(array(1, 2, 3), (x, i) -> x + i); array(1, 3, 5) ``` ## How was this patch tested? Added tests. Author: Takuya UESHIN Closes #21954 from ueshin/issues/SPARK-23908/transform. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/02f96779 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/02f96779 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/02f96779 Branch: refs/heads/master Commit: 02f967795b7e8ccf2738d567928e47c38c1134e1 Parents: 0df6bf8 Author: Takuya UESHIN Authored: Thu Aug 2 13:00:33 2018 -0700 Committer: Xiao Li Committed: Thu Aug 2 13:00:33 2018 -0700 -- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 2 + .../spark/sql/catalyst/analysis/Analyzer.scala | 3 + .../catalyst/analysis/FunctionRegistry.scala| 1 + .../analysis/higherOrderFunctions.scala | 166 +++ .../expressions/higherOrderFunctions.scala | 212 +++ .../spark/sql/catalyst/parser/AstBuilder.scala | 10 + .../analysis/ResolveLambdaVariablesSuite.scala | 89 .../expressions/HigherOrderFunctionsSuite.scala | 97 + .../catalyst/parser/ExpressionParserSuite.scala | 5 + .../spark/sql/catalyst/plans/PlanTest.scala | 2 + .../sql-tests/inputs/higher-order-functions.sql | 26 +++ .../results/higher-order-functions.sql.out | 81 +++ .../spark/sql/DataFrameFunctionsSuite.scala | 153 + 13 files changed, 847 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/02f96779/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 -- 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 2aca10f..9ad6f30 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 @@ -591,6 +591,8 @@ primaryExpression (OVER windowSpec)? #functionCall | qualifiedName '(' trimOption=(BOTH | LEADING | TRAILING) argument+=expression FROM argument+=expression ')' #functionCall +| IDENTIFIER '->' expression #lambda +| '(' IDENTIFIER (',' IDENTIFIER)+ ')' '->' expression #lambda | value=primaryExpression '[' index=valueExpression ']' #subscript | identifier #columnReference | base=primaryExpression '.' fieldName=identifier #dereference http://git-wip-us.apache.org/repos/asf/spark/blob/02f96779/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 76dc867..7f235ac 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 @@ -180,6 +180,8 @@ class Analyzer( ResolveAggregateFunctions :: TimeWindowing :: ResolveInlineTables(conf) :: + ResolveHigherOrderFunctions(catalog) :: + ResolveLambdaVariables(conf) :: ResolveTimeZone(conf) :: ResolveRandomSeed :: TypeCoercion.typeCoercionRules(conf) ++ @@ -878,6 +880,7 @@ class Analyzer( } private def resolve(e: Expression, q: LogicalPlan): Expression = e match { + case f: LambdaFunction if !f.bound => f case u @ UnresolvedAttribute(nameParts) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = http://git-wip-us.apache.org/repos/asf/spark/blob/02f96779/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala --
spark git commit: [SPARK-24598][DOCS] State in the documentation the behavior when arithmetic operations cause overflow
Repository: spark Updated Branches: refs/heads/master 15fc23722 -> ad2e63662 [SPARK-24598][DOCS] State in the documentation the behavior when arithmetic operations cause overflow ## What changes were proposed in this pull request? According to the discussion in https://github.com/apache/spark/pull/21599, changing the behavior of arithmetic operations so that they can check for overflow is not nice in a minor release. What we can do for 2.4 is warn users about the current behavior in the documentation, so that they are aware of the issue and can take proper actions. ## How was this patch tested? NA Author: Marco Gaido Closes #21967 from mgaido91/SPARK-24598_doc. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ad2e6366 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ad2e6366 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ad2e6366 Branch: refs/heads/master Commit: ad2e63662885b67b1e94030b13fdae4f7366dc4a Parents: 15fc237 Author: Marco Gaido Authored: Thu Aug 2 09:28:13 2018 -0700 Committer: Xiao Li Committed: Thu Aug 2 09:28:13 2018 -0700 -- docs/sql-programming-guide.md | 7 +++ 1 file changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ad2e6366/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 5f1eee8..0900f83 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -3072,3 +3072,10 @@ Specifically: - In aggregations, all NaN values are grouped together. - NaN is treated as a normal value in join keys. - NaN values go last when in ascending order, larger than any other numeric value. + + ## Arithmetic operations + +Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. +This means that in case an operation causes an overflow, the result is the same that the same operation +returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, +the result is a negative number). - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24957][SQL][FOLLOW-UP] Clean the code for AVERAGE
Repository: spark Updated Branches: refs/heads/master c9914cf04 -> 166f34618 [SPARK-24957][SQL][FOLLOW-UP] Clean the code for AVERAGE ## What changes were proposed in this pull request? This PR is to refactor the code in AVERAGE by dsl. ## How was this patch tested? N/A Author: Xiao Li Closes #21951 from gatorsmile/refactor1. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/166f3461 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/166f3461 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/166f3461 Branch: refs/heads/master Commit: 166f346185cc0b27a7e2b2a3b42df277e5901f2f Parents: c9914cf Author: Xiao Li Authored: Wed Aug 1 23:00:17 2018 -0700 Committer: Xiao Li Committed: Wed Aug 1 23:00:17 2018 -0700 -- .../scala/org/apache/spark/sql/catalyst/dsl/package.scala | 1 + .../sql/catalyst/expressions/aggregate/Average.scala | 10 -- 2 files changed, 5 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/166f3461/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 89e8c99..9870854 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -166,6 +166,7 @@ package object dsl { def maxDistinct(e: Expression): Expression = Max(e).toAggregateExpression(isDistinct = true) def upper(e: Expression): Expression = Upper(e) def lower(e: Expression): Expression = Lower(e) +def coalesce(args: Expression*): Expression = Coalesce(args) def sqrt(e: Expression): Expression = Sqrt(e) def abs(e: Expression): Expression = Abs(e) def star(names: String*): Expression = names match { http://git-wip-us.apache.org/repos/asf/spark/blob/166f3461/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala index 9ccf5aa..f1fad77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala @@ -46,7 +46,7 @@ abstract class AverageLike(child: Expression) extends DeclarativeAggregate { override lazy val aggBufferAttributes = sum :: count :: Nil override lazy val initialValues = Seq( -/* sum = */ Cast(Literal(0), sumDataType), +/* sum = */ Literal(0).cast(sumDataType), /* count = */ Literal(0L) ) @@ -58,18 +58,16 @@ abstract class AverageLike(child: Expression) extends DeclarativeAggregate { // If all input are nulls, count will be 0 and we will get null after the division. override lazy val evaluateExpression = child.dataType match { case _: DecimalType => - Cast( -DecimalPrecision.decimalAndDecimal(sum / Cast(count, DecimalType.LongDecimal)), -resultType) + DecimalPrecision.decimalAndDecimal(sum / count.cast(DecimalType.LongDecimal)).cast(resultType) case _ => - Cast(sum, resultType) / Cast(count, resultType) + sum.cast(resultType) / count.cast(resultType) } protected def updateExpressionsDef: Seq[Expression] = Seq( /* sum = */ Add( sum, - Coalesce(Cast(child, sumDataType) :: Cast(Literal(0), sumDataType) :: Nil)), + coalesce(child.cast(sumDataType), Literal(0).cast(sumDataType))), /* count = */ If(IsNull(child), count, count + 1L) ) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24957][SQL][BACKPORT-2.2] Average with decimal followed by aggregation returns wrong result
Repository: spark Updated Branches: refs/heads/branch-2.2 c4b37696f -> 22ce8051f [SPARK-24957][SQL][BACKPORT-2.2] Average with decimal followed by aggregation returns wrong result ## What changes were proposed in this pull request? When we do an average, the result is computed dividing the sum of the values by their count. In the case the result is a DecimalType, the way we are casting/managing the precision and scale is not really optimized and it is not coherent with what we do normally. In particular, a problem can happen when the Divide operand returns a result which contains a precision and scale different by the ones which are expected as output of the Divide operand. In the case reported in the JIRA, for instance, the result of the Divide operand is a Decimal(38, 36), while the output data type for Divide is 38, 22. This is not an issue when the Divide is followed by a CheckOverflow or a Cast to the right data type, as these operations return a decimal with the defined precision and scale. Despite in the Average operator we do have a Cast, this may be bypassed if the result of Divide is the same type which it is casted to, hence the issue reported in the JIRA may arise. The PR proposes to use the normal rules/handling of the arithmetic operators with Decimal data type, so we both reuse the existing code (having a single logic for operations between decimals) and we fix this problem as the result is always guarded by CheckOverflow. ## How was this patch tested? added UT Author: Marco Gaido Closes #21949 from mgaido91/SPARK-24957_2.2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/22ce8051 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/22ce8051 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/22ce8051 Branch: refs/heads/branch-2.2 Commit: 22ce8051f22099f032813591d17d84506f6c0096 Parents: c4b3769 Author: Marco Gaido Authored: Wed Aug 1 15:59:26 2018 -0700 Committer: Xiao Li Committed: Wed Aug 1 15:59:26 2018 -0700 -- .../spark/sql/catalyst/analysis/DecimalPrecision.scala | 2 +- .../sql/catalyst/expressions/aggregate/Average.scala | 9 - .../sql/hive/execution/AggregationQuerySuite.scala | 13 + 3 files changed, 18 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/22ce8051/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala index fd2ac78..a48801c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala @@ -85,7 +85,7 @@ object DecimalPrecision extends Rule[LogicalPlan] { } /** Decimal precision promotion for +, -, *, /, %, pmod, and binary comparison. */ - private val decimalAndDecimal: PartialFunction[Expression, Expression] = { + private[catalyst] val decimalAndDecimal: PartialFunction[Expression, Expression] = { // Skip nodes whose children have not been resolved yet case e if !e.childrenResolved => e http://git-wip-us.apache.org/repos/asf/spark/blob/22ce8051/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala index 708bdbf..f80df75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{DecimalPrecision, TypeCheckResult} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.TypeUtils @@ -77,10 +77,9 @@ case class Average(child: Expression) extends DeclarativeAggregate with Implicit // If all input are nulls, count will be 0 and we will get null after the division. override lazy val evaluateExpression = child.dataType match { -case DecimalType.Fixed(p, s) => - // increase the precision and scale to prevent precision loss - val dt = DecimalType
spark git commit: [SPARK-24990][SQL] merge ReadSupport and ReadSupportWithSchema
Repository: spark Updated Branches: refs/heads/master 9f558601e -> ce084d3e0 [SPARK-24990][SQL] merge ReadSupport and ReadSupportWithSchema ## What changes were proposed in this pull request? Regarding user-specified schema, data sources may have 3 different behaviors: 1. must have a user-specified schema 2. can't have a user-specified schema 3. can accept the user-specified if it's given, or infer the schema. I added `ReadSupportWithSchema` to support these behaviors, following data source v1. But it turns out we don't need this extra interface. We can just add a `createReader(schema, options)` to `ReadSupport` and make it call `createReader(options)` by default. TODO: also fix the streaming API in followup PRs. ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #21946 from cloud-fan/ds-schema. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ce084d3e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ce084d3e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ce084d3e Branch: refs/heads/master Commit: ce084d3e06b14897174426665dada0464260da89 Parents: 9f55860 Author: Wenchen Fan Authored: Wed Aug 1 15:57:54 2018 -0700 Committer: Xiao Li Committed: Wed Aug 1 15:57:54 2018 -0700 -- .../spark/sql/sources/v2/ReadSupport.java | 25 ++ .../sql/sources/v2/ReadSupportWithSchema.java | 49 .../sql/sources/v2/reader/DataSourceReader.java | 3 +- .../org/apache/spark/sql/DataFrameReader.scala | 4 +- .../datasources/v2/DataSourceV2Relation.scala | 20 +--- .../v2/JavaSchemaRequiredDataSource.java| 9 +++- .../sql/sources/v2/DataSourceV2Suite.scala | 16 --- 7 files changed, 47 insertions(+), 79 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ce084d3e/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java -- diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java index b2526de..80ac08e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java @@ -18,7 +18,9 @@ package org.apache.spark.sql.sources.v2; import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.DataSourceRegister; import org.apache.spark.sql.sources.v2.reader.DataSourceReader; +import org.apache.spark.sql.types.StructType; /** * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to @@ -33,6 +35,29 @@ public interface ReadSupport extends DataSourceV2 { * If this method fails (by throwing an exception), the action will fail and no Spark job will be * submitted. * + * @param schema the user specified schema. + * @param options the options for the returned data source reader, which is an immutable + *case-insensitive string-to-string map. + * + * By default this method throws {@link UnsupportedOperationException}, implementations should + * override this method to handle user specified schema. + */ + default DataSourceReader createReader(StructType schema, DataSourceOptions options) { +String name; +if (this instanceof DataSourceRegister) { + name = ((DataSourceRegister) this).shortName(); +} else { + name = this.getClass().getName(); +} +throw new UnsupportedOperationException(name + " does not support user specified schema"); + } + + /** + * Creates a {@link DataSourceReader} to scan the data from this data source. + * + * If this method fails (by throwing an exception), the action will fail and no Spark job will be + * submitted. + * * @param options the options for the returned data source reader, which is an immutable *case-insensitive string-to-string map. */ http://git-wip-us.apache.org/repos/asf/spark/blob/ce084d3e/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupportWithSchema.java -- diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupportWithSchema.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupportWithSchema.java deleted file mode 100644 index f316599..000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupportWithSchema.java +++ /dev/null @@ -1,49 +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.
spark git commit: [SPARK-24937][SQL] Datasource partition table should load empty static partitions
Repository: spark Updated Branches: refs/heads/master f5113ea8d -> 9f558601e [SPARK-24937][SQL] Datasource partition table should load empty static partitions ## What changes were proposed in this pull request? How to reproduce: ```sql spark-sql> CREATE TABLE tbl AS SELECT 1; spark-sql> CREATE TABLE tbl1 (c1 BIGINT, day STRING, hour STRING) > USING parquet > PARTITIONED BY (day, hour); spark-sql> INSERT INTO TABLE tbl1 PARTITION (day = '2018-07-25', hour='01') SELECT * FROM tbl where 1=0; spark-sql> SHOW PARTITIONS tbl1; spark-sql> CREATE TABLE tbl2 (c1 BIGINT) > PARTITIONED BY (day STRING, hour STRING); spark-sql> INSERT INTO TABLE tbl2 PARTITION (day = '2018-07-25', hour='01') SELECT * FROM tbl where 1=0; spark-sql> SHOW PARTITIONS tbl2; day=2018-07-25/hour=01 spark-sql> ``` 1. Users will be confused about whether the partition data of `tbl1` is generated. 2. Inconsistent with Hive table behavior. This pr fix this issues. ## How was this patch tested? unit tests Author: Yuming Wang Closes #21883 from wangyum/SPARK-24937. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9f558601 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9f558601 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9f558601 Branch: refs/heads/master Commit: 9f558601e822b7596e4bcc141d5c91a5a8859628 Parents: f5113ea Author: Yuming Wang Authored: Wed Aug 1 13:58:29 2018 -0700 Committer: Xiao Li Committed: Wed Aug 1 13:58:29 2018 -0700 -- .../InsertIntoHadoopFsRelationCommand.scala | 10 +++- .../datasources/PartitioningUtils.scala | 6 +- .../spark/sql/execution/command/DDLSuite.scala | 62 3 files changed, 76 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9f558601/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.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 8a2e00d..2ae21b7 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 @@ -171,7 +171,15 @@ case class InsertIntoHadoopFsRelationCommand( // update metastore partition metadata - refreshUpdatedPartitions(updatedPartitionPaths) + if (updatedPartitionPaths.isEmpty && staticPartitions.nonEmpty +&& partitionColumns.length == staticPartitions.size) { +// Avoid empty static partition can't loaded to datasource table. +val staticPathFragment = + PartitioningUtils.getPathFragment(staticPartitions, partitionColumns) +refreshUpdatedPartitions(Set(staticPathFragment)) + } else { +refreshUpdatedPartitions(updatedPartitionPaths) + } // refresh cached files in FileIndex fileIndex.foreach(_.refresh()) http://git-wip-us.apache.org/repos/asf/spark/blob/9f558601/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala -- 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 c8a5f98..3183fd3 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -284,6 +284,10 @@ object PartitioningUtils { }.mkString("/") } + def getPathFragment(spec: TablePartitionSpec, partitionColumns: Seq[Attribute]): String = { +getPathFragment(spec, StructType.fromAttributes(partitionColumns)) + } + /** * Normalize the column names in partition specification, w.r.t. the real partition column names * and case sensitivity. e.g., if the partition sp
spark git commit: [SPARK-24982][SQL] UDAF resolution should not throw AssertionError
Repository: spark Updated Branches: refs/heads/master 1f7e22c72 -> 1efffb799 [SPARK-24982][SQL] UDAF resolution should not throw AssertionError ## What changes were proposed in this pull request? When user calls anUDAF with the wrong number of arguments, Spark previously throws an AssertionError, which is not supposed to be a user-facing exception. This patch updates it to throw AnalysisException instead, so it is consistent with a regular UDF. ## How was this patch tested? Updated test case udaf.sql. Author: Reynold Xin Closes #21938 from rxin/SPARK-24982. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1efffb79 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1efffb79 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1efffb79 Branch: refs/heads/master Commit: 1efffb7993ecebe5dc1f9ebd924e7503bfd9668c Parents: 1f7e22c Author: Reynold Xin Authored: Wed Aug 1 00:15:31 2018 -0700 Committer: Xiao Li Committed: Wed Aug 1 00:15:31 2018 -0700 -- .../spark/sql/catalyst/catalog/SessionCatalog.scala | 15 --- .../test/resources/sql-tests/results/udaf.sql.out| 4 ++-- 2 files changed, 14 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1efffb79/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala -- 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 b09b81e..2f60eb3 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 @@ -34,7 +34,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder -import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, ImplicitCastInputTypes} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.StringUtils @@ -1124,13 +1124,22 @@ class SessionCatalog( name: String, clazz: Class[_], input: Seq[Expression]): Expression = { +// Unfortunately we need to use reflection here because UserDefinedAggregateFunction +// and ScalaUDAF are defined in sql/core module. val clsForUDAF = Utils.classForName("org.apache.spark.sql.expressions.UserDefinedAggregateFunction") if (clsForUDAF.isAssignableFrom(clazz)) { val cls = Utils.classForName("org.apache.spark.sql.execution.aggregate.ScalaUDAF") - cls.getConstructor(classOf[Seq[Expression]], clsForUDAF, classOf[Int], classOf[Int]) + val e = cls.getConstructor(classOf[Seq[Expression]], clsForUDAF, classOf[Int], classOf[Int]) .newInstance(input, clazz.newInstance().asInstanceOf[Object], Int.box(1), Int.box(1)) -.asInstanceOf[Expression] +.asInstanceOf[ImplicitCastInputTypes] + + // Check input argument size + if (e.inputTypes.size != input.size) { +throw new AnalysisException(s"Invalid number of arguments for function $name. " + + s"Expected: ${e.inputTypes.size}; Found: ${input.size}") + } + e } else { throw new AnalysisException(s"No handler for UDAF '${clazz.getCanonicalName}'. " + s"Use sparkSession.udf.register(...) instead.") http://git-wip-us.apache.org/repos/asf/spark/blob/1efffb79/sql/core/src/test/resources/sql-tests/results/udaf.sql.out -- diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index 4815a57..87824ab 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -33,8 +33,8 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 -- !query 3 schema struct<> -- !query 3 output -java.lang.AssertionError -assertion failed: Incorrect number of children +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 -- !query 4 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commi
spark git commit: [SPARK-24951][SQL] Table valued functions should throw AnalysisException
Repository: spark Updated Branches: refs/heads/master 5f3441e54 -> 1f7e22c72 [SPARK-24951][SQL] Table valued functions should throw AnalysisException ## What changes were proposed in this pull request? Previously TVF resolution could throw IllegalArgumentException if the data type is null type. This patch replaces that exception with AnalysisException, enriched with positional information, to improve error message reporting and to be more consistent with rest of Spark SQL. ## How was this patch tested? Updated the test case in table-valued-functions.sql.out, which is how I identified this problem in the first place. Author: Reynold Xin Closes #21934 from rxin/SPARK-24951. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1f7e22c7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1f7e22c7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1f7e22c7 Branch: refs/heads/master Commit: 1f7e22c72c89fc2c0e729dde0948bc6bdf8f7628 Parents: 5f3441e Author: Reynold Xin Authored: Tue Jul 31 22:25:40 2018 -0700 Committer: Xiao Li Committed: Tue Jul 31 22:25:40 2018 -0700 -- .../analysis/ResolveTableValuedFunctions.scala | 34 ++-- .../results/table-valued-functions.sql.out | 9 -- 2 files changed, 32 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1f7e22c7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala index 7358f9e..983e4b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import java.util.Locale +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Range} import org.apache.spark.sql.catalyst.rules._ @@ -68,9 +69,11 @@ object ResolveTableValuedFunctions extends Rule[LogicalPlan] { : (ArgumentList, Seq[Any] => LogicalPlan) = { (ArgumentList(args: _*), pf orElse { - case args => - throw new IllegalArgumentException( - "Invalid arguments for resolved function: " + args.mkString(", ")) + case arguments => + // This is caught again by the apply function and rethrow with richer information about + // position, etc, for a better error message. + throw new AnalysisException( + "Invalid arguments for resolved function: " + arguments.mkString(", ")) }) } @@ -105,22 +108,35 @@ object ResolveTableValuedFunctions extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case u: UnresolvedTableValuedFunction if u.functionArgs.forall(_.resolved) => + // The whole resolution is somewhat difficult to understand here due to too much abstractions. + // We should probably rewrite the following at some point. Reynold was just here to improve + // error messages and didn't have time to do a proper rewrite. val resolvedFunc = builtinFunctions.get(u.functionName.toLowerCase(Locale.ROOT)) match { case Some(tvf) => + + def failAnalysis(): Nothing = { +val argTypes = u.functionArgs.map(_.dataType.typeName).mkString(", ") +u.failAnalysis( + s"""error: table-valued function ${u.functionName} with alternatives: + |${tvf.keys.map(_.toString).toSeq.sorted.map(x => s" ($x)").mkString("\n")} + |cannot be applied to: ($argTypes)""".stripMargin) + } + val resolved = tvf.flatMap { case (argList, resolver) => argList.implicitCast(u.functionArgs) match { case Some(casted) => -Some(resolver(casted.map(_.eval( +try { + Some(resolver(casted.map(_.eval( +} catch { + case e: AnalysisException => +failAnalysis() +} case _ => None } } resolved.headOption.getOrElse { -val argTypes = u.functionArgs.map(_.dataType.typeName).mkString(", ") -u.failAnalysis( - s"""error: table-valued function ${u.functionName} with alterna
spark git commit: [SPARK-24536] Validate that an evaluated limit clause cannot be null
Repository: spark Updated Branches: refs/heads/branch-2.3 25ea27b09 -> fc3df4517 [SPARK-24536] Validate that an evaluated limit clause cannot be null It proposes a version in which nullable expressions are not valid in the limit clause It was tested with unit and e2e tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Mauro Palsgraaf Closes #21807 from mauropalsgraaf/SPARK-24536. (cherry picked from commit 4ac2126bc64bad1b4cbe1c697b4bcafacd67c96c) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fc3df451 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fc3df451 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fc3df451 Branch: refs/heads/branch-2.3 Commit: fc3df45177d176cc0fe43049b6f8df372f7ea0e0 Parents: 25ea27b Author: Mauro Palsgraaf Authored: Tue Jul 31 08:18:08 2018 -0700 Committer: Xiao Li Committed: Tue Jul 31 08:22:25 2018 -0700 -- .../sql/catalyst/analysis/CheckAnalysis.scala | 14 +++--- .../catalyst/analysis/AnalysisErrorSuite.scala | 6 +++ .../test/resources/sql-tests/inputs/limit.sql | 5 +++ .../resources/sql-tests/results/limit.sql.out | 45 ++-- 4 files changed, 51 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fc3df451/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala -- 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 0d189b4..beb11d7 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 @@ -66,11 +66,15 @@ trait CheckAnalysis extends PredicateHelper { limitExpr.sql) case e if e.dataType != IntegerType => failAnalysis( s"The limit expression must be integer type, but got " + - e.dataType.simpleString) - case e if e.eval().asInstanceOf[Int] < 0 => failAnalysis( -"The limit expression must be equal to or greater than 0, but got " + - e.eval().asInstanceOf[Int]) - case e => // OK + e.dataType.catalogString) + case e => +e.eval() match { + case null => failAnalysis( +s"The evaluated limit expression must not be null, but got ${limitExpr.sql}") + case v: Int if v < 0 => failAnalysis( +s"The limit expression must be equal to or greater than 0, but got $v") + case _ => // OK +} } } http://git-wip-us.apache.org/repos/asf/spark/blob/fc3df451/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 5d2f8e7..70325b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -393,6 +393,12 @@ class AnalysisErrorSuite extends AnalysisTest { ) errorTest( +"an evaluated limit class must not be null", +testRelation.limit(Literal(null, IntegerType)), +"The evaluated limit expression must not be null, but got " :: Nil + ) + + errorTest( "num_rows in limit clause must be equal to or greater than 0", listRelation.limit(-1), "The limit expression must be equal to or greater than 0, but got -1" :: Nil http://git-wip-us.apache.org/repos/asf/spark/blob/fc3df451/sql/core/src/test/resources/sql-tests/inputs/limit.sql -- diff --git a/sql/core/src/test/resources/sql-tests/inputs/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/limit.sql index f21912a..b4c73cf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/limit.sql @@ -13,6 +13,11 @@ SELECT * FROM testdata LIMIT CAST(1 AS int); SELECT * FROM testdata LIMIT -1; SELECT * FROM testData TABLESAMPLE (-1 ROWS); + +SELECT * FROM testdata LIMIT CAST(1 AS INT); +-- evaluated limit must not be null +SELECT * FROM testdata LIMIT CAST(NULL AS INT); + -- limit must be foldable SELECT * FROM testdata LIMIT key > 3; http://git-wip-us.apache.org/repos/asf/spark/blob/fc3df451/sql/core/src/test/resources/sql-tests/results/li
spark git commit: [SPARK-24536] Validate that an evaluated limit clause cannot be null
Repository: spark Updated Branches: refs/heads/master b4fd75fb9 -> 4ac2126bc [SPARK-24536] Validate that an evaluated limit clause cannot be null ## What changes were proposed in this pull request? It proposes a version in which nullable expressions are not valid in the limit clause ## How was this patch tested? It was tested with unit and e2e tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Mauro Palsgraaf Closes #21807 from mauropalsgraaf/SPARK-24536. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4ac2126b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4ac2126b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4ac2126b Branch: refs/heads/master Commit: 4ac2126bc64bad1b4cbe1c697b4bcafacd67c96c Parents: b4fd75f Author: Mauro Palsgraaf Authored: Tue Jul 31 08:18:08 2018 -0700 Committer: Xiao Li Committed: Tue Jul 31 08:18:08 2018 -0700 -- .../sql/catalyst/analysis/CheckAnalysis.scala | 12 -- .../catalyst/analysis/AnalysisErrorSuite.scala | 6 +++ .../test/resources/sql-tests/inputs/limit.sql | 5 +++ .../resources/sql-tests/results/limit.sql.out | 45 ++-- 4 files changed, 50 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4ac2126b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala -- 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 f9478a1..4addc83 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 @@ -68,10 +68,14 @@ trait CheckAnalysis extends PredicateHelper { case e if e.dataType != IntegerType => failAnalysis( s"The limit expression must be integer type, but got " + e.dataType.catalogString) - case e if e.eval().asInstanceOf[Int] < 0 => failAnalysis( -"The limit expression must be equal to or greater than 0, but got " + - e.eval().asInstanceOf[Int]) - case e => // OK + case e => +e.eval() match { + case null => failAnalysis( +s"The evaluated limit expression must not be null, but got ${limitExpr.sql}") + case v: Int if v < 0 => failAnalysis( +s"The limit expression must be equal to or greater than 0, but got $v") + case _ => // OK +} } } http://git-wip-us.apache.org/repos/asf/spark/blob/4ac2126b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index f4cfed4..ae8d77b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -400,6 +400,12 @@ class AnalysisErrorSuite extends AnalysisTest { ) errorTest( +"an evaluated limit class must not be null", +testRelation.limit(Literal(null, IntegerType)), +"The evaluated limit expression must not be null, but got " :: Nil + ) + + errorTest( "num_rows in limit clause must be equal to or greater than 0", listRelation.limit(-1), "The limit expression must be equal to or greater than 0, but got -1" :: Nil http://git-wip-us.apache.org/repos/asf/spark/blob/4ac2126b/sql/core/src/test/resources/sql-tests/inputs/limit.sql -- diff --git a/sql/core/src/test/resources/sql-tests/inputs/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/limit.sql index f21912a..b4c73cf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/limit.sql @@ -13,6 +13,11 @@ SELECT * FROM testdata LIMIT CAST(1 AS int); SELECT * FROM testdata LIMIT -1; SELECT * FROM testData TABLESAMPLE (-1 ROWS); + +SELECT * FROM testdata LIMIT CAST(1 AS INT); +-- evaluated limit must not be null +SELECT * FROM testdata LIMIT CAST(NULL AS INT); + -- limit must be foldable SELECT * FROM testdata LIMIT key > 3; http://git-wip-us.apache.org/repos/asf/spark/blob/4ac2126b/sql/core/src/test/resources/sql-tests/results/limit.sql.out
spark git commit: [SPARK-24972][SQL] PivotFirst could not handle pivot columns of complex types
Repository: spark Updated Branches: refs/heads/master 8141d5592 -> b4fd75fb9 [SPARK-24972][SQL] PivotFirst could not handle pivot columns of complex types ## What changes were proposed in this pull request? When the pivot column is of a complex type, the eval() result will be an UnsafeRow, while the keys of the HashMap for column value matching is a GenericInternalRow. As a result, there will be no match and the result will always be empty. So for a pivot column of complex-types, we should: 1) If the complex-type is not comparable (orderable), throw an Exception. It cannot be a pivot column. 2) Otherwise, if it goes through the `PivotFirst` code path, `PivotFirst` should use a TreeMap instead of HashMap for such columns. This PR has also reverted the walk-around in Analyzer that had been introduced to avoid this `PivotFirst` issue. ## How was this patch tested? Added UT. Author: maryannxue Closes #21926 from maryannxue/pivot_followup. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b4fd75fb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b4fd75fb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b4fd75fb Branch: refs/heads/master Commit: b4fd75fb9b615cfe592ad269cf20d02b483a0d33 Parents: 8141d55 Author: maryannxue Authored: Mon Jul 30 23:43:53 2018 -0700 Committer: Xiao Li Committed: Mon Jul 30 23:43:53 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 12 +- .../expressions/aggregate/PivotFirst.scala | 11 +- .../test/resources/sql-tests/inputs/pivot.sql | 78 - .../resources/sql-tests/results/pivot.sql.out | 116 +-- 4 files changed, 199 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b4fd75fb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1488ede..76dc867 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 @@ -529,6 +529,10 @@ class Analyzer( || (p.groupByExprsOpt.isDefined && !p.groupByExprsOpt.get.forall(_.resolved)) || !p.pivotColumn.resolved || !p.pivotValues.forall(_.resolved) => p case Pivot(groupByExprsOpt, pivotColumn, pivotValues, aggregates, child) => +if (!RowOrdering.isOrderable(pivotColumn.dataType)) { + throw new AnalysisException( +s"Invalid pivot column '${pivotColumn}'. Pivot columns must be comparable.") +} // Check all aggregate expressions. aggregates.foreach(checkValidAggregateExpression) // Check all pivot values are literal and match pivot column data type. @@ -574,10 +578,14 @@ class Analyzer( // Since evaluating |pivotValues| if statements for each input row can get slow this is an // alternate plan that instead uses two steps of aggregation. val namedAggExps: Seq[NamedExpression] = aggregates.map(a => Alias(a, a.sql)()) - val bigGroup = groupByExprs ++ pivotColumn.references + val namedPivotCol = pivotColumn match { +case n: NamedExpression => n +case _ => Alias(pivotColumn, "__pivot_col")() + } + val bigGroup = groupByExprs :+ namedPivotCol val firstAgg = Aggregate(bigGroup, bigGroup ++ namedAggExps, child) val pivotAggs = namedAggExps.map { a => -Alias(PivotFirst(pivotColumn, a.toAttribute, evalPivotValues) +Alias(PivotFirst(namedPivotCol.toAttribute, a.toAttribute, evalPivotValues) .toAggregateExpression() , "__pivot_" + a.sql)() } http://git-wip-us.apache.org/repos/asf/spark/blob/b4fd75fb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala index 5237148..33bc5b5 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 @@ -17,11 +17,11 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import scala.collection.immutable.HashMap +import scala.collection.immutable.{
spark git commit: [SPARK-24865][SQL] Remove AnalysisBarrier addendum
Repository: spark Updated Branches: refs/heads/master d6b7545b5 -> abbb4ab4d [SPARK-24865][SQL] Remove AnalysisBarrier addendum ## What changes were proposed in this pull request? I didn't want to pollute the diff in the previous PR and left some TODOs. This is a follow-up to address those TODOs. ## How was this patch tested? Should be covered by existing tests. Author: Reynold Xin Closes #21896 from rxin/SPARK-24865-addendum. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/abbb4ab4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/abbb4ab4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/abbb4ab4 Branch: refs/heads/master Commit: abbb4ab4d8b12ba2d94b16407c0d62ae207ee4fa Parents: d6b7545 Author: Reynold Xin Authored: Mon Jul 30 14:05:45 2018 -0700 Committer: Xiao Li Committed: Mon Jul 30 14:05:45 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 1 - .../scala/org/apache/spark/sql/Dataset.scala| 88 ++-- .../spark/sql/RelationalGroupedDataset.scala| 13 ++- .../continuous/ContinuousAggregationSuite.scala | 4 +- 4 files changed, 50 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/abbb4ab4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9965cd6..1488ede 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 @@ -799,7 +799,6 @@ class Analyzer( right case Some((oldRelation, newRelation)) => val attributeRewrites = AttributeMap(oldRelation.output.zip(newRelation.output)) - // TODO(rxin): Why do we need transformUp here? right transformUp { case r if r == oldRelation => newRelation } transformUp { http://git-wip-us.apache.org/repos/asf/spark/blob/abbb4ab4/sql/core/src/main/scala/org/apache/spark/sql/Dataset.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 d36c8d1..3b0a6d8 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 @@ -195,10 +195,6 @@ class Dataset[T] private[sql]( } } - // Wraps analyzed logical plans with an analysis barrier so we won't traverse/resolve it again. - // TODO(rxin): remove this later. - @transient private[sql] val planWithBarrier = logicalPlan - /** * Currently [[ExpressionEncoder]] is the only implementation of [[Encoder]], here we turn the * passed in encoder to [[ExpressionEncoder]] explicitly, and mark it implicit so that we can use @@ -427,7 +423,7 @@ class Dataset[T] private[sql]( */ @Experimental @InterfaceStability.Evolving - def as[U : Encoder]: Dataset[U] = Dataset[U](sparkSession, planWithBarrier) + def as[U : Encoder]: Dataset[U] = Dataset[U](sparkSession, logicalPlan) /** * Converts this strongly typed collection of data to generic `DataFrame` with columns renamed. @@ -681,7 +677,7 @@ class Dataset[T] private[sql]( require(parsedDelay.milliseconds >= 0 && parsedDelay.months >= 0, s"delay threshold ($delayThreshold) should not be negative.") EliminateEventTimeWatermark( - EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, planWithBarrier)) + EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, logicalPlan)) } /** @@ -854,7 +850,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ def join(right: Dataset[_]): DataFrame = withPlan { -Join(planWithBarrier, right.planWithBarrier, joinType = Inner, None) +Join(logicalPlan, right.logicalPlan, joinType = Inner, None) } /** @@ -932,7 +928,7 @@ class Dataset[T] private[sql]( // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right // by creating a new instance for one of the branch. val joined = sparkSession.sessionState.executePlan( - Join(planWithBarrier, right.planWithBarrier, joinType = JoinType(joinType), None)) + Join(logicalPlan, right.logicalPlan, joinType = JoinType(joinType), None)) .analyzed.asInstanceOf[Join] withPlan { @@ -993,7 +989,7 @@ class Dataset[T] private[sql]( // Trigger analysis so in the case of self-join, the analyzer will clone the plan.
spark git commit: [SPARK-22814][SQL] Support Date/Timestamp in a JDBC partition column
Repository: spark Updated Branches: refs/heads/master b90bfe3c4 -> 47d84e4d0 [SPARK-22814][SQL] Support Date/Timestamp in a JDBC partition column ## What changes were proposed in this pull request? This pr supported Date/Timestamp in a JDBC partition column (a numeric column is only supported in the master). This pr also modified code to verify a partition column type; ``` val jdbcTable = spark.read .option("partitionColumn", "text") .option("lowerBound", "aaa") .option("upperBound", "zzz") .option("numPartitions", 2) .jdbc("jdbc:postgresql:postgres", "t", options) // with this pr org.apache.spark.sql.AnalysisException: Partition column type should be numeric, date, or timestamp, but string found.; at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.verifyAndGetNormalizedPartitionColumn(JDBCRelation.scala:165) at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.columnPartition(JDBCRelation.scala:85) at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:36) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:317) // without this pr java.lang.NumberFormatException: For input string: "aaa" at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) at java.lang.Long.parseLong(Long.java:589) at java.lang.Long.parseLong(Long.java:631) at scala.collection.immutable.StringLike$class.toLong(StringLike.scala:277) ``` Closes #1 ## How was this patch tested? Added tests in `JDBCSuite`. Author: Takeshi Yamamuro Closes #21834 from maropu/SPARK-22814. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/47d84e4d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/47d84e4d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/47d84e4d Branch: refs/heads/master Commit: 47d84e4d0e56e14f9402770dceaf0b4302c00e98 Parents: b90bfe3 Author: Takeshi Yamamuro Authored: Mon Jul 30 07:42:00 2018 -0700 Committer: Xiao Li Committed: Mon Jul 30 07:42:00 2018 -0700 -- docs/sql-programming-guide.md | 4 +- .../spark/sql/jdbc/OracleIntegrationSuite.scala | 86 +-- .../spark/sql/catalyst/util/DateTimeUtils.scala | 10 +- .../datasources/PartitioningUtils.scala | 2 +- .../datasources/jdbc/JDBCOptions.scala | 4 +- .../datasources/jdbc/JDBCRelation.scala | 107 +++ .../datasources/jdbc/JdbcRelationProvider.scala | 21 +--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 77 - 8 files changed, 258 insertions(+), 53 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/47d84e4d/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4b013c6..cff521c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1345,8 +1345,8 @@ the following case-insensitive options: These options must all be specified if any of them is specified. In addition, numPartitions must be specified. They describe how to partition the table when reading in parallel from multiple workers. - partitionColumn must be a numeric column from the table in question. Notice - that lowerBound and upperBound are just used to decide the + partitionColumn must be a numeric, date, or timestamp 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. This option applies only to reading. http://git-wip-us.apache.org/repos/asf/spark/blob/47d84e4d/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala -- diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 8512496..09a2cd8 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.jdbc +import java.math.BigDecimal import java.sql.{Connection, Date, Timestamp} import java.util.{Properties, TimeZone} -import java.math.BigDecimal -import org.apache.spark.sql.{DataFrame, QueryTest, Row, SaveMode} +import org.apache.spark.sql.{Row, SaveMode} import org.
spark git commit: [SPARK-24771][BUILD] Upgrade Apache AVRO to 1.8.2
Repository: spark Updated Branches: refs/heads/master fca0b8528 -> b90bfe3c4 [SPARK-24771][BUILD] Upgrade Apache AVRO to 1.8.2 ## What changes were proposed in this pull request? Upgrade Apache Avro from 1.7.7 to 1.8.2. The major new features: 1. More logical types. From the spec of 1.8.2 https://avro.apache.org/docs/1.8.2/spec.html#Logical+Types we can see comparing to [1.7.7](https://avro.apache.org/docs/1.7.7/spec.html#Logical+Types), the new version support: - Date - Time (millisecond precision) - Time (microsecond precision) - Timestamp (millisecond precision) - Timestamp (microsecond precision) - Duration 2. Single-object encoding: https://avro.apache.org/docs/1.8.2/spec.html#single_object_encoding This PR aims to update Apache Spark to support these new features. ## How was this patch tested? Unit test Author: Gengliang Wang Closes #21761 from gengliangwang/upgrade_avro_1.8. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b90bfe3c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b90bfe3c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b90bfe3c Branch: refs/heads/master Commit: b90bfe3c42eb9b51e6131a8f8923bcddfccd75bb Parents: fca0b85 Author: Gengliang Wang Authored: Mon Jul 30 07:30:47 2018 -0700 Committer: Xiao Li Committed: Mon Jul 30 07:30:47 2018 -0700 -- dev/deps/spark-deps-hadoop-2.6 | 10 +- dev/deps/spark-deps-hadoop-2.7 | 10 +- dev/deps/spark-deps-hadoop-3.1 | 10 +- pom.xml| 2 +- sql/core/pom.xml | 13 - 5 files changed, 16 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b90bfe3c/dev/deps/spark-deps-hadoop-2.6 -- diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index ff6d5c3..4ef61b2 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -18,9 +18,9 @@ arrow-format-0.8.0.jar arrow-memory-0.8.0.jar arrow-vector-0.8.0.jar automaton-1.11-8.jar -avro-1.7.7.jar -avro-ipc-1.7.7.jar -avro-mapred-1.7.7-hadoop2.jar +avro-1.8.2.jar +avro-ipc-1.8.2.jar +avro-mapred-1.8.2-hadoop2.jar base64-2.3.8.jar bcprov-jdk15on-1.58.jar bonecp-0.8.0.RELEASE.jar @@ -37,7 +37,7 @@ commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar commons-compiler-3.0.8.jar -commons-compress-1.4.1.jar +commons-compress-1.8.1.jar commons-configuration-1.6.jar commons-crypto-1.0.0.jar commons-dbcp-1.4.jar @@ -196,7 +196,7 @@ validation-api-1.1.0.Final.jar xbean-asm6-shaded-4.8.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar -xz-1.0.jar +xz-1.5.jar zjsonpatch-0.3.0.jar zookeeper-3.4.6.jar zstd-jni-1.3.2-2.jar http://git-wip-us.apache.org/repos/asf/spark/blob/b90bfe3c/dev/deps/spark-deps-hadoop-2.7 -- diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 72a94f8..a74ce1f 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -18,9 +18,9 @@ arrow-format-0.8.0.jar arrow-memory-0.8.0.jar arrow-vector-0.8.0.jar automaton-1.11-8.jar -avro-1.7.7.jar -avro-ipc-1.7.7.jar -avro-mapred-1.7.7-hadoop2.jar +avro-1.8.2.jar +avro-ipc-1.8.2.jar +avro-mapred-1.8.2-hadoop2.jar base64-2.3.8.jar bcprov-jdk15on-1.58.jar bonecp-0.8.0.RELEASE.jar @@ -37,7 +37,7 @@ commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar commons-compiler-3.0.8.jar -commons-compress-1.4.1.jar +commons-compress-1.8.1.jar commons-configuration-1.6.jar commons-crypto-1.0.0.jar commons-dbcp-1.4.jar @@ -197,7 +197,7 @@ validation-api-1.1.0.Final.jar xbean-asm6-shaded-4.8.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar -xz-1.0.jar +xz-1.5.jar zjsonpatch-0.3.0.jar zookeeper-3.4.6.jar zstd-jni-1.3.2-2.jar http://git-wip-us.apache.org/repos/asf/spark/blob/b90bfe3c/dev/deps/spark-deps-hadoop-3.1 -- diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 3409dc4..e0fcca0 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -16,9 +16,9 @@ arrow-format-0.8.0.jar arrow-memory-0.8.0.jar arrow-vector-0.8.0.jar automaton-1.11-8.jar -avro-1.7.7.jar -avro-ipc-1.7.7.jar -avro-mapred-1.7.7-hadoop2.jar +avro-1.8.2.jar +avro-ipc-1.8.2.jar +avro-mapred-1.8.2-hadoop2.jar base64-2.3.8.jar bcprov-jdk15on-1.58.jar bonecp-0.8.0.RELEASE.jar @@ -34,7 +34,7 @@ commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar commons-compiler-3.0.8.jar -commons-compress-1.4.1.jar +commons-compress-1.8.1.jar commons-configuration2-2.1.1.jar commons-crypto-1.0.0.jar commons-daemon-1.0
spark git commit: [SPARK-21274][SQL] Implement INTERSECT ALL clause
Repository: spark Updated Branches: refs/heads/master 6690924c4 -> 65a4bc143 [SPARK-21274][SQL] Implement INTERSECT ALL clause ## What changes were proposed in this pull request? Implements INTERSECT ALL clause through query rewrites using existing operators in Spark. Please refer to [Link](https://drive.google.com/open?id=1nyW0T0b_ajUduQoPgZLAsyHK8s3_dko3ulQuxaLpUXE) for the design. Input Query ``` SQL SELECT c1 FROM ut1 INTERSECT ALL SELECT c1 FROM ut2 ``` Rewritten Query ```SQL SELECT c1 FROM ( SELECT replicate_row(min_count, c1) FROM ( SELECT c1, IF (vcol1_cnt > vcol2_cnt, vcol2_cnt, vcol1_cnt) AS min_count FROM ( SELECT c1, count(vcol1) as vcol1_cnt, count(vcol2) as vcol2_cnt FROM ( SELECT c1, true as vcol1, null as vcol2 FROM ut1 UNION ALL SELECT c1, null as vcol1, true as vcol2 FROM ut2 ) AS union_all GROUP BY c1 HAVING vcol1_cnt >= 1 AND vcol2_cnt >= 1 ) ) ) ``` ## How was this patch tested? Added test cases in SQLQueryTestSuite, DataFrameSuite, SetOperationSuite Author: Dilip Biswal Closes #21886 from dilipbiswal/dkb_intersect_all_final. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/65a4bc14 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/65a4bc14 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/65a4bc14 Branch: refs/heads/master Commit: 65a4bc143ab5dc2ced589dc107bbafa8a7290931 Parents: 6690924 Author: Dilip Biswal Authored: Sun Jul 29 22:11:01 2018 -0700 Committer: Xiao Li Committed: Sun Jul 29 22:11:01 2018 -0700 -- python/pyspark/sql/dataframe.py | 22 ++ .../spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/analysis/TypeCoercion.scala| 4 +- .../analysis/UnsupportedOperationChecker.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 81 ++- .../spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 7 +- .../catalyst/optimizer/SetOperationSuite.scala | 32 ++- .../sql/catalyst/parser/PlanParserSuite.scala | 1 - .../scala/org/apache/spark/sql/Dataset.scala| 19 +- .../spark/sql/execution/SparkStrategies.scala | 8 +- .../sql-tests/inputs/intersect-all.sql | 123 ++ .../sql-tests/results/intersect-all.sql.out | 241 +++ .../org/apache/spark/sql/DataFrameSuite.scala | 54 + .../org/apache/spark/sql/test/SQLTestData.scala | 13 + 15 files changed, 599 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/65a4bc14/python/pyspark/sql/dataframe.py -- diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index b2e0a5b..07fb260 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1500,6 +1500,28 @@ class DataFrame(object): """ return DataFrame(self._jdf.intersect(other._jdf), self.sql_ctx) +@since(2.4) +def intersectAll(self, other): +""" Return a new :class:`DataFrame` containing rows in both this dataframe and other +dataframe while preserving duplicates. + +This is equivalent to `INTERSECT ALL` in SQL. +>>> df1 = spark.createDataFrame([("a", 1), ("a", 1), ("b", 3), ("c", 4)], ["C1", "C2"]) +>>> df2 = spark.createDataFrame([("a", 1), ("a", 1), ("b", 3)], ["C1", "C2"]) + +>>> df1.intersectAll(df2).sort("C1", "C2").show() ++---+---+ +| C1| C2| ++---+---+ +| a| 1| +| a| 1| +| b| 3| ++---+---+ + +Also as standard in SQL, this function resolves columns by position (not by name). +""" +return DataFrame(self._jdf.intersectAll(other._jdf), self.sql_ctx) + @since(1.3) def subtract(self, other): """ Return a new :class:`DataFrame` containing rows in this frame http://git-wip-us.apache.org/repos/asf/spark/blob/65a4bc14/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8abb1c7..9965cd6 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 @@ -914,7 +914,7 @@ cl
spark git commit: [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error
Repository: spark Updated Branches: refs/heads/branch-2.1 7d50fec3f -> a3eb07db3 [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error When join key is long or int in broadcast join, Spark will use `LongToUnsafeRowMap` to store key-values of the table witch will be broadcasted. But, when `LongToUnsafeRowMap` is broadcasted to executors, and it is too big to hold in memory, it will be stored in disk. At that time, because `write` uses a variable `cursor` to determine how many bytes in `page` of `LongToUnsafeRowMap` will be write out and the `cursor` was not restore when deserializing, executor will write out nothing from page into disk. ## What changes were proposed in this pull request? Restore cursor value when deserializing. Author: liulijia Closes #21772 from liutang123/SPARK-24809. (cherry picked from commit 2c54aae1bc2fa3da26917c89e6201fb2108d9fab) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a3eb07db Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a3eb07db Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a3eb07db Branch: refs/heads/branch-2.1 Commit: a3eb07db3be80be663ca66f1e9a11fcef8ab6c20 Parents: 7d50fec Author: liulijia Authored: Sun Jul 29 13:13:00 2018 -0700 Committer: Xiao Li Committed: Sun Jul 29 13:14:57 2018 -0700 -- .../sql/execution/joins/HashedRelation.scala| 2 ++ .../execution/joins/HashedRelationSuite.scala | 29 2 files changed, 31 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a3eb07db/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index f7e8ea6..206afcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -741,6 +741,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap array = readLongArray(readBuffer, length) val pageLength = readLong().toInt page = readLongArray(readBuffer, pageLength) +// Restore cursor variable to make this map able to be serialized again on executors. +cursor = pageLength * 8 + Platform.LONG_ARRAY_OFFSET } override def readExternal(in: ObjectInput): Unit = { http://git-wip-us.apache.org/repos/asf/spark/blob/a3eb07db/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index f0288c8..9c9e9dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -277,6 +277,35 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { map.free() } + test("SPARK-24809: Serializing LongToUnsafeRowMap in executor may result in data error") { +val unsafeProj = UnsafeProjection.create(Array[DataType](LongType)) +val originalMap = new LongToUnsafeRowMap(mm, 1) + +val key1 = 1L +val value1 = 4852306286022334418L + +val key2 = 2L +val value2 = 8813607448788216010L + +originalMap.append(key1, unsafeProj(InternalRow(value1))) +originalMap.append(key2, unsafeProj(InternalRow(value2))) +originalMap.optimize() + +val ser = sparkContext.env.serializer.newInstance() +// Simulate serialize/deserialize twice on driver and executor +val firstTimeSerialized = ser.deserialize[LongToUnsafeRowMap](ser.serialize(originalMap)) +val secondTimeSerialized = + ser.deserialize[LongToUnsafeRowMap](ser.serialize(firstTimeSerialized)) + +val resultRow = new UnsafeRow(1) +assert(secondTimeSerialized.getValue(key1, resultRow).getLong(0) === value1) +assert(secondTimeSerialized.getValue(key2, resultRow).getLong(0) === value2) + +originalMap.free() +firstTimeSerialized.free() +secondTimeSerialized.free() + } + test("Spark-14521") { val ser = new KryoSerializer( (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.
spark git commit: [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error
Repository: spark Updated Branches: refs/heads/branch-2.3 d5f340f27 -> 71eb7d468 [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error When join key is long or int in broadcast join, Spark will use `LongToUnsafeRowMap` to store key-values of the table witch will be broadcasted. But, when `LongToUnsafeRowMap` is broadcasted to executors, and it is too big to hold in memory, it will be stored in disk. At that time, because `write` uses a variable `cursor` to determine how many bytes in `page` of `LongToUnsafeRowMap` will be write out and the `cursor` was not restore when deserializing, executor will write out nothing from page into disk. ## What changes were proposed in this pull request? Restore cursor value when deserializing. Author: liulijia Closes #21772 from liutang123/SPARK-24809. (cherry picked from commit 2c54aae1bc2fa3da26917c89e6201fb2108d9fab) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/71eb7d46 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/71eb7d46 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/71eb7d46 Branch: refs/heads/branch-2.3 Commit: 71eb7d4682a7e85e4de580ffe110da961d84817f Parents: d5f340f Author: liulijia Authored: Sun Jul 29 13:13:00 2018 -0700 Committer: Xiao Li Committed: Sun Jul 29 13:13:22 2018 -0700 -- .../sql/execution/joins/HashedRelation.scala| 2 ++ .../execution/joins/HashedRelationSuite.scala | 29 2 files changed, 31 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/71eb7d46/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 20ce01f..86eb47a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -772,6 +772,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap array = readLongArray(readBuffer, length) val pageLength = readLong().toInt page = readLongArray(readBuffer, pageLength) +// Restore cursor variable to make this map able to be serialized again on executors. +cursor = pageLength * 8 + Platform.LONG_ARRAY_OFFSET } override def readExternal(in: ObjectInput): Unit = { http://git-wip-us.apache.org/repos/asf/spark/blob/71eb7d46/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 037cc2e..d9b34dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -278,6 +278,35 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { map.free() } + test("SPARK-24809: Serializing LongToUnsafeRowMap in executor may result in data error") { +val unsafeProj = UnsafeProjection.create(Array[DataType](LongType)) +val originalMap = new LongToUnsafeRowMap(mm, 1) + +val key1 = 1L +val value1 = 4852306286022334418L + +val key2 = 2L +val value2 = 8813607448788216010L + +originalMap.append(key1, unsafeProj(InternalRow(value1))) +originalMap.append(key2, unsafeProj(InternalRow(value2))) +originalMap.optimize() + +val ser = sparkContext.env.serializer.newInstance() +// Simulate serialize/deserialize twice on driver and executor +val firstTimeSerialized = ser.deserialize[LongToUnsafeRowMap](ser.serialize(originalMap)) +val secondTimeSerialized = + ser.deserialize[LongToUnsafeRowMap](ser.serialize(firstTimeSerialized)) + +val resultRow = new UnsafeRow(1) +assert(secondTimeSerialized.getValue(key1, resultRow).getLong(0) === value1) +assert(secondTimeSerialized.getValue(key2, resultRow).getLong(0) === value2) + +originalMap.free() +firstTimeSerialized.free() +secondTimeSerialized.free() + } + test("Spark-14521") { val ser = new KryoSerializer( (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.
spark git commit: [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error
Repository: spark Updated Branches: refs/heads/branch-2.2 73764737d -> f52d0c451 [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error When join key is long or int in broadcast join, Spark will use `LongToUnsafeRowMap` to store key-values of the table witch will be broadcasted. But, when `LongToUnsafeRowMap` is broadcasted to executors, and it is too big to hold in memory, it will be stored in disk. At that time, because `write` uses a variable `cursor` to determine how many bytes in `page` of `LongToUnsafeRowMap` will be write out and the `cursor` was not restore when deserializing, executor will write out nothing from page into disk. ## What changes were proposed in this pull request? Restore cursor value when deserializing. Author: liulijia Closes #21772 from liutang123/SPARK-24809. (cherry picked from commit 2c54aae1bc2fa3da26917c89e6201fb2108d9fab) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f52d0c45 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f52d0c45 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f52d0c45 Branch: refs/heads/branch-2.2 Commit: f52d0c4515f3f0ceaea03c661fb7739c70c25236 Parents: 7376473 Author: liulijia Authored: Sun Jul 29 13:13:00 2018 -0700 Committer: Xiao Li Committed: Sun Jul 29 13:13:57 2018 -0700 -- .../sql/execution/joins/HashedRelation.scala| 2 ++ .../execution/joins/HashedRelationSuite.scala | 29 2 files changed, 31 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f52d0c45/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 07ee3d0..78190bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -741,6 +741,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap array = readLongArray(readBuffer, length) val pageLength = readLong().toInt page = readLongArray(readBuffer, pageLength) +// Restore cursor variable to make this map able to be serialized again on executors. +cursor = pageLength * 8 + Platform.LONG_ARRAY_OFFSET } override def readExternal(in: ObjectInput): Unit = { http://git-wip-us.apache.org/repos/asf/spark/blob/f52d0c45/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index f0288c8..9c9e9dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -277,6 +277,35 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { map.free() } + test("SPARK-24809: Serializing LongToUnsafeRowMap in executor may result in data error") { +val unsafeProj = UnsafeProjection.create(Array[DataType](LongType)) +val originalMap = new LongToUnsafeRowMap(mm, 1) + +val key1 = 1L +val value1 = 4852306286022334418L + +val key2 = 2L +val value2 = 8813607448788216010L + +originalMap.append(key1, unsafeProj(InternalRow(value1))) +originalMap.append(key2, unsafeProj(InternalRow(value2))) +originalMap.optimize() + +val ser = sparkContext.env.serializer.newInstance() +// Simulate serialize/deserialize twice on driver and executor +val firstTimeSerialized = ser.deserialize[LongToUnsafeRowMap](ser.serialize(originalMap)) +val secondTimeSerialized = + ser.deserialize[LongToUnsafeRowMap](ser.serialize(firstTimeSerialized)) + +val resultRow = new UnsafeRow(1) +assert(secondTimeSerialized.getValue(key1, resultRow).getLong(0) === value1) +assert(secondTimeSerialized.getValue(key2, resultRow).getLong(0) === value2) + +originalMap.free() +firstTimeSerialized.free() +secondTimeSerialized.free() + } + test("Spark-14521") { val ser = new KryoSerializer( (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.
spark git commit: [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error
Repository: spark Updated Branches: refs/heads/master 8fe5d2c39 -> 2c54aae1b [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error When join key is long or int in broadcast join, Spark will use `LongToUnsafeRowMap` to store key-values of the table witch will be broadcasted. But, when `LongToUnsafeRowMap` is broadcasted to executors, and it is too big to hold in memory, it will be stored in disk. At that time, because `write` uses a variable `cursor` to determine how many bytes in `page` of `LongToUnsafeRowMap` will be write out and the `cursor` was not restore when deserializing, executor will write out nothing from page into disk. ## What changes were proposed in this pull request? Restore cursor value when deserializing. Author: liulijia Closes #21772 from liutang123/SPARK-24809. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2c54aae1 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2c54aae1 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2c54aae1 Branch: refs/heads/master Commit: 2c54aae1bc2fa3da26917c89e6201fb2108d9fab Parents: 8fe5d2c Author: liulijia Authored: Sun Jul 29 13:13:00 2018 -0700 Committer: Xiao Li Committed: Sun Jul 29 13:13:00 2018 -0700 -- .../sql/execution/joins/HashedRelation.scala| 2 ++ .../execution/joins/HashedRelationSuite.scala | 29 2 files changed, 31 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2c54aae1/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 20ce01f..86eb47a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -772,6 +772,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap array = readLongArray(readBuffer, length) val pageLength = readLong().toInt page = readLongArray(readBuffer, pageLength) +// Restore cursor variable to make this map able to be serialized again on executors. +cursor = pageLength * 8 + Platform.LONG_ARRAY_OFFSET } override def readExternal(in: ObjectInput): Unit = { http://git-wip-us.apache.org/repos/asf/spark/blob/2c54aae1/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 037cc2e..d9b34dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -278,6 +278,35 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { map.free() } + test("SPARK-24809: Serializing LongToUnsafeRowMap in executor may result in data error") { +val unsafeProj = UnsafeProjection.create(Array[DataType](LongType)) +val originalMap = new LongToUnsafeRowMap(mm, 1) + +val key1 = 1L +val value1 = 4852306286022334418L + +val key2 = 2L +val value2 = 8813607448788216010L + +originalMap.append(key1, unsafeProj(InternalRow(value1))) +originalMap.append(key2, unsafeProj(InternalRow(value2))) +originalMap.optimize() + +val ser = sparkContext.env.serializer.newInstance() +// Simulate serialize/deserialize twice on driver and executor +val firstTimeSerialized = ser.deserialize[LongToUnsafeRowMap](ser.serialize(originalMap)) +val secondTimeSerialized = + ser.deserialize[LongToUnsafeRowMap](ser.serialize(firstTimeSerialized)) + +val resultRow = new UnsafeRow(1) +assert(secondTimeSerialized.getValue(key1, resultRow).getLong(0) === value1) +assert(secondTimeSerialized.getValue(key2, resultRow).getLong(0) === value2) + +originalMap.free() +firstTimeSerialized.free() +secondTimeSerialized.free() + } + test("Spark-14521") { val ser = new KryoSerializer( (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR] Update docs for functions.scala to make it clear not all the built-in functions are defined there
Repository: spark Updated Branches: refs/heads/master 34ebcc6b5 -> 6424b146c [MINOR] Update docs for functions.scala to make it clear not all the built-in functions are defined there The title summarizes the change. Author: Reynold Xin Closes #21318 from rxin/functions. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6424b146 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6424b146 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6424b146 Branch: refs/heads/master Commit: 6424b146c91fdca734a3ec972067e8e1f88e8b9e Parents: 34ebcc6 Author: Reynold Xin Authored: Fri Jul 27 17:24:55 2018 -0700 Committer: Xiao Li Committed: Fri Jul 27 17:24:55 2018 -0700 -- .../main/scala/org/apache/spark/sql/functions.scala | 16 +++- 1 file changed, 15 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6424b146/sql/core/src/main/scala/org/apache/spark/sql/functions.scala -- 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 bcd0c94..2772958 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 @@ -39,7 +39,21 @@ import org.apache.spark.util.Utils /** - * Functions available for DataFrame operations. + * Commonly used functions available for DataFrame operations. Using functions defined here provides + * a little bit more compile-time safety to make sure the function exists. + * + * Spark also includes more built-in functions that are less common and are not defined here. + * You can still access them (and all the functions defined here) using the `functions.expr()` API + * and calling them through a SQL expression string. You can find the entire list of functions for + * the latest version of Spark at https://spark.apache.org/docs/latest/api/sql/index.html. + * + * As an example, `isnan` is a function that is defined here. You can use `isnan(col("myCol"))` + * to invoke the `isnan` function. This way the programming language's compiler ensures `isnan` + * exists and is of the proper form. You can also use `expr("isnan(myCol)")` function to invoke the + * same function. In this case, Spark itself will ensure `isnan` exists when it analyzes the query. + * + * `regr_count` is an example of a function that is built-in but not defined here, because it is + * less commonly used. To invoke it, use `expr("regr_count(yCol, xCol)")`. * * @groupname udf_funcs UDF functions * @groupname agg_funcs Aggregate functions - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MINOR] Improve documentation for HiveStringType's
Repository: spark Updated Branches: refs/heads/master 10f1f1965 -> 34ebcc6b5 [MINOR] Improve documentation for HiveStringType's The diff should be self-explanatory. Author: Reynold Xin Closes #21897 from rxin/hivestringtypedoc. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/34ebcc6b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/34ebcc6b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/34ebcc6b Branch: refs/heads/master Commit: 34ebcc6b5246c1a47e6d3b2dbb23e368de25219e Parents: 10f1f19 Author: Reynold Xin Authored: Fri Jul 27 15:34:06 2018 -0700 Committer: Xiao Li Committed: Fri Jul 27 15:34:06 2018 -0700 -- .../scala/org/apache/spark/sql/types/HiveStringType.scala| 8 ++-- 1 file changed, 6 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/34ebcc6b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala index e0bca93..4eb3226 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala @@ -56,14 +56,18 @@ object HiveStringType { } /** - * Hive char type. + * Hive char type. Similar to other HiveStringType's, these datatypes should only used for + * parsing, and should NOT be used anywhere else. Any instance of these data types should be + * replaced by a [[StringType]] before analysis. */ case class CharType(length: Int) extends HiveStringType { override def simpleString: String = s"char($length)" } /** - * Hive varchar type. + * Hive varchar type. Similar to other HiveStringType's, these datatypes should only used for + * parsing, and should NOT be used anywhere else. Any instance of these data types should be + * replaced by a [[StringType]] before analysis. */ case class VarcharType(length: Int) extends HiveStringType { override def simpleString: String = s"varchar($length)" - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-21274][SQL] Implement EXCEPT ALL clause.
Repository: spark Updated Branches: refs/heads/master 5828f41a5 -> 10f1f1965 [SPARK-21274][SQL] Implement EXCEPT ALL clause. ## What changes were proposed in this pull request? Implements EXCEPT ALL clause through query rewrites using existing operators in Spark. In this PR, an internal UDTF (replicate_rows) is added to aid in preserving duplicate rows. Please refer to [Link](https://drive.google.com/open?id=1nyW0T0b_ajUduQoPgZLAsyHK8s3_dko3ulQuxaLpUXE) for the design. **Note** This proposed UDTF is kept as a internal function that is purely used to aid with this particular rewrite to give us flexibility to change to a more generalized UDTF in future. Input Query ``` SQL SELECT c1 FROM ut1 EXCEPT ALL SELECT c1 FROM ut2 ``` Rewritten Query ```SQL SELECT c1 FROM ( SELECT replicate_rows(sum_val, c1) FROM ( SELECT c1, sum_val FROM ( SELECT c1, sum(vcol) AS sum_val FROM ( SELECT 1L as vcol, c1 FROM ut1 UNION ALL SELECT -1L as vcol, c1 FROM ut2 ) AS union_all GROUP BY union_all.c1 ) WHERE sum_val > 0 ) ) ``` ## How was this patch tested? Added test cases in SQLQueryTestSuite, DataFrameSuite and SetOperationSuite Author: Dilip Biswal Closes #21857 from dilipbiswal/dkb_except_all_final. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/10f1f196 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/10f1f196 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/10f1f196 Branch: refs/heads/master Commit: 10f1f196595df66cb82d1fb9e27cc7ef0a176766 Parents: 5828f41 Author: Dilip Biswal Authored: Fri Jul 27 13:47:33 2018 -0700 Committer: Xiao Li Committed: Fri Jul 27 13:47:33 2018 -0700 -- python/pyspark/sql/dataframe.py | 25 ++ .../spark/sql/catalyst/analysis/Analyzer.scala | 5 +- .../sql/catalyst/analysis/TypeCoercion.scala| 12 +- .../analysis/UnsupportedOperationChecker.scala | 2 +- .../sql/catalyst/expressions/generators.scala | 26 ++ .../sql/catalyst/optimizer/Optimizer.scala | 61 +++- .../optimizer/ReplaceExceptWithFilter.scala | 2 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 7 +- .../catalyst/optimizer/SetOperationSuite.scala | 24 +- .../sql/catalyst/parser/ErrorParserSuite.scala | 3 - .../sql/catalyst/parser/PlanParserSuite.scala | 1 - .../scala/org/apache/spark/sql/Dataset.scala| 16 + .../spark/sql/execution/SparkStrategies.scala | 6 +- .../resources/sql-tests/inputs/except-all.sql | 146 + .../sql-tests/results/except-all.sql.out| 319 +++ .../org/apache/spark/sql/DataFrameSuite.scala | 70 +++- 17 files changed, 708 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/10f1f196/python/pyspark/sql/dataframe.py -- diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c40aea9..b2e0a5b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -293,6 +293,31 @@ class DataFrame(object): else: print(self._jdf.queryExecution().simpleString()) +@since(2.4) +def exceptAll(self, other): +"""Return a new :class:`DataFrame` containing rows in this :class:`DataFrame` but +not in another :class:`DataFrame` while preserving duplicates. + +This is equivalent to `EXCEPT ALL` in SQL. + +>>> df1 = spark.createDataFrame( +... [("a", 1), ("a", 1), ("a", 1), ("a", 2), ("b", 3), ("c", 4)], ["C1", "C2"]) +>>> df2 = spark.createDataFrame([("a", 1), ("b", 3)], ["C1", "C2"]) + +>>> df1.exceptAll(df2).show() ++---+---+ +| C1| C2| ++---+---+ +| a| 1| +| a| 1| +| a| 2| +| c| 4| ++---+---+ + +Also as standard in SQL, this function resolves columns by position (not by name). +""" +return DataFrame(self._jdf.exceptAll(other._jdf), self.sql_ctx) + @since(1.3) def isLocal(self): """Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally http://git-wip-us.apache.org/repos/asf/spark/blob/10f1f196/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d18509f..8abb1c7 100644 --- a/sql/catalyst/src/main
spark git commit: [SPARK-24927][BUILD][BRANCH-2.3] The scope of snappy-java cannot be "provided"
Repository: spark Updated Branches: refs/heads/master ef6c8395c -> c9bec1d37 [SPARK-24927][BUILD][BRANCH-2.3] The scope of snappy-java cannot be "provided" ## What changes were proposed in this pull request? Please see [SPARK-24927][1] for more details. [1]: https://issues.apache.org/jira/browse/SPARK-24927 ## How was this patch tested? Manually tested. Author: Cheng Lian Closes #21879 from liancheng/spark-24927. (cherry picked from commit d5f340f27706bd9767f23ac9726f904028916814) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c9bec1d3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c9bec1d3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c9bec1d3 Branch: refs/heads/master Commit: c9bec1d3717e5aeb6d3ec95d4c78111bfc33e0ca Parents: ef6c839 Author: Cheng Lian Authored: Fri Jul 27 08:57:48 2018 -0700 Committer: Xiao Li Committed: Fri Jul 27 08:58:42 2018 -0700 -- pom.xml | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c9bec1d3/pom.xml -- diff --git a/pom.xml b/pom.xml index d75db0f..f320844 100644 --- a/pom.xml +++ b/pom.xml @@ -538,7 +538,6 @@ org.xerial.snappy snappy-java ${snappy.version} -${hadoop.deps.scope} org.lz4 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24927][BUILD][BRANCH-2.3] The scope of snappy-java cannot be "provided"
Repository: spark Updated Branches: refs/heads/branch-2.2 f339e2fd7 -> 73764737d [SPARK-24927][BUILD][BRANCH-2.3] The scope of snappy-java cannot be "provided" ## What changes were proposed in this pull request? Please see [SPARK-24927][1] for more details. [1]: https://issues.apache.org/jira/browse/SPARK-24927 ## How was this patch tested? Manually tested. Author: Cheng Lian Closes #21879 from liancheng/spark-24927. (cherry picked from commit d5f340f27706bd9767f23ac9726f904028916814) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/73764737 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/73764737 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/73764737 Branch: refs/heads/branch-2.2 Commit: 73764737d2230003d57210a68a87c5977bef6782 Parents: f339e2f Author: Cheng Lian Authored: Fri Jul 27 08:57:48 2018 -0700 Committer: Xiao Li Committed: Fri Jul 27 08:58:23 2018 -0700 -- pom.xml | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/73764737/pom.xml -- diff --git a/pom.xml b/pom.xml index cf98296..6ab9989 100644 --- a/pom.xml +++ b/pom.xml @@ -510,7 +510,6 @@ org.xerial.snappy snappy-java ${snappy.version} -${hadoop.deps.scope} net.jpountz.lz4 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24927][BUILD][BRANCH-2.3] The scope of snappy-java cannot be "provided"
Repository: spark Updated Branches: refs/heads/branch-2.3 fa552c3c1 -> d5f340f27 [SPARK-24927][BUILD][BRANCH-2.3] The scope of snappy-java cannot be "provided" ## What changes were proposed in this pull request? Please see [SPARK-24927][1] for more details. [1]: https://issues.apache.org/jira/browse/SPARK-24927 ## How was this patch tested? Manually tested. Author: Cheng Lian Closes #21879 from liancheng/spark-24927. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d5f340f2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d5f340f2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d5f340f2 Branch: refs/heads/branch-2.3 Commit: d5f340f27706bd9767f23ac9726f904028916814 Parents: fa552c3 Author: Cheng Lian Authored: Fri Jul 27 08:57:48 2018 -0700 Committer: Xiao Li Committed: Fri Jul 27 08:57:48 2018 -0700 -- pom.xml | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d5f340f2/pom.xml -- diff --git a/pom.xml b/pom.xml index ea6f27c..179e9d2 100644 --- a/pom.xml +++ b/pom.xml @@ -536,7 +536,6 @@ org.xerial.snappy snappy-java ${snappy.version} -${hadoop.deps.scope} org.lz4 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24288][SQL] Add a JDBC Option to enable preventing predicate pushdown
Repository: spark Updated Branches: refs/heads/master e6e9031d7 -> 21fcac164 [SPARK-24288][SQL] Add a JDBC Option to enable preventing predicate pushdown ## What changes were proposed in this pull request? Add a JDBC Option "pushDownPredicate" (default `true`) to allow/disallow predicate push-down in JDBC data source. ## How was this patch tested? Add a test in `JDBCSuite` Author: maryannxue Closes #21875 from maryannxue/spark-24288. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/21fcac16 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/21fcac16 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/21fcac16 Branch: refs/heads/master Commit: 21fcac1645bf01c453ddd4cb64c566895e66ea4f Parents: e6e9031 Author: maryannxue Authored: Thu Jul 26 23:47:32 2018 -0700 Committer: Xiao Li Committed: Thu Jul 26 23:47:32 2018 -0700 -- docs/sql-programming-guide.md | 7 +++ .../datasources/jdbc/JDBCOptions.scala | 4 ++ .../datasources/jdbc/JDBCRelation.scala | 6 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 66 ++-- 4 files changed, 63 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/21fcac16/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e815e5b..4b013c6 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1435,6 +1435,13 @@ the following case-insensitive options: The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. + + +pushDownPredicate + + The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. + + http://git-wip-us.apache.org/repos/asf/spark/blob/21fcac16/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala -- 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 574aed4..d80efce 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 @@ -183,6 +183,9 @@ class JDBCOptions( } // An option to execute custom SQL before fetching data from the remote DB val sessionInitStatement = parameters.get(JDBC_SESSION_INIT_STATEMENT) + + // An option to allow/disallow pushing down predicate into JDBC data source + val pushDownPredicate = parameters.getOrElse(JDBC_PUSHDOWN_PREDICATE, "true").toBoolean } class JdbcOptionsInWrite( @@ -234,4 +237,5 @@ object JDBCOptions { val JDBC_BATCH_INSERT_SIZE = newOption("batchsize") val JDBC_TXN_ISOLATION_LEVEL = newOption("isolationLevel") val JDBC_SESSION_INIT_STATEMENT = newOption("sessionInitStatement") + val JDBC_PUSHDOWN_PREDICATE = newOption("pushDownPredicate") } http://git-wip-us.apache.org/repos/asf/spark/blob/21fcac16/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala -- 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 97e2d25..4f78f59 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 @@ -172,7 +172,11 @@ private[sql] case class JDBCRelation( // Check if JDBCRDD.compileFilter can accept input filters override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { -filters.filter(JDBCRDD.compileFilter(_, JdbcDialects.get
spark git commit: [SPARK-24919][BUILD] New linter rule for sparkContext.hadoopConfiguration
Repository: spark Updated Branches: refs/heads/master 2c8274568 -> fa09d9192 [SPARK-24919][BUILD] New linter rule for sparkContext.hadoopConfiguration ## What changes were proposed in this pull request? In most cases, we should use `spark.sessionState.newHadoopConf()` instead of `sparkContext.hadoopConfiguration`, so that the hadoop configurations specified in Spark session configuration will come into effect. Add a rule matching `spark.sparkContext.hadoopConfiguration` or `spark.sqlContext.sparkContext.hadoopConfiguration` to prevent the usage. ## How was this patch tested? Unit test Author: Gengliang Wang Closes #21873 from gengliangwang/linterRule. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fa09d919 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fa09d919 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fa09d919 Branch: refs/heads/master Commit: fa09d91925c07a58dea285d6cf85a751664f89ff Parents: 2c82745 Author: Gengliang Wang Authored: Thu Jul 26 16:50:59 2018 -0700 Committer: Xiao Li Committed: Thu Jul 26 16:50:59 2018 -0700 -- .../org/apache/spark/sql/avro/AvroSuite.scala | 26 +--- .../org/apache/spark/ml/image/HadoopUtils.scala | 4 +++ .../apache/spark/ml/clustering/LDASuite.scala | 2 +- scalastyle-config.xml | 13 ++ .../HadoopFileLinesReaderSuite.scala| 22 - .../spark/sql/hive/execution/HiveDDLSuite.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 11 ++--- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- 8 files changed, 45 insertions(+), 37 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fa09d919/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala -- diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 865a145..a93309e 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -638,12 +638,8 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { intercept[FileNotFoundException] { withTempPath { dir => FileUtils.touch(new File(dir, "test")) -val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration -try { - hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") +withSQLConf(AvroFileFormat.IgnoreFilesWithoutExtensionProperty -> "true") { spark.read.format("avro").load(dir.toString) -} finally { - hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) } } } @@ -717,15 +713,10 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { Files.createFile(new File(tempSaveDir, "non-avro").toPath) - val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration - val count = try { -hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") + withSQLConf(AvroFileFormat.IgnoreFilesWithoutExtensionProperty -> "true") { val newDf = spark.read.format("avro").load(tempSaveDir) -newDf.count() - } finally { -hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) +assert(newDf.count() == 8) } - assert(count == 8) } } @@ -888,20 +879,15 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { Paths.get(new URL(episodesAvro).toURI), Paths.get(dir.getCanonicalPath, "episodes")) - val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration - val count = try { -hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") + val hadoopConf = spark.sessionState.newHadoopConf() + withSQLConf(AvroFileFormat.IgnoreFilesWithoutExtensionProperty -> "true") { val newDf = spark .read .option("ignoreExtension", "true") .format("avro") .load(s"${dir.getCanonicalPath}/episodes") -newDf.count() - } finally { -hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) +assert(newDf.count() == 8) } - - assert(count == 8) } } } http://git-wip-us.apache.org/repos/asf/spark/blob/fa09d919/mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/i
spark git commit: [SPARK-24307][CORE] Add conf to revert to old code.
Repository: spark Updated Branches: refs/heads/master e3486e1b9 -> 2c8274568 [SPARK-24307][CORE] Add conf to revert to old code. In case there are any issues in converting FileSegmentManagedBuffer to ChunkedByteBuffer, add a conf to go back to old code path. Followup to 7e847646d1f377f46dc3154dea37148d4e557a03 Author: Imran Rashid Closes #21867 from squito/SPARK-24307-p2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2c827456 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2c827456 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2c827456 Branch: refs/heads/master Commit: 2c82745686f4456c4d5c84040a431dcb5b6cb60b Parents: e3486e1 Author: Imran Rashid Authored: Thu Jul 26 12:13:27 2018 -0700 Committer: Xiao Li Committed: Thu Jul 26 12:13:27 2018 -0700 -- .../scala/org/apache/spark/storage/BlockManager.scala| 11 ++- 1 file changed, 10 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2c827456/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1db0327..5cd21e3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -132,6 +132,8 @@ private[spark] class BlockManager( conf.getBoolean("spark.shuffle.service.enabled", false) private val chunkSize = conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt + private val remoteReadNioBufferConversion = +conf.getBoolean("spark.network.remoteReadNioBufferConversion", false) val diskBlockManager = { // Only perform cleanup if an external service is not serving our shuffle files. @@ -731,7 +733,14 @@ private[spark] class BlockManager( } if (data != null) { -return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize)) +// SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to +// ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if +// new path is stable. +if (remoteReadNioBufferConversion) { + return Some(new ChunkedByteBuffer(data.nioByteBuffer())) +} else { + return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize)) +} } logDebug(s"The value of block $blockId is null") } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24795][CORE] Implement barrier execution mode
Repository: spark Updated Branches: refs/heads/master 5ed7660d1 -> e3486e1b9 [SPARK-24795][CORE] Implement barrier execution mode ## What changes were proposed in this pull request? Propose new APIs and modify job/task scheduling to support barrier execution mode, which requires all tasks in a same barrier stage start at the same time, and retry all tasks in case some tasks fail in the middle. The barrier execution mode is useful for some ML/DL workloads. The proposed API changes include: - `RDDBarrier` that marks an RDD as barrier (Spark must launch all the tasks together for the current stage). - `BarrierTaskContext` that support global sync of all tasks in a barrier stage, and provide extra `BarrierTaskInfo`s. In DAGScheduler, we retry all tasks of a barrier stage in case some tasks fail in the middle, this is achieved by unregistering map outputs for a shuffleId (for ShuffleMapStage) or clear the finished partitions in an active job (for ResultStage). ## How was this patch tested? Add `RDDBarrierSuite` to ensure we convert RDDs correctly; Add new test cases in `DAGSchedulerSuite` to ensure we do task scheduling correctly; Add new test cases in `SparkContextSuite` to ensure the barrier execution mode actually works (both under local mode and local cluster mode). Add new test cases in `TaskSchedulerImplSuite` to ensure we schedule tasks for barrier taskSet together. Author: Xingbo Jiang Closes #21758 from jiangxb1987/barrier-execution-mode. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e3486e1b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e3486e1b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e3486e1b Branch: refs/heads/master Commit: e3486e1b9556e00bc9c392a5b8440ab366780f9b Parents: 5ed7660 Author: Xingbo Jiang Authored: Thu Jul 26 12:09:01 2018 -0700 Committer: Xiao Li Committed: Thu Jul 26 12:09:01 2018 -0700 -- .../org/apache/spark/BarrierTaskContext.scala | 42 ++ .../apache/spark/BarrierTaskContextImpl.scala | 49 +++ .../org/apache/spark/BarrierTaskInfo.scala | 31 + .../org/apache/spark/MapOutputTracker.scala | 12 ++ .../org/apache/spark/rdd/MapPartitionsRDD.scala | 15 ++- .../main/scala/org/apache/spark/rdd/RDD.scala | 27 +++- .../scala/org/apache/spark/rdd/RDDBarrier.scala | 52 .../org/apache/spark/rdd/ShuffledRDD.scala | 2 + .../org/apache/spark/scheduler/ActiveJob.scala | 6 + .../apache/spark/scheduler/DAGScheduler.scala | 131 --- .../org/apache/spark/scheduler/ResultTask.scala | 9 +- .../apache/spark/scheduler/ShuffleMapTask.scala | 7 +- .../org/apache/spark/scheduler/Stage.scala | 8 +- .../scala/org/apache/spark/scheduler/Task.scala | 41 -- .../spark/scheduler/TaskDescription.scala | 7 +- .../spark/scheduler/TaskSchedulerImpl.scala | 66 -- .../apache/spark/scheduler/TaskSetManager.scala | 9 +- .../apache/spark/scheduler/WorkerOffer.scala| 8 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 6 +- .../scheduler/local/LocalSchedulerBackend.scala | 3 +- .../org/apache/spark/SparkContextSuite.scala| 42 ++ .../apache/spark/executor/ExecutorSuite.scala | 1 + .../org/apache/spark/rdd/RDDBarrierSuite.scala | 43 ++ .../spark/scheduler/DAGSchedulerSuite.scala | 58 .../org/apache/spark/scheduler/FakeTask.scala | 24 +++- .../spark/scheduler/TaskDescriptionSuite.scala | 2 + .../scheduler/TaskSchedulerImplSuite.scala | 34 + .../MesosFineGrainedSchedulerBackendSuite.scala | 2 + 28 files changed, 673 insertions(+), 64 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e3486e1b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala new file mode 100644 index 000..4c35862 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -0,0 +1,42 @@ +/* + * 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
spark git commit: [SPARK-24802][SQL][FOLLOW-UP] Add a new config for Optimization Rule Exclusion
Repository: spark Updated Branches: refs/heads/master 58353d7f4 -> 5ed7660d1 [SPARK-24802][SQL][FOLLOW-UP] Add a new config for Optimization Rule Exclusion ## What changes were proposed in this pull request? This is an extension to the original PR, in which rule exclusion did not work for classes derived from Optimizer, e.g., SparkOptimizer. To solve this issue, Optimizer and its derived classes will define/override `defaultBatches` and `nonExcludableRules` in order to define its default rule set as well as rules that cannot be excluded by the SQL config. In the meantime, Optimizer's `batches` method is dedicated to the rule exclusion logic and is defined "final". ## How was this patch tested? Added UT. Author: maryannxue Closes #21876 from maryannxue/rule-exclusion. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5ed7660d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5ed7660d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5ed7660d Branch: refs/heads/master Commit: 5ed7660d14022eb65396e28496c06e47c1dbab1d Parents: 58353d7 Author: maryannxue Authored: Thu Jul 26 11:06:23 2018 -0700 Committer: Xiao Li Committed: Thu Jul 26 11:06:23 2018 -0700 -- .../sql/catalyst/optimizer/Optimizer.scala | 24 - .../optimizer/OptimizerExtendableSuite.scala| 2 +- .../optimizer/OptimizerRuleExclusionSuite.scala | 53 +++- ...timizerStructuralIntegrityCheckerSuite.scala | 2 +- .../spark/sql/execution/SparkOptimizer.scala| 5 +- 5 files changed, 69 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5ed7660d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index adb1350..3c264eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -46,6 +46,13 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations) + /** + * Defines the default rule batches in the Optimizer. + * + * Implementations of this class should override this method, and [[nonExcludableRules]] if + * necessary, instead of [[batches]]. The rule batches that eventually run in the Optimizer, + * i.e., returned by [[batches]], will be (defaultBatches - (excludedRules - nonExcludableRules)). + */ def defaultBatches: Seq[Batch] = { val operatorOptimizationRuleSet = Seq( @@ -160,6 +167,14 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) UpdateNullabilityInAttributeReferences) } + /** + * Defines rules that cannot be excluded from the Optimizer even if they are specified in + * SQL config "excludedRules". + * + * Implementations of this class can override this method if necessary. The rule batches + * that eventually run in the Optimizer, i.e., returned by [[batches]], will be + * (defaultBatches - (excludedRules - nonExcludableRules)). + */ def nonExcludableRules: Seq[String] = EliminateDistinct.ruleName :: EliminateSubqueryAliases.ruleName :: @@ -202,7 +217,14 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) */ def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil - override def batches: Seq[Batch] = { + /** + * Returns (defaultBatches - (excludedRules - nonExcludableRules)), the rule batches that + * eventually run in the Optimizer. + * + * Implementations of this class should override [[defaultBatches]], and [[nonExcludableRules]] + * if necessary, instead of this method. + */ + final override def batches: Seq[Batch] = { val excludedRulesConf = SQLConf.get.optimizerExcludedRules.toSeq.flatMap(Utils.stringToSeq) val excludedRules = excludedRulesConf.filter { ruleName => http://git-wip-us.apache.org/repos/asf/spark/blob/5ed7660d/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerExtendableSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerExtendableSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerExtendableSuite.scala index 7112c03..36b083a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerExtendableSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/cata
spark git commit: [SPARK-24867][SQL] Add AnalysisBarrier to DataFrameWriter
Repository: spark Updated Branches: refs/heads/branch-2.3 740606eb8 -> fa552c3c1 [SPARK-24867][SQL] Add AnalysisBarrier to DataFrameWriter ```Scala val udf1 = udf({(x: Int, y: Int) => x + y}) val df = spark.range(0, 3).toDF("a") .withColumn("b", udf1($"a", udf1($"a", lit(10 df.cache() df.write.saveAsTable("t") ``` Cache is not being used because the plans do not match with the cached plan. This is a regression caused by the changes we made in AnalysisBarrier, since not all the Analyzer rules are idempotent. Added a test. Also found a bug in the DSV1 write path. This is not a regression. Thus, opened a separate JIRA https://issues.apache.org/jira/browse/SPARK-24869 Author: Xiao Li Closes #21821 from gatorsmile/testMaster22. (cherry picked from commit d2e7deb59f641e93778b763d5396f73d38f9a785) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fa552c3c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fa552c3c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fa552c3c Branch: refs/heads/branch-2.3 Commit: fa552c3c1102404fe98c72a5b83cffbc5ba41df3 Parents: 740606e Author: Xiao Li Authored: Wed Jul 25 17:22:37 2018 -0700 Committer: Xiao Li Committed: Wed Jul 25 17:24:32 2018 -0700 -- .../org/apache/spark/sql/DataFrameWriter.scala | 10 +++-- .../spark/sql/execution/command/ddl.scala | 7 ++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 42 +++- 3 files changed, 51 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fa552c3c/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala -- 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 ed7a910..6c9fb52 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 @@ -254,7 +254,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val writer = ws.createWriter(jobId, df.logicalPlan.schema, mode, options) if (writer.isPresent) { runCommand(df.sparkSession, "save") { - WriteToDataSourceV2(writer.get(), df.logicalPlan) + WriteToDataSourceV2(writer.get(), df.planWithBarrier) } } @@ -275,7 +275,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { sparkSession = df.sparkSession, className = source, partitionColumns = partitioningColumns.getOrElse(Nil), -options = extraOptions.toMap).planForWriting(mode, AnalysisBarrier(df.logicalPlan)) +options = extraOptions.toMap).planForWriting(mode, df.planWithBarrier) } } @@ -323,7 +323,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { InsertIntoTable( table = UnresolvedRelation(tableIdent), partition = Map.empty[String, Option[String]], -query = df.logicalPlan, +query = df.planWithBarrier, overwrite = mode == SaveMode.Overwrite, ifPartitionNotExists = false) } @@ -455,7 +455,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { partitionColumnNames = partitioningColumns.getOrElse(Nil), bucketSpec = getBucketSpec) -runCommand(df.sparkSession, "saveAsTable")(CreateTable(tableDesc, mode, Some(df.logicalPlan))) +runCommand(df.sparkSession, "saveAsTable") { + CreateTable(tableDesc, mode, Some(df.planWithBarrier)) +} } /** http://git-wip-us.apache.org/repos/asf/spark/blob/fa552c3c/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala -- 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 0f4831b..28313f2 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 @@ -29,7 +29,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Resolver} +import org.apache.spark.sql.catalyst.analysis.{EliminateBarriers, NoSuchTableException, Resolver} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.
spark git commit: [SPARK-24867][SQL] Add AnalysisBarrier to DataFrameWriter
Repository: spark Updated Branches: refs/heads/master 17f469bc8 -> d2e7deb59 [SPARK-24867][SQL] Add AnalysisBarrier to DataFrameWriter ## What changes were proposed in this pull request? ```Scala val udf1 = udf({(x: Int, y: Int) => x + y}) val df = spark.range(0, 3).toDF("a") .withColumn("b", udf1($"a", udf1($"a", lit(10 df.cache() df.write.saveAsTable("t") ``` Cache is not being used because the plans do not match with the cached plan. This is a regression caused by the changes we made in AnalysisBarrier, since not all the Analyzer rules are idempotent. ## How was this patch tested? Added a test. Also found a bug in the DSV1 write path. This is not a regression. Thus, opened a separate JIRA https://issues.apache.org/jira/browse/SPARK-24869 Author: Xiao Li Closes #21821 from gatorsmile/testMaster22. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d2e7deb5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d2e7deb5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d2e7deb5 Branch: refs/heads/master Commit: d2e7deb59f641e93778b763d5396f73d38f9a785 Parents: 17f469b Author: Xiao Li Authored: Wed Jul 25 17:22:37 2018 -0700 Committer: Xiao Li Committed: Wed Jul 25 17:22:37 2018 -0700 -- .../org/apache/spark/sql/DataFrameWriter.scala | 10 +++-- .../spark/sql/execution/command/ddl.scala | 7 ++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 42 +++- 3 files changed, 51 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d2e7deb5/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala -- 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 b9fa43f..39c0e10 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 @@ -254,7 +254,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val writer = ws.createWriter(jobId, df.logicalPlan.schema, mode, options) if (writer.isPresent) { runCommand(df.sparkSession, "save") { - WriteToDataSourceV2(writer.get(), df.logicalPlan) + WriteToDataSourceV2(writer.get(), df.planWithBarrier) } } @@ -275,7 +275,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { sparkSession = df.sparkSession, className = source, partitionColumns = partitioningColumns.getOrElse(Nil), -options = extraOptions.toMap).planForWriting(mode, AnalysisBarrier(df.logicalPlan)) +options = extraOptions.toMap).planForWriting(mode, df.planWithBarrier) } } @@ -323,7 +323,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { InsertIntoTable( table = UnresolvedRelation(tableIdent), partition = Map.empty[String, Option[String]], -query = df.logicalPlan, +query = df.planWithBarrier, overwrite = mode == SaveMode.Overwrite, ifPartitionNotExists = false) } @@ -459,7 +459,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { partitionColumnNames = partitioningColumns.getOrElse(Nil), bucketSpec = getBucketSpec) -runCommand(df.sparkSession, "saveAsTable")(CreateTable(tableDesc, mode, Some(df.logicalPlan))) +runCommand(df.sparkSession, "saveAsTable") { + CreateTable(tableDesc, mode, Some(df.planWithBarrier)) +} } /** http://git-wip-us.apache.org/repos/asf/spark/blob/d2e7deb5/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala -- 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 04bf8c6..c7f7e4d 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 @@ -29,7 +29,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Resolver} +import org.apache.spark.sql.catalyst.analysis.{EliminateBarriers, NoSuchTableException, Resolver} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, Attribu
spark git commit: [SPARK-24860][SQL] Support setting of partitionOverWriteMode in output options for writing DataFrame
Repository: spark Updated Branches: refs/heads/master 0c83f718e -> 17f469bc8 [SPARK-24860][SQL] Support setting of partitionOverWriteMode in output options for writing DataFrame ## What changes were proposed in this pull request? Besides spark setting spark.sql.sources.partitionOverwriteMode also allow setting partitionOverWriteMode per write ## How was this patch tested? Added unit test in InsertSuite Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Koert Kuipers Closes #21818 from koertkuipers/feat-partition-overwrite-mode-per-write. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/17f469bc Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/17f469bc Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/17f469bc Branch: refs/heads/master Commit: 17f469bc808e076b45fffcedb0147991fa4c41f3 Parents: 0c83f71 Author: Koert Kuipers Authored: Wed Jul 25 13:06:03 2018 -0700 Committer: Xiao Li Committed: Wed Jul 25 13:06:03 2018 -0700 -- .../org/apache/spark/sql/internal/SQLConf.scala | 6 +- .../InsertIntoHadoopFsRelationCommand.scala | 9 +++-- .../apache/spark/sql/sources/InsertSuite.scala | 20 3 files changed, 32 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/17f469bc/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d7c830d..53423e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1360,7 +1360,11 @@ object SQLConf { "overwriting. In dynamic mode, Spark doesn't delete partitions ahead, and only overwrite " + "those partitions that have data written into it at runtime. By default we use static " + "mode to keep the same behavior of Spark prior to 2.3. Note that this config doesn't " + -"affect Hive serde tables, as they are always overwritten with dynamic mode.") +"affect Hive serde tables, as they are always overwritten with dynamic mode. This can " + +"also be set as an output option for a data source using key partitionOverwriteMode " + +"(which takes precendence over this setting), e.g. " + +"dataframe.write.option(\"partitionOverwriteMode\", \"dynamic\").save(path)." + ) .stringConf .transform(_.toUpperCase(Locale.ROOT)) .checkValues(PartitionOverwriteMode.values.map(_.toString)) http://git-wip-us.apache.org/repos/asf/spark/blob/17f469bc/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.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 dd7ef0d..8a2e00d 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogT import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode @@ -91,8 +92,12 @@ case class InsertIntoHadoopFsRelationCommand( val pathExists = fs.exists(qualifiedOutputPath) -val enableDynamicOverwrite = - sparkSession.sessionState.conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC +val parameters = CaseInsensitiveMap(options) + +val partitionOverwriteMode = parameters.get("partitionOverwriteMode") + .map(mode => PartitionOverwriteMode.withName(mode.toUpperCase)) + .getOrElse(sparkSession.sessionState.conf.partitionOverwriteMode) +val enableDynamicOverwrite = partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC // This config only makes sense when we are overwriting a partitioned dataset with dynamic // partition columns. val dynamicPartitionOverwrite = enableDynamicOv
spark git commit: [SPARK-24849][SPARK-24911][SQL] Converting a value of StructType to a DDL string
Repository: spark Updated Branches: refs/heads/master 571a6f057 -> 2f77616e1 [SPARK-24849][SPARK-24911][SQL] Converting a value of StructType to a DDL string ## What changes were proposed in this pull request? In the PR, I propose to extend the `StructType`/`StructField` classes by new method `toDDL` which converts a value of the `StructType`/`StructField` type to a string formatted in DDL style. The resulted string can be used in a table creation. The `toDDL` method of `StructField` is reused in `SHOW CREATE TABLE`. In this way the PR fixes the bug of unquoted names of nested fields. ## How was this patch tested? I add a test for checking the new method and 2 round trip tests: `fromDDL` -> `toDDL` and `toDDL` -> `fromDDL` Author: Maxim Gekk Closes #21803 from MaxGekk/to-ddl. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2f77616e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2f77616e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2f77616e Branch: refs/heads/master Commit: 2f77616e1dc593fb9c376a8bd72416198cf3d6f5 Parents: 571a6f0 Author: Maxim Gekk Authored: Wed Jul 25 11:09:12 2018 -0700 Committer: Xiao Li Committed: Wed Jul 25 11:09:12 2018 -0700 -- .../spark/sql/catalyst/util/package.scala | 12 +++ .../apache/spark/sql/types/StructField.scala| 13 .../org/apache/spark/sql/types/StructType.scala | 10 +- .../spark/sql/types/StructTypeSuite.scala | 33 .../spark/sql/execution/command/tables.scala| 23 +++--- .../spark/sql/hive/ShowCreateTableSuite.scala | 15 + 6 files changed, 86 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2f77616e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 4005087..0978e92 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -155,6 +155,18 @@ package object util { def toPrettySQL(e: Expression): String = usePrettyExpression(e).sql + + def escapeSingleQuotedString(str: String): String = { +val builder = StringBuilder.newBuilder + +str.foreach { + case '\'' => builder ++= s"\\\'" + case ch => builder += ch +} + +builder.toString() + } + /* FIX ME implicit class debugLogging(a: Any) { def debugLogging() { http://git-wip-us.apache.org/repos/asf/spark/blob/2f77616e/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala -- 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 2c18fdc..902cae9 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 @@ -21,6 +21,7 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} /** * A field inside a StructType. @@ -74,4 +75,16 @@ case class StructField( def getComment(): Option[String] = { if (metadata.contains("comment")) Option(metadata.getString("comment")) else None } + + /** + * Returns a string containing a schema in DDL format. For example, the following value: + * `StructField("eventId", IntegerType)` will be converted to `eventId` INT. + */ + def toDDL: String = { +val comment = getComment() + .map(escapeSingleQuotedString) + .map(" COMMENT '" + _ + "'") + +s"${quoteIdentifier(name)} ${dataType.sql}${comment.getOrElse("")}" + } } http://git-wip-us.apache.org/repos/asf/spark/blob/2f77616e/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala -- 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 b13e95f..c5ca169 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 @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.{Attrib
[2/3] spark-website git commit: spark summit eu 2018
http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/news/spark-2-2-1-released.html -- diff --git a/site/news/spark-2-2-1-released.html b/site/news/spark-2-2-1-released.html index df7c2f0..b9d465f 100644 --- a/site/news/spark-2-2-1-released.html +++ b/site/news/spark-2-2-1-released.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/news/spark-2-2-2-released.html -- diff --git a/site/news/spark-2-2-2-released.html b/site/news/spark-2-2-2-released.html index 1c46315..2ab8b62 100644 --- a/site/news/spark-2-2-2-released.html +++ b/site/news/spark-2-2-2-released.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/news/spark-2-3-0-released.html -- diff --git a/site/news/spark-2-3-0-released.html b/site/news/spark-2-3-0-released.html index 1d99ab0..d6fcf03 100644 --- a/site/news/spark-2-3-0-released.html +++ b/site/news/spark-2-3-0-released.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/news/spark-2-3-1-released.html -- diff --git a/site/news/spark-2-3-1-released.html b/site/news/spark-2-3-1-released.html index 06b718a..16d475b 100644 --- a/site/news/spark-2-3-1-released.html +++ b/site/news/spark-2-3-1-released.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/news/spark-2.0.0-preview.html -- diff --git a/site/news/spark-2.0.0-preview.html b/site/news/spark-2.0.0-preview.html index da54cef..342ef2e 100644 --- a/site/news/spark-2.0.0-preview.html +++ b/site/news/spark-2.0.0-preview.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/news/spark-accepted-into-apache-incubator.html -- diff --git a/site/news/spark-accepted-into-apache-incubator.html b/site/news/spark-accepted-into-apache-incubator.html index 696c6d0..1f48ed2 100644 --- a/site/news/spark-accepted-into-apache-incubator.html +++ b/site/news/spark-accepted-into-apache-incubator.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI
[1/3] spark-website git commit: spark summit eu 2018
Repository: spark-website Updated Branches: refs/heads/asf-site f5d7dfafe -> d86cffd19 http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/releases/spark-release-1-1-1.html -- diff --git a/site/releases/spark-release-1-1-1.html b/site/releases/spark-release-1-1-1.html index f46e47a..bb5c345 100644 --- a/site/releases/spark-release-1-1-1.html +++ b/site/releases/spark-release-1-1-1.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/releases/spark-release-1-2-0.html -- diff --git a/site/releases/spark-release-1-2-0.html b/site/releases/spark-release-1-2-0.html index 4fe4f0a..e7b1ee2 100644 --- a/site/releases/spark-release-1-2-0.html +++ b/site/releases/spark-release-1-2-0.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/releases/spark-release-1-2-1.html -- diff --git a/site/releases/spark-release-1-2-1.html b/site/releases/spark-release-1-2-1.html index a334c62..068cdee 100644 --- a/site/releases/spark-release-1-2-1.html +++ b/site/releases/spark-release-1-2-1.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/releases/spark-release-1-2-2.html -- diff --git a/site/releases/spark-release-1-2-2.html b/site/releases/spark-release-1-2-2.html index 63876f7..2a5ec4d 100644 --- a/site/releases/spark-release-1-2-2.html +++ b/site/releases/spark-release-1-2-2.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/releases/spark-release-1-3-0.html -- diff --git a/site/releases/spark-release-1-3-0.html b/site/releases/spark-release-1-3-0.html index 567dc1d..480b28f 100644 --- a/site/releases/spark-release-1-3-0.html +++ b/site/releases/spark-release-1-3-0.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@ -171,9 +174,6 @@ Spark 2.3.1 released (Jun 08, 2018) - Spark+AI Summit (June 4-6th, 2018, San Francisco) agenda posted - (Mar 01, 2018) - Archive http://git-wip-us.apache.org/repos/asf/spark-website/blob/d86cffd1/site/releases/spark-release-1-3-1.html -- diff --git a/site/releases/spark-release-1-3-1.html b/site/releases/spark-release-1-3-1.html index 66f50db..88fb4cc 100644 --- a/site/releases/spark-release-1-3-1.html +++ b/site/releases/spark-release-1-3-1.html @@ -162,6 +162,9 @@ Latest News + Spark+AI Summit (October 2-4th, 2018, London) agenda posted + (Jul 24, 2018) + Spark 2.2.2 released (Jul 02, 2018) @@
[3/3] spark-website git commit: spark summit eu 2018
spark summit eu 2018 Project: http://git-wip-us.apache.org/repos/asf/spark-website/repo Commit: http://git-wip-us.apache.org/repos/asf/spark-website/commit/d86cffd1 Tree: http://git-wip-us.apache.org/repos/asf/spark-website/tree/d86cffd1 Diff: http://git-wip-us.apache.org/repos/asf/spark-website/diff/d86cffd1 Branch: refs/heads/asf-site Commit: d86cffd1918f9cb8f25d7beb0ae65f814f5232bd Parents: f5d7dfa Author: Xiao Li Authored: Wed Jul 25 09:06:55 2018 -0700 Committer: Xiao Li Committed: Wed Jul 25 09:06:55 2018 -0700 -- ...07-24-spark-summit-oct-2018-agenda-posted.md | 15 + site/committers.html| 6 +- site/community.html | 6 +- site/contributing.html | 6 +- site/developer-tools.html | 6 +- site/documentation.html | 6 +- site/downloads.html | 6 +- site/examples.html | 6 +- site/faq.html | 6 +- site/graphx/index.html | 6 +- site/history.html | 6 +- site/improvement-proposals.html | 6 +- site/index.html | 6 +- site/mailing-lists.html | 8 +- site/mllib/index.html | 6 +- site/news/amp-camp-2013-registration-ope.html | 6 +- .../news/announcing-the-first-spark-summit.html | 6 +- .../news/fourth-spark-screencast-published.html | 6 +- site/news/index.html| 15 +- site/news/nsdi-paper.html | 6 +- site/news/one-month-to-spark-summit-2015.html | 6 +- .../proposals-open-for-spark-summit-east.html | 6 +- ...registration-open-for-spark-summit-east.html | 6 +- .../news/run-spark-and-shark-on-amazon-emr.html | 6 +- site/news/spark-0-6-1-and-0-5-2-released.html | 6 +- site/news/spark-0-6-2-released.html | 6 +- site/news/spark-0-7-0-released.html | 6 +- site/news/spark-0-7-2-released.html | 6 +- site/news/spark-0-7-3-released.html | 6 +- site/news/spark-0-8-0-released.html | 6 +- site/news/spark-0-8-1-released.html | 6 +- site/news/spark-0-9-0-released.html | 6 +- site/news/spark-0-9-1-released.html | 6 +- site/news/spark-0-9-2-released.html | 6 +- site/news/spark-1-0-0-released.html | 6 +- site/news/spark-1-0-1-released.html | 6 +- site/news/spark-1-0-2-released.html | 6 +- site/news/spark-1-1-0-released.html | 6 +- site/news/spark-1-1-1-released.html | 6 +- site/news/spark-1-2-0-released.html | 6 +- site/news/spark-1-2-1-released.html | 6 +- site/news/spark-1-2-2-released.html | 6 +- site/news/spark-1-3-0-released.html | 6 +- site/news/spark-1-4-0-released.html | 6 +- site/news/spark-1-4-1-released.html | 6 +- site/news/spark-1-5-0-released.html | 6 +- site/news/spark-1-5-1-released.html | 6 +- site/news/spark-1-5-2-released.html | 6 +- site/news/spark-1-6-0-released.html | 6 +- site/news/spark-1-6-1-released.html | 6 +- site/news/spark-1-6-2-released.html | 6 +- site/news/spark-1-6-3-released.html | 6 +- site/news/spark-2-0-0-released.html | 6 +- site/news/spark-2-0-1-released.html | 6 +- site/news/spark-2-0-2-released.html | 6 +- site/news/spark-2-1-0-released.html | 6 +- site/news/spark-2-1-1-released.html | 6 +- site/news/spark-2-1-2-released.html | 6 +- site/news/spark-2-1-3-released.html | 6 +- site/news/spark-2-2-0-released.html | 6 +- site/news/spark-2-2-1-released.html | 6 +- site/news/spark-2-2-2-released.html | 6 +- site/news/spark-2-3-0-released.html | 6 +- site/news/spark-2-3-1-released.html | 6 +- site/news/spark-2.0.0-preview.html | 6 +- .../spark-accepted-into-apache-incubator.html | 6 +- site/news/spark-and-shark-in-the-news.html | 6 +- site/news/spark-becomes-tlp.html| 6 +- site/news/spark-featured-in-wired.html | 6 +- .../spark-mailing-lists-moving-to-apache.html | 6 +- site/news/spark-meetups.html| 6 +- site/news/spark-screencasts-published.html | 6 +- site/news/spark-summit-2013-is-a-wrap.html | 6 +- site/news/spark-summit-2014-videos-posted.html | 6 +- site/news/spark-summit-2015-videos-posted.html | 6 +- site/news/spark-summit-agenda-posted.html |
spark git commit: [SPARK-24768][FOLLOWUP][SQL] Avro migration followup: change artifactId to spark-avro
Repository: spark Updated Branches: refs/heads/master 7a5fd4a91 -> c44eb561e [SPARK-24768][FOLLOWUP][SQL] Avro migration followup: change artifactId to spark-avro ## What changes were proposed in this pull request? After rethinking on the artifactId, I think it should be `spark-avro` instead of `spark-sql-avro`, which is simpler, and consistent with the previous artifactId. I think we need to change it before Spark 2.4 release. Also a tiny change: use `spark.sessionState.newHadoopConf()` to get the hadoop configuration, thus the related hadoop configurations in SQLConf will come into effect. ## How was this patch tested? Unit test Author: Gengliang Wang Closes #21866 from gengliangwang/avro_followup. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c44eb561 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c44eb561 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c44eb561 Branch: refs/heads/master Commit: c44eb561ec371af0405710d2e9358f9797655145 Parents: 7a5fd4a Author: Gengliang Wang Authored: Wed Jul 25 08:42:45 2018 -0700 Committer: Xiao Li Committed: Wed Jul 25 08:42:45 2018 -0700 -- external/avro/pom.xml | 2 +- .../src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c44eb561/external/avro/pom.xml -- diff --git a/external/avro/pom.xml b/external/avro/pom.xml index ad7df1f..8f118ba 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -25,7 +25,7 @@ ../../pom.xml - spark-sql-avro_2.11 + spark-avro_2.11 avro http://git-wip-us.apache.org/repos/asf/spark/blob/c44eb561/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index b043252..c6b3c13 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -56,7 +56,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { spark: SparkSession, options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { -val conf = spark.sparkContext.hadoopConfiguration +val conf = spark.sessionState.newHadoopConf() val parsedOptions = new AvroOptions(options, conf) // Schema evolution is not supported yet. Here we only pick a single random sample file to - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-18874][SQL][FOLLOW-UP] Improvement type mismatched message
Repository: spark Updated Branches: refs/heads/master 78e0a725e -> 7a5fd4a91 [SPARK-18874][SQL][FOLLOW-UP] Improvement type mismatched message ## What changes were proposed in this pull request? Improvement `IN` predicate type mismatched message: ```sql Mismatched columns: [(, t, 4, ., `, t, 4, a, `, :, d, o, u, b, l, e, ,, , t, 5, ., `, t, 5, a, `, :, d, e, c, i, m, a, l, (, 1, 8, ,, 0, ), ), (, t, 4, ., `, t, 4, c, `, :, s, t, r, i, n, g, ,, , t, 5, ., `, t, 5, c, `, :, b, i, g, i, n, t, )] ``` After this patch: ```sql Mismatched columns: [(t4.`t4a`:double, t5.`t5a`:decimal(18,0)), (t4.`t4c`:string, t5.`t5c`:bigint)] ``` ## How was this patch tested? unit tests Author: Yuming Wang Closes #21863 from wangyum/SPARK-18874. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7a5fd4a9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7a5fd4a9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7a5fd4a9 Branch: refs/heads/master Commit: 7a5fd4a91e19ee32b365eaf5678c627ad6c6d4c2 Parents: 78e0a72 Author: Yuming Wang Authored: Tue Jul 24 23:59:13 2018 -0700 Committer: Xiao Li Committed: Tue Jul 24 23:59:13 2018 -0700 -- .../sql/catalyst/expressions/predicates.scala | 2 +- .../negative-cases/subq-input-typecheck.sql | 16 - .../negative-cases/subq-input-typecheck.sql.out | 66 3 files changed, 70 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7a5fd4a9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 699601e..f4077f78 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 @@ -189,7 +189,7 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { } else { val mismatchedColumns = valExprs.zip(childOutputs).flatMap { case (l, r) if l.dataType != r.dataType => -s"(${l.sql}:${l.dataType.catalogString}, ${r.sql}:${r.dataType.catalogString})" +Seq(s"(${l.sql}:${l.dataType.catalogString}, ${r.sql}:${r.dataType.catalogString})") case _ => None } TypeCheckResult.TypeCheckFailure( http://git-wip-us.apache.org/repos/asf/spark/blob/7a5fd4a9/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql -- diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql index b15f4da..95b115a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql @@ -13,6 +13,14 @@ CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (3, 1, 2) AS t3(t3a, t3b, t3c); +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES + (CAST(1 AS DOUBLE), CAST(2 AS STRING), CAST(3 AS STRING)) +AS t1(t4a, t4b, t4c); + +CREATE TEMPORARY VIEW t5 AS SELECT * FROM VALUES + (CAST(1 AS DECIMAL(18, 0)), CAST(2 AS STRING), CAST(3 AS BIGINT)) +AS t1(t5a, t5b, t5c); + -- TC 01.01 SELECT ( SELECT max(t2b), min(t2b) @@ -44,4 +52,10 @@ WHERE (t1a, t1b) IN (SELECT t2a FROM t2 WHERE t1a = t2a); - +-- TC 01.05 +SELECT * FROM t4 +WHERE +(t4a, t4b, t4c) IN (SELECT t5a, + t5b, + t5c +FROM t5); http://git-wip-us.apache.org/repos/asf/spark/blob/7a5fd4a9/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out -- diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index 70aeb93..dcd3005 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 7 +-- Number of queries: 10 -- !query 0 @@ -33,6 +33,2
spark git commit: [SPARK-23957][SQL] Sorts in subqueries are redundant and can be removed
Repository: spark Updated Branches: refs/heads/master d4c341589 -> afb062753 [SPARK-23957][SQL] Sorts in subqueries are redundant and can be removed ## What changes were proposed in this pull request? Thanks to henryr for the original idea at https://github.com/apache/spark/pull/21049 Description from the original PR : Subqueries (at least in SQL) have 'bag of tuples' semantics. Ordering them is therefore redundant (unless combined with a limit). This patch removes the top sort operators from the subquery plans. This closes https://github.com/apache/spark/pull/21049. ## How was this patch tested? Added test cases in SubquerySuite to cover in, exists and scalar subqueries. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Dilip Biswal Closes #21853 from dilipbiswal/SPARK-23957. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/afb06275 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/afb06275 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/afb06275 Branch: refs/heads/master Commit: afb0627536494c654ce5dd72db648f1ee7da641c Parents: d4c3415 Author: Dilip Biswal Authored: Tue Jul 24 20:46:27 2018 -0700 Committer: Xiao Li Committed: Tue Jul 24 20:46:27 2018 -0700 -- .../sql/catalyst/optimizer/Optimizer.scala | 12 +- .../org/apache/spark/sql/SubquerySuite.scala| 300 ++- 2 files changed, 310 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/afb06275/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 5ed7412..adb1350 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -180,10 +180,20 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) * Optimize all the subqueries inside expression. */ object OptimizeSubqueries extends Rule[LogicalPlan] { +private def removeTopLevelSort(plan: LogicalPlan): LogicalPlan = { + plan match { +case Sort(_, _, child) => child +case Project(fields, child) => Project(fields, removeTopLevelSort(child)) +case other => other + } +} def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case s: SubqueryExpression => val Subquery(newPlan) = Optimizer.this.execute(Subquery(s.plan)) -s.withNewPlan(newPlan) +// At this point we have an optimized subquery plan that we are going to attach +// to this subquery expression. Here we can safely remove any top level sort +// in the plan as tuples produced by a subquery are un-ordered. +s.withNewPlan(removeTopLevelSort(newPlan)) } } http://git-wip-us.apache.org/repos/asf/spark/blob/afb06275/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index acef62d..cbffed9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.plans.logical.Join +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.test.SharedSQLContext class SubquerySuite extends QueryTest with SharedSQLContext { @@ -970,4 +973,299 @@ class SubquerySuite extends QueryTest with SharedSQLContext { Row("3", "b") :: Row("4", "b") :: Nil) } } + + private def getNumSortsInQuery(query: String): Int = { +val plan = sql(query).queryExecution.optimizedPlan +getNumSorts(plan) + getSubqueryExpressions(plan).map{s => getNumSorts(s.plan)}.sum + } + + private def getSubqueryExpressions(plan: LogicalPlan): Seq[SubqueryExpression] = { +val subqueryExpressions = ArrayBuffer.empty[SubqueryExpression] +plan transformAllExpressions { + case s: SubqueryExpression => +subqueryExpressions ++= (getSubqueryExpressions(s.plan) :+ s) +s +} +subqueryExpressions + } + + private def getNumSorts(plan: LogicalPlan): Int = { +plan.collect { case s: Sort => s
spark git commit: [SPARK-24890][SQL] Short circuiting the `if` condition when `trueValue` and `falseValue` are the same
Repository: spark Updated Branches: refs/heads/master c26b09216 -> d4c341589 [SPARK-24890][SQL] Short circuiting the `if` condition when `trueValue` and `falseValue` are the same ## What changes were proposed in this pull request? When `trueValue` and `falseValue` are semantic equivalence, the condition expression in `if` can be removed to avoid extra computation in runtime. ## How was this patch tested? Test added. Author: DB Tsai Closes #21848 from dbtsai/short-circuit-if. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d4c34158 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d4c34158 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d4c34158 Branch: refs/heads/master Commit: d4c341589499099654ed4febf235f19897a21601 Parents: c26b092 Author: DB Tsai Authored: Tue Jul 24 20:21:11 2018 -0700 Committer: Xiao Li Committed: Tue Jul 24 20:21:11 2018 -0700 -- .../sql/catalyst/optimizer/expressions.scala| 7 -- .../optimizer/SimplifyConditionalSuite.scala| 24 +++- .../apache/spark/sql/test/SQLTestUtils.scala| 2 +- 3 files changed, 29 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d4c34158/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index cf17f59..4696699 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -390,6 +390,8 @@ object SimplifyConditionals extends Rule[LogicalPlan] with PredicateHelper { case If(TrueLiteral, trueValue, _) => trueValue case If(FalseLiteral, _, falseValue) => falseValue case If(Literal(null, _), _, falseValue) => falseValue + case If(cond, trueValue, falseValue) +if cond.deterministic && trueValue.semanticEquals(falseValue) => trueValue case e @ CaseWhen(branches, elseValue) if branches.exists(x => falseOrNullLiteral(x._1)) => // If there are branches that are always false, remove them. @@ -403,14 +405,14 @@ object SimplifyConditionals extends Rule[LogicalPlan] with PredicateHelper { e.copy(branches = newBranches) } - case e @ CaseWhen(branches, _) if branches.headOption.map(_._1) == Some(TrueLiteral) => + case CaseWhen(branches, _) if branches.headOption.map(_._1).contains(TrueLiteral) => // If the first branch is a true literal, remove the entire CaseWhen and use the value // from that. Note that CaseWhen.branches should never be empty, and as a result the // headOption (rather than head) added above is just an extra (and unnecessary) safeguard. branches.head._2 case CaseWhen(branches, _) if branches.exists(_._1 == TrueLiteral) => -// a branc with a TRue condition eliminates all following branches, +// a branch with a true condition eliminates all following branches, // these branches can be pruned away val (h, t) = branches.span(_._1 != TrueLiteral) CaseWhen( h :+ t.head, None) @@ -651,6 +653,7 @@ object SimplifyCaseConversionExpressions extends Rule[LogicalPlan] { } } + /** * Combine nested [[Concat]] expressions. */ http://git-wip-us.apache.org/repos/asf/spark/blob/d4c34158/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala index b597c8e..e210874 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} @@ -29,7 +31,8 @@ import org.apache.spark.sql.types.{IntegerType, NullType} class SimplifyConditionalSuite extends PlanTest with PredicateHelper { object Optimize extends RuleExe
spark git commit: [SPARK-24891][SQL] Fix HandleNullInputsForUDF rule
Repository: spark Updated Branches: refs/heads/branch-2.3 740a23d7d -> 6a5999286 [SPARK-24891][SQL] Fix HandleNullInputsForUDF rule The HandleNullInputsForUDF would always add a new `If` node every time it is applied. That would cause a difference between the same plan being analyzed once and being analyzed twice (or more), thus raising issues like plan not matched in the cache manager. The solution is to mark the arguments as null-checked, which is to add a "KnownNotNull" node above those arguments, when adding the UDF under an `If` node, because clearly the UDF will not be called when any of those arguments is null. Add new tests under sql/UDFSuite and AnalysisSuite. Author: maryannxue Closes #21851 from maryannxue/spark-24891. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6a599928 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6a599928 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6a599928 Branch: refs/heads/branch-2.3 Commit: 6a59992866a971abf6052e479ba48c6abded4d04 Parents: 740a23d Author: maryannxue Authored: Tue Jul 24 19:35:34 2018 -0700 Committer: Xiao Li Committed: Tue Jul 24 19:39:23 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 22 .../expressions/constraintExpressions.scala | 35 .../sql/catalyst/analysis/AnalysisSuite.scala | 16 +++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 31 - 4 files changed, 94 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6a599928/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2858bee..5963c14 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.objects.{LambdaVariable, MapObjects, NewInstance, UnresolvedMapObjects} +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -2046,14 +2046,24 @@ class Analyzer( val parameterTypes = ScalaReflection.getParameterTypes(func) assert(parameterTypes.length == inputs.length) + // TODO: skip null handling for not-nullable primitive inputs after we can completely + // trust the `nullable` information. + // (cls, expr) => cls.isPrimitive && expr.nullable + val needsNullCheck = (cls: Class[_], expr: Expression) => +cls.isPrimitive && !expr.isInstanceOf[KnowNotNull] val inputsNullCheck = parameterTypes.zip(inputs) -// TODO: skip null handling for not-nullable primitive inputs after we can completely -// trust the `nullable` information. -// .filter { case (cls, expr) => cls.isPrimitive && expr.nullable } -.filter { case (cls, _) => cls.isPrimitive } +.filter { case (cls, expr) => needsNullCheck(cls, expr) } .map { case (_, expr) => IsNull(expr) } .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) - inputsNullCheck.map(If(_, Literal.create(null, udf.dataType), udf)).getOrElse(udf) + // Once we add an `If` check above the udf, it is safe to mark those checked inputs + // as not nullable (i.e., wrap them with `KnownNotNull`), because the null-returning + // branch of `If` will be called if any of these checked inputs is null. Thus we can + // prevent this rule from being applied repeatedly. + val newInputs = parameterTypes.zip(inputs).map{ case (cls, expr) => +if (needsNullCheck(cls, expr)) KnowNotNull(expr) else expr } + inputsNullCheck +.map(If(_, Literal.create(null, udf.dataType), udf.copy(children = newInputs))) +.getOrElse(udf) } } } http://git-wip-us.apache.org/repos/asf/spark/blob/6a599928/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala -- diff --git a/sql/catalyst/src/main/scala/org/apa
spark git commit: [SPARK-24891][SQL] Fix HandleNullInputsForUDF rule
Repository: spark Updated Branches: refs/heads/master 15fff7903 -> c26b09216 [SPARK-24891][SQL] Fix HandleNullInputsForUDF rule ## What changes were proposed in this pull request? The HandleNullInputsForUDF would always add a new `If` node every time it is applied. That would cause a difference between the same plan being analyzed once and being analyzed twice (or more), thus raising issues like plan not matched in the cache manager. The solution is to mark the arguments as null-checked, which is to add a "KnownNotNull" node above those arguments, when adding the UDF under an `If` node, because clearly the UDF will not be called when any of those arguments is null. ## How was this patch tested? Add new tests under sql/UDFSuite and AnalysisSuite. Author: maryannxue Closes #21851 from maryannxue/spark-24891. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c26b0921 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c26b0921 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c26b0921 Branch: refs/heads/master Commit: c26b0921693814f0726507f16b836d82e2e8cfe0 Parents: 15fff79 Author: maryannxue Authored: Tue Jul 24 19:35:34 2018 -0700 Committer: Xiao Li Committed: Tue Jul 24 19:35:34 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 22 .../expressions/constraintExpressions.scala | 35 .../sql/catalyst/analysis/AnalysisSuite.scala | 16 +++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 31 - 4 files changed, 94 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c26b0921/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 866396c..4f474f4 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.objects.{LambdaVariable, MapObjects, NewInstance, UnresolvedMapObjects} +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -2145,14 +2145,24 @@ class Analyzer( val parameterTypes = ScalaReflection.getParameterTypes(func) assert(parameterTypes.length == inputs.length) + // TODO: skip null handling for not-nullable primitive inputs after we can completely + // trust the `nullable` information. + // (cls, expr) => cls.isPrimitive && expr.nullable + val needsNullCheck = (cls: Class[_], expr: Expression) => +cls.isPrimitive && !expr.isInstanceOf[KnowNotNull] val inputsNullCheck = parameterTypes.zip(inputs) -// TODO: skip null handling for not-nullable primitive inputs after we can completely -// trust the `nullable` information. -// .filter { case (cls, expr) => cls.isPrimitive && expr.nullable } -.filter { case (cls, _) => cls.isPrimitive } +.filter { case (cls, expr) => needsNullCheck(cls, expr) } .map { case (_, expr) => IsNull(expr) } .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) - inputsNullCheck.map(If(_, Literal.create(null, udf.dataType), udf)).getOrElse(udf) + // Once we add an `If` check above the udf, it is safe to mark those checked inputs + // as not nullable (i.e., wrap them with `KnownNotNull`), because the null-returning + // branch of `If` will be called if any of these checked inputs is null. Thus we can + // prevent this rule from being applied repeatedly. + val newInputs = parameterTypes.zip(inputs).map{ case (cls, expr) => +if (needsNullCheck(cls, expr)) KnowNotNull(expr) else expr } + inputsNullCheck +.map(If(_, Literal.create(null, udf.dataType), udf.copy(children = newInputs))) +.getOrElse(udf) } } } http://git-wip-us.apache.org/repos/asf/spark/blob/c26b0921/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala
spark git commit: [SPARK-24812][SQL] Last Access Time in the table description is not valid
Repository: spark Updated Branches: refs/heads/master 9d27541a8 -> d4a277f0c [SPARK-24812][SQL] Last Access Time in the table description is not valid ## What changes were proposed in this pull request? Last Access Time will always displayed wrong date Thu Jan 01 05:30:00 IST 1970 when user run DESC FORMATTED table command In hive its displayed as "UNKNOWN" which makes more sense than displaying wrong date. seems to be a limitation as of now even from hive, better we can follow the hive behavior unless the limitation has been resolved from hive. spark client output ![spark_desc table](https://user-images.githubusercontent.com/12999161/42753448-ddeea66a-88a5-11e8-94aa-ef8d017f94c5.png) Hive client output ![hive_behaviour](https://user-images.githubusercontent.com/12999161/42753489-f4fd366e-88a5-11e8-83b0-0f3a53ce83dd.png) ## How was this patch tested? UT has been added which makes sure that the wrong date "Thu Jan 01 05:30:00 IST 1970 " shall not be added as value for the Last Access property Author: s71955 Closes #21775 from sujith71955/master_hive. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d4a277f0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d4a277f0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d4a277f0 Branch: refs/heads/master Commit: d4a277f0ce2d6e1832d87cae8faec38c5bc730f4 Parents: 9d27541 Author: s71955 Authored: Tue Jul 24 11:31:27 2018 -0700 Committer: Xiao Li Committed: Tue Jul 24 11:31:27 2018 -0700 -- docs/sql-programming-guide.md | 1 + .../apache/spark/sql/catalyst/catalog/interface.scala | 5 - .../apache/spark/sql/hive/execution/HiveDDLSuite.scala | 13 + 3 files changed, 18 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d4a277f0/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4bab58a..e815e5b 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1850,6 +1850,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see ## Upgrading From Spark SQL 2.3 to 2.4 + - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. http://git-wip-us.apache.org/repos/asf/spark/blob/d4a277f0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.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 c6105c5..a4ead53 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 @@ -114,7 +114,10 @@ case class CatalogTablePartition( map.put("Partition Parameters", s"{${parameters.map(p => p._1 + "=" + p._2).mkString(", ")}}") } map.put("Created Time", new Date(createTime).toString) -map.put("Last Access", new Date(lastAccessTime).toString) +val lastAccess = { + if (-1 == lastAccessTime) "UNKNOWN" else new Date(lastAccessTime).toString +} +map.put("Last Access", lastAccess) stats.foreach(s => map.put("Partition Statistics", s.simpleString)) map } http://git-wip-us.apache.org/repos/asf/spark/blob/d4a277f0/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
spark git commit: [SPARK-23325] Use InternalRow when reading with DataSourceV2.
Repository: spark Updated Branches: refs/heads/master 3d5c61e5f -> 9d27541a8 [SPARK-23325] Use InternalRow when reading with DataSourceV2. ## What changes were proposed in this pull request? This updates the DataSourceV2 API to use InternalRow instead of Row for the default case with no scan mix-ins. Support for readers that produce Row is added through SupportsDeprecatedScanRow, which matches the previous API. Readers that used Row now implement this class and should be migrated to InternalRow. Readers that previously implemented SupportsScanUnsafeRow have been migrated to use no SupportsScan mix-ins and produce InternalRow. ## How was this patch tested? This uses existing tests. Author: Ryan Blue Closes #21118 from rdblue/SPARK-23325-datasource-v2-internal-row. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9d27541a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9d27541a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9d27541a Branch: refs/heads/master Commit: 9d27541a856d95635386cbc98f2bb1f1f2f30c13 Parents: 3d5c61e Author: Ryan Blue Authored: Tue Jul 24 10:46:36 2018 -0700 Committer: Xiao Li Committed: Tue Jul 24 10:46:36 2018 -0700 -- .../sql/kafka010/KafkaContinuousReader.scala| 16 --- .../sql/kafka010/KafkaMicroBatchReader.scala| 21 - .../kafka010/KafkaMicroBatchSourceSuite.scala | 2 +- .../sql/sources/v2/reader/DataSourceReader.java | 6 +-- .../sources/v2/reader/InputPartitionReader.java | 7 +-- .../v2/reader/SupportsDeprecatedScanRow.java| 39 + .../v2/reader/SupportsScanColumnarBatch.java| 4 +- .../v2/reader/SupportsScanUnsafeRow.java| 46 .../datasources/v2/DataSourceRDD.scala | 1 - .../datasources/v2/DataSourceV2ScanExec.scala | 17 .../datasources/v2/DataSourceV2Strategy.scala | 13 +++--- .../continuous/ContinuousDataSourceRDD.scala| 26 +-- .../continuous/ContinuousQueuedDataReader.scala | 8 ++-- .../continuous/ContinuousRateStreamSource.scala | 4 +- .../spark/sql/execution/streaming/memory.scala | 16 +++ .../sources/ContinuousMemoryStream.scala| 7 +-- .../sources/RateStreamMicroBatchReader.scala| 4 +- .../execution/streaming/sources/socket.scala| 7 +-- .../sources/v2/JavaAdvancedDataSourceV2.java| 4 +- .../v2/JavaPartitionAwareDataSource.java| 4 +- .../v2/JavaSchemaRequiredDataSource.java| 5 ++- .../sql/sources/v2/JavaSimpleDataSourceV2.java | 5 ++- .../sources/v2/JavaUnsafeRowDataSourceV2.java | 9 ++-- .../sources/RateStreamProviderSuite.scala | 6 +-- .../sql/sources/v2/DataSourceV2Suite.scala | 30 +++-- .../sources/v2/SimpleWritableDataSource.scala | 7 +-- .../sql/streaming/StreamingQuerySuite.scala | 6 +-- .../ContinuousQueuedDataReaderSuite.scala | 6 +-- .../sources/StreamingDataSourceV2Suite.scala| 7 +-- 29 files changed, 168 insertions(+), 165 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9d27541a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala -- diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala index badaa69..48b91df 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala @@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} import org.apache.spark.sql.sources.v2.reader._ @@ -53,7 +54,7 @@ class KafkaContinuousReader( metadataPath: String, initialOffsets: KafkaOffsetRangeLimit, failOnDataLoss: Boolean) - extends ContinuousReader with SupportsScanUnsafeRow with Logging { + extends ContinuousReader with Logging { private lazy val session = SparkSession.getActiveSession.get private lazy val sc = session.sparkContext @@ -86,7 +87,7 @@ class KafkaContinuousReader( KafkaSourceOffset(JsonUtils.partitionOffsets(json)) } - override def planUnsafeInputPartitions(): ju.List[InputPartition[UnsafeRow]] = { + override def planInputPartitions(): ju.List[InputPa
spark git commit: [SPARK-24870][SQL] Cache can't work normally if there are case letters in SQL
Repository: spark Updated Branches: refs/heads/master d2436a852 -> 13a67b070 [SPARK-24870][SQL] Cache can't work normally if there are case letters in SQL ## What changes were proposed in this pull request? Modified the canonicalized to not case-insensitive. Before the PR, cache can't work normally if there are case letters in SQL, for example: sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive") sql("select key, sum(case when Key > 0 then 1 else 0 end) as positiveNum " + "from src group by key").cache().createOrReplaceTempView("src_cache") sql( s"""select a.key from (select key from src_cache where positiveNum = 1)a left join (select key from src_cache )b on a.key=b.key """).explain The physical plan of the sql is: ![image](https://user-images.githubusercontent.com/26834091/42979518-3decf0fa-8c05-11e8-9837-d5e4c334cb1f.png) The subquery "select key from src_cache where positiveNum = 1" on the left of join can use the cache data, but the subquery "select key from src_cache" on the right of join cannot use the cache data. ## How was this patch tested? new added test Author: 10129659 Closes #21823 from eatoncys/canonicalized. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/13a67b07 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/13a67b07 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/13a67b07 Branch: refs/heads/master Commit: 13a67b070d335bb257d13dacadea3450885c3d81 Parents: d2436a8 Author: 10129659 Authored: Mon Jul 23 23:05:08 2018 -0700 Committer: Xiao Li Committed: Mon Jul 23 23:05:08 2018 -0700 -- .../apache/spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../apache/spark/sql/execution/SameResultSuite.scala | 15 +++ 2 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/13a67b07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 4b4722b..b1ffdca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -284,7 +284,7 @@ object QueryPlan extends PredicateHelper { if (ordinal == -1) { ar } else { - ar.withExprId(ExprId(ordinal)) + ar.withExprId(ExprId(ordinal)).canonicalized } }.canonicalized.asInstanceOf[T] } http://git-wip-us.apache.org/repos/asf/spark/blob/13a67b07/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala index aaf51b5..d088e24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala @@ -18,8 +18,11 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.IntegerType /** * Tests for the sameResult function for [[SparkPlan]]s. @@ -58,4 +61,16 @@ class SameResultSuite extends QueryTest with SharedSQLContext { val df4 = spark.range(10).agg(sumDistinct($"id")) assert(df3.queryExecution.executedPlan.sameResult(df4.queryExecution.executedPlan)) } + + test("Canonicalized result is case-insensitive") { +val a = AttributeReference("A", IntegerType)() +val b = AttributeReference("B", IntegerType)() +val planUppercase = Project(Seq(a), LocalRelation(a, b)) + +val c = AttributeReference("a", IntegerType)() +val d = AttributeReference("b", IntegerType)() +val planLowercase = Project(Seq(c), LocalRelation(c, d)) + +assert(planUppercase.sameResult(planLowercase)) + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24339][SQL] Prunes the unused columns from child of ScriptTransformation
Repository: spark Updated Branches: refs/heads/master 61f0ca4f1 -> cfc3e1aaa [SPARK-24339][SQL] Prunes the unused columns from child of ScriptTransformation ## What changes were proposed in this pull request? Modify the strategy in ColumnPruning to add a Project between ScriptTransformation and its child, this strategy can reduce the scan time especially in the scenario of the table has many columns. ## How was this patch tested? Add UT in ColumnPruningSuite and ScriptTransformationSuite. Author: Yuanjian Li Closes #21839 from xuanyuanking/SPARK-24339. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cfc3e1aa Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cfc3e1aa Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cfc3e1aa Branch: refs/heads/master Commit: cfc3e1aaa44b58da660be3378effdc48e088b9d3 Parents: 61f0ca4 Author: Yuanjian Li Authored: Mon Jul 23 13:04:39 2018 -0700 Committer: Xiao Li Committed: Mon Jul 23 13:04:39 2018 -0700 -- .../sql/catalyst/optimizer/Optimizer.scala | 5 +++- .../catalyst/optimizer/ColumnPruningSuite.scala | 24 .../execution/ScriptTransformationSuite.scala | 19 3 files changed, 47 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/cfc3e1aa/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 6faecd3..5ed7412 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -501,13 +501,16 @@ object ColumnPruning extends Rule[LogicalPlan] { case d @ DeserializeToObject(_, _, child) if (child.outputSet -- d.references).nonEmpty => d.copy(child = prunedChild(child, d.references)) -// Prunes the unused columns from child of Aggregate/Expand/Generate +// Prunes the unused columns from child of Aggregate/Expand/Generate/ScriptTransformation case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => a.copy(child = prunedChild(child, a.references)) case f @ FlatMapGroupsInPandas(_, _, _, child) if (child.outputSet -- f.references).nonEmpty => f.copy(child = prunedChild(child, f.references)) case e @ Expand(_, _, child) if (child.outputSet -- e.references).nonEmpty => e.copy(child = prunedChild(child, e.references)) +case s @ ScriptTransformation(_, _, _, child, _) +if (child.outputSet -- s.references).nonEmpty => + s.copy(child = prunedChild(child, s.references)) // prune unrequired references case p @ Project(_, g: Generate) if p.references != g.outputSet => http://git-wip-us.apache.org/repos/asf/spark/blob/cfc3e1aa/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala index 8b05ba3..f6db3c9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -140,6 +140,30 @@ class ColumnPruningSuite extends PlanTest { comparePlans(optimized, expected) } + test("Column pruning for ScriptTransformation") { +val input = LocalRelation('a.int, 'b.string, 'c.double) +val query = + ScriptTransformation( +Seq('a, 'b), +"func", +Seq.empty, +input, +null).analyze +val optimized = Optimize.execute(query) + +val expected = + ScriptTransformation( +Seq('a, 'b), +"func", +Seq.empty, +Project( + Seq('a, 'b), + input), +null).analyze + +comparePlans(optimized, expected) + } + test("Column pruning on Filter") { val input = LocalRelation('a.int, 'b.string, 'c.double) val plan1 = Filter('a > 1, input).analyze http://git-wip-us.apache.org/repos/asf/spark/blob/cfc3e1aa/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/ap
spark git commit: [SPARK-24850][SQL] fix str representation of CachedRDDBuilder
Repository: spark Updated Branches: refs/heads/master 08e315f63 -> 2edf17eff [SPARK-24850][SQL] fix str representation of CachedRDDBuilder ## What changes were proposed in this pull request? As of https://github.com/apache/spark/pull/21018, InMemoryRelation includes its cacheBuilder when logging query plans. This PR changes the string representation of the CachedRDDBuilder to not include the cached spark plan. ## How was this patch tested? spark-shell, query: ``` var df_cached = spark.read.format("csv").option("header", "true").load("test.csv").cache() 0 to 1 foreach { _ => df_cached = df_cached.join(spark.read.format("csv").option("header", "true").load("test.csv"), "A").cache() } df_cached.explain ``` as of master results in: ``` == Physical Plan == InMemoryTableScan [A#10, B#11, B#35, B#87] +- InMemoryRelation [A#10, B#11, B#35, B#87], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(2) Project [A#10, B#11, B#35, B#87] +- *(2) BroadcastHashJoin [A#10], [A#86], Inner, BuildRight :- *(2) Filter isnotnull(A#10) : +- InMemoryTableScan [A#10, B#11, B#35], [isnotnull(A#10)] : +- InMemoryRelation [A#10, B#11, B#35], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(2) Project [A#10, B#11, B#35] +- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight :- *(2) Filter isnotnull(A#10) : +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)] : +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ,None) : +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false])) +- *(1) Filter isnotnull(A#34) +- InMemoryTableScan [A#34, B#35], [isnotnull(A#34)] +- InMemoryRelation [A#34, B#35], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ,None) +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ,None) : +- *(2) Project [A#10, B#11, B#35] : +- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight : :- *(2) Filter isnotnull(A#10) : : +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)] : : +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ,None) : : +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct : +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false])) : +- *(1) Filter isnotnull(A#34) : +- InMemoryTableScan [A#34, B#35], [isnotnull(A#34)] : +- InMemoryRelation [A#34, B#35], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ,None) : +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false])) +- *(1) Filter isnotnull(A#86) +- InMemoryTableScan [A#86, B#87], [isnotnull(A#86)] +- InMemoryRelation [A#86, B#87], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ,None) +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ,None) +- *(2) Project [A#10, B#11, B#35, B#87] +- *(2) BroadcastHashJoin [A#10], [A#86], Inner, BuildRight :- *(2) Filter isnotnull(A#10) : +- InMemoryTableScan [A#10, B#11, B#35], [isnotnull(A#10)] : +- InMemoryRelation [A#10, B#11, B#35], CachedRDDBuilder(true,1,StorageLevel(disk, memory, deserialized, 1 replicas),*(2) Project [A#10, B#11, B#35] +- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight :- *(2) Filter isnotnull(A#10) : +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)] : +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true,100
spark git commit: [SPARK-24887][SQL] Avro: use SerializableConfiguration in Spark utils to deduplicate code
Repository: spark Updated Branches: refs/heads/master 434319e73 -> 08e315f63 [SPARK-24887][SQL] Avro: use SerializableConfiguration in Spark utils to deduplicate code ## What changes were proposed in this pull request? To implement the method `buildReader` in `FileFormat`, it is required to serialize the hadoop configuration for executors. Previous spark-avro uses its own class `SerializableConfiguration` for the serialization. As now it is part of Spark, we can use SerializableConfiguration in Spark util to deduplicate the code. ## How was this patch tested? Unit test Author: Gengliang Wang Closes #21846 from gengliangwang/removeSerializableConfiguration. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/08e315f6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/08e315f6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/08e315f6 Branch: refs/heads/master Commit: 08e315f6330984b757f241079dfc9e1028e5cd0a Parents: 434319e Author: Gengliang Wang Authored: Mon Jul 23 08:31:48 2018 -0700 Committer: Xiao Li Committed: Mon Jul 23 08:31:48 2018 -0700 -- .../apache/spark/sql/avro/AvroFileFormat.scala | 44 + .../avro/SerializableConfigurationSuite.scala | 50 2 files changed, 2 insertions(+), 92 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/08e315f6/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 078efab..b043252 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -23,8 +23,6 @@ import java.util.zip.Deflater import scala.util.control.NonFatal -import com.esotericsoftware.kryo.{Kryo, KryoSerializable} -import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.avro.Schema import org.apache.avro.file.{DataFileConstants, DataFileReader} import org.apache.avro.generic.{GenericDatumReader, GenericRecord} @@ -41,6 +39,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.sources.{DataSourceRegister, Filter} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { private val log = LoggerFactory.getLogger(getClass) @@ -157,7 +156,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { val broadcastedConf = - spark.sparkContext.broadcast(new AvroFileFormat.SerializableConfiguration(hadoopConf)) + spark.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) val parsedOptions = new AvroOptions(options, hadoopConf) (file: PartitionedFile) => { @@ -233,43 +232,4 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { private[avro] object AvroFileFormat { val IgnoreFilesWithoutExtensionProperty = "avro.mapred.ignore.inputs.without.extension" - - class SerializableConfiguration(@transient var value: Configuration) - extends Serializable with KryoSerializable { -@transient private[avro] lazy val log = LoggerFactory.getLogger(getClass) - -private def writeObject(out: ObjectOutputStream): Unit = tryOrIOException { - out.defaultWriteObject() - value.write(out) -} - -private def readObject(in: ObjectInputStream): Unit = tryOrIOException { - value = new Configuration(false) - value.readFields(in) -} - -private def tryOrIOException[T](block: => T): T = { - try { -block - } catch { -case e: IOException => - log.error("Exception encountered", e) - throw e -case NonFatal(e) => - log.error("Exception encountered", e) - throw new IOException(e) - } -} - -def write(kryo: Kryo, out: Output): Unit = { - val dos = new DataOutputStream(out) - value.write(dos) - dos.flush() -} - -def read(kryo: Kryo, in: Input): Unit = { - value = new Configuration(false) - value.readFields(new DataInputStream(in)) -} - } } http://git-wip-us.apache.org/repos/asf/spark/blob/08e315f6/external/avro/src/test/scala/org/apache/spark/sql/avro/SerializableConfigurationSuite.scala -- dif
spark git commit: [SPARK-24802][SQL] Add a new config for Optimization Rule Exclusion
Repository: spark Updated Branches: refs/heads/master ab18b02e6 -> 434319e73 [SPARK-24802][SQL] Add a new config for Optimization Rule Exclusion ## What changes were proposed in this pull request? Since Spark has provided fairly clear interfaces for adding user-defined optimization rules, it would be nice to have an easy-to-use interface for excluding an optimization rule from the Spark query optimizer as well. This would make customizing Spark optimizer easier and sometimes could debugging issues too. - Add a new config spark.sql.optimizer.excludedRules, with the value being a list of rule names separated by comma. - Modify the current batches method to remove the excluded rules from the default batches. Log the rules that have been excluded. - Split the existing default batches into "post-analysis batches" and "optimization batches" so that only rules in the "optimization batches" can be excluded. ## How was this patch tested? Add a new test suite: OptimizerRuleExclusionSuite Author: maryannxue Closes #21764 from maryannxue/rule-exclusion. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/434319e7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/434319e7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/434319e7 Branch: refs/heads/master Commit: 434319e73f8cb6e080671bdde42a72228bd814ef Parents: ab18b02 Author: maryannxue Authored: Mon Jul 23 08:25:24 2018 -0700 Committer: Xiao Li Committed: Mon Jul 23 08:25:24 2018 -0700 -- .../sql/catalyst/optimizer/Optimizer.scala | 53 +- .../org/apache/spark/sql/internal/SQLConf.scala | 10 ++ .../optimizer/OptimizerRuleExclusionSuite.scala | 101 +++ 3 files changed, 163 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/434319e7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 2cc27d8..6faecd3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -46,7 +46,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations) - def batches: Seq[Batch] = { + def defaultBatches: Seq[Batch] = { val operatorOptimizationRuleSet = Seq( // Operator push down @@ -160,6 +160,22 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) UpdateNullabilityInAttributeReferences) } + def nonExcludableRules: Seq[String] = +EliminateDistinct.ruleName :: + EliminateSubqueryAliases.ruleName :: + EliminateView.ruleName :: + ReplaceExpressions.ruleName :: + ComputeCurrentTime.ruleName :: + GetCurrentDatabase(sessionCatalog).ruleName :: + RewriteDistinctAggregates.ruleName :: + ReplaceDeduplicateWithAggregate.ruleName :: + ReplaceIntersectWithSemiJoin.ruleName :: + ReplaceExceptWithFilter.ruleName :: + ReplaceExceptWithAntiJoin.ruleName :: + ReplaceDistinctWithAggregate.ruleName :: + PullupCorrelatedPredicates.ruleName :: + RewritePredicateSubquery.ruleName :: Nil + /** * Optimize all the subqueries inside expression. */ @@ -175,6 +191,41 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) * Override to provide additional rules for the operator optimization batch. */ def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil + + override def batches: Seq[Batch] = { +val excludedRulesConf = + SQLConf.get.optimizerExcludedRules.toSeq.flatMap(Utils.stringToSeq) +val excludedRules = excludedRulesConf.filter { ruleName => + val nonExcludable = nonExcludableRules.contains(ruleName) + if (nonExcludable) { +logWarning(s"Optimization rule '${ruleName}' was not excluded from the optimizer " + + s"because this rule is a non-excludable rule.") + } + !nonExcludable +} +if (excludedRules.isEmpty) { + defaultBatches +} else { + defaultBatches.flatMap { batch => +val filteredRules = batch.rules.filter { rule => + val exclude = excludedRules.contains(rule.ruleName) + if (exclude) { +logInfo(s"Optimization rule '${rule.ruleName}' is excluded from the optimizer.") + } + !exclude +} +if (batch.rules == filteredRules) { + Some(batch) +} else if (filteredRules.nonEmpty) { + Some(Batch(batch.nam
spark git commit: [SPARK-24811][SQL] Avro: add new function from_avro and to_avro
Repository: spark Updated Branches: refs/heads/master 81af88687 -> 8817c68f5 [SPARK-24811][SQL] Avro: add new function from_avro and to_avro ## What changes were proposed in this pull request? 1. Add a new function from_avro for parsing a binary column of avro format and converting it into its corresponding catalyst value. 2. Add a new function to_avro for converting a column into binary of avro format with the specified schema. I created #21774 for this, but it failed the build https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.6/7902/ Additional changes In this PR: 1. Add `scalacheck` dependency in pom.xml to resolve the failure. 2. Update the `log4j.properties` to make it consistent with other modules. ## How was this patch tested? Unit test Compile with different commands: ``` ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.6 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.7 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-3.1 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ``` Author: Gengliang Wang Closes #21838 from gengliangwang/from_and_to_avro. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8817c68f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8817c68f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8817c68f Branch: refs/heads/master Commit: 8817c68f5099901753585716e00281736938bca0 Parents: 81af886 Author: Gengliang Wang Authored: Sun Jul 22 17:36:57 2018 -0700 Committer: Xiao Li Committed: Sun Jul 22 17:36:57 2018 -0700 -- external/avro/pom.xml | 5 + .../spark/sql/avro/AvroDataToCatalyst.scala | 68 +++ .../spark/sql/avro/CatalystDataToAvro.scala | 69 .../org/apache/spark/sql/avro/package.scala | 31 .../avro/src/test/resources/log4j.properties| 39 + .../avro/AvroCatalystDataConversionSuite.scala | 175 +++ .../spark/sql/avro/AvroFunctionsSuite.scala | 83 + .../expressions/ExpressionEvalHelper.scala | 6 + 8 files changed, 446 insertions(+), 30 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8817c68f/external/avro/pom.xml -- diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 42e865b..ad7df1f 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -62,6 +62,11 @@ test + org.scalacheck + scalacheck_${scala.binary.version} + test + + org.apache.spark spark-tags_${scala.binary.version} http://git-wip-us.apache.org/repos/asf/spark/blob/8817c68f/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala new file mode 100644 index 000..6671b3f --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.avro.Schema +import org.apache.avro.generic.GenericDatumReader +import org.apache.avro.io.{BinaryDecoder, DecoderFactory} + +import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} +import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType} + +case class AvroDa
spark git commit: [SPARK-24836][SQL] New option for Avro datasource - ignoreExtension
Repository: spark Updated Branches: refs/heads/master bbd6f0c25 -> 106880edc [SPARK-24836][SQL] New option for Avro datasource - ignoreExtension ## What changes were proposed in this pull request? I propose to add new option for AVRO datasource which should control ignoring of files without `.avro` extension in read. The option has name `ignoreExtension` with default value `true`. If both options `ignoreExtension` and `avro.mapred.ignore.inputs.without.extension` are set, `ignoreExtension` overrides the former one. Here is an example of usage: ``` spark .read .option("ignoreExtension", false) .avro("path to avro files") ``` ## How was this patch tested? I added a test which checks the option directly and a test for checking that new option overrides hadoop's config. Author: Maxim Gekk Closes #21798 from MaxGekk/avro-ignore-extension. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/106880ed Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/106880ed Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/106880ed Branch: refs/heads/master Commit: 106880edcd67bc20e8610a16f8ce6aa250268eeb Parents: bbd6f0c Author: Maxim Gekk Authored: Fri Jul 20 20:04:40 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 20:04:40 2018 -0700 -- .../apache/spark/sql/avro/AvroFileFormat.scala | 33 +--- .../org/apache/spark/sql/avro/AvroOptions.scala | 29 +-- .../org/apache/spark/sql/avro/AvroSuite.scala | 55 +++- 3 files changed, 91 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/106880ed/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 780e457..078efab 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -58,21 +58,19 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { val conf = spark.sparkContext.hadoopConfiguration -val parsedOptions = new AvroOptions(options) +val parsedOptions = new AvroOptions(options, conf) // Schema evolution is not supported yet. Here we only pick a single random sample file to // figure out the schema of the whole dataset. val sampleFile = - if (AvroFileFormat.ignoreFilesWithoutExtensions(conf)) { -files.find(_.getPath.getName.endsWith(".avro")).getOrElse { - throw new FileNotFoundException( -"No Avro files found. Hadoop option \"avro.mapred.ignore.inputs.without.extension\" " + - " is set to true. Do all input files have \".avro\" extension?" - ) + if (parsedOptions.ignoreExtension) { +files.headOption.getOrElse { + throw new FileNotFoundException("Files for schema inferring have been not found.") } } else { -files.headOption.getOrElse { - throw new FileNotFoundException("No Avro files found.") +files.find(_.getPath.getName.endsWith(".avro")).getOrElse { + throw new FileNotFoundException( +"No Avro files found. If files don't have .avro extension, set ignoreExtension to true") } } @@ -115,7 +113,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { -val parsedOptions = new AvroOptions(options) +val parsedOptions = new AvroOptions(options, spark.sessionState.newHadoopConf()) val outputAvroSchema = SchemaConverters.toAvroType( dataSchema, nullable = false, parsedOptions.recordName, parsedOptions.recordNamespace) @@ -160,7 +158,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { val broadcastedConf = spark.sparkContext.broadcast(new AvroFileFormat.SerializableConfiguration(hadoopConf)) -val parsedOptions = new AvroOptions(options) +val parsedOptions = new AvroOptions(options, hadoopConf) (file: PartitionedFile) => { val log = LoggerFactory.getLogger(classOf[AvroFileFormat]) @@ -171,9 +169,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Doing input file filtering is improper because we may generate empty tasks that process no // input files but stress the scheduler. We should probably add a more general in
spark git commit: [SPARK-24879][SQL] Fix NPE in Hive partition pruning filter pushdown
Repository: spark Updated Branches: refs/heads/branch-2.3 db1f3cc76 -> bd6bfacb2 [SPARK-24879][SQL] Fix NPE in Hive partition pruning filter pushdown ## What changes were proposed in this pull request? We get a NPE when we have a filter on a partition column of the form `col in (x, null)`. This is due to the filter converter in HiveShim not handling `null`s correctly. This patch fixes this bug while still pushing down as much of the partition pruning predicates as possible, by filtering out `null`s from any `in` predicate. Since Hive only supports very simple partition pruning filters, this change should preserve correctness. ## How was this patch tested? Unit tests, manual tests Author: William Sheu Closes #21832 from PenguinToast/partition-pruning-npe. (cherry picked from commit bbd6f0c25fe19dc6c946e63cac7b98d0f78b3463) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bd6bfacb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bd6bfacb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bd6bfacb Branch: refs/heads/branch-2.3 Commit: bd6bfacb2ef473b858cc91e0692591deaea26118 Parents: db1f3cc Author: William Sheu Authored: Fri Jul 20 19:59:28 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 20:00:17 2018 -0700 -- .../apache/spark/sql/hive/client/HiveShim.scala | 19 ++- .../spark/sql/hive/client/FiltersSuite.scala | 14 ++ 2 files changed, 32 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bd6bfacb/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 60fe31f..68acea7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -599,6 +599,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { object ExtractableLiteral { def unapply(expr: Expression): Option[String] = expr match { +case Literal(null, _) => None // `null`s can be cast as other types; we want to avoid NPEs. case Literal(value, _: IntegralType) => Some(value.toString) case Literal(value, _: StringType) => Some(quoteStringLiteral(value.toString)) case _ => None @@ -607,7 +608,23 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { object ExtractableLiterals { def unapply(exprs: Seq[Expression]): Option[Seq[String]] = { -val extractables = exprs.map(ExtractableLiteral.unapply) +// SPARK-24879: The Hive metastore filter parser does not support "null", but we still want +// to push down as many predicates as we can while still maintaining correctness. +// In SQL, the `IN` expression evaluates as follows: +// > `1 in (2, NULL)` -> NULL +// > `1 in (1, NULL)` -> true +// > `1 in (2)` -> false +// Since Hive metastore filters are NULL-intolerant binary operations joined only by +// `AND` and `OR`, we can treat `NULL` as `false` and thus rewrite `1 in (2, NULL)` as +// `1 in (2)`. +// If the Hive metastore begins supporting NULL-tolerant predicates and Spark starts +// pushing down these predicates, then this optimization will become incorrect and need +// to be changed. +val extractables = exprs +.filter { + case Literal(null, _) => false + case _ => true +}.map(ExtractableLiteral.unapply) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { http://git-wip-us.apache.org/repos/asf/spark/blob/bd6bfacb/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 1976569..2a4efd0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -72,6 +72,20 @@ class FiltersSuite extends SparkFunSuite with Logging with PlanTest { (Literal("p2\" and q=\"q2") === a("stringcol", StringType)) :: Nil, """stringcol = 'p1" and q="q1' and 'p2" and q="q2' = stringcol""") + filterTest("SPARK-24879 null literals should be ignored for IN constructs", +(a("intcol", IntegerType) in (Lit
spark git commit: [SPARK-24879][SQL] Fix NPE in Hive partition pruning filter pushdown
Repository: spark Updated Branches: refs/heads/master 96f312076 -> bbd6f0c25 [SPARK-24879][SQL] Fix NPE in Hive partition pruning filter pushdown ## What changes were proposed in this pull request? We get a NPE when we have a filter on a partition column of the form `col in (x, null)`. This is due to the filter converter in HiveShim not handling `null`s correctly. This patch fixes this bug while still pushing down as much of the partition pruning predicates as possible, by filtering out `null`s from any `in` predicate. Since Hive only supports very simple partition pruning filters, this change should preserve correctness. ## How was this patch tested? Unit tests, manual tests Author: William Sheu Closes #21832 from PenguinToast/partition-pruning-npe. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bbd6f0c2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bbd6f0c2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bbd6f0c2 Branch: refs/heads/master Commit: bbd6f0c25fe19dc6c946e63cac7b98d0f78b3463 Parents: 96f3120 Author: William Sheu Authored: Fri Jul 20 19:59:28 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 19:59:28 2018 -0700 -- .../apache/spark/sql/hive/client/HiveShim.scala | 19 ++- .../spark/sql/hive/client/FiltersSuite.scala | 14 ++ 2 files changed, 32 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bbd6f0c2/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 933384e..bc9d4cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -598,6 +598,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { object ExtractableLiteral { def unapply(expr: Expression): Option[String] = expr match { +case Literal(null, _) => None // `null`s can be cast as other types; we want to avoid NPEs. case Literal(value, _: IntegralType) => Some(value.toString) case Literal(value, _: StringType) => Some(quoteStringLiteral(value.toString)) case _ => None @@ -606,7 +607,23 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { object ExtractableLiterals { def unapply(exprs: Seq[Expression]): Option[Seq[String]] = { -val extractables = exprs.map(ExtractableLiteral.unapply) +// SPARK-24879: The Hive metastore filter parser does not support "null", but we still want +// to push down as many predicates as we can while still maintaining correctness. +// In SQL, the `IN` expression evaluates as follows: +// > `1 in (2, NULL)` -> NULL +// > `1 in (1, NULL)` -> true +// > `1 in (2)` -> false +// Since Hive metastore filters are NULL-intolerant binary operations joined only by +// `AND` and `OR`, we can treat `NULL` as `false` and thus rewrite `1 in (2, NULL)` as +// `1 in (2)`. +// If the Hive metastore begins supporting NULL-tolerant predicates and Spark starts +// pushing down these predicates, then this optimization will become incorrect and need +// to be changed. +val extractables = exprs +.filter { + case Literal(null, _) => false + case _ => true +}.map(ExtractableLiteral.unapply) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { http://git-wip-us.apache.org/repos/asf/spark/blob/bbd6f0c2/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 1976569..2a4efd0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -72,6 +72,20 @@ class FiltersSuite extends SparkFunSuite with Logging with PlanTest { (Literal("p2\" and q=\"q2") === a("stringcol", StringType)) :: Nil, """stringcol = 'p1" and q="q1' and 'p2" and q="q2' = stringcol""") + filterTest("SPARK-24879 null literals should be ignored for IN constructs", +(a("intcol", IntegerType) in (Literal(1), Literal(null))) :: Nil, +"(intcol = 1)") + + // Applying the predicate `x IN (NULL)` sho
spark git commit: [PYSPARK][TEST][MINOR] Fix UDFInitializationTests
Repository: spark Updated Branches: refs/heads/master 597bdeff2 -> 96f312076 [PYSPARK][TEST][MINOR] Fix UDFInitializationTests ## What changes were proposed in this pull request? Fix a typo in pyspark sql tests Author: William Sheu Closes #21833 from PenguinToast/fix-test-typo. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/96f31207 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/96f31207 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/96f31207 Branch: refs/heads/master Commit: 96f3120760ba0a83ef6347327ecfb130487e02dd Parents: 597bdef Author: William Sheu Authored: Fri Jul 20 19:48:32 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 19:48:32 2018 -0700 -- python/pyspark/sql/tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/96f31207/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 565654e..2d6b9f0 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -3595,7 +3595,7 @@ class UDFInitializationTests(unittest.TestCase): SparkSession._instantiatedSession.stop() if SparkContext._active_spark_context is not None: -SparkContext._active_spark_contex.stop() +SparkContext._active_spark_context.stop() def test_udf_init_shouldnt_initalize_context(self): from pyspark.sql.functions import UserDefinedFunction - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24880][BUILD] Fix the group id for spark-kubernetes-integration-tests
Repository: spark Updated Branches: refs/heads/master 00b864aa7 -> f765bb782 [SPARK-24880][BUILD] Fix the group id for spark-kubernetes-integration-tests ## What changes were proposed in this pull request? The correct group id should be `org.apache.spark`. This is causing the nightly build failure: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-maven-snapshots/2295/console ` [ERROR] Failed to execute goal org.apache.maven.plugins:maven-deploy-plugin:2.8.2:deploy (default-deploy) on project spark-kubernetes-integration-tests_2.11: Failed to deploy artifacts: Could not transfer artifact spark-kubernetes-integration-tests:spark-kubernetes-integration-tests_2.11:jar:2.4.0-20180720.101629-1 from/to apache.snapshots.https (https://repository.apache.org/content/repositories/snapshots): Access denied to: https://repository.apache.org/content/repositories/snapshots/spark-kubernetes-integration-tests/spark-kubernetes-integration-tests_2.11/2.4.0-SNAPSHOT/spark-kubernetes-integration-tests_2.11-2.4.0-20180720.101629-1.jar, ReasonPhrase: Forbidden. -> [Help 1] [ERROR] ` ## How was this patch tested? Jenkins. Author: zsxwing Closes #21831 from zsxwing/fix-k8s-test. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f765bb78 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f765bb78 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f765bb78 Branch: refs/heads/master Commit: f765bb7823a60440cb42819edd98b14f65e13b18 Parents: 00b864a Author: zsxwing Authored: Fri Jul 20 15:23:04 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 15:23:04 2018 -0700 -- resource-managers/kubernetes/integration-tests/pom.xml | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f765bb78/resource-managers/kubernetes/integration-tests/pom.xml -- diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 29334cc..614705c 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -25,7 +25,6 @@ spark-kubernetes-integration-tests_2.11 - spark-kubernetes-integration-tests 1.3.0 1.4.0 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24876][SQL] Avro: simplify schema serialization
Repository: spark Updated Branches: refs/heads/master 2333a34d3 -> 00b864aa7 [SPARK-24876][SQL] Avro: simplify schema serialization ## What changes were proposed in this pull request? Previously in the refactoring of Avro Serializer and Deserializer, a new class SerializableSchema is created for serializing the Avro schema: https://github.com/apache/spark/pull/21762/files#diff-01fea32e6ec6bcf6f34d06282e08705aR37 On second thought, we can use `toString` method for serialization. After that, parse the JSON format schema on executor. This makes the code much simpler. ## How was this patch tested? Unit test Author: Gengliang Wang Closes #21829 from gengliangwang/removeSerializableSchema. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/00b864aa Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/00b864aa Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/00b864aa Branch: refs/heads/master Commit: 00b864aa7054a34f3d7a118d92eae0b3c28b86e5 Parents: 2333a34 Author: Gengliang Wang Authored: Fri Jul 20 14:57:59 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 14:57:59 2018 -0700 -- .../apache/spark/sql/avro/AvroFileFormat.scala | 2 +- .../sql/avro/AvroOutputWriterFactory.scala | 14 +++- .../spark/sql/avro/SerializableSchema.scala | 69 .../sql/avro/SerializableSchemaSuite.scala | 56 4 files changed, 12 insertions(+), 129 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/00b864aa/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 1d0f40e..780e457 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -146,7 +146,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { log.error(s"unsupported compression codec $unknown") } -new AvroOutputWriterFactory(dataSchema, new SerializableSchema(outputAvroSchema)) +new AvroOutputWriterFactory(dataSchema, outputAvroSchema.toString) } override def buildReader( http://git-wip-us.apache.org/repos/asf/spark/blob/00b864aa/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala index 18a6d93..116020e 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala @@ -17,14 +17,22 @@ package org.apache.spark.sql.avro +import org.apache.avro.Schema import org.apache.hadoop.mapreduce.TaskAttemptContext import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} import org.apache.spark.sql.types.StructType +/** + * A factory that produces [[AvroOutputWriter]]. + * @param catalystSchema Catalyst schema of input data. + * @param avroSchemaAsJsonString Avro schema of output result, in JSON string format. + */ private[avro] class AvroOutputWriterFactory( -schema: StructType, -avroSchema: SerializableSchema) extends OutputWriterFactory { +catalystSchema: StructType, +avroSchemaAsJsonString: String) extends OutputWriterFactory { + + private lazy val avroSchema = new Schema.Parser().parse(avroSchemaAsJsonString) override def getFileExtension(context: TaskAttemptContext): String = ".avro" @@ -32,6 +40,6 @@ private[avro] class AvroOutputWriterFactory( path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { -new AvroOutputWriter(path, context, schema, avroSchema.value) +new AvroOutputWriter(path, context, catalystSchema, avroSchema) } } http://git-wip-us.apache.org/repos/asf/spark/blob/00b864aa/external/avro/src/main/scala/org/apache/spark/sql/avro/SerializableSchema.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/SerializableSchema.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/SerializableSchema.scala deleted file mode 100644 index ec0ddc7..000 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/SerializableSchema.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache
spark git commit: [SPARK-22880][SQL] Add cascadeTruncate option to JDBC datasource
Repository: spark Updated Branches: refs/heads/master 9ad77b303 -> 2333a34d3 [SPARK-22880][SQL] Add cascadeTruncate option to JDBC datasource This commit adds the `cascadeTruncate` option to the JDBC datasource API, for databases that support this functionality (PostgreSQL and Oracle at the moment). This allows for applying a cascading truncate that affects tables that have foreign key constraints on the table being truncated. ## What changes were proposed in this pull request? Add `cascadeTruncate` option to JDBC datasource API. Allow this to affect the `TRUNCATE` query for databases that support this option. ## How was this patch tested? Existing tests for `truncateQuery` were updated. Also, an additional test was added to ensure that the correct syntax was applied, and that enabling the config for databases that do not support this option does not result in invalid queries. Author: Daniel van der Ende Closes #20057 from danielvdende/SPARK-22880. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2333a34d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2333a34d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2333a34d Branch: refs/heads/master Commit: 2333a34d390f2fa19b939b8007be0deb31f31d3c Parents: 9ad77b3 Author: Daniel van der Ende Authored: Fri Jul 20 13:03:57 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 13:03:57 2018 -0700 -- docs/sql-programming-guide.md | 7 +++ .../datasources/jdbc/JDBCOptions.scala | 3 ++ .../execution/datasources/jdbc/JdbcUtils.scala | 7 ++- .../spark/sql/jdbc/AggregatedDialect.scala | 13 +- .../apache/spark/sql/jdbc/DerbyDialect.scala| 2 + .../apache/spark/sql/jdbc/JdbcDialects.scala| 20 - .../apache/spark/sql/jdbc/OracleDialect.scala | 16 +++ .../apache/spark/sql/jdbc/PostgresDialect.scala | 29 .../apache/spark/sql/jdbc/TeradataDialect.scala | 18 .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 46 +--- 10 files changed, 140 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2333a34d/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index ad23dae..4bab58a 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1407,6 +1407,13 @@ the following case-insensitive options: 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. + + +cascadeTruncate + +This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE (in the case of PostgreSQL a TRUNCATE TABLE ONLY t CASCADE is executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. This option applies only to writing. It defaults to the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate in each JDBCDialect. + + createTableOptions http://git-wip-us.apache.org/repos/asf/spark/blob/2333a34d/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala -- 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 eea966d..574aed4 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 @@ -157,6 +157,8 @@ class JDBCOptions( // // if to truncate the table from the JDBC database val isTruncate = parameters.getOrElse(JDBC_TRUNCATE, "false").toBoolean + + val isCascadeTruncate: Option[Boolean] = parameters.get(JDBC_CASCADE_TRUNCATE).map(_.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 @
spark git commit: Revert "[SPARK-24811][SQL] Avro: add new function from_avro and to_avro"
Repository: spark Updated Branches: refs/heads/master 3cb1b5780 -> 9ad77b303 Revert "[SPARK-24811][SQL] Avro: add new function from_avro and to_avro" This reverts commit 244bcff19463d82ec72baf15bc0a5209f21f2ef3. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9ad77b30 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9ad77b30 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9ad77b30 Branch: refs/heads/master Commit: 9ad77b3037b476b726b773c38d1cd264d89d51e2 Parents: 3cb1b57 Author: Xiao Li Authored: Fri Jul 20 12:55:38 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 12:55:38 2018 -0700 -- .../spark/sql/avro/AvroDataToCatalyst.scala | 68 --- .../spark/sql/avro/CatalystDataToAvro.scala | 69 .../org/apache/spark/sql/avro/package.scala | 31 .../avro/AvroCatalystDataConversionSuite.scala | 175 --- .../spark/sql/avro/AvroFunctionsSuite.scala | 83 - .../expressions/ExpressionEvalHelper.scala | 6 - 6 files changed, 432 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9ad77b30/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala deleted file mode 100644 index 6671b3f..000 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - *http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.avro.Schema -import org.apache.avro.generic.GenericDatumReader -import org.apache.avro.io.{BinaryDecoder, DecoderFactory} - -import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters} -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} -import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType} - -case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) - extends UnaryExpression with ExpectsInputTypes { - - override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) - - override lazy val dataType: DataType = SchemaConverters.toSqlType(avroSchema).dataType - - override def nullable: Boolean = true - - @transient private lazy val avroSchema = new Schema.Parser().parse(jsonFormatSchema) - - @transient private lazy val reader = new GenericDatumReader[Any](avroSchema) - - @transient private lazy val deserializer = new AvroDeserializer(avroSchema, dataType) - - @transient private var decoder: BinaryDecoder = _ - - @transient private var result: Any = _ - - override def nullSafeEval(input: Any): Any = { -val binary = input.asInstanceOf[Array[Byte]] -decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) -result = reader.read(result, decoder) -deserializer.deserialize(result) - } - - override def simpleString: String = { -s"from_avro(${child.sql}, ${dataType.simpleString})" - } - - override def sql: String = { -s"from_avro(${child.sql}, ${dataType.catalogString})" - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { -val expr = ctx.addReferenceObj("this", this) -defineCodeGen(ctx, ev, input => - s"(${CodeGenerator.boxedType(dataType)})$expr.nullSafeEval($input)") - } -} http://git-wip-us.apache.org/repos/asf/spark/blob/9ad77b30/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala deleted file mode 10064
spark git commit: [SPARK-24811][SQL] Avro: add new function from_avro and to_avro
Repository: spark Updated Branches: refs/heads/master cc4d64bb1 -> 244bcff19 [SPARK-24811][SQL] Avro: add new function from_avro and to_avro ## What changes were proposed in this pull request? Add a new function from_avro for parsing a binary column of avro format and converting it into its corresponding catalyst value. Add a new function to_avro for converting a column into binary of avro format with the specified schema. This PR is in progress. Will add test cases. ## How was this patch tested? Author: Gengliang Wang Closes #21774 from gengliangwang/from_and_to_avro. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/244bcff1 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/244bcff1 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/244bcff1 Branch: refs/heads/master Commit: 244bcff19463d82ec72baf15bc0a5209f21f2ef3 Parents: cc4d64b Author: Gengliang Wang Authored: Fri Jul 20 09:19:29 2018 -0700 Committer: Xiao Li Committed: Fri Jul 20 09:19:29 2018 -0700 -- .../spark/sql/avro/AvroDataToCatalyst.scala | 68 +++ .../spark/sql/avro/CatalystDataToAvro.scala | 69 .../org/apache/spark/sql/avro/package.scala | 31 .../avro/AvroCatalystDataConversionSuite.scala | 175 +++ .../spark/sql/avro/AvroFunctionsSuite.scala | 83 + .../expressions/ExpressionEvalHelper.scala | 6 + 6 files changed, 432 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/244bcff1/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala new file mode 100644 index 000..6671b3f --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.avro.Schema +import org.apache.avro.generic.GenericDatumReader +import org.apache.avro.io.{BinaryDecoder, DecoderFactory} + +import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} +import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType} + +case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) + extends UnaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) + + override lazy val dataType: DataType = SchemaConverters.toSqlType(avroSchema).dataType + + override def nullable: Boolean = true + + @transient private lazy val avroSchema = new Schema.Parser().parse(jsonFormatSchema) + + @transient private lazy val reader = new GenericDatumReader[Any](avroSchema) + + @transient private lazy val deserializer = new AvroDeserializer(avroSchema, dataType) + + @transient private var decoder: BinaryDecoder = _ + + @transient private var result: Any = _ + + override def nullSafeEval(input: Any): Any = { +val binary = input.asInstanceOf[Array[Byte]] +decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) +result = reader.read(result, decoder) +deserializer.deserialize(result) + } + + override def simpleString: String = { +s"from_avro(${child.sql}, ${dataType.simpleString})" + } + + override def sql: String = { +s"from_avro(${child.sql}, ${dataType.catalogString})" + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { +val expr = ctx.addReferenceObj("this", this) +defineCodeGen(ctx, ev, input => + s"(${CodeGenerator.boxedType(dataType)})$expr.nullSafeEval($input)") + } +} http://git-wip-us.apache.org/repos/
spark git commit: [SPARK-24424][SQL] Support ANSI-SQL compliant syntax for GROUPING SET
Repository: spark Updated Branches: refs/heads/master a5925c163 -> 2b91d9918 [SPARK-24424][SQL] Support ANSI-SQL compliant syntax for GROUPING SET ## What changes were proposed in this pull request? Enhances the parser and analyzer to support ANSI compliant syntax for GROUPING SET. As part of this change we derive the grouping expressions from user supplied groupings in the grouping sets clause. ```SQL SELECT c1, c2, max(c3) FROM t1 GROUP BY GROUPING SETS ((c1), (c1, c2)) ``` ## How was this patch tested? Added tests in SQLQueryTestSuite and ResolveGroupingAnalyticsSuite. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Dilip Biswal Closes #21813 from dilipbiswal/spark-24424. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2b91d991 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2b91d991 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2b91d991 Branch: refs/heads/master Commit: 2b91d9918c8eaec6c32a502e2f08b63c475d3335 Parents: a5925c1 Author: Dilip Biswal Authored: Thu Jul 19 23:52:53 2018 -0700 Committer: Xiao Li Committed: Thu Jul 19 23:52:53 2018 -0700 -- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 1 + .../spark/sql/catalyst/analysis/Analyzer.scala | 22 +++- .../ResolveGroupingAnalyticsSuite.scala | 28 + .../resources/sql-tests/inputs/grouping_set.sql | 36 ++ .../sql-tests/results/grouping_set.sql.out | 126 ++- 5 files changed, 210 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2b91d991/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 -- 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 1b43874..2aca10f 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 @@ -406,6 +406,7 @@ aggregation WITH kind=ROLLUP | WITH kind=CUBE | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')? +| GROUP BY kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')' ; groupingSet http://git-wip-us.apache.org/repos/asf/spark/blob/2b91d991/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7c5504d..957c468 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 @@ -442,17 +442,35 @@ class Analyzer( child: LogicalPlan): LogicalPlan = { val gid = AttributeReference(VirtualColumn.groupingIdName, IntegerType, false)() + // In case of ANSI-SQL compliant syntax for GROUPING SETS, groupByExprs is optional and + // can be null. In such case, we derive the groupByExprs from the user supplied values for + // grouping sets. + val finalGroupByExpressions = if (groupByExprs == Nil) { +selectedGroupByExprs.flatten.foldLeft(Seq.empty[Expression]) { (result, currentExpr) => + // Only unique expressions are included in the group by expressions and is determined + // based on their semantic equality. Example. grouping sets ((a * b), (b * a)) results + // in grouping expression (a * b) + if (result.find(_.semanticEquals(currentExpr)).isDefined) { +result + } else { +result :+ currentExpr + } +} + } else { +groupByExprs + } + // Expand works by setting grouping expressions to null as determined by the // `selectedGroupByExprs`. To prevent these null values from being used in an aggregate // instead of the original value we need to create new aliases for all group by expressions // that will only be used for the intended purpose. - val groupByAliases = constructGroupByAlias(groupByExprs) + val groupByAliases = constructGroupByAlias(finalGroupByExpressions) val expand = constructExpand(selectedGroupByExprs, child, groupByAliases, gid) val groupingAttrs = expand.output.drop(child.output.length) val aggregations = constructAggregateExprs( -groupByExprs, aggregationExprs, groupByAliases, groupingAttrs, gid) +finalGroupByExpressions, aggregationExprs, groupByA
[2/2] spark git commit: [SPARK-24268][SQL] Use datatype.catalogString in error messages
[SPARK-24268][SQL] Use datatype.catalogString in error messages ## What changes were proposed in this pull request? As stated in https://github.com/apache/spark/pull/21321, in the error messages we should use `catalogString`. This is not the case, as SPARK-22893 used `simpleString` in order to have the same representation everywhere and it missed some places. The PR unifies the messages using alway the `catalogString` representation of the dataTypes in the messages. ## How was this patch tested? existing/modified UTs Author: Marco Gaido Closes #21804 from mgaido91/SPARK-24268_catalog. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a5925c16 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a5925c16 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a5925c16 Branch: refs/heads/master Commit: a5925c1631e25c2dcc3c2948cea31e993ce66a97 Parents: 1462b17 Author: Marco Gaido Authored: Thu Jul 19 23:29:29 2018 -0700 Committer: Xiao Li Committed: Thu Jul 19 23:29:29 2018 -0700 -- .../spark/sql/kafka010/KafkaWriteTask.scala | 6 +++--- .../apache/spark/sql/kafka010/KafkaWriter.scala | 6 +++--- .../sql/kafka010/KafkaContinuousSinkSuite.scala | 4 ++-- .../spark/sql/kafka010/KafkaSinkSuite.scala | 4 ++-- .../scala/org/apache/spark/ml/feature/DCT.scala | 3 ++- .../apache/spark/ml/feature/FeatureHasher.scala | 5 +++-- .../org/apache/spark/ml/feature/HashingTF.scala | 2 +- .../org/apache/spark/ml/feature/Interaction.scala | 3 ++- .../scala/org/apache/spark/ml/feature/NGram.scala | 3 ++- .../apache/spark/ml/feature/OneHotEncoder.scala | 3 ++- .../org/apache/spark/ml/feature/RFormula.scala| 2 +- .../spark/ml/feature/StopWordsRemover.scala | 4 ++-- .../org/apache/spark/ml/feature/Tokenizer.scala | 3 ++- .../apache/spark/ml/feature/VectorAssembler.scala | 2 +- .../scala/org/apache/spark/ml/fpm/FPGrowth.scala | 2 +- .../org/apache/spark/ml/util/SchemaUtils.scala| 11 +++ .../BinaryClassificationEvaluatorSuite.scala | 4 ++-- .../apache/spark/ml/feature/RFormulaSuite.scala | 2 +- .../spark/ml/feature/VectorAssemblerSuite.scala | 6 +++--- .../apache/spark/ml/recommendation/ALSSuite.scala | 2 +- .../regression/AFTSurvivalRegressionSuite.scala | 2 +- .../org/apache/spark/ml/util/MLTestingUtils.scala | 6 +++--- .../spark/sql/catalyst/analysis/Analyzer.scala| 8 .../sql/catalyst/analysis/CheckAnalysis.scala | 18 +- .../apache/spark/sql/catalyst/analysis/view.scala | 3 ++- .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../catalyst/expressions/ExpectsInputTypes.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 4 ++-- .../spark/sql/catalyst/expressions/ScalaUDF.scala | 5 +++-- .../sql/catalyst/expressions/SortOrder.scala | 2 +- .../aggregate/ApproximatePercentile.scala | 4 ++-- .../sql/catalyst/expressions/arithmetic.scala | 4 ++-- .../expressions/codegen/CodeGenerator.scala | 4 ++-- .../expressions/collectionOperations.scala| 14 +++--- .../catalyst/expressions/complexTypeCreator.scala | 8 .../expressions/complexTypeExtractors.scala | 2 +- .../expressions/conditionalExpressions.scala | 4 ++-- .../sql/catalyst/expressions/generators.scala | 8 .../catalyst/expressions/jsonExpressions.scala| 6 +++--- .../catalyst/expressions/objects/objects.scala| 2 +- .../sql/catalyst/expressions/predicates.scala | 2 +- .../catalyst/expressions/stringExpressions.scala | 7 --- .../catalyst/expressions/windowExpressions.scala | 6 +++--- .../sql/catalyst/json/JacksonGenerator.scala | 8 .../spark/sql/catalyst/json/JacksonParser.scala | 6 -- .../spark/sql/catalyst/json/JacksonUtils.scala| 2 +- .../spark/sql/catalyst/json/JsonInferSchema.scala | 6 -- .../spark/sql/catalyst/util/TypeUtils.scala | 7 --- .../apache/spark/sql/types/AbstractDataType.scala | 9 + .../org/apache/spark/sql/types/ArrayType.scala| 5 +++-- .../org/apache/spark/sql/types/DecimalType.scala | 3 ++- .../org/apache/spark/sql/types/ObjectType.scala | 3 ++- .../org/apache/spark/sql/types/StructType.scala | 5 +++-- .../catalyst/analysis/AnalysisErrorSuite.scala| 2 +- .../analysis/ExpressionTypeCheckingSuite.scala| 16 .../catalyst/parser/ExpressionParserSuite.scala | 2 +- .../apache/spark/sql/types/DataTypeSuite.scala| 2 +- .../parquet/VectorizedColumnReader.java | 2 +- .../spark/sql/RelationalGroupedDataset.scala | 2 +- .../spark/sql/execution/arrow/ArrowUtils.scala| 6 -- .../spark/sql/execution/arrow/ArrowWriter.scala | 2 +- .../spark/sql/execution/columnar/ColumnType.scala | 2 +- .../execution/datasou
[1/2] spark git commit: [SPARK-24268][SQL] Use datatype.catalogString in error messages
Repository: spark Updated Branches: refs/heads/master 1462b1766 -> a5925c163 http://git-wip-us.apache.org/repos/asf/spark/blob/a5925c16/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java -- diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index d5969b5..31ef090 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -244,7 +244,7 @@ public class VectorizedColumnReader { return new SchemaColumnConvertNotSupportedException( Arrays.toString(descriptor.getPath()), descriptor.getType().toString(), - column.dataType().toString()); + column.dataType().catalogString()); } /** http://git-wip-us.apache.org/repos/asf/spark/blob/a5925c16/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala -- 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 c6449cd..b068493 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 @@ -452,7 +452,7 @@ class RelationalGroupedDataset protected[sql]( require(expr.evalType == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, "Must pass a grouped map udf") require(expr.dataType.isInstanceOf[StructType], - "The returnType of the udf must be a StructType") + s"The returnType of the udf must be a ${StructType.simpleString}") val groupingNamedExpressions = groupingExprs.map { case ne: NamedExpression => ne http://git-wip-us.apache.org/repos/asf/spark/blob/a5925c16/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala index 93c8127..533097a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala @@ -47,11 +47,13 @@ object ArrowUtils { case DateType => new ArrowType.Date(DateUnit.DAY) case TimestampType => if (timeZoneId == null) { -throw new UnsupportedOperationException("TimestampType must supply timeZoneId parameter") +throw new UnsupportedOperationException( + s"${TimestampType.catalogString} must supply timeZoneId parameter") } else { new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId) } -case _ => throw new UnsupportedOperationException(s"Unsupported data type: ${dt.simpleString}") +case _ => + throw new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}") } def fromArrowType(dt: ArrowType): DataType = dt match { http://git-wip-us.apache.org/repos/asf/spark/blob/a5925c16/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala index 66888fc..3de6ea8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala @@ -68,7 +68,7 @@ object ArrowWriter { } new StructWriter(vector, children.toArray) case (dt, _) => -throw new UnsupportedOperationException(s"Unsupported data type: ${dt.simpleString}") +throw new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}") } } } http://git-wip-us.apache.org/repos/asf/spark/blob/a5925c16/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala -- 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 e9b150f..542a10f 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 @@ -717,7 +717,7 @@ private[column
spark git commit: [SPARK-24163][SPARK-24164][SQL] Support column list as the pivot column in Pivot
Repository: spark Updated Branches: refs/heads/master 1272b2034 -> cd203e0df [SPARK-24163][SPARK-24164][SQL] Support column list as the pivot column in Pivot ## What changes were proposed in this pull request? 1. Extend the Parser to enable parsing a column list as the pivot column. 2. Extend the Parser and the Pivot node to enable parsing complex expressions with aliases as the pivot value. 3. Add type check and constant check in Analyzer for Pivot node. ## How was this patch tested? Add tests in pivot.sql Author: maryannxue Closes #21720 from maryannxue/spark-24164. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cd203e0d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cd203e0d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cd203e0d Branch: refs/heads/master Commit: cd203e0dfc0758a2a90297e8c74c22a1212db846 Parents: 1272b20 Author: maryannxue Authored: Wed Jul 18 13:33:26 2018 -0700 Committer: Xiao Li Committed: Wed Jul 18 13:33:26 2018 -0700 -- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 11 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 47 ++-- .../spark/sql/catalyst/parser/AstBuilder.scala | 22 +- .../plans/logical/basicLogicalOperators.scala | 2 +- .../test/resources/sql-tests/inputs/pivot.sql | 92 .../resources/sql-tests/results/pivot.sql.out | 230 +++ 6 files changed, 348 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/cd203e0d/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 -- 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 dc95751..1b43874 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 @@ -414,7 +414,16 @@ groupingSet ; pivotClause -: PIVOT '(' aggregates=namedExpressionSeq FOR pivotColumn=identifier IN '(' pivotValues+=constant (',' pivotValues+=constant)* ')' ')' +: PIVOT '(' aggregates=namedExpressionSeq FOR pivotColumn IN '(' pivotValues+=pivotValue (',' pivotValues+=pivotValue)* ')' ')' +; + +pivotColumn +: identifiers+=identifier +| '(' identifiers+=identifier (',' identifiers+=identifier)* ')' +; + +pivotValue +: expression (AS? identifier)? ; lateralView http://git-wip-us.apache.org/repos/asf/spark/blob/cd203e0d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 36f14cc..59c371e 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 @@ -509,17 +509,39 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan transform { case p: Pivot if !p.childrenResolved || !p.aggregates.forall(_.resolved) || (p.groupByExprsOpt.isDefined && !p.groupByExprsOpt.get.forall(_.resolved)) -|| !p.pivotColumn.resolved => p +|| !p.pivotColumn.resolved || !p.pivotValues.forall(_.resolved) => p case Pivot(groupByExprsOpt, pivotColumn, pivotValues, aggregates, child) => // Check all aggregate expressions. aggregates.foreach(checkValidAggregateExpression) +// Check all pivot values are literal and match pivot column data type. +val evalPivotValues = pivotValues.map { value => + val foldable = value match { +case Alias(v, _) => v.foldable +case _ => value.foldable + } + if (!foldable) { +throw new AnalysisException( + s"Literal expressions required for pivot values, found '$value'") + } + if (!Cast.canCast(value.dataType, pivotColumn.dataType)) { +throw new AnalysisException(s"Invalid pivot value '$value': " + + s"value data type ${value.dataType.simpleString} does not match " + + s"pivot column data type ${pivotColumn.dataType.catalogString}") + } + Cast(value, pivotColumn.dataType, Some(conf.sessionLocalTimeZone)).eval(EmptyRow) +} // Group-by expressions coming from SQL are implicit and need to be deduced. val groupByExprs = groupByExprsOpt.getOrElse( (child.outputSet -- aggregates.flatMap(_.references) --
spark git commit: [SPARK-24576][BUILD] Upgrade Apache ORC to 1.5.2
Repository: spark Updated Branches: refs/heads/master fc2e18963 -> 3b59d326c [SPARK-24576][BUILD] Upgrade Apache ORC to 1.5.2 ## What changes were proposed in this pull request? This issue aims to upgrade Apache ORC library from 1.4.4 to 1.5.2 in order to bring the following benefits into Apache Spark. - [ORC-91](https://issues.apache.org/jira/browse/ORC-91) Support for variable length blocks in HDFS (The current space wasted in ORC to padding is known to be 5%.) - [ORC-344](https://issues.apache.org/jira/browse/ORC-344) Support for using Decimal64ColumnVector In addition to that, Apache Hive 3.1 and 3.2 will use ORC 1.5.1 ([HIVE-19669](https://issues.apache.org/jira/browse/HIVE-19465)) and 1.5.2 ([HIVE-19792](https://issues.apache.org/jira/browse/HIVE-19792)) respectively. This will improve the compatibility between Apache Spark and Apache Hive by sharing the common library. ## How was this patch tested? Pass the Jenkins with all existing tests. Author: Dongjoon Hyun Closes #21582 from dongjoon-hyun/SPARK-24576. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3b59d326 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3b59d326 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3b59d326 Branch: refs/heads/master Commit: 3b59d326c77bec96e5fb856d827139e0389394ba Parents: fc2e189 Author: Dongjoon Hyun Authored: Tue Jul 17 23:52:17 2018 -0700 Committer: Xiao Li Committed: Tue Jul 17 23:52:17 2018 -0700 -- dev/deps/spark-deps-hadoop-2.6 | 7 ++--- dev/deps/spark-deps-hadoop-2.7 | 7 ++--- dev/deps/spark-deps-hadoop-3.1 | 7 ++--- pom.xml | 2 +- sql/core/pom.xml| 28 .../datasources/orc/OrcFileFormat.scala | 15 ++- .../datasources/orc/OrcSerializer.scala | 2 +- 7 files changed, 56 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3b59d326/dev/deps/spark-deps-hadoop-2.6 -- diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index f50a0aa..ff6d5c3 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -2,7 +2,7 @@ JavaEWAH-0.3.2.jar RoaringBitmap-0.5.11.jar ST4-4.0.4.jar activation-1.1.1.jar -aircompressor-0.8.jar +aircompressor-0.10.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.jar @@ -157,8 +157,9 @@ objenesis-2.1.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.4.4-nohive.jar -orc-mapreduce-1.4.4-nohive.jar +orc-core-1.5.2-nohive.jar +orc-mapreduce-1.5.2-nohive.jar +orc-shims-1.5.2.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar http://git-wip-us.apache.org/repos/asf/spark/blob/3b59d326/dev/deps/spark-deps-hadoop-2.7 -- diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 774f9dc..72a94f8 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -2,7 +2,7 @@ JavaEWAH-0.3.2.jar RoaringBitmap-0.5.11.jar ST4-4.0.4.jar activation-1.1.1.jar -aircompressor-0.8.jar +aircompressor-0.10.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.jar @@ -158,8 +158,9 @@ objenesis-2.1.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.4.4-nohive.jar -orc-mapreduce-1.4.4-nohive.jar +orc-core-1.5.2-nohive.jar +orc-mapreduce-1.5.2-nohive.jar +orc-shims-1.5.2.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar http://git-wip-us.apache.org/repos/asf/spark/blob/3b59d326/dev/deps/spark-deps-hadoop-3.1 -- diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 19c05ad..3409dc4 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -4,7 +4,7 @@ RoaringBitmap-0.5.11.jar ST4-4.0.4.jar accessors-smart-1.2.jar activation-1.1.1.jar -aircompressor-0.8.jar +aircompressor-0.10.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.jar @@ -176,8 +176,9 @@ okhttp-2.7.5.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.4.4-nohive.jar -orc-mapreduce-1.4.4-nohive.jar +orc-core-1.5.2-nohive.jar +orc-mapreduce-1.5.2-nohive.jar +orc-shims-1.5.2.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar http://git-wip-us.apache.org/repos/asf/spark/blob/3b59d326/pom.xml -- diff --git a/pom.xml b/pom.xml index 1892bbe..649221d 100644 --- a/pom.xml +++ b/pom.xml @@ -131,7 +131,7 @@ 1.2.1 10.12.1.1 1.10.0 -
spark git commit: [SPARK-24681][SQL] Verify nested column names in Hive metastore
Repository: spark Updated Branches: refs/heads/master 912634b00 -> 2a4dd6f06 [SPARK-24681][SQL] Verify nested column names in Hive metastore ## What changes were proposed in this pull request? This pr added code to check if nested column names do not include ',', ':', and ';' because Hive metastore can't handle these characters in nested column names; ref: https://github.com/apache/hive/blob/release-1.2.1/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java#L239 ## How was this patch tested? Added tests in `HiveDDLSuite`. Author: Takeshi Yamamuro Closes #21711 from maropu/SPARK-24681. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2a4dd6f0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2a4dd6f0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2a4dd6f0 Branch: refs/heads/master Commit: 2a4dd6f06cfd2f58fda9786c88809e6de695444e Parents: 912634b Author: Takeshi Yamamuro Authored: Tue Jul 17 14:15:30 2018 -0700 Committer: Xiao Li Committed: Tue Jul 17 14:15:30 2018 -0700 -- .../spark/sql/hive/HiveExternalCatalog.scala| 34 .../spark/sql/hive/execution/HiveDDLSuite.scala | 19 +++ 2 files changed, 46 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2a4dd6f0/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala -- 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 44480ce..7f28fc4 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 @@ -138,17 +138,37 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } /** - * Checks the validity of data column names. Hive metastore disallows the table to use comma in - * data column names. Partition columns do not have such a restriction. Views do not have such - * a restriction. + * Checks the validity of data column names. Hive metastore disallows the table to use some + * special characters (',', ':', and ';') in data column names, including nested column names. + * Partition columns do not have such a restriction. Views do not have such a restriction. */ private def verifyDataSchema( tableName: TableIdentifier, tableType: CatalogTableType, dataSchema: StructType): Unit = { if (tableType != VIEW) { - dataSchema.map(_.name).foreach { colName => -if (colName.contains(",")) { - throw new AnalysisException("Cannot create a table having a column whose name contains " + -s"commas in Hive metastore. Table: $tableName; Column: $colName") + val invalidChars = Seq(",", ":", ";") + def verifyNestedColumnNames(schema: StructType): Unit = schema.foreach { f => +f.dataType match { + case st: StructType => verifyNestedColumnNames(st) + case _ if invalidChars.exists(f.name.contains) => +val invalidCharsString = invalidChars.map(c => s"'$c'").mkString(", ") +val errMsg = "Cannot create a table having a nested column whose name contains " + + s"invalid characters ($invalidCharsString) in Hive metastore. Table: $tableName; " + + s"Column: ${f.name}" +throw new AnalysisException(errMsg) + case _ => +} + } + + dataSchema.foreach { f => +f.dataType match { + // Checks top-level column names + case _ if f.name.contains(",") => +throw new AnalysisException("Cannot create a table having a column whose name " + + s"contains commas in Hive metastore. Table: $tableName; Column: ${f.name}") + // Checks nested column names + case st: StructType => +verifyNestedColumnNames(st) + case _ => } } } http://git-wip-us.apache.org/repos/asf/spark/blob/2a4dd6f0/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala -- 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 0341c3b..31fd4c5 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionE
spark git commit: [SPARK-24402][SQL] Optimize `In` expression when only one element in the collection or collection is empty
Repository: spark Updated Branches: refs/heads/master ba437fc5c -> 0f0d1865f [SPARK-24402][SQL] Optimize `In` expression when only one element in the collection or collection is empty ## What changes were proposed in this pull request? Two new rules in the logical plan optimizers are added. 1. When there is only one element in the **`Collection`**, the physical plan will be optimized to **`EqualTo`**, so predicate pushdown can be used. ```scala profileDF.filter( $"profileID".isInCollection(Set(6))).explain(true) """ |== Physical Plan == |*(1) Project [profileID#0] |+- *(1) Filter (isnotnull(profileID#0) && (profileID#0 = 6)) | +- *(1) FileScan parquet [profileID#0] Batched: true, Format: Parquet, | PartitionFilters: [], | PushedFilters: [IsNotNull(profileID), EqualTo(profileID,6)], | ReadSchema: struct """.stripMargin ``` 2. When the **`Collection`** is empty, and the input is nullable, the logical plan will be simplified to ```scala profileDF.filter( $"profileID".isInCollection(Set())).explain(true) """ |== Optimized Logical Plan == |Filter if (isnull(profileID#0)) null else false |+- Relation[profileID#0] parquet """.stripMargin ``` TODO: 1. For multiple conditions with numbers less than certain thresholds, we should still allow predicate pushdown. 2. Optimize the **`In`** using **`tableswitch`** or **`lookupswitch`** when the numbers of the categories are low, and they are **`Int`**, **`Long`**. 3. The default immutable hash trees set is slow for query, and we should do benchmark for using different set implementation for faster query. 4. **`filter(if (condition) null else false)`** can be optimized to false. ## How was this patch tested? Couple new tests are added. Author: DB Tsai Closes #21442 from dbtsai/optimize-in. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0f0d1865 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0f0d1865 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0f0d1865 Branch: refs/heads/master Commit: 0f0d1865f581a9158d73505471953656b173beba Parents: ba437fc Author: DB Tsai Authored: Mon Jul 16 15:33:39 2018 -0700 Committer: Xiao Li Committed: Mon Jul 16 15:33:39 2018 -0700 -- .../sql/catalyst/optimizer/expressions.scala| 13 +--- .../catalyst/optimizer/OptimizeInSuite.scala| 32 2 files changed, 41 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0f0d1865/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 1d363b8..f78a0ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -218,15 +218,20 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] { object OptimizeIn extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsDown { - case In(v, list) if list.isEmpty && !v.nullable => FalseLiteral + case In(v, list) if list.isEmpty => +// When v is not nullable, the following expression will be optimized +// to FalseLiteral which is tested in OptimizeInSuite.scala +If(IsNotNull(v), FalseLiteral, Literal(null, BooleanType)) case expr @ In(v, list) if expr.inSetConvertible => val newList = ExpressionSet(list).toSeq -if (newList.size > SQLConf.get.optimizerInSetConversionThreshold) { +if (newList.length == 1 && !newList.isInstanceOf[ListQuery]) { + EqualTo(v, newList.head) +} else if (newList.length > SQLConf.get.optimizerInSetConversionThreshold) { val hSet = newList.map(e => e.eval(EmptyRow)) InSet(v, HashSet() ++ hSet) -} else if (newList.size < list.size) { +} else if (newList.length < list.length) { expr.copy(list = newList) -} else { // newList.length == list.length +} else { // newList.length == list.length && newList.length > 1 expr } } http://git-wip-us.apache.org/repos/asf/spark/blob/0f0d1865/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/t
spark git commit: [SPARK-24805][SQL] Do not ignore avro files without extensions by default
Repository: spark Updated Branches: refs/heads/master b0c95a1d6 -> ba437fc5c [SPARK-24805][SQL] Do not ignore avro files without extensions by default ## What changes were proposed in this pull request? In the PR, I propose to change default behaviour of AVRO datasource which currently ignores files without `.avro` extension in read by default. This PR sets the default value for `avro.mapred.ignore.inputs.without.extension` to `false` in the case if the parameter is not set by an user. ## How was this patch tested? Added a test file without extension in AVRO format, and new test for reading the file with and wihout specified schema. Author: Maxim Gekk Author: Maxim Gekk Closes #21769 from MaxGekk/avro-without-extension. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ba437fc5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ba437fc5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ba437fc5 Branch: refs/heads/master Commit: ba437fc5c73b95ee4c59327abf3161c58f64cb12 Parents: b0c95a1 Author: Maxim Gekk Authored: Mon Jul 16 14:35:44 2018 -0700 Committer: Xiao Li Committed: Mon Jul 16 14:35:44 2018 -0700 -- .../apache/spark/sql/avro/AvroFileFormat.scala | 14 +++--- .../org/apache/spark/sql/avro/AvroSuite.scala | 45 +--- 2 files changed, 47 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ba437fc5/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala -- diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index fb93033..9eb2064 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -62,7 +62,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Schema evolution is not supported yet. Here we only pick a single random sample file to // figure out the schema of the whole dataset. val sampleFile = - if (conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, true)) { + if (AvroFileFormat.ignoreFilesWithoutExtensions(conf)) { files.find(_.getPath.getName.endsWith(".avro")).getOrElse { throw new FileNotFoundException( "No Avro files found. Hadoop option \"avro.mapred.ignore.inputs.without.extension\" " + @@ -170,10 +170,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Doing input file filtering is improper because we may generate empty tasks that process no // input files but stress the scheduler. We should probably add a more general input file // filtering mechanism for `FileFormat` data sources. See SPARK-16317. - if ( -conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, true) && -!file.filePath.endsWith(".avro") - ) { + if (AvroFileFormat.ignoreFilesWithoutExtensions(conf) && !file.filePath.endsWith(".avro")) { Iterator.empty } else { val reader = { @@ -278,4 +275,11 @@ private[avro] object AvroFileFormat { value.readFields(new DataInputStream(in)) } } + + def ignoreFilesWithoutExtensions(conf: Configuration): Boolean = { +// Files without .avro extensions are not ignored by default +val defaultValue = false + +conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, defaultValue) + } } http://git-wip-us.apache.org/repos/asf/spark/blob/ba437fc5/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala -- diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 9c6526b..446b421 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.avro import java.io._ -import java.nio.file.Files +import java.net.URL +import java.nio.file.{Files, Path, Paths} import java.sql.{Date, Timestamp} import java.util.{TimeZone, UUID} @@ -622,7 +623,12 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { intercept[FileNotFoundException] { withTempPath { dir => FileUtils.touch(new File(dir, "test")) -spark.read.avro(dir.toString) +val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration +try { + hadoopConf.set
spark git commit: [SPARK-23901][SQL] Removing masking functions
Repository: spark Updated Branches: refs/heads/master b045315e5 -> b0c95a1d6 [SPARK-23901][SQL] Removing masking functions The PR reverts #21246. Author: Marek Novotny Closes #21786 from mn-mikke/SPARK-23901. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b0c95a1d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b0c95a1d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b0c95a1d Branch: refs/heads/master Commit: b0c95a1d69df752bd62e49838a98268f6847 Parents: b045315 Author: Marek Novotny Authored: Mon Jul 16 14:28:35 2018 -0700 Committer: Xiao Li Committed: Mon Jul 16 14:28:35 2018 -0700 -- .../expressions/MaskExpressionsUtils.java | 80 --- .../catalyst/analysis/FunctionRegistry.scala| 8 - .../catalyst/expressions/maskExpressions.scala | 569 --- .../expressions/MaskExpressionsSuite.scala | 236 .../scala/org/apache/spark/sql/functions.scala | 119 .../spark/sql/DataFrameFunctionsSuite.scala | 107 6 files changed, 1119 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b0c95a1d/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/MaskExpressionsUtils.java -- diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/MaskExpressionsUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/MaskExpressionsUtils.java deleted file mode 100644 index 0587990..000 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/MaskExpressionsUtils.java +++ /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.sql.catalyst.expressions; - -/** - * Contains all the Utils methods used in the masking expressions. - */ -public class MaskExpressionsUtils { - static final int UNMASKED_VAL = -1; - - /** - * Returns the masking character for {@param c} or {@param c} is it should not be masked. - * @param c the character to transform - * @param maskedUpperChar the character to use instead of a uppercase letter - * @param maskedLowerChar the character to use instead of a lowercase letter - * @param maskedDigitChar the character to use instead of a digit - * @param maskedOtherChar the character to use instead of a any other character - * @return masking character for {@param c} - */ - public static int transformChar( - final int c, - int maskedUpperChar, - int maskedLowerChar, - int maskedDigitChar, - int maskedOtherChar) { -switch(Character.getType(c)) { - case Character.UPPERCASE_LETTER: -if(maskedUpperChar != UNMASKED_VAL) { - return maskedUpperChar; -} -break; - - case Character.LOWERCASE_LETTER: -if(maskedLowerChar != UNMASKED_VAL) { - return maskedLowerChar; -} -break; - - case Character.DECIMAL_DIGIT_NUMBER: -if(maskedDigitChar != UNMASKED_VAL) { - return maskedDigitChar; -} -break; - - default: -if(maskedOtherChar != UNMASKED_VAL) { - return maskedOtherChar; -} -break; -} - -return c; - } - - /** - * Returns the replacement char to use according to the {@param rep} specified by the user and - * the {@param def} default. - */ - public static int getReplacementChar(String rep, int def) { -if (rep != null && rep.length() > 0) { - return rep.codePointAt(0); -} -return def; - } -} http://git-wip-us.apache.org/repos/asf/spark/blob/b0c95a1d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 1d9e470..d696ce9 100644 --- a/s
spark git commit: [SPARK-24810][SQL] Fix paths to test files in AvroSuite
Repository: spark Updated Branches: refs/heads/master d463533de -> 9f929458f [SPARK-24810][SQL] Fix paths to test files in AvroSuite ## What changes were proposed in this pull request? In the PR, I propose to move `testFile()` to the common trait `SQLTestUtilsBase` and wrap test files in `AvroSuite` by the method `testFile()` which returns full paths to test files in the resource folder. Author: Maxim Gekk Closes #21773 from MaxGekk/test-file. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9f929458 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9f929458 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9f929458 Branch: refs/heads/master Commit: 9f929458fb0a8a106f3b5a6ed3ee2cd3faa85770 Parents: d463533 Author: Maxim Gekk Authored: Sun Jul 15 23:01:36 2018 -0700 Committer: Xiao Li Committed: Sun Jul 15 23:01:36 2018 -0700 -- .../org/apache/spark/sql/avro/AvroSuite.scala | 79 +++- .../execution/datasources/csv/CSVSuite.scala| 4 - .../execution/datasources/json/JsonSuite.scala | 4 - .../datasources/text/WholeTextFileSuite.scala | 11 +-- .../apache/spark/sql/test/SQLTestUtils.scala| 7 ++ 5 files changed, 53 insertions(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9f929458/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala -- diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 6ed6656..9c6526b 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -36,8 +36,8 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { - val episodesFile = "src/test/resources/episodes.avro" - val testFile = "src/test/resources/test.avro" + val episodesAvro = testFile("episodes.avro") + val testAvro = testFile("test.avro") override protected def beforeAll(): Unit = { super.beforeAll() @@ -45,18 +45,18 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } def checkReloadMatchesSaved(originalFile: String, newFile: String): Unit = { -val originalEntries = spark.read.avro(testFile).collect() +val originalEntries = spark.read.avro(testAvro).collect() val newEntries = spark.read.avro(newFile) checkAnswer(newEntries, originalEntries) } test("reading from multiple paths") { -val df = spark.read.avro(episodesFile, episodesFile) +val df = spark.read.avro(episodesAvro, episodesAvro) assert(df.count == 16) } test("reading and writing partitioned data") { -val df = spark.read.avro(episodesFile) +val df = spark.read.avro(episodesAvro) val fields = List("title", "air_date", "doctor") for (field <- fields) { withTempPath { dir => @@ -72,14 +72,14 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } test("request no fields") { -val df = spark.read.avro(episodesFile) +val df = spark.read.avro(episodesAvro) df.createOrReplaceTempView("avro_table") assert(spark.sql("select count(*) from avro_table").collect().head === Row(8)) } test("convert formats") { withTempPath { dir => - val df = spark.read.avro(episodesFile) + val df = spark.read.avro(episodesAvro) df.write.parquet(dir.getCanonicalPath) assert(spark.read.parquet(dir.getCanonicalPath).count() === df.count) } @@ -87,7 +87,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { test("rearrange internal schema") { withTempPath { dir => - val df = spark.read.avro(episodesFile) + val df = spark.read.avro(episodesAvro) df.select("doctor", "title").write.avro(dir.getCanonicalPath) } } @@ -362,7 +362,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val deflateDir = s"$dir/deflate" val snappyDir = s"$dir/snappy" - val df = spark.read.avro(testFile) + val df = spark.read.avro(testAvro) spark.conf.set(AVRO_COMPRESSION_CODEC, "uncompressed") df.write.avro(uncompressDir) spark.conf.set(AVRO_COMPRESSION_CODEC, "deflate") @@ -381,49 +381,49 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } test("dsl test") { -val results = spark.read.avro(episodesFile).select("title").collect() +val results = spark.read.avro(episodesAvro).select("title").collect() assert(results.length === 8
spark git commit: [SPARK-24676][SQL] Project required data from CSV parsed data when column pruning disabled
Repository: spark Updated Branches: refs/heads/master bcf7121ed -> d463533de [SPARK-24676][SQL] Project required data from CSV parsed data when column pruning disabled ## What changes were proposed in this pull request? This pr modified code to project required data from CSV parsed data when column pruning disabled. In the current master, an exception below happens if `spark.sql.csv.parser.columnPruning.enabled` is false. This is because required formats and CSV parsed formats are different from each other; ``` ./bin/spark-shell --conf spark.sql.csv.parser.columnPruning.enabled=false scala> val dir = "/tmp/spark-csv/csv" scala> spark.range(10).selectExpr("id % 2 AS p", "id").write.mode("overwrite").partitionBy("p").csv(dir) scala> spark.read.csv(dir).selectExpr("sum(p)").collect() 18/06/25 13:48:46 ERROR Executor: Exception in task 2.0 in stage 2.0 (TID 7) java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Integer at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:101) at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getInt(rows.scala:41) ... ``` ## How was this patch tested? Added tests in `CSVSuite`. Author: Takeshi Yamamuro Closes #21657 from maropu/SPARK-24676. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d463533d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d463533d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d463533d Branch: refs/heads/master Commit: d463533ded89a05e9f77e590fd3de2ffa212d68b Parents: bcf7121 Author: Takeshi Yamamuro Authored: Sun Jul 15 20:22:09 2018 -0700 Committer: Xiao Li Committed: Sun Jul 15 20:22:09 2018 -0700 -- .../datasources/csv/UnivocityParser.scala | 54 +++- .../execution/datasources/csv/CSVSuite.scala| 29 +++ 2 files changed, 70 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d463533d/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala index aa545e1..79143cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala @@ -33,29 +33,49 @@ import org.apache.spark.sql.execution.datasources.FailureSafeParser import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String + +/** + * Constructs a parser for a given schema that translates CSV data to an [[InternalRow]]. + * + * @param dataSchema The CSV data schema that is specified by the user, or inferred from underlying + * data files. + * @param requiredSchema The schema of the data that should be output for each row. This should be a + * subset of the columns in dataSchema. + * @param options Configuration options for a CSV parser. + */ class UnivocityParser( dataSchema: StructType, requiredSchema: StructType, val options: CSVOptions) extends Logging { require(requiredSchema.toSet.subsetOf(dataSchema.toSet), -"requiredSchema should be the subset of schema.") +s"requiredSchema (${requiredSchema.catalogString}) should be the subset of " + + s"dataSchema (${dataSchema.catalogString}).") def this(schema: StructType, options: CSVOptions) = this(schema, schema, options) // A `ValueConverter` is responsible for converting the given value to a desired type. private type ValueConverter = String => Any + // This index is used to reorder parsed tokens + private val tokenIndexArr = +requiredSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray + + // When column pruning is enabled, the parser only parses the required columns based on + // their positions in the data schema. + private val parsedSchema = if (options.columnPruning) requiredSchema else dataSchema + val tokenizer = { val parserSetting = options.asParserSettings -if (options.columnPruning && requiredSchema.length < dataSchema.length) { - val tokenIndexArr = requiredSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))) +// When to-be-parsed schema is shorter than the to-be-read data schema, we let Univocity CSV +// parser select a sequence of fields for reading by their positions. +// if (options.columnPruning && requiredSchema.length < dataSchema.length) { +if (parsedSchema.length < dataSchema.length)
spark git commit: [SPARK-24807][CORE] Adding files/jars twice: output a warning and add a note
Repository: spark Updated Branches: refs/heads/master 3e7dc8296 -> 69993217f [SPARK-24807][CORE] Adding files/jars twice: output a warning and add a note ## What changes were proposed in this pull request? In the PR, I propose to output an warning if the `addFile()` or `addJar()` methods are callled more than once for the same path. Currently, overwriting of already added files is not supported. New comments and warning are reflected the existing behaviour. Author: Maxim Gekk Closes #21771 from MaxGekk/warning-on-adding-file. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/69993217 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/69993217 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/69993217 Branch: refs/heads/master Commit: 69993217fc4f5e5e41a297702389e86fe534dc2f Parents: 3e7dc82 Author: Maxim Gekk Authored: Sat Jul 14 22:07:49 2018 -0700 Committer: Xiao Li Committed: Sat Jul 14 22:07:49 2018 -0700 -- R/pkg/R/context.R | 2 ++ core/src/main/scala/org/apache/spark/SparkContext.scala | 12 .../org/apache/spark/api/java/JavaSparkContext.scala| 6 ++ python/pyspark/context.py | 4 4 files changed, 24 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/69993217/R/pkg/R/context.R -- diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 8ec727d..3e996a5 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -305,6 +305,8 @@ setCheckpointDirSC <- function(sc, dirName) { #' Currently directories are only supported for Hadoop-supported filesystems. #' Refer Hadoop-supported filesystems at \url{https://wiki.apache.org/hadoop/HCFS}. #' +#' Note: A path can be added only once. Subsequent additions of the same path are ignored. +#' #' @rdname spark.addFile #' @param path The path of the file to be added #' @param recursive Whether to add files recursively from the path. Default is FALSE. http://git-wip-us.apache.org/repos/asf/spark/blob/69993217/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 74bfb5d..531384a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1496,6 +1496,8 @@ class SparkContext(config: SparkConf) extends Logging { * @param path can be either a local file, a file in HDFS (or other Hadoop-supported * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, * use `SparkFiles.get(fileName)` to find its download location. + * + * @note A path can be added only once. Subsequent additions of the same path are ignored. */ def addFile(path: String): Unit = { addFile(path, false) @@ -1516,6 +1518,8 @@ class SparkContext(config: SparkConf) extends Logging { * use `SparkFiles.get(fileName)` to find its download location. * @param recursive if true, a directory can be given in `path`. Currently directories are * only supported for Hadoop-supported filesystems. + * + * @note A path can be added only once. Subsequent additions of the same path are ignored. */ def addFile(path: String, recursive: Boolean): Unit = { val uri = new Path(path).toUri @@ -1555,6 +1559,9 @@ class SparkContext(config: SparkConf) extends Logging { Utils.fetchFile(uri.toString, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, hadoopConfiguration, timestamp, useCache = false) postEnvironmentUpdate() +} else { + logWarning(s"The path $path has been added already. Overwriting of added paths " + + "is not supported in the current version.") } } @@ -1803,6 +1810,8 @@ class SparkContext(config: SparkConf) extends Logging { * * @param path can be either a local file, a file in HDFS (or other Hadoop-supported filesystems), * an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node. + * + * @note A path can be added only once. Subsequent additions of the same path are ignored. */ def addJar(path: String) { def addJarFile(file: File): String = { @@ -1849,6 +1858,9 @@ class SparkContext(config: SparkConf) extends Logging { if (addedJars.putIfAbsent(key, timestamp).isEmpty) { logInfo(s"Added JAR $path at $key with timestamp $timestamp") postEnvironmentUpdate() +} else { + logWarning(s"The jar $path has been added already. Overwriting of added jars " + +"is not supported in th
spark git commit: [SPARK-24776][SQL] Avro unit test: deduplicate code and replace deprecated methods
Repository: spark Updated Branches: refs/heads/master 43e4e851b -> 3e7dc8296 [SPARK-24776][SQL] Avro unit test: deduplicate code and replace deprecated methods ## What changes were proposed in this pull request? Improve Avro unit test: 1. use QueryTest/SharedSQLContext/SQLTestUtils, instead of the duplicated test utils. 2. replace deprecated methods This is a follow up PR for #21760, the PR passes pull request tests but failed in: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.6/7842/ This PR is to fix it. ## How was this patch tested? Unit test. Compile with different commands: ``` ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.6 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.7 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-3.1 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ``` Author: Gengliang Wang Closes #21768 from gengliangwang/improve_avro_test. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3e7dc829 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3e7dc829 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3e7dc829 Branch: refs/heads/master Commit: 3e7dc82960fd3339eee16d83df66761ae6e3fe3d Parents: 43e4e85 Author: Gengliang Wang Authored: Sat Jul 14 21:36:56 2018 -0700 Committer: Xiao Li Committed: Sat Jul 14 21:36:56 2018 -0700 -- .../org/apache/spark/sql/avro/AvroSuite.scala | 98 ++-- .../org/apache/spark/sql/avro/TestUtils.scala | 156 --- 2 files changed, 45 insertions(+), 209 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3e7dc829/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala -- diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index c6c1e40..4f94d82 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -31,32 +31,24 @@ import org.apache.avro.generic.{GenericData, GenericDatumWriter, GenericRecord} import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed} import org.apache.commons.io.FileUtils -import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.avro.SchemaConverters.IncompatibleSchemaException +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ -class AvroSuite extends SparkFunSuite { +class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val episodesFile = "src/test/resources/episodes.avro" val testFile = "src/test/resources/test.avro" - private var spark: SparkSession = _ - override protected def beforeAll(): Unit = { super.beforeAll() -spark = SparkSession.builder() - .master("local[2]") - .appName("AvroSuite") - .config("spark.sql.files.maxPartitionBytes", 1024) - .getOrCreate() - } - - override protected def afterAll(): Unit = { -try { - spark.sparkContext.stop() -} finally { - super.afterAll() -} +spark.conf.set("spark.sql.files.maxPartitionBytes", 1024) + } + + def checkReloadMatchesSaved(originalFile: String, newFile: String): Unit = { +val originalEntries = spark.read.avro(testFile).collect() +val newEntries = spark.read.avro(newFile) +checkAnswer(newEntries, originalEntries) } test("reading from multiple paths") { @@ -68,7 +60,7 @@ class AvroSuite extends SparkFunSuite { val df = spark.read.avro(episodesFile) val fields = List("title", "air_date", "doctor") for (field <- fields) { - TestUtils.withTempDir { dir => + withTempPath { dir => val outputDir = s"$dir/${UUID.randomUUID}" df.write.partitionBy(field).avro(outputDir) val input = spark.read.avro(outputDir) @@ -82,12 +74,12 @@ class AvroSuite extends SparkFunSuite { test("request no fields") { val df = spark.read.avro(episodesFile) -df.registerTempTable("avro_table") +df.createOrReplaceTempView("avro_table") assert(spark.sql("select count(*) from avro_table").collect().head === Row(8)) } test("convert formats") { -TestUtils.withTempDir { dir => +withTempPath { dir => val df = spark.read.avro(episodesFile) df.write.parquet(dir.getCanonicalPath) assert(spark.read.parquet(dir.getCanonical
spark git commit: [SPARK-23831][SQL] Add org.apache.derby to IsolatedClientLoader
Repository: spark Updated Branches: refs/heads/master 3b6005b8a -> a75571b46 [SPARK-23831][SQL] Add org.apache.derby to IsolatedClientLoader ## What changes were proposed in this pull request? Add `org.apache.derby` to `IsolatedClientLoader`, otherwise it may throw an exception: ```scala ... [info] Cause: java.sql.SQLException: Failed to start database 'metastore_db' with class loader org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$12439ab23, see the next exception for details. [info] at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source) [info] at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source) [info] at org.apache.derby.impl.jdbc.Util.seeNextException(Unknown Source) [info] at org.apache.derby.impl.jdbc.EmbedConnection.bootDatabase(Unknown Source) [info] at org.apache.derby.impl.jdbc.EmbedConnection.(Unknown Source) [info] at org.apache.derby.jdbc.InternalDriver$1.run(Unknown Source) ... ``` ## How was this patch tested? unit tests and manual tests Author: Yuming Wang Closes #20944 from wangyum/SPARK-23831. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a75571b4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a75571b4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a75571b4 Branch: refs/heads/master Commit: a75571b46f813005a6d4b076ec39081ffab11844 Parents: 3b6005b Author: Yuming Wang Authored: Fri Jul 13 14:07:52 2018 -0700 Committer: Xiao Li Committed: Fri Jul 13 14:07:52 2018 -0700 -- .../apache/spark/sql/hive/client/IsolatedClientLoader.scala| 1 + .../org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala | 6 ++ 2 files changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a75571b4/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 2f34f69..6a90c44 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -182,6 +182,7 @@ private[hive] class IsolatedClientLoader( name.startsWith("org.slf4j") || name.startsWith("org.apache.log4j") || // log4j1.x name.startsWith("org.apache.logging.log4j") || // log4j2 +name.startsWith("org.apache.derby.") || name.startsWith("org.apache.spark.") || (sharesHadoopClasses && isHadoopClass) || name.startsWith("scala.") || http://git-wip-us.apache.org/repos/asf/spark/blob/a75571b4/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala -- 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 0a522b6..1de258f 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 @@ -113,4 +113,10 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { catalog.createDatabase(newDb("dbWithNullDesc").copy(description = null), ignoreIfExists = false) assert(catalog.getDatabase("dbWithNullDesc").description == "") } + + test("SPARK-23831: Add org.apache.derby to IsolatedClientLoader") { +val client1 = HiveUtils.newClientForMetadata(new SparkConf, new Configuration) +val client2 = HiveUtils.newClientForMetadata(new SparkConf, new Configuration) +assert(!client1.equals(client2)) + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: Revert "[SPARK-24776][SQL] Avro unit test: use SQLTestUtils and replace deprecated methods"
Repository: spark Updated Branches: refs/heads/master c1b62e420 -> 3bcb1b481 Revert "[SPARK-24776][SQL] Avro unit test: use SQLTestUtils and replace deprecated methods" This reverts commit c1b62e420a43aa7da36733ccdbec057d87ac1b43. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3bcb1b48 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3bcb1b48 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3bcb1b48 Branch: refs/heads/master Commit: 3bcb1b481423aedf1ac531ad582c7cb8685f1e3c Parents: c1b62e4 Author: Xiao Li Authored: Fri Jul 13 10:06:26 2018 -0700 Committer: Xiao Li Committed: Fri Jul 13 10:06:26 2018 -0700 -- .../org/apache/spark/sql/avro/AvroSuite.scala | 114 +++--- .../org/apache/spark/sql/avro/TestUtils.scala | 156 +++ 2 files changed, 217 insertions(+), 53 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3bcb1b48/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala -- diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 108b347..c6c1e40 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -31,24 +31,32 @@ import org.apache.avro.generic.{GenericData, GenericDatumWriter, GenericRecord} import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed} import org.apache.commons.io.FileUtils +import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.avro.SchemaConverters.IncompatibleSchemaException -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ -class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +class AvroSuite extends SparkFunSuite { val episodesFile = "src/test/resources/episodes.avro" val testFile = "src/test/resources/test.avro" + private var spark: SparkSession = _ + override protected def beforeAll(): Unit = { super.beforeAll() -spark.conf.set("spark.sql.files.maxPartitionBytes", 1024) - } - - def checkReloadMatchesSaved(originalFile: String, newFile: String): Unit = { -val originalEntries = spark.read.avro(testFile).collect() -val newEntries = spark.read.avro(newFile) -checkAnswer(newEntries, originalEntries) +spark = SparkSession.builder() + .master("local[2]") + .appName("AvroSuite") + .config("spark.sql.files.maxPartitionBytes", 1024) + .getOrCreate() + } + + override protected def afterAll(): Unit = { +try { + spark.sparkContext.stop() +} finally { + super.afterAll() +} } test("reading from multiple paths") { @@ -60,7 +68,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val df = spark.read.avro(episodesFile) val fields = List("title", "air_date", "doctor") for (field <- fields) { - withTempPath { dir => + TestUtils.withTempDir { dir => val outputDir = s"$dir/${UUID.randomUUID}" df.write.partitionBy(field).avro(outputDir) val input = spark.read.avro(outputDir) @@ -74,12 +82,12 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { test("request no fields") { val df = spark.read.avro(episodesFile) -df.createOrReplaceTempView("avro_table") +df.registerTempTable("avro_table") assert(spark.sql("select count(*) from avro_table").collect().head === Row(8)) } test("convert formats") { -withTempPath { dir => +TestUtils.withTempDir { dir => val df = spark.read.avro(episodesFile) df.write.parquet(dir.getCanonicalPath) assert(spark.read.parquet(dir.getCanonicalPath).count() === df.count) @@ -87,16 +95,15 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } test("rearrange internal schema") { -withTempPath { dir => +TestUtils.withTempDir { dir => val df = spark.read.avro(episodesFile) df.select("doctor", "title").write.avro(dir.getCanonicalPath) } } test("test NULL avro type") { -withTempPath { dir => - val fields = -Seq(new Field("null", Schema.create(Type.NULL), "doc", null.asInstanceOf[Any])).asJava +TestUtils.withTempDir { dir => + val fields = Seq(new Field("null", Schema.create(Type.NULL), "doc", null)).asJava val schema = Schema.createRecord("name", "docs", "namespace", false) schema.setFields(fields) val datumWriter = new GenericDatumWriter[GenericRecord](schema) @@ -115,11 +122,11 @@ class AvroSuite extends QueryTest wi
spark git commit: [SPARK-24776][SQL] Avro unit test: use SQLTestUtils and replace deprecated methods
Repository: spark Updated Branches: refs/heads/master dfd7ac988 -> c1b62e420 [SPARK-24776][SQL] Avro unit test: use SQLTestUtils and replace deprecated methods ## What changes were proposed in this pull request? Improve Avro unit test: 1. use QueryTest/SharedSQLContext/SQLTestUtils, instead of the duplicated test utils. 2. replace deprecated methods ## How was this patch tested? Unit test Author: Gengliang Wang Closes #21760 from gengliangwang/improve_avro_test. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c1b62e42 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c1b62e42 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c1b62e42 Branch: refs/heads/master Commit: c1b62e420a43aa7da36733ccdbec057d87ac1b43 Parents: dfd7ac9 Author: Gengliang Wang Authored: Fri Jul 13 08:55:46 2018 -0700 Committer: Xiao Li Committed: Fri Jul 13 08:55:46 2018 -0700 -- .../org/apache/spark/sql/avro/AvroSuite.scala | 114 +++--- .../org/apache/spark/sql/avro/TestUtils.scala | 156 --- 2 files changed, 53 insertions(+), 217 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c1b62e42/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala -- diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index c6c1e40..108b347 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -31,32 +31,24 @@ import org.apache.avro.generic.{GenericData, GenericDatumWriter, GenericRecord} import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed} import org.apache.commons.io.FileUtils -import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.avro.SchemaConverters.IncompatibleSchemaException +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ -class AvroSuite extends SparkFunSuite { +class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val episodesFile = "src/test/resources/episodes.avro" val testFile = "src/test/resources/test.avro" - private var spark: SparkSession = _ - override protected def beforeAll(): Unit = { super.beforeAll() -spark = SparkSession.builder() - .master("local[2]") - .appName("AvroSuite") - .config("spark.sql.files.maxPartitionBytes", 1024) - .getOrCreate() - } - - override protected def afterAll(): Unit = { -try { - spark.sparkContext.stop() -} finally { - super.afterAll() -} +spark.conf.set("spark.sql.files.maxPartitionBytes", 1024) + } + + def checkReloadMatchesSaved(originalFile: String, newFile: String): Unit = { +val originalEntries = spark.read.avro(testFile).collect() +val newEntries = spark.read.avro(newFile) +checkAnswer(newEntries, originalEntries) } test("reading from multiple paths") { @@ -68,7 +60,7 @@ class AvroSuite extends SparkFunSuite { val df = spark.read.avro(episodesFile) val fields = List("title", "air_date", "doctor") for (field <- fields) { - TestUtils.withTempDir { dir => + withTempPath { dir => val outputDir = s"$dir/${UUID.randomUUID}" df.write.partitionBy(field).avro(outputDir) val input = spark.read.avro(outputDir) @@ -82,12 +74,12 @@ class AvroSuite extends SparkFunSuite { test("request no fields") { val df = spark.read.avro(episodesFile) -df.registerTempTable("avro_table") +df.createOrReplaceTempView("avro_table") assert(spark.sql("select count(*) from avro_table").collect().head === Row(8)) } test("convert formats") { -TestUtils.withTempDir { dir => +withTempPath { dir => val df = spark.read.avro(episodesFile) df.write.parquet(dir.getCanonicalPath) assert(spark.read.parquet(dir.getCanonicalPath).count() === df.count) @@ -95,15 +87,16 @@ class AvroSuite extends SparkFunSuite { } test("rearrange internal schema") { -TestUtils.withTempDir { dir => +withTempPath { dir => val df = spark.read.avro(episodesFile) df.select("doctor", "title").write.avro(dir.getCanonicalPath) } } test("test NULL avro type") { -TestUtils.withTempDir { dir => - val fields = Seq(new Field("null", Schema.create(Type.NULL), "doc", null)).asJava +withTempPath { dir => + val fields = +Seq(new Field("null", Schema.create(Type.NULL), "doc", null.asInstanceOf[Any])).asJava val schema = Schema.createRecord("name", "docs", "namespace", false) schema.setFields
spark git commit: [SPARK-24781][SQL] Using a reference from Dataset in Filter/Sort might not work
Repository: spark Updated Branches: refs/heads/branch-2.3 32429256f -> 9cf375f5b [SPARK-24781][SQL] Using a reference from Dataset in Filter/Sort might not work ## What changes were proposed in this pull request? When we use a reference from Dataset in filter or sort, which was not used in the prior select, an AnalysisException occurs, e.g., ```scala val df = Seq(("test1", 0), ("test2", 1)).toDF("name", "id") df.select(df("name")).filter(df("id") === 0).show() ``` ```scala org.apache.spark.sql.AnalysisException: Resolved attribute(s) id#6 missing from name#5 in operator !Filter (id#6 = 0).;; !Filter (id#6 = 0) +- AnalysisBarrier +- Project [name#5] +- Project [_1#2 AS name#5, _2#3 AS id#6] +- LocalRelation [_1#2, _2#3] ``` This change updates the rule `ResolveMissingReferences` so `Filter` and `Sort` with non-empty `missingInputs` will also be transformed. ## How was this patch tested? Added tests. Author: Liang-Chi Hsieh Closes #21745 from viirya/SPARK-24781. (cherry picked from commit dfd7ac9887f89b9b51b7b143ab54d01f11cfcdb5) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9cf375f5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9cf375f5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9cf375f5 Branch: refs/heads/branch-2.3 Commit: 9cf375f5be3d359912bde9b6ba5766425e8cf3bb Parents: 3242925 Author: Liang-Chi Hsieh Authored: Fri Jul 13 08:25:00 2018 -0700 Committer: Xiao Li Committed: Fri Jul 13 08:25:14 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 30 +++- .../org/apache/spark/sql/DataFrameSuite.scala | 25 2 files changed, 48 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9cf375f5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a584cb8..2858bee 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 @@ -1114,7 +1114,8 @@ class Analyzer( case sa @ Sort(_, _, AnalysisBarrier(child: Aggregate)) => sa case sa @ Sort(_, _, child: Aggregate) => sa - case s @ Sort(order, _, child) if !s.resolved && child.resolved => + case s @ Sort(order, _, child) + if (!s.resolved || s.missingInput.nonEmpty) && child.resolved => val (newOrder, newChild) = resolveExprsAndAddMissingAttrs(order, child) val ordering = newOrder.map(_.asInstanceOf[SortOrder]) if (child.output == newChild.output) { @@ -1125,7 +1126,7 @@ class Analyzer( Project(child.output, newSort) } - case f @ Filter(cond, child) if !f.resolved && child.resolved => + case f @ Filter(cond, child) if (!f.resolved || f.missingInput.nonEmpty) && child.resolved => val (newCond, newChild) = resolveExprsAndAddMissingAttrs(Seq(cond), child) if (child.output == newChild.output) { f.copy(condition = newCond.head) @@ -1136,10 +1137,17 @@ class Analyzer( } } +/** + * This method tries to resolve expressions and find missing attributes recursively. Specially, + * when the expressions used in `Sort` or `Filter` contain unresolved attributes or resolved + * attributes which are missed from child output. This method tries to find the missing + * attributes out and add into the projection. + */ private def resolveExprsAndAddMissingAttrs( exprs: Seq[Expression], plan: LogicalPlan): (Seq[Expression], LogicalPlan) = { - if (exprs.forall(_.resolved)) { -// All given expressions are resolved, no need to continue anymore. + // Missing attributes can be unresolved attributes or resolved attributes which are not in + // the output attributes of the plan. + if (exprs.forall(e => e.resolved && e.references.subsetOf(plan.outputSet))) { (exprs, plan) } else { plan match { @@ -1150,15 +1158,19 @@ class Analyzer( (newExprs, AnalysisBarrier(newChild)) case p: Project => +// Resolving expressions against current plan. val maybeResolvedExprs = exprs.map(resolveExpression(_, p)) +// Recursively resolving expressions on the child of current plan. val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, p.child) -val missingAttrs = AttributeSet(newExprs) -- Att
spark git commit: [SPARK-24781][SQL] Using a reference from Dataset in Filter/Sort might not work
Repository: spark Updated Branches: refs/heads/master 0f24c6f8a -> dfd7ac988 [SPARK-24781][SQL] Using a reference from Dataset in Filter/Sort might not work ## What changes were proposed in this pull request? When we use a reference from Dataset in filter or sort, which was not used in the prior select, an AnalysisException occurs, e.g., ```scala val df = Seq(("test1", 0), ("test2", 1)).toDF("name", "id") df.select(df("name")).filter(df("id") === 0).show() ``` ```scala org.apache.spark.sql.AnalysisException: Resolved attribute(s) id#6 missing from name#5 in operator !Filter (id#6 = 0).;; !Filter (id#6 = 0) +- AnalysisBarrier +- Project [name#5] +- Project [_1#2 AS name#5, _2#3 AS id#6] +- LocalRelation [_1#2, _2#3] ``` This change updates the rule `ResolveMissingReferences` so `Filter` and `Sort` with non-empty `missingInputs` will also be transformed. ## How was this patch tested? Added tests. Author: Liang-Chi Hsieh Closes #21745 from viirya/SPARK-24781. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dfd7ac98 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dfd7ac98 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dfd7ac98 Branch: refs/heads/master Commit: dfd7ac9887f89b9b51b7b143ab54d01f11cfcdb5 Parents: 0f24c6f Author: Liang-Chi Hsieh Authored: Fri Jul 13 08:25:00 2018 -0700 Committer: Xiao Li Committed: Fri Jul 13 08:25:00 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 30 +++- .../org/apache/spark/sql/DataFrameSuite.scala | 25 2 files changed, 48 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/dfd7ac98/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 960ee27..36f14cc 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 @@ -1132,7 +1132,8 @@ class Analyzer( case sa @ Sort(_, _, AnalysisBarrier(child: Aggregate)) => sa case sa @ Sort(_, _, child: Aggregate) => sa - case s @ Sort(order, _, child) if !s.resolved && child.resolved => + case s @ Sort(order, _, child) + if (!s.resolved || s.missingInput.nonEmpty) && child.resolved => val (newOrder, newChild) = resolveExprsAndAddMissingAttrs(order, child) val ordering = newOrder.map(_.asInstanceOf[SortOrder]) if (child.output == newChild.output) { @@ -1143,7 +1144,7 @@ class Analyzer( Project(child.output, newSort) } - case f @ Filter(cond, child) if !f.resolved && child.resolved => + case f @ Filter(cond, child) if (!f.resolved || f.missingInput.nonEmpty) && child.resolved => val (newCond, newChild) = resolveExprsAndAddMissingAttrs(Seq(cond), child) if (child.output == newChild.output) { f.copy(condition = newCond.head) @@ -1154,10 +1155,17 @@ class Analyzer( } } +/** + * This method tries to resolve expressions and find missing attributes recursively. Specially, + * when the expressions used in `Sort` or `Filter` contain unresolved attributes or resolved + * attributes which are missed from child output. This method tries to find the missing + * attributes out and add into the projection. + */ private def resolveExprsAndAddMissingAttrs( exprs: Seq[Expression], plan: LogicalPlan): (Seq[Expression], LogicalPlan) = { - if (exprs.forall(_.resolved)) { -// All given expressions are resolved, no need to continue anymore. + // Missing attributes can be unresolved attributes or resolved attributes which are not in + // the output attributes of the plan. + if (exprs.forall(e => e.resolved && e.references.subsetOf(plan.outputSet))) { (exprs, plan) } else { plan match { @@ -1168,15 +1176,19 @@ class Analyzer( (newExprs, AnalysisBarrier(newChild)) case p: Project => +// Resolving expressions against current plan. val maybeResolvedExprs = exprs.map(resolveExpression(_, p)) +// Recursively resolving expressions on the child of current plan. val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, p.child) -val missingAttrs = AttributeSet(newExprs) -- AttributeSet(maybeResolvedExprs) +// If some attributes used by expressions are resolvable on
spark git commit: [SPARK-23486] cache the function name from the external catalog for lookupFunctions
Repository: spark Updated Branches: refs/heads/master e0f4f206b -> 0ce11d0e3 [SPARK-23486] cache the function name from the external catalog for lookupFunctions ## What changes were proposed in this pull request? This PR will cache the function name from external catalog, it is used by lookupFunctions in the analyzer, and it is cached for each query plan. The original problem is reported in the [ spark-19737](https://issues.apache.org/jira/browse/SPARK-19737) ## How was this patch tested? create new test file LookupFunctionsSuite and add test case in SessionCatalogSuite Author: Kevin Yu Closes #20795 from kevinyu98/spark-23486. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0ce11d0e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0ce11d0e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0ce11d0e Branch: refs/heads/master Commit: 0ce11d0e3a7c8c48d9f7305d2dd39c7b281b2a53 Parents: e0f4f20 Author: Kevin Yu Authored: Thu Jul 12 22:20:06 2018 -0700 Committer: Xiao Li Committed: Thu Jul 12 22:20:06 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 45 ++-- .../sql/catalyst/catalog/SessionCatalog.scala | 16 +++ .../analysis/LookupFunctionsSuite.scala | 104 +++ .../catalyst/catalog/SessionCatalogSuite.scala | 36 +++ .../spark/sql/hive/HiveSessionCatalog.scala | 4 + 5 files changed, 199 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0ce11d0e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9749893..960ee27 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 @@ -17,6 +17,9 @@ package org.apache.spark.sql.catalyst.analysis +import java.util.Locale + +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.Random @@ -1208,16 +1211,46 @@ class Analyzer( * only performs simple existence check according to the function identifier to quickly identify * undefined functions without triggering relation resolution, which may incur potentially * expensive partition/schema discovery process in some cases. - * + * In order to avoid duplicate external functions lookup, the external function identifier will + * store in the local hash set externalFunctionNameSet. * @see [[ResolveFunctions]] * @see https://issues.apache.org/jira/browse/SPARK-19737 */ object LookupFunctions extends Rule[LogicalPlan] { -override def apply(plan: LogicalPlan): LogicalPlan = plan.transformAllExpressions { - case f: UnresolvedFunction if !catalog.functionExists(f.name) => -withPosition(f) { - throw new NoSuchFunctionException(f.name.database.getOrElse("default"), f.name.funcName) -} +override def apply(plan: LogicalPlan): LogicalPlan = { + val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]() + plan.transformAllExpressions { +case f: UnresolvedFunction + if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f +case f: UnresolvedFunction if catalog.isRegisteredFunction(f.name) => f +case f: UnresolvedFunction if catalog.isPersistentFunction(f.name) => + externalFunctionNameSet.add(normalizeFuncName(f.name)) + f +case f: UnresolvedFunction => + withPosition(f) { +throw new NoSuchFunctionException(f.name.database.getOrElse(catalog.getCurrentDatabase), + f.name.funcName) + } + } +} + +def normalizeFuncName(name: FunctionIdentifier): FunctionIdentifier = { + val funcName = if (conf.caseSensitiveAnalysis) { +name.funcName + } else { +name.funcName.toLowerCase(Locale.ROOT) + } + + val databaseName = name.database match { +case Some(a) => formatDatabaseName(a) +case None => catalog.getCurrentDatabase + } + + FunctionIdentifier(funcName, Some(databaseName)) +} + +protected def formatDatabaseName(name: String): String = { + if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT) } } http://git-wip-us.apache.org/repos/asf/spark/blob/0ce11d0e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala -- diff --git a/sql/catalyst/src/main/scal
spark git commit: [SPARK-24790][SQL] Allow complex aggregate expressions in Pivot
Repository: spark Updated Branches: refs/heads/master 11384893b -> 75725057b [SPARK-24790][SQL] Allow complex aggregate expressions in Pivot ## What changes were proposed in this pull request? Relax the check to allow complex aggregate expressions, like `ceil(sum(col1))` or `sum(col1) + 1`, which roughly means any aggregate expression that could appear in an Aggregate plan except pandas UDF (due to the fact that it is not supported in pivot yet). ## How was this patch tested? Added 2 tests in pivot.sql Author: maryannxue Closes #21753 from maryannxue/pivot-relax-syntax. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/75725057 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/75725057 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/75725057 Branch: refs/heads/master Commit: 75725057b3ffdb0891844b10bd707bb0830f92ca Parents: 1138489 Author: maryannxue Authored: Thu Jul 12 16:54:03 2018 -0700 Committer: Xiao Li Committed: Thu Jul 12 16:54:03 2018 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 24 +++--- .../test/resources/sql-tests/inputs/pivot.sql | 18 +++ .../resources/sql-tests/results/pivot.sql.out | 34 ++-- 3 files changed, 62 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/75725057/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c078efd..9749893 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 @@ -509,12 +509,7 @@ class Analyzer( || !p.pivotColumn.resolved => p case Pivot(groupByExprsOpt, pivotColumn, pivotValues, aggregates, child) => // Check all aggregate expressions. -aggregates.foreach { e => - if (!isAggregateExpression(e)) { - throw new AnalysisException( -s"Aggregate expression required for pivot, found '$e'") - } -} +aggregates.foreach(checkValidAggregateExpression) // Group-by expressions coming from SQL are implicit and need to be deduced. val groupByExprs = groupByExprsOpt.getOrElse( (child.outputSet -- aggregates.flatMap(_.references) -- pivotColumn.references).toSeq) @@ -586,12 +581,17 @@ class Analyzer( } } -private def isAggregateExpression(expr: Expression): Boolean = { - expr match { -case Alias(e, _) => isAggregateExpression(e) -case AggregateExpression(_, _, _, _) => true -case _ => false - } +// Support any aggregate expression that can appear in an Aggregate plan except Pandas UDF. +// TODO: Support Pandas UDF. +private def checkValidAggregateExpression(expr: Expression): Unit = expr match { + case _: AggregateExpression => // OK and leave the argument check to CheckAnalysis. + case expr: PythonUDF if PythonUDF.isGroupedAggPandasUDF(expr) => +failAnalysis("Pandas UDF aggregate expressions are currently not supported in pivot.") + case e: Attribute => +failAnalysis( + s"Aggregate expression required for pivot, but '${e.sql}' " + + s"did not appear in any aggregate function.") + case e => e.children.foreach(checkValidAggregateExpression) } } http://git-wip-us.apache.org/repos/asf/spark/blob/75725057/sql/core/src/test/resources/sql-tests/inputs/pivot.sql -- diff --git a/sql/core/src/test/resources/sql-tests/inputs/pivot.sql b/sql/core/src/test/resources/sql-tests/inputs/pivot.sql index 01dea6c..b3d53ad 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pivot.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pivot.sql @@ -111,3 +111,21 @@ PIVOT ( sum(earnings) FOR year IN (2012, 2013) ); + +-- pivot with complex aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + ceil(sum(earnings)), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +); + +-- pivot with invalid arguments in aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(avg(earnings)) + FOR course IN ('dotNET', 'Java') +); http://git-wip-us.apache.org/repos/asf/spark/blob/75725057/sql/core/src/test/resources/sql-tests/results/pivot.sql.out -- diff --git a/sql
spark git commit: [SPARK-24208][SQL][FOLLOWUP] Move test cases to proper locations
Repository: spark Updated Branches: refs/heads/master 07704c971 -> 11384893b [SPARK-24208][SQL][FOLLOWUP] Move test cases to proper locations ## What changes were proposed in this pull request? The PR is a followup to move the test cases introduced by the original PR in their proper location. ## How was this patch tested? moved UTs Author: Marco Gaido Closes #21751 from mgaido91/SPARK-24208_followup. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/11384893 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/11384893 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/11384893 Branch: refs/heads/master Commit: 11384893b6ad09c0c8bc6a350bb9540d0d704bb4 Parents: 07704c9 Author: Marco Gaido Authored: Thu Jul 12 15:13:26 2018 -0700 Committer: Xiao Li Committed: Thu Jul 12 15:13:26 2018 -0700 -- python/pyspark/sql/tests.py | 32 ++-- .../sql/catalyst/analysis/AnalysisSuite.scala | 18 +++ .../apache/spark/sql/GroupedDatasetSuite.scala | 12 3 files changed, 34 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/11384893/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 4404dbe..565654e 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -5471,6 +5471,22 @@ class GroupedMapPandasUDFTests(ReusedSQLTestCase): self.assertEqual(r.a, 'hi') self.assertEqual(r.b, 1) +def test_self_join_with_pandas(self): +import pyspark.sql.functions as F + +@F.pandas_udf('key long, col string', F.PandasUDFType.GROUPED_MAP) +def dummy_pandas_udf(df): +return df[['key', 'col']] + +df = self.spark.createDataFrame([Row(key=1, col='A'), Row(key=1, col='B'), + Row(key=2, col='C')]) +df_with_pandas = df.groupBy('key').apply(dummy_pandas_udf) + +# this was throwing an AnalysisException before SPARK-24208 +res = df_with_pandas.alias('temp0').join(df_with_pandas.alias('temp1'), + F.col('temp0.key') == F.col('temp1.key')) +self.assertEquals(res.count(), 5) + @unittest.skipIf( not _have_pandas or not _have_pyarrow, @@ -5925,22 +5941,6 @@ class GroupedAggPandasUDFTests(ReusedSQLTestCase): 'mixture.*aggregate function.*group aggregate pandas UDF'): df.groupby(df.id).agg(mean_udf(df.v), mean(df.v)).collect() -def test_self_join_with_pandas(self): -import pyspark.sql.functions as F - -@F.pandas_udf('key long, col string', F.PandasUDFType.GROUPED_MAP) -def dummy_pandas_udf(df): -return df[['key', 'col']] - -df = self.spark.createDataFrame([Row(key=1, col='A'), Row(key=1, col='B'), - Row(key=2, col='C')]) -dfWithPandas = df.groupBy('key').apply(dummy_pandas_udf) - -# this was throwing an AnalysisException before SPARK-24208 -res = dfWithPandas.alias('temp0').join(dfWithPandas.alias('temp1'), - F.col('temp0.key') == F.col('temp1.key')) -self.assertEquals(res.count(), 5) - @unittest.skipIf( not _have_pandas or not _have_pyarrow, http://git-wip-us.apache.org/repos/asf/spark/blob/11384893/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala -- 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 cd85795..bbcdf6c 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 @@ -21,6 +21,7 @@ import java.util.TimeZone import org.scalatest.Matchers +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -557,4 +558,21 @@ class AnalysisSuite extends AnalysisTest with Matchers { SubqueryAlias("tbl", testRelation))) assertAnalysisError(barrier, Seq("cannot resolve '`tbl.b`'")) } + + test("SPARK-24208: analysis fails on self-join with FlatMapGroupsInPandas") { +val pythonUdf = PythonUDF("pyUDF", null, + StructType(Seq(StructField("a", LongType))), + Seq.empty, + PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, +
spark git commit: [SPARK-23007][SQL][TEST] Add read schema suite for file-based data sources
Repository: spark Updated Branches: refs/heads/master 395860a98 -> 07704c971 [SPARK-23007][SQL][TEST] Add read schema suite for file-based data sources ## What changes were proposed in this pull request? The reader schema is said to be evolved (or projected) when it changed after the data is written. The followings are already supported in file-based data sources. Note that partition columns are not maintained in files. In this PR, `column` means `non-partition column`. 1. Add a column 2. Hide a column 3. Change a column position 4. Change a column type (upcast) This issue aims to guarantee users a backward-compatible read-schema test coverage on file-based data sources and to prevent future regressions by *adding read schema tests explicitly*. Here, we consider safe changes without data loss. For example, data type change should be from small types to larger types like `int`-to-`long`, not vice versa. As of today, in the master branch, file-based data sources have the following coverage. File Format | Coverage | Note --- | -- | TEXT | N/A| Schema consists of a single string column. CSV| 1, 2, 4| JSON | 1, 2, 3, 4| ORC| 1, 2, 3, 4| Native vectorized ORC reader has the widest coverage among ORC formats. PARQUET | 1, 2, 3| ## How was this patch tested? Pass the Jenkins with newly added test suites. Author: Dongjoon Hyun Closes #20208 from dongjoon-hyun/SPARK-SCHEMA-EVOLUTION. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/07704c97 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/07704c97 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/07704c97 Branch: refs/heads/master Commit: 07704c971cbc92bff15e15f8c42fab9afaab3ef7 Parents: 395860a Author: Dongjoon Hyun Authored: Thu Jul 12 14:08:49 2018 -0700 Committer: Xiao Li Committed: Thu Jul 12 14:08:49 2018 -0700 -- .../execution/datasources/ReadSchemaSuite.scala | 181 +++ .../execution/datasources/ReadSchemaTest.scala | 493 +++ 2 files changed, 674 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/07704c97/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala new file mode 100644 index 000..23c58e1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala @@ -0,0 +1,181 @@ +/* + * 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.internal.SQLConf + +/** + * Read schema suites have the following hierarchy and aims to guarantee users + * a backward-compatible read-schema change coverage on file-based data sources, and + * to prevent future regressions. + * + * ReadSchemaSuite + * -> CSVReadSchemaSuite + * -> HeaderCSVReadSchemaSuite + * + * -> JsonReadSchemaSuite + * + * -> OrcReadSchemaSuite + * -> VectorizedOrcReadSchemaSuite + * + * -> ParquetReadSchemaSuite + * -> VectorizedParquetReadSchemaSuite + * -> MergedParquetReadSchemaSuite + */ + +/** + * All file-based data sources supports column addition and removal at the end. + */ +abstract class ReadSchemaSuite + extends AddColumnTest + with HideColumnAtTheEndTest { + + var originalConf: Boolean = _ +} + +class CSVReadSchemaSuite + extends ReadSchemaSuite + with IntegralTypeTest + with ToDoubleTypeTest + with ToDecimalTypeTest + with ToStringTypeTest { + + override val format: String = "csv" +} + +class HeaderCSVReadSchemaSuite + extends ReadSchemaSuite + with IntegralTypeTest + with ToDoubleTypeTest + with ToDecimalTypeTest + with ToStringTyp
[2/2] spark git commit: [SPARK-24768][SQL] Have a built-in AVRO data source implementation
[SPARK-24768][SQL] Have a built-in AVRO data source implementation ## What changes were proposed in this pull request? Apache Avro (https://avro.apache.org) is a popular data serialization format. It is widely used in the Spark and Hadoop ecosystem, especially for Kafka-based data pipelines. Using the external package https://github.com/databricks/spark-avro, Spark SQL can read and write the avro data. Making spark-Avro built-in can provide a better experience for first-time users of Spark SQL and structured streaming. We expect the built-in Avro data source can further improve the adoption of structured streaming. The proposal is to inline code from spark-avro package (https://github.com/databricks/spark-avro). The target release is Spark 2.4. [Built-in AVRO Data Source In Spark 2.4.pdf](https://github.com/apache/spark/files/2181511/Built-in.AVRO.Data.Source.In.Spark.2.4.pdf) ## How was this patch tested? Unit test Author: Gengliang Wang Closes #21742 from gengliangwang/export_avro. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/395860a9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/395860a9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/395860a9 Branch: refs/heads/master Commit: 395860a986987886df6d60fd9b26afd818b2cb39 Parents: 1055c94 Author: Gengliang Wang Authored: Thu Jul 12 13:55:25 2018 -0700 Committer: Xiao Li Committed: Thu Jul 12 13:55:25 2018 -0700 -- dev/run-tests.py| 2 +- dev/sparktestsupport/modules.py | 10 + external/avro/pom.xml | 73 ++ apache.spark.sql.sources.DataSourceRegister | 1 + .../apache/spark/sql/avro/AvroFileFormat.scala | 289 +++ .../spark/sql/avro/AvroOutputWriter.scala | 164 .../sql/avro/AvroOutputWriterFactory.scala | 38 + .../spark/sql/avro/SchemaConverters.scala | 406 ++ .../org/apache/spark/sql/avro/package.scala | 39 + external/avro/src/test/resources/episodes.avro | Bin 0 -> 597 bytes .../avro/src/test/resources/log4j.properties| 49 ++ .../test-random-partitioned/part-r-0.avro | Bin 0 -> 1768 bytes .../test-random-partitioned/part-r-1.avro | Bin 0 -> 2313 bytes .../test-random-partitioned/part-r-2.avro | Bin 0 -> 1621 bytes .../test-random-partitioned/part-r-3.avro | Bin 0 -> 2117 bytes .../test-random-partitioned/part-r-4.avro | Bin 0 -> 3282 bytes .../test-random-partitioned/part-r-5.avro | Bin 0 -> 1550 bytes .../test-random-partitioned/part-r-6.avro | Bin 0 -> 1729 bytes .../test-random-partitioned/part-r-7.avro | Bin 0 -> 1897 bytes .../test-random-partitioned/part-r-8.avro | Bin 0 -> 3420 bytes .../test-random-partitioned/part-r-9.avro | Bin 0 -> 1796 bytes .../test-random-partitioned/part-r-00010.avro | Bin 0 -> 3872 bytes external/avro/src/test/resources/test.avro | Bin 0 -> 1365 bytes external/avro/src/test/resources/test.avsc | 53 ++ external/avro/src/test/resources/test.json | 42 + .../org/apache/spark/sql/avro/AvroSuite.scala | 812 +++ .../avro/SerializableConfigurationSuite.scala | 50 ++ .../org/apache/spark/sql/avro/TestUtils.scala | 156 pom.xml | 1 + project/SparkBuild.scala| 12 +- 30 files changed, 2191 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/395860a9/dev/run-tests.py -- diff --git a/dev/run-tests.py b/dev/run-tests.py index cd45908..d9d3789 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', 'sql-kafka-0-10', 'examples', 'hive-thriftserver', +['sql', 'avro', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver', 'pyspark-sql', 'repl', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] """ modules_to_test = set() http://git-wip-us.apache.org/repos/asf/spark/blob/395860a9/dev/sparktestsupport/modules.py -- diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index dfea762..2aa3555 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -170,6 +170,16 @@ hive_thriftserver = Module( ] ) +avro = Module( +name="avro", +dependencies=[sql], +source_file_regexes=[ +"external/avro", +], +sbt_test_goals=[ +"avro/test", +] +) sql_kafka = Module(
[1/2] spark git commit: [SPARK-24768][SQL] Have a built-in AVRO data source implementation
Repository: spark Updated Branches: refs/heads/master 1055c94cd -> 395860a98 http://git-wip-us.apache.org/repos/asf/spark/blob/395860a9/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala -- diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala new file mode 100644 index 000..c6c1e40 --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -0,0 +1,812 @@ +/* + * 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.avro + +import java.io._ +import java.nio.file.Files +import java.sql.{Date, Timestamp} +import java.util.{TimeZone, UUID} + +import scala.collection.JavaConverters._ + +import org.apache.avro.Schema +import org.apache.avro.Schema.{Field, Type} +import org.apache.avro.file.DataFileWriter +import org.apache.avro.generic.{GenericData, GenericDatumWriter, GenericRecord} +import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed} +import org.apache.commons.io.FileUtils + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql._ +import org.apache.spark.sql.avro.SchemaConverters.IncompatibleSchemaException +import org.apache.spark.sql.types._ + +class AvroSuite extends SparkFunSuite { + val episodesFile = "src/test/resources/episodes.avro" + val testFile = "src/test/resources/test.avro" + + private var spark: SparkSession = _ + + override protected def beforeAll(): Unit = { +super.beforeAll() +spark = SparkSession.builder() + .master("local[2]") + .appName("AvroSuite") + .config("spark.sql.files.maxPartitionBytes", 1024) + .getOrCreate() + } + + override protected def afterAll(): Unit = { +try { + spark.sparkContext.stop() +} finally { + super.afterAll() +} + } + + test("reading from multiple paths") { +val df = spark.read.avro(episodesFile, episodesFile) +assert(df.count == 16) + } + + test("reading and writing partitioned data") { +val df = spark.read.avro(episodesFile) +val fields = List("title", "air_date", "doctor") +for (field <- fields) { + TestUtils.withTempDir { dir => +val outputDir = s"$dir/${UUID.randomUUID}" +df.write.partitionBy(field).avro(outputDir) +val input = spark.read.avro(outputDir) +// makes sure that no fields got dropped. +// We convert Rows to Seqs in order to work around SPARK-10325 +assert(input.select(field).collect().map(_.toSeq).toSet === + df.select(field).collect().map(_.toSeq).toSet) + } +} + } + + test("request no fields") { +val df = spark.read.avro(episodesFile) +df.registerTempTable("avro_table") +assert(spark.sql("select count(*) from avro_table").collect().head === Row(8)) + } + + test("convert formats") { +TestUtils.withTempDir { dir => + val df = spark.read.avro(episodesFile) + df.write.parquet(dir.getCanonicalPath) + assert(spark.read.parquet(dir.getCanonicalPath).count() === df.count) +} + } + + test("rearrange internal schema") { +TestUtils.withTempDir { dir => + val df = spark.read.avro(episodesFile) + df.select("doctor", "title").write.avro(dir.getCanonicalPath) +} + } + + test("test NULL avro type") { +TestUtils.withTempDir { dir => + val fields = Seq(new Field("null", Schema.create(Type.NULL), "doc", null)).asJava + val schema = Schema.createRecord("name", "docs", "namespace", false) + schema.setFields(fields) + val datumWriter = new GenericDatumWriter[GenericRecord](schema) + val dataFileWriter = new DataFileWriter[GenericRecord](datumWriter) + dataFileWriter.create(schema, new File(s"$dir.avro")) + val avroRec = new GenericData.Record(schema) + avroRec.put("null", null) + dataFileWriter.append(avroRec) + dataFileWriter.flush() + dataFileWriter.close() + + intercept[IncompatibleSchemaException] { +spark.read.avro(s"$dir.avro") + } +} + } + + test("union(int, long) is read as long") { +TestUtils.withTempDir { dir => + val avroSchema
spark git commit: [SPARK-24761][SQL] Adding of isModifiable() to RuntimeConfig
Repository: spark Updated Branches: refs/heads/master e008ad175 -> 3ab48f985 [SPARK-24761][SQL] Adding of isModifiable() to RuntimeConfig ## What changes were proposed in this pull request? In the PR, I propose to extend `RuntimeConfig` by new method `isModifiable()` which returns `true` if a config parameter can be modified at runtime (for current session state). For static SQL and core parameters, the method returns `false`. ## How was this patch tested? Added new test to `RuntimeConfigSuite` for checking Spark core and SQL parameters. Author: Maxim Gekk Closes #21730 from MaxGekk/is-modifiable. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3ab48f98 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3ab48f98 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3ab48f98 Branch: refs/heads/master Commit: 3ab48f985c7f96bc9143caad99bf3df7cc984583 Parents: e008ad1 Author: Maxim Gekk Authored: Wed Jul 11 17:38:43 2018 -0700 Committer: Xiao Li Committed: Wed Jul 11 17:38:43 2018 -0700 -- python/pyspark/sql/conf.py| 8 .../scala/org/apache/spark/sql/internal/SQLConf.scala | 4 .../scala/org/apache/spark/sql/RuntimeConfig.scala| 11 +++ .../org/apache/spark/sql/RuntimeConfigSuite.scala | 14 ++ 4 files changed, 37 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3ab48f98/python/pyspark/sql/conf.py -- diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index db49040..f80bf59 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -63,6 +63,14 @@ class RuntimeConfig(object): raise TypeError("expected %s '%s' to be a string (was '%s')" % (identifier, obj, type(obj).__name__)) +@ignore_unicode_prefix +@since(2.4) +def isModifiable(self, key): +"""Indicates whether the configuration property with the given key +is modifiable in the current session. +""" +return self._jconf.isModifiable(key) + def _test(): import os http://git-wip-us.apache.org/repos/asf/spark/blob/3ab48f98/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index ae56cc9..14dd528 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1907,4 +1907,8 @@ class SQLConf extends Serializable with Logging { } cloned } + + def isModifiable(key: String): Boolean = { +sqlConfEntries.containsKey(key) && !staticConfKeys.contains(key) + } } http://git-wip-us.apache.org/repos/asf/spark/blob/3ab48f98/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala -- 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 b352e33..3c39579 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 @@ -133,6 +133,17 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { } /** + * Indicates whether the configuration property with the given key + * is modifiable in the current session. + * + * @return `true` if the configuration property is modifiable. For static SQL, Spark Core, + * invalid (not existing) and other non-modifiable configuration properties, + * the returned value is `false`. + * @since 2.4.0 + */ + def isModifiable(key: String): Boolean = sqlConf.isModifiable(key) + + /** * Returns whether a particular key is set. */ protected[sql] def contains(key: String): Boolean = { http://git-wip-us.apache.org/repos/asf/spark/blob/3ab48f98/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index cfe2e9f..cdcea09 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -54,4 +54,18 @@ class RuntimeConfigSuite extends SparkFunSuite { conf.get("k1") } } + + test("SPARK-24761: is a config
spark git commit: [SPARK-24782][SQL] Simplify conf retrieval in SQL expressions
Repository: spark Updated Branches: refs/heads/master ff7f6ef75 -> e008ad175 [SPARK-24782][SQL] Simplify conf retrieval in SQL expressions ## What changes were proposed in this pull request? The PR simplifies the retrieval of config in `size`, as we can access them from tasks too thanks to SPARK-24250. ## How was this patch tested? existing UTs Author: Marco Gaido Closes #21736 from mgaido91/SPARK-24605_followup. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e008ad17 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e008ad17 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e008ad17 Branch: refs/heads/master Commit: e008ad175256a3192fdcbd2c4793044d52f46d57 Parents: ff7f6ef Author: Marco Gaido Authored: Wed Jul 11 17:30:43 2018 -0700 Committer: Xiao Li Committed: Wed Jul 11 17:30:43 2018 -0700 -- .../expressions/collectionOperations.scala | 10 +-- .../catalyst/expressions/jsonExpressions.scala | 16 ++--- .../spark/sql/catalyst/plans/QueryPlan.scala| 2 - .../CollectionExpressionsSuite.scala| 27 .../expressions/JsonExpressionsSuite.scala | 65 ++-- .../scala/org/apache/spark/sql/functions.scala | 4 +- 6 files changed, 57 insertions(+), 67 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e008ad17/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala -- 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 879603b..e217d37 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 @@ -89,15 +89,9 @@ trait BinaryArrayExpressionWithImplicitCast extends BinaryExpression > SELECT _FUNC_(NULL); -1 """) -case class Size( -child: Expression, -legacySizeOfNull: Boolean) - extends UnaryExpression with ExpectsInputTypes { +case class Size(child: Expression) extends UnaryExpression with ExpectsInputTypes { - def this(child: Expression) = -this( - child, - legacySizeOfNull = SQLConf.get.getConf(SQLConf.LEGACY_SIZE_OF_NULL)) + val legacySizeOfNull = SQLConf.get.legacySizeOfNull override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(ArrayType, MapType)) http://git-wip-us.apache.org/repos/asf/spark/blob/e008ad17/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 8cd8605..63943b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -514,10 +514,11 @@ case class JsonToStructs( schema: DataType, options: Map[String, String], child: Expression, -timeZoneId: Option[String], -forceNullableSchema: Boolean) +timeZoneId: Option[String] = None) extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { + val forceNullableSchema = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA) + // The JSON input data might be missing certain fields. We force the nullability // of the user-provided schema to avoid data corruptions. In particular, the parquet-mr encoder // can generate incorrect files if values are missing in columns declared as non-nullable. @@ -531,8 +532,7 @@ case class JsonToStructs( schema = JsonExprUtils.evalSchemaExpr(schema), options = options, child = child, - timeZoneId = None, - forceNullableSchema = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)) + timeZoneId = None) def this(child: Expression, schema: Expression) = this(child, schema, Map.empty[String, String]) @@ -541,13 +541,7 @@ case class JsonToStructs( schema = JsonExprUtils.evalSchemaExpr(schema), options = JsonExprUtils.convertToMapData(options), child = child, - timeZoneId = None, - forceNullableSchema = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)) - - // Used in `org.apache.spark.sql.functions` - def this(schema: DataType, options: Map[String,
spark git commit: [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas
Repository: spark Updated Branches: refs/heads/branch-2.3 86457a16d -> 32429256f [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas A self-join on a dataset which contains a `FlatMapGroupsInPandas` fails because of duplicate attributes. This happens because we are not dealing with this specific case in our `dedupAttr` rules. The PR fix the issue by adding the management of the specific case added UT + manual tests Author: Marco Gaido Author: Marco Gaido Closes #21737 from mgaido91/SPARK-24208. (cherry picked from commit ebf4bfb966389342bfd9bdb8e3b612828c18730c) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/32429256 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/32429256 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/32429256 Branch: refs/heads/branch-2.3 Commit: 32429256f3e659c648462e5b2740747645740c97 Parents: 86457a1 Author: Marco Gaido Authored: Wed Jul 11 09:29:19 2018 -0700 Committer: Xiao Li Committed: Wed Jul 11 09:35:44 2018 -0700 -- python/pyspark/sql/tests.py | 16 .../spark/sql/catalyst/analysis/Analyzer.scala | 4 .../org/apache/spark/sql/GroupedDatasetSuite.scala | 12 3 files changed, 32 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/32429256/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index aa7d8eb..6bfb329 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -4691,6 +4691,22 @@ class GroupedMapPandasUDFTests(ReusedSQLTestCase): result = df.groupby('time').apply(foo_udf).sort('time') self.assertPandasEqual(df.toPandas(), result.toPandas()) +def test_self_join_with_pandas(self): +import pyspark.sql.functions as F + +@F.pandas_udf('key long, col string', F.PandasUDFType.GROUPED_MAP) +def dummy_pandas_udf(df): +return df[['key', 'col']] + +df = self.spark.createDataFrame([Row(key=1, col='A'), Row(key=1, col='B'), + Row(key=2, col='C')]) +dfWithPandas = df.groupBy('key').apply(dummy_pandas_udf) + +# this was throwing an AnalysisException before SPARK-24208 +res = dfWithPandas.alias('temp0').join(dfWithPandas.alias('temp1'), + F.col('temp0.key') == F.col('temp1.key')) +self.assertEquals(res.count(), 5) + if __name__ == "__main__": from pyspark.sql.tests import * http://git-wip-us.apache.org/repos/asf/spark/blob/32429256/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8597d83..a584cb8 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 @@ -723,6 +723,10 @@ class Analyzer( if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) +case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) +if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + (oldVersion, oldVersion.copy(output = output.map(_.newInstance( + case oldVersion: Generate if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => val newOutput = oldVersion.generatorOutput.map(_.newInstance()) http://git-wip-us.apache.org/repos/asf/spark/blob/32429256/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala index 218a1b7..9699fad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala @@ -93,4 +93,16 @@ class GroupedDatasetSuite extends QueryTest with SharedSQLContext { } datasetWithUDF.unpersist(true) } + + test("SPARK-24208: analysis fails on self-join with FlatMapGroupsInPandas") { +val df = datasetWithUDF.groupBy("s").flatMapGroupsInPandas(PythonUDF( + "pyUDF", + null, + StructType(Seq(St
spark git commit: [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas
Repository: spark Updated Branches: refs/heads/master 592cc8458 -> ebf4bfb96 [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas ## What changes were proposed in this pull request? A self-join on a dataset which contains a `FlatMapGroupsInPandas` fails because of duplicate attributes. This happens because we are not dealing with this specific case in our `dedupAttr` rules. The PR fix the issue by adding the management of the specific case ## How was this patch tested? added UT + manual tests Author: Marco Gaido Author: Marco Gaido Closes #21737 from mgaido91/SPARK-24208. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ebf4bfb9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ebf4bfb9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ebf4bfb9 Branch: refs/heads/master Commit: ebf4bfb966389342bfd9bdb8e3b612828c18730c Parents: 592cc84 Author: Marco Gaido Authored: Wed Jul 11 09:29:19 2018 -0700 Committer: Xiao Li Committed: Wed Jul 11 09:29:19 2018 -0700 -- python/pyspark/sql/tests.py | 16 .../spark/sql/catalyst/analysis/Analyzer.scala | 4 .../org/apache/spark/sql/GroupedDatasetSuite.scala | 12 3 files changed, 32 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ebf4bfb9/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 8d73806..4404dbe 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -5925,6 +5925,22 @@ class GroupedAggPandasUDFTests(ReusedSQLTestCase): 'mixture.*aggregate function.*group aggregate pandas UDF'): df.groupby(df.id).agg(mean_udf(df.v), mean(df.v)).collect() +def test_self_join_with_pandas(self): +import pyspark.sql.functions as F + +@F.pandas_udf('key long, col string', F.PandasUDFType.GROUPED_MAP) +def dummy_pandas_udf(df): +return df[['key', 'col']] + +df = self.spark.createDataFrame([Row(key=1, col='A'), Row(key=1, col='B'), + Row(key=2, col='C')]) +dfWithPandas = df.groupBy('key').apply(dummy_pandas_udf) + +# this was throwing an AnalysisException before SPARK-24208 +res = dfWithPandas.alias('temp0').join(dfWithPandas.alias('temp1'), + F.col('temp0.key') == F.col('temp1.key')) +self.assertEquals(res.count(), 5) + @unittest.skipIf( not _have_pandas or not _have_pyarrow, http://git-wip-us.apache.org/repos/asf/spark/blob/ebf4bfb9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e187133..c078efd 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 @@ -738,6 +738,10 @@ class Analyzer( if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) +case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) +if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + (oldVersion, oldVersion.copy(output = output.map(_.newInstance( + case oldVersion: Generate if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => val newOutput = oldVersion.generatorOutput.map(_.newInstance()) http://git-wip-us.apache.org/repos/asf/spark/blob/ebf4bfb9/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala index 147c0b6..bd54ea4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GroupedDatasetSuite.scala @@ -93,4 +93,16 @@ class GroupedDatasetSuite extends QueryTest with SharedSQLContext { } datasetWithUDF.unpersist(true) } + + test("SPARK-24208: analysis fails on self-join with FlatMapGroupsInPandas") { +val df = datasetWithUDF.groupBy("s").flatMapGroupsInPandas(PythonUDF( + "pyUDF", + null, + StructType
spark git commit: [SPARK-24759][SQL] No reordering keys for broadcast hash join
Repository: spark Updated Branches: refs/heads/master aec966b05 -> eb6e98803 [SPARK-24759][SQL] No reordering keys for broadcast hash join ## What changes were proposed in this pull request? As the implementation of the broadcast hash join is independent of the input hash partitioning, reordering keys is not necessary. Thus, we solve this issue by simply removing the broadcast hash join from the reordering rule in EnsureRequirements. ## How was this patch tested? N/A Author: Xiao Li Closes #21728 from gatorsmile/cleanER. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/eb6e9880 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/eb6e9880 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/eb6e9880 Branch: refs/heads/master Commit: eb6e9880397dbac8b0b9ebc0796150b6924fc566 Parents: aec966b Author: Xiao Li Authored: Mon Jul 9 14:53:14 2018 -0700 Committer: Xiao Li Committed: Mon Jul 9 14:53:14 2018 -0700 -- .../spark/sql/execution/exchange/EnsureRequirements.scala | 7 --- 1 file changed, 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/eb6e9880/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index ad95879..d96ecba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -279,13 +279,6 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { */ private def reorderJoinPredicates(plan: SparkPlan): SparkPlan = { plan match { - case BroadcastHashJoinExec(leftKeys, rightKeys, joinType, buildSide, condition, left, -right) => -val (reorderedLeftKeys, reorderedRightKeys) = - reorderJoinKeys(leftKeys, rightKeys, left.outputPartitioning, right.outputPartitioning) -BroadcastHashJoinExec(reorderedLeftKeys, reorderedRightKeys, joinType, buildSide, condition, - left, right) - case ShuffledHashJoinExec(leftKeys, rightKeys, joinType, buildSide, condition, left, right) => val (reorderedLeftKeys, reorderedRightKeys) = reorderJoinKeys(leftKeys, rightKeys, left.outputPartitioning, right.outputPartitioning) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: Revert "[SPARK-24268][SQL] Use datatype.simpleString in error messages"
Repository: spark Updated Branches: refs/heads/master 1bd3d61f4 -> aec966b05 Revert "[SPARK-24268][SQL] Use datatype.simpleString in error messages" This reverts commit 1bd3d61f4191767a94b71b42f4d00706b703e84f. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/aec966b0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/aec966b0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/aec966b0 Branch: refs/heads/master Commit: aec966b05e8df9d459dae88d091de1923e50e2dc Parents: 1bd3d61 Author: Xiao Li Authored: Mon Jul 9 14:24:23 2018 -0700 Committer: Xiao Li Committed: Mon Jul 9 14:24:23 2018 -0700 -- .../apache/spark/sql/kafka010/KafkaWriteTask.scala | 6 +++--- .../org/apache/spark/sql/kafka010/KafkaWriter.scala | 6 +++--- .../sql/kafka010/KafkaContinuousSinkSuite.scala | 4 ++-- .../apache/spark/sql/kafka010/KafkaSinkSuite.scala | 4 ++-- .../scala/org/apache/spark/ml/feature/DCT.scala | 3 +-- .../org/apache/spark/ml/feature/FeatureHasher.scala | 5 ++--- .../org/apache/spark/ml/feature/HashingTF.scala | 2 +- .../org/apache/spark/ml/feature/Interaction.scala | 3 +-- .../scala/org/apache/spark/ml/feature/NGram.scala | 2 +- .../org/apache/spark/ml/feature/OneHotEncoder.scala | 3 +-- .../org/apache/spark/ml/feature/RFormula.scala | 2 +- .../apache/spark/ml/feature/StopWordsRemover.scala | 4 ++-- .../org/apache/spark/ml/feature/Tokenizer.scala | 3 +-- .../apache/spark/ml/feature/VectorAssembler.scala | 2 +- .../scala/org/apache/spark/ml/fpm/FPGrowth.scala| 2 +- .../org/apache/spark/ml/util/SchemaUtils.scala | 11 --- .../BinaryClassificationEvaluatorSuite.scala| 4 ++-- .../org/apache/spark/ml/feature/RFormulaSuite.scala | 2 +- .../spark/ml/feature/VectorAssemblerSuite.scala | 6 +++--- .../apache/spark/ml/recommendation/ALSSuite.scala | 2 +- .../ml/regression/AFTSurvivalRegressionSuite.scala | 2 +- .../org/apache/spark/ml/util/MLTestingUtils.scala | 6 +++--- .../catalyst/expressions/complexTypeCreator.scala | 4 ++-- .../sql/catalyst/expressions/jsonExpressions.scala | 2 +- .../catalyst/expressions/stringExpressions.scala| 5 ++--- .../spark/sql/catalyst/json/JacksonGenerator.scala | 4 ++-- .../spark/sql/catalyst/json/JacksonParser.scala | 6 ++ .../spark/sql/catalyst/json/JsonInferSchema.scala | 6 ++ .../apache/spark/sql/catalyst/util/TypeUtils.scala | 5 ++--- .../apache/spark/sql/types/AbstractDataType.scala | 9 - .../org/apache/spark/sql/types/ArrayType.scala | 5 ++--- .../org/apache/spark/sql/types/DecimalType.scala| 3 +-- .../org/apache/spark/sql/types/ObjectType.scala | 3 +-- .../org/apache/spark/sql/types/StructType.scala | 5 ++--- .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 2 +- .../analysis/ExpressionTypeCheckingSuite.scala | 16 .../sql/catalyst/parser/ExpressionParserSuite.scala | 2 +- .../org/apache/spark/sql/types/DataTypeSuite.scala | 2 +- .../datasources/parquet/VectorizedColumnReader.java | 2 +- .../apache/spark/sql/RelationalGroupedDataset.scala | 2 +- .../spark/sql/execution/arrow/ArrowUtils.scala | 3 +-- .../sql/execution/datasources/orc/OrcFilters.scala | 2 +- .../parquet/ParquetSchemaConverter.scala| 2 +- .../spark/sql/execution/stat/StatFunctions.scala| 2 +- .../sql-tests/results/json-functions.sql.out| 4 ++-- .../resources/sql-tests/results/literals.sql.out| 6 +++--- .../datasources/parquet/ParquetSchemaSuite.scala| 4 ++-- .../sql/hive/execution/HiveTableScanExec.scala | 6 +++--- 48 files changed, 88 insertions(+), 108 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/aec966b0/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala -- diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 59a8470..d90630a 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -110,7 +110,7 @@ private[kafka010] abstract class KafkaRowWriter( case t => throw new IllegalStateException(s"${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + s"attribute unsupported type $t. ${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + - s"must be a ${StringType.simpleString}") + "must be a StringType") } val keyExpression = inputSchema.find(_.name == KafkaWriter.KEY_ATTRIBUTE_NAME) .g
spark git commit: [SPARK-24675][SQL] Rename table: validate existence of new location
Repository: spark Updated Branches: refs/heads/master ac78bcce0 -> 33952cfa8 [SPARK-24675][SQL] Rename table: validate existence of new location ## What changes were proposed in this pull request? If table is renamed to a existing new location, data won't show up. ``` scala> Seq("hello").toDF("a").write.format("parquet").saveAsTable("t") scala> sql("select * from t").show() +-+ |a| +-+ |hello| +-+ scala> sql("alter table t rename to test") res2: org.apache.spark.sql.DataFrame = [] scala> sql("select * from test").show() +---+ | a| +---+ +---+ ``` The file layout is like ``` $ tree test test âââ gabage âââ t âââ _SUCCESS âââ part-0-856b0f10-08f1-42d6-9eb3-7719261f3d5e-c000.snappy.parquet ``` In Hive, if the new location exists, the renaming will fail even the location is empty. We should have the same validation in Catalog, in case of unexpected bugs. ## How was this patch tested? New unit test. Author: Gengliang Wang Closes #21655 from gengliangwang/validate_rename_table. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/33952cfa Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/33952cfa Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/33952cfa Branch: refs/heads/master Commit: 33952cfa8182c1e925083e18c63c6152dcc3c8b4 Parents: ac78bcc Author: Gengliang Wang Authored: Thu Jul 5 09:25:19 2018 -0700 Committer: Xiao Li Committed: Thu Jul 5 09:25:19 2018 -0700 -- docs/sql-programming-guide.md | 1 + .../sql/catalyst/catalog/SessionCatalog.scala | 20 .../spark/sql/execution/command/DDLSuite.scala | 18 ++ 3 files changed, 39 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/33952cfa/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index cd7329b..ad23dae 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1850,6 +1850,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.hive.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. - In version 2.3 and earlier, `to_utc_timestamp` and `from_utc_timestamp` respect the timezone in the input timestamp string, which breaks the assumption that the input timestamp is in a specific timezone. Therefore, these 2 functions can return unexpected results. In version 2.4 and later, this problem has been fixed. `to_utc_timestamp` and `from_utc_timestamp` will return null if the input timestamp string contains timezone. As an example, `from_utc_timestamp('2000-10-10 00:00:00', 'GMT+1')` will return `2000-10-10 01:00:00` in both Spark 2.3 and 2.4. However, `from_utc_timestamp('2000-10-10 00:00:00+00:00', 'GMT+1')`, assuming a local timezone of GMT+8, will return `2000-10-10 09:00:00` in Spark 2.3 but `null` in 2.4. For people w
spark git commit: [SPARK-22384][SQL][FOLLOWUP] Refine partition pruning when attribute is wrapped in Cast
Repository: spark Updated Branches: refs/heads/master ca8243f30 -> bf764a33b [SPARK-22384][SQL][FOLLOWUP] Refine partition pruning when attribute is wrapped in Cast ## What changes were proposed in this pull request? As mentioned in https://github.com/apache/spark/pull/21586 , `Cast.mayTruncate` is not 100% safe, string to boolean is allowed. Since changing `Cast.mayTruncate` also changes the behavior of Dataset, here I propose to add a new `Cast.canSafeCast` for partition pruning. ## How was this patch tested? new test cases Author: Wenchen Fan Closes #21712 from cloud-fan/safeCast. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bf764a33 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bf764a33 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bf764a33 Branch: refs/heads/master Commit: bf764a33bef617aa9bae535a5ea73d6a3e278d42 Parents: ca8243f Author: Wenchen Fan Authored: Wed Jul 4 18:36:09 2018 -0700 Committer: Xiao Li Committed: Wed Jul 4 18:36:09 2018 -0700 -- .../spark/sql/catalyst/expressions/Cast.scala | 20 .../apache/spark/sql/hive/client/HiveShim.scala | 5 +++-- .../spark/sql/hive/client/HiveClientSuite.scala | 20 ++-- 3 files changed, 41 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bf764a33/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala -- 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 699ea53..7971ae6 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 @@ -134,6 +134,26 @@ object Cast { toPrecedence > 0 && fromPrecedence > toPrecedence } + /** + * Returns true iff we can safely cast the `from` type to `to` type without any truncating or + * precision lose, e.g. int -> long, date -> timestamp. + */ + def canSafeCast(from: AtomicType, to: AtomicType): Boolean = (from, to) match { +case _ if from == to => true +case (from: NumericType, to: DecimalType) if to.isWiderThan(from) => true +case (from: DecimalType, to: NumericType) if from.isTighterThan(to) => true +case (from, to) if legalNumericPrecedence(from, to) => true +case (DateType, TimestampType) => true +case (_, StringType) => true +case _ => false + } + + private def legalNumericPrecedence(from: DataType, to: DataType): Boolean = { +val fromPrecedence = TypeCoercion.numericPrecedence.indexOf(from) +val toPrecedence = TypeCoercion.numericPrecedence.indexOf(to) +fromPrecedence >= 0 && fromPrecedence < toPrecedence + } + def forceNullable(from: DataType, to: DataType): Boolean = (from, to) match { case (NullType, _) => true case (_, _) if from == to => false http://git-wip-us.apache.org/repos/asf/spark/blob/bf764a33/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 8620f3f..933384e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, CatalogUtils, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{IntegralType, StringType} +import org.apache.spark.sql.types.{AtomicType, IntegralType, StringType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -660,7 +660,8 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { def unapply(expr: Expression): Option[Attribute] = { expr match { case attr: Attribute => Some(attr) - case Cast(child, dt, _) if !Cast.mayTruncate(child.dataType, dt) => unapply(child) + case Cast(child @ AtomicType(), dt: AtomicType, _) + if Cast.canSafeCast(child.dataType.asInstanceOf[AtomicType], dt) => unapply(child) case _ => None } } http://git-wip-us.apache.org/repos/asf/spark/blob/bf764a33/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala ---
spark git commit: [SPARK-24696][SQL] ColumnPruning rule fails to remove extra Project
Repository: spark Updated Branches: refs/heads/branch-2.3 8ff4b9727 -> 3c0af793f [SPARK-24696][SQL] ColumnPruning rule fails to remove extra Project The ColumnPruning rule tries adding an extra Project if an input node produces fields more than needed, but as a post-processing step, it needs to remove the lower Project in the form of "Project - Filter - Project" otherwise it would conflict with PushPredicatesThroughProject and would thus cause a infinite optimization loop. The current post-processing method is defined as: ``` private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transform { case p1 Project(_, f Filter(_, p2 Project(_, child))) if p2.outputSet.subsetOf(child.outputSet) => p1.copy(child = f.copy(child = child)) } ``` This method works well when there is only one Filter but would not if there's two or more Filters. In this case, there is a deterministic filter and a non-deterministic filter so they stay as separate filter nodes and cannot be combined together. An simplified illustration of the optimization process that forms the infinite loop is shown below (F1 stands for the 1st filter, F2 for the 2nd filter, P for project, S for scan of relation, PredicatePushDown as abbrev. of PushPredicatesThroughProject): ``` F1 - F2 - P - S PredicatePushDown =>F1 - P - F2 - S ColumnPruning =>F1 - P - F2 - P - S =>F1 - P - F2 - S(Project removed) PredicatePushDown =>P - F1 - F2 - S ColumnPruning =>P - F1 - P - F2 - S =>P - F1 - P - F2 - P - S =>P - F1 - F2 - P - S(only one Project removed) RemoveRedundantProject =>F1 - F2 - P - S(goes back to the loop start) ``` So the problem is the ColumnPruning rule adds a Project under a Filter (and fails to remove it in the end), and that new Project triggers PushPredicateThroughProject. Once the filters have been push through the Project, a new Project will be added by the ColumnPruning rule and this goes on and on. The fix should be when adding Projects, the rule applies top-down, but later when removing extra Projects, the process should go bottom-up to ensure all extra Projects can be matched. Added a optimization rule test in ColumnPruningSuite; and a end-to-end test in SQLQuerySuite. Author: maryannxue Closes #21674 from maryannxue/spark-24696. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3c0af793 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3c0af793 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3c0af793 Branch: refs/heads/branch-2.3 Commit: 3c0af793f9e050f5d8dfb2f5daab6c0043c39748 Parents: 8ff4b97 Author: maryannxue Authored: Fri Jun 29 23:46:12 2018 -0700 Committer: Xiao Li Committed: Fri Jun 29 23:57:09 2018 -0700 -- .../sql/catalyst/optimizer/Optimizer.scala | 5 +++-- .../catalyst/optimizer/ColumnPruningSuite.scala | 9 - .../org/apache/spark/sql/SQLQuerySuite.scala| 21 3 files changed, 32 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3c0af793/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c77e0f8..38799c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -524,9 +524,10 @@ object ColumnPruning extends Rule[LogicalPlan] { /** * The Project before Filter is not necessary but conflict with PushPredicatesThroughProject, - * so remove it. + * so remove it. Since the Projects have been added top-down, we need to remove in bottom-up + * order, otherwise lower Projects can be missed. */ - private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transform { + private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transformUp { case p1 @ Project(_, f @ Filter(_, p2 @ Project(_, child))) if p2.outputSet.subsetOf(child.outputSet) => p1.copy(child = f.copy(child = child)) http://git-wip-us.apache.org/repos/asf/spark/blob/3c0af793/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite
spark git commit: simplify rand in dsl/package.scala
Repository: spark Updated Branches: refs/heads/branch-2.3 0f534d3da -> 8ff4b9727 simplify rand in dsl/package.scala (cherry picked from commit d54d8b86301581142293341af25fd78b3278a2e8) Signed-off-by: Xiao Li Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8ff4b972 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8ff4b972 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8ff4b972 Branch: refs/heads/branch-2.3 Commit: 8ff4b97274e58f5944506b25481c6eb44238a4cd Parents: 0f534d3 Author: Xiao Li Authored: Fri Jun 29 23:51:13 2018 -0700 Committer: Xiao Li Committed: Fri Jun 29 23:53:00 2018 -0700 -- .../src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8ff4b972/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index efb2eba..89e8c99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -149,6 +149,7 @@ package object dsl { } } +def rand(e: Long): Expression = Rand(e) def sum(e: Expression): Expression = Sum(e).toAggregateExpression() def sumDistinct(e: Expression): Expression = Sum(e).toAggregateExpression(isDistinct = true) def count(e: Expression): Expression = Count(e).toAggregateExpression() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org