Skip to content

Commit

Permalink
Use subqueries eliminated plan to compare.
Browse files Browse the repository at this point in the history
  • Loading branch information
ueshin committed Jun 14, 2016
1 parent 9a1621a commit e51774f
Show file tree
Hide file tree
Showing 4 changed files with 21 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.plans.logical

import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, analysis}
import org.apache.spark.sql.types.{StructField, StructType}
Expand Down Expand Up @@ -56,10 +57,12 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil)

override protected def stringArgs = Iterator(output)

override def sameResult(plan: LogicalPlan): Boolean = plan match {
case LocalRelation(otherOutput, otherData) =>
otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data
case _ => false
override def sameResult(plan: LogicalPlan): Boolean = {
EliminateSubQueries(plan) match {
case LocalRelation(otherOutput, otherData) =>
otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data
case _ => false
}
}

override lazy val statistics =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, MultiInstanceRelation}
import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
import org.apache.spark.sql.sources.{HadoopFsRelation, BaseRelation}
Expand Down Expand Up @@ -79,9 +79,11 @@ private[sql] case class LogicalRDD(
override def newInstance(): LogicalRDD.this.type =
LogicalRDD(output.map(_.newInstance()), rdd)(sqlContext).asInstanceOf[this.type]

override def sameResult(plan: LogicalPlan): Boolean = plan match {
case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id
case _ => false
override def sameResult(plan: LogicalPlan): Boolean = {
EliminateSubQueries(plan) match {
case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id
case _ => false
}
}

@transient override lazy val statistics: Statistics = Statistics(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.apache.spark.sql.execution.datasources

import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, MultiInstanceRelation}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.sources.BaseRelation
Expand Down Expand Up @@ -57,9 +57,11 @@ case class LogicalRelation(
com.google.common.base.Objects.hashCode(relation, output)
}

override def sameResult(otherPlan: LogicalPlan): Boolean = otherPlan match {
case LogicalRelation(otherRelation, _) => relation == otherRelation
case _ => false
override def sameResult(otherPlan: LogicalPlan): Boolean = {
EliminateSubQueries(otherPlan) match {
case LogicalRelation(otherRelation, _) => relation == otherRelation
case _ => false
}
}

// When comparing two LogicalRelations from within LogicalPlan.sameResult, we only need
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.hadoop.hive.ql.metadata._
import org.apache.hadoop.hive.ql.plan.TableDesc

import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog}
import org.apache.spark.sql.catalyst.analysis.{Catalog, EliminateSubQueries, MultiInstanceRelation, OverrideCatalog}
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical
Expand Down Expand Up @@ -832,7 +832,7 @@ private[hive] case class MetastoreRelation

/** Only compare database and tablename, not alias. */
override def sameResult(plan: LogicalPlan): Boolean = {
plan match {
EliminateSubQueries(plan) match {
case mr: MetastoreRelation =>
mr.databaseName == databaseName && mr.tableName == tableName
case _ => false
Expand Down

0 comments on commit e51774f

Please sign in to comment.