Skip to content

Commit

Permalink
Move SQLConf back. Assign default sizeInBytes to SparkLogicalPlan.
Browse files Browse the repository at this point in the history
  • Loading branch information
concretevitamin committed Jul 29, 2014
1 parent 73412be commit 73cde01
Show file tree
Hide file tree
Showing 4 changed files with 25 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,29 +19,24 @@ package org.apache.spark.sql.catalyst.plans.logical

import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.SQLConf
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.types.StructType
import org.apache.spark.sql.catalyst.trees

abstract class LogicalPlan extends QueryPlan[LogicalPlan] with SQLConf {
abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
self: Product =>

// TODO: make a case class?
/**
* Estimates of various statistics. The default estimation logic simply sums up the corresponding
* statistic produced by the children. To override this behavior, override `statistics` and
* assign it a overriden version of `Statistics`.
*/
protected class Statistics {
lazy val childrenStats = children.map(_.statistics)

lazy val numTuples: Long = childrenStats.map(_.numTuples).sum

lazy val sizeInBytes: Long = {
val sum = childrenStats.map(_.sizeInBytes).sum
if (sum == 0) statsDefaultSizeInBytes else sum
}
lazy val sizeInBytes: Long = childrenStats.map(_.sizeInBytes).sum
}

/**
* Estimates of various statistics.
*/
lazy val statistics: Statistics = new Statistics

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.planning
package org.apache.spark.sql

import java.util.Properties

Expand All @@ -40,8 +40,8 @@ private object SQLConf {
trait SQLConf {
import SQLConf._

import SQLConf._
protected[spark] val settings = confSettings
import org.apache.spark.sql.SQLConf._
@transient protected[spark] val settings = confSettings

/** ************************ Spark SQL Params/Hints ******************* */
// TODO: refactor so that these hints accessors don't pollute the name space of SQLContext?
Expand Down
2 changes: 0 additions & 2 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.sql


import scala.language.implicitConversions
import scala.reflect.runtime.universe.TypeTag

Expand All @@ -30,7 +29,6 @@ import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.dsl.ExpressionConversions
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.optimizer.Optimizer
import org.apache.spark.sql.catalyst.planning.SQLConf
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.catalyst.types._
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.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Logging, Row}
import org.apache.spark.sql.{Logging, Row, SQLConf}
import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions.GenericRow
Expand Down Expand Up @@ -67,7 +67,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
*/
@DeveloperApi
case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
extends LogicalPlan with MultiInstanceRelation {
extends LogicalPlan with MultiInstanceRelation with SQLConf {

def output = alreadyPlanned.output
override def references = Set.empty
Expand All @@ -80,6 +80,19 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
case _ => sys.error("Multiple instance of the same relation detected.")
}).asInstanceOf[this.type]
}

override lazy val statistics = new Statistics {
// If this is wrapping around ExistingRdd and no reasonable estimation logic is implemented,
// return a default value.
override lazy val sizeInBytes: Long = {
val defaultSum = childrenStats.map(_.sizeInBytes).sum
alreadyPlanned match {
case e: ExistingRdd if defaultSum == 0 => statsDefaultSizeInBytes
case _ => defaultSum
}
}
}

}

private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] {
Expand Down

0 comments on commit 73cde01

Please sign in to comment.