Repository: spark
Updated Branches:
  refs/heads/master 9426fd0c2 -> 56247c1d1


[SPARK-25727][FOLLOWUP] Move outputOrdering to case class field for 
InMemoryRelation

## What changes were proposed in this pull request?

The PR addresses [the 
comment](https://github.com/apache/spark/pull/22715#discussion_r225024084) in 
the previous one. `outputOrdering` becomes a field of `InMemoryRelation`.

## How was this patch tested?

existing UTs

Closes #22726 from mgaido91/SPARK-25727_followup.

Authored-by: Marco Gaido <marcogaid...@gmail.com>
Signed-off-by: gatorsmile <gatorsm...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/56247c1d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/56247c1d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/56247c1d

Branch: refs/heads/master
Commit: 56247c1d1790f7cf9b902d5924aa8a92961b6691
Parents: 9426fd0
Author: Marco Gaido <marcogaid...@gmail.com>
Authored: Mon Oct 15 10:12:45 2018 -0700
Committer: gatorsmile <gatorsm...@gmail.com>
Committed: Mon Oct 15 10:12:45 2018 -0700

----------------------------------------------------------------------
 .../execution/columnar/InMemoryRelation.scala   | 30 ++++++++++----------
 1 file changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/56247c1d/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 b752b77..3b65885 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
@@ -149,30 +149,30 @@ object InMemoryRelation {
       tableName: Option[String],
       logicalPlan: LogicalPlan): InMemoryRelation = {
     val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, 
storageLevel, child, tableName)()
-    new InMemoryRelation(child.output, cacheBuilder)(
-      statsOfPlanToCache = logicalPlan.stats, outputOrdering = 
logicalPlan.outputOrdering)
+    new InMemoryRelation(child.output, cacheBuilder, 
logicalPlan.outputOrdering)(
+      statsOfPlanToCache = logicalPlan.stats)
   }
 
   def apply(cacheBuilder: CachedRDDBuilder, logicalPlan: LogicalPlan): 
InMemoryRelation = {
-    new InMemoryRelation(cacheBuilder.cachedPlan.output, cacheBuilder)(
-      statsOfPlanToCache = logicalPlan.stats, outputOrdering = 
logicalPlan.outputOrdering)
+    new InMemoryRelation(cacheBuilder.cachedPlan.output, cacheBuilder, 
logicalPlan.outputOrdering)(
+      statsOfPlanToCache = logicalPlan.stats)
   }
 }
 
 case class InMemoryRelation(
     output: Seq[Attribute],
-    @transient cacheBuilder: CachedRDDBuilder)(
-    statsOfPlanToCache: Statistics,
-    override val outputOrdering: Seq[SortOrder])
+    @transient cacheBuilder: CachedRDDBuilder,
+    override val outputOrdering: Seq[SortOrder])(
+    statsOfPlanToCache: Statistics)
   extends logical.LeafNode with MultiInstanceRelation {
 
   override protected def innerChildren: Seq[SparkPlan] = Seq(cachedPlan)
 
   override def doCanonicalize(): logical.LogicalPlan =
     copy(output = output.map(QueryPlan.normalizeExprId(_, cachedPlan.output)),
-      cacheBuilder)(
-      statsOfPlanToCache,
-      outputOrdering)
+      cacheBuilder,
+      outputOrdering)(
+      statsOfPlanToCache)
 
   override def producedAttributes: AttributeSet = outputSet
 
@@ -195,18 +195,18 @@ case class InMemoryRelation(
   }
 
   def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = {
-    InMemoryRelation(newOutput, cacheBuilder)(statsOfPlanToCache, 
outputOrdering)
+    InMemoryRelation(newOutput, cacheBuilder, 
outputOrdering)(statsOfPlanToCache)
   }
 
   override def newInstance(): this.type = {
     new InMemoryRelation(
       output.map(_.newInstance()),
-      cacheBuilder)(
-        statsOfPlanToCache,
-        outputOrdering).asInstanceOf[this.type]
+      cacheBuilder,
+      outputOrdering)(
+        statsOfPlanToCache).asInstanceOf[this.type]
   }
 
-  override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache, 
outputOrdering)
+  override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache)
 
   override def simpleString: String =
     s"InMemoryRelation [${Utils.truncatedString(output, ", ")}], 
${cacheBuilder.storageLevel}"


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to