From 1d30cc53850e1455cef8586bbc83ab4257d47f89 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 13 Oct 2018 11:55:56 -0700 Subject: [PATCH 1/2] fix --- .../apache/spark/sql/execution/columnar/InMemoryRelation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1a8fbaca53f59..b752b771f43a2 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}" From cff78e992b91ae81f3b08f421919d81c45c244cc Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 13 Oct 2018 16:17:59 -0700 Subject: [PATCH 2/2] add a test --- .../sql/execution/columnar/InMemoryColumnarQuerySuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) 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 efc2f20a907f1..b1b23e4439878 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(