Repository: spark
Updated Branches:
refs/heads/master 6bbceb9fe -> 6c3f2c6a6
[SPARK-25727][SQL] Add outputOrdering to otherCopyArgs in InMemoryRelation
## What changes were proposed in this pull request?
Add `outputOrdering ` to `otherCopyArgs` in InMemoryRelation so that this field
will be copied when we doing the tree transformation.
```
val data = Seq(100).toDF("count").cache()
data.queryExecution.optimizedPlan.toJSON
```
The above code can generate the following error:
```
assertion failed: InMemoryRelation fields: output, cacheBuilder,
statsOfPlanToCache, outputOrdering, values: List(count#178),
CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1
replicas),*(1) Project [value#176 AS count#178]
+- LocalTableScan [value#176]
,None), Statistics(sizeInBytes=12.0 B, hints=none)
java.lang.AssertionError: assertion failed: InMemoryRelation fields: output,
cacheBuilder, statsOfPlanToCache, outputOrdering, values: List(count#178),
CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1
replicas),*(1) Project [value#176 AS count#178]
+- LocalTableScan [value#176]
,None), Statistics(sizeInBytes=12.0 B, hints=none)
at scala.Predef$.assert(Predef.scala:170)
at
org.apache.spark.sql.catalyst.trees.TreeNode.jsonFields(TreeNode.scala:611)
at
org.apache.spark.sql.catalyst.trees.TreeNode.org$apache$spark$sql$catalyst$trees$TreeNode$$collectJsonValue$1(TreeNode.scala:599)
at
org.apache.spark.sql.catalyst.trees.TreeNode.jsonValue(TreeNode.scala:604)
at
org.apache.spark.sql.catalyst.trees.TreeNode.toJSON(TreeNode.scala:590)
```
## How was this patch tested?
Added a test
Closes #22715 from gatorsmile/copyArgs1.
Authored-by: gatorsmile <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6c3f2c6a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6c3f2c6a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6c3f2c6a
Branch: refs/heads/master
Commit: 6c3f2c6a6aa69f80de5504961cfd61b9a61ea7ce
Parents: 6bbceb9
Author: gatorsmile <[email protected]>
Authored: Sat Oct 13 22:10:17 2018 -0700
Committer: Dongjoon Hyun <[email protected]>
Committed: Sat Oct 13 22:10:17 2018 -0700
----------------------------------------------------------------------
.../apache/spark/sql/execution/columnar/InMemoryRelation.scala | 2 +-
.../sql/execution/columnar/InMemoryColumnarQuerySuite.scala | 6 ++++++
2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/6c3f2c6a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
----------------------------------------------------------------------
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
index 1a8fbac..b752b77 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
@@ -206,7 +206,7 @@ case class InMemoryRelation(
outputOrdering).asInstanceOf[this.type]
}
- override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache)
+ override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache,
outputOrdering)
override def simpleString: String =
s"InMemoryRelation [${Utils.truncatedString(output, ", ")}],
${cacheBuilder.storageLevel}"
http://git-wip-us.apache.org/repos/asf/spark/blob/6c3f2c6a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
----------------------------------------------------------------------
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
index efc2f20..b1b23e4 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
@@ -488,6 +488,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with
SharedSQLContext {
}
}
+ test("SPARK-25727 - otherCopyArgs in InMemoryRelation does not include
outputOrdering") {
+ val data = Seq(100).toDF("count").cache()
+ val json = data.queryExecution.optimizedPlan.toJSON
+ assert(json.contains("outputOrdering") &&
json.contains("statsOfPlanToCache"))
+ }
+
test("SPARK-22673: InMemoryRelation should utilize existing stats of the
plan to be cached") {
// This test case depends on the size of parquet in statistics.
withSQLConf(
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]