Skip to content

Commit

Permalink
Use BigInt for stat; for logical leaves, by default throw an exception.
Browse files Browse the repository at this point in the history
Also cleanups & scaladoc fixes per review comments.
  • Loading branch information
concretevitamin committed Jul 29, 2014
1 parent 2f2fb89 commit 8663e84
Show file tree
Hide file tree
Showing 6 changed files with 37 additions and 37 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.trees
abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
self: Product =>

// TODO: handle overflow?
/**
* Estimates of various statistics. The default estimation logic simply lazily multiplies the
* corresponding statistic produced by the children. To override this behavior, override
Expand All @@ -40,7 +39,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
* defaults to the product of children's `sizeInBytes`.
*/
case class Statistics(
sizeInBytes: Long
sizeInBytes: BigInt
)
lazy val statistics: Statistics = Statistics(
sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product
Expand Down Expand Up @@ -112,7 +111,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] {
self: Product =>

override lazy val statistics = Statistics(sizeInBytes = 1L)
override lazy val statistics: Statistics =
throw new UnsupportedOperationException("default leaf nodes don't have meaningful Statistics")

// Leaf nodes by definition cannot reference any input attributes.
override def references = Set.empty
Expand Down
10 changes: 4 additions & 6 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,13 @@ import java.util.Properties
import scala.collection.JavaConverters._

/**
* A trait that enables the setting and getting of mutable config parameters/hints. The central
* location for storing them is uniquely located in the same-name private companion object.
* Therefore, all classes that mix in this trait share all the hints.
* A trait that enables the setting and getting of mutable config parameters/hints.
*
* In the presence of a SQLContext, these can be set and queried either by passing SET commands
* into Spark SQL's DSL functions (sql(), hql(), etc.). Otherwise, users of this trait can
* In the presence of a SQLContext, these can be set and queried by passing SET commands
* into Spark SQL's query functions (sql(), hql(), etc.). Otherwise, users of this trait can
* modify the hints by programmatically calling the setters and getters of this trait.
*
* SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads).
* SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads).
*/
trait SQLConf {
import SQLConf._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,14 +82,9 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ
}

@transient override lazy val statistics = Statistics(
sizeInBytes = {
alreadyPlanned match {
// TODO: Instead of returning a default value here, find a way to return a meaningful
// size estimate for RDDs. See PR 1238 for more discussions.
case e: ExistingRdd => sqlContext.statsDefaultSizeInBytes
case _ => 1L // TODO: consider adding statistics to physical plans as well.
}
}
// TODO: Instead of returning a default value here, find a way to return a meaningful size
// estimate for RDDs. See PR 1238 for more discussions.
sizeInBytes = BigInt(sqlContext.statsDefaultSizeInBytes)
)

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.sql.execution

import scala.util.Try

import org.apache.spark.sql.{SQLContext, execution}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning._
Expand Down Expand Up @@ -72,19 +74,22 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {

def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right)
if sqlContext.autoConvertJoinSize > 0 &&
right.statistics.sizeInBytes <= sqlContext.autoConvertJoinSize =>
makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight)
if Try(sqlContext.autoConvertJoinSize > 0 &&
right.statistics.sizeInBytes <= sqlContext.autoConvertJoinSize).getOrElse(false) =>
makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight)

case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right)
if sqlContext.autoConvertJoinSize > 0 &&
left.statistics.sizeInBytes <= sqlContext.autoConvertJoinSize =>
if Try(sqlContext.autoConvertJoinSize > 0 &&
left.statistics.sizeInBytes <= sqlContext.autoConvertJoinSize).getOrElse(false) =>
makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft)

case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) =>
val buildSide =
if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) BuildRight
else BuildLeft
if (Try(right.statistics.sizeInBytes <= left.statistics.sizeInBytes).getOrElse(false)) {
BuildRight
} else {
BuildLeft
}
val hashJoin =
execution.ShuffledHashJoin(
leftKeys, rightKeys, buildSide, planLater(left), planLater(right))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.hadoop.hive.serde2.Deserializer

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.Logging
import org.apache.spark.sql.{SQLContext, Logging}
import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical
Expand Down Expand Up @@ -66,8 +66,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
// Since HiveQL is case insensitive for table names we make them all lowercase.
MetastoreRelation(
databaseName, tblName, alias)(
table.getTTable, partitions.map(part => part.getTPartition))(
hive.hiveconf, table.getPath)
table.getTTable, partitions.map(part => part.getTPartition))(hive)
}

def createTable(
Expand Down Expand Up @@ -252,7 +251,7 @@ object HiveMetastoreTypes extends RegexParsers {
private[hive] case class MetastoreRelation
(databaseName: String, tableName: String, alias: Option[String])
(val table: TTable, val partitions: Seq[TPartition])
(@transient hiveConf: HiveConf, @transient path: Path)
(@transient sqlContext: SQLContext)
extends LeafNode {

self: Product =>
Expand All @@ -270,15 +269,17 @@ private[hive] case class MetastoreRelation
}

@transient override lazy val statistics = Statistics(
// TODO: check if this estimate is valid for tables after partition pruning.
sizeInBytes = {
// NOTE: kind of hacky, but this should be relatively cheap if parameters for the table are
// populated into the metastore. An alternative would be going through Hadoop's FileSystem
// API, which can be expensive if a lot of RPCs are involved. Besides `totalSize`, there are
// also `numFiles`, `numRows`, `rawDataSize` keys we can look at in the future.
val sizeMaybeFromMetastore =
Option(hiveQlTable.getParameters.get("totalSize")).map(_.toLong).getOrElse(-1L)
math.max(sizeMaybeFromMetastore, 1L)
// TODO: check if this estimate is valid for tables after partition pruning.
// NOTE: getting `totalSize` directly from params is kind of hacky, but this should be
// relatively cheap if parameters for the table are populated into the metastore. An
// alternative would be going through Hadoop's FileSystem API, which can be expensive if a lot
// of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`,
// `rawDataSize` keys that we can look at in the future.
BigInt(
Option(hiveQlTable.getParameters.get("totalSize"))
.map(_.toLong)
.getOrElse(sqlContext.statsDefaultSizeInBytes))
}
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,8 @@ class StatisticsSuite extends QueryTest {
mr.statistics.sizeInBytes
}
assert(sizes.size === 1)
assert(sizes(0) == 5812, s"expected exact size 5812 for test table 'src', got ${sizes(0)}")
assert(sizes(0).equals(BigInt(5812)),
s"expected exact size 5812 for test table 'src', got: ${sizes(0)}")
}

test("auto converts to broadcast hash join, by size estimate of a relation") {
Expand Down

0 comments on commit 8663e84

Please sign in to comment.