Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
add ut
  • Loading branch information
lianhuiwang committed May 11, 2016
commit 5de46b05092e77fc68dda8ef4dc1d549b4ef9373
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,13 @@ import java.nio.charset.StandardCharsets
import java.sql.Timestamp

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.execution.adaptive.QueryFragmentTransformer
import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.adaptive.QueryFragmentTransformer
import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec}
import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange}
import org.apache.spark.sql.internal.SQLConf
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,18 @@

package org.apache.spark.sql.execution.adaptive

import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.{LinkedBlockingDeque, BlockingQueue}
import java.util.{HashMap => JHashMap, Map => JMap}
import java.util.concurrent.{BlockingQueue, LinkedBlockingDeque}
import java.util.concurrent.atomic.AtomicBoolean

import org.apache.spark.sql.catalyst.rules.Rule

import scala.concurrent.ExecutionContext.Implicits.global
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.ExecutionContext.Implicits.global

import org.apache.spark.{MapOutputStatistics, SimpleFutureAction, ShuffleDependency}
import org.apache.spark.{MapOutputStatistics, ShuffleDependency, SimpleFutureAction}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.execution.{CollapseCodegenStages, SortExec, SparkPlan}
import org.apache.spark.sql.execution.aggregate.TungstenAggregate
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchange}
Expand Down Expand Up @@ -150,7 +149,7 @@ trait QueryFragment extends SparkPlan {
}

case agg @ TungstenAggregate(_, _, _, _, _, _, input @ FragmentInput(_))
if (!input.isOptimized())=> {
if (!input.isOptimized()) => {
logInfo("Begin optimize agg, operator =\n" + agg.toString)
optimizeAggregate(agg, input)
}
Expand Down Expand Up @@ -188,7 +187,7 @@ trait QueryFragment extends SparkPlan {
Utils.estimatePartitionStartIndices(aggStatistics.toArray, minNumPostShufflePartitions,
advisoryTargetPostShuffleInputSize)
}
val shuffledRowRdd= childFragments(0).getExchange().preparePostShuffleRDD(
val shuffledRowRdd = childFragments(0).getExchange().preparePostShuffleRDD(
shuffleDependencies(fragmentsIndex.get(childFragments(0))), partitionStartIndices)
childFragments(0).getFragmentInput().setShuffleRdd(shuffledRowRdd)
childFragments(0).getFragmentInput().setOptimized()
Expand Down Expand Up @@ -223,7 +222,7 @@ trait QueryFragment extends SparkPlan {

val leftFragment = childFragments(0)
val rightFragment = childFragments(1)
val leftShuffledRowRdd= leftFragment.getExchange().preparePostShuffleRDD(
val leftShuffledRowRdd = leftFragment.getExchange().preparePostShuffleRDD(
shuffleDependencies(fragmentsIndex.get(leftFragment)), partitionStartIndices)
val rightShuffledRowRdd = rightFragment.getExchange().preparePostShuffleRDD(
shuffleDependencies(fragmentsIndex.get(rightFragment)), partitionStartIndices)
Expand All @@ -238,7 +237,17 @@ trait QueryFragment extends SparkPlan {
if (sqlContext.conf.autoBroadcastJoinThreshold > 0) {
val leftSizeInBytes = childSizeInBytes(0)
val rightSizeInBytes = childSizeInBytes(1)
if (leftSizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold) {
val joinType = joinPlan.joinType
def canBuildLeft(joinType: JoinType): Boolean = joinType match {
case Inner | RightOuter => true
case _ => false
}
def canBuildRight(joinType: JoinType): Boolean = joinType match {
case Inner | LeftOuter | LeftSemi | LeftAnti => true
case j: ExistenceJoin => true
case _ => false
}
if (leftSizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold && canBuildLeft(joinType)) {
val keys = Utils.rewriteKeyExpr(joinPlan.leftKeys).map(
BindReferences.bindReference(_, left.child.output))
newOperator = BroadcastHashJoinExec(
Expand All @@ -249,7 +258,8 @@ trait QueryFragment extends SparkPlan {
joinPlan.condition,
BroadcastExchangeExec(HashedRelationBroadcastMode(keys), left.child),
right.child)
} else if (rightSizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold) {
} else if (rightSizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold
&& canBuildRight(joinType)) {
val keys = Utils.rewriteKeyExpr(joinPlan.rightKeys).map(
BindReferences.bindReference(_, right.child.output))
newOperator = BroadcastHashJoinExec(
Expand All @@ -260,13 +270,13 @@ trait QueryFragment extends SparkPlan {
joinPlan.condition,
left.child,
BroadcastExchangeExec(HashedRelationBroadcastMode(keys), right.child))
}
}
}
newOperator
}

/** Returns a string representation of the nodes in this tree */
override def treeString: String =
override def treeString: String =
executedPlan.generateTreeString(0, Nil, new StringBuilder).toString

override def simpleString: String = "QueryFragment"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@

package org.apache.spark.sql.execution.adaptive

import scala.collection.mutable.{ArrayBuffer, Queue}

import org.apache.spark.MapOutputStatistics
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.types.{IntegerType, LongType, IntegralType}
import org.apache.spark.MapOutputStatistics
import org.apache.spark.sql.execution.SparkPlan

import scala.collection.mutable.{Queue, ArrayBuffer}
import org.apache.spark.sql.types.{IntegralType, LongType}

/**
* Utility functions used by the query fragment.
Expand All @@ -49,7 +49,9 @@ private[sql] object Utils extends Logging {
private[sql] def findLeafFragment(root: QueryFragment): Seq[QueryFragment] = {
val result = new ArrayBuffer[QueryFragment]
val queue = new Queue[QueryFragment]
queue.enqueue(root)
if (!root.children.isEmpty) {
root.children.foreach(c => queue.enqueue(c))
}
while (queue.nonEmpty) {
val current = queue.dequeue()
if (current.children.isEmpty) {
Expand Down