Skip to content

Commit f76679b

Browse files
committed
merge master
2 parents 689495d + b22952f commit f76679b

File tree

5 files changed

+22
-103
lines changed

5 files changed

+22
-103
lines changed

core/src/main/scala/org/apache/spark/InterruptibleIterator.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,11 +17,15 @@
1717

1818
package org.apache.spark
1919

20+
import org.apache.spark.annotation.DeveloperApi
21+
2022
/**
23+
* :: DeveloperApi ::
2124
* An iterator that wraps around an existing iterator to provide task killing functionality.
2225
* It works by checking the interrupted flag in [[TaskContext]].
2326
*/
24-
private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T])
27+
@DeveloperApi
28+
class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T])
2529
extends Iterator[T] {
2630

2731
def hasNext: Boolean = {

core/src/main/scala/org/apache/spark/Partitioner.scala

Lines changed: 0 additions & 61 deletions
Original file line numberDiff line numberDiff line change
@@ -157,64 +157,3 @@ class RangePartitioner[K : Ordering : ClassTag, V](
157157
}
158158
}
159159

160-
/**
161-
* A [[org.apache.spark.Partitioner]] that partitions records into specified bounds
162-
* Default value is 1000. Once all partitions have bounds elements, the partitioner
163-
* allocates 1 element per partition so eventually the smaller partitions are at most
164-
* off by 1 key compared to the larger partitions.
165-
*/
166-
class BoundaryPartitioner[K : Ordering : ClassTag, V](
167-
partitions: Int,
168-
@transient rdd: RDD[_ <: Product2[K,V]],
169-
private val boundary: Int = 1000)
170-
extends Partitioner {
171-
172-
// this array keeps track of keys assigned to a partition
173-
// counts[0] refers to # of keys in partition 0 and so on
174-
private val counts: Array[Int] = {
175-
new Array[Int](numPartitions)
176-
}
177-
178-
def numPartitions = math.abs(partitions)
179-
180-
/*
181-
* Ideally, this should've been calculated based on # partitions and total keys
182-
* But we are not calling count on RDD here to avoid calling an action.
183-
* User has the flexibility of calling count and passing in any appropriate boundary
184-
*/
185-
def keysPerPartition = boundary
186-
187-
var currPartition = 0
188-
189-
/*
190-
* Pick current partition for the key until we hit the bound for keys / partition,
191-
* start allocating to next partition at that time.
192-
*
193-
* NOTE: In case where we have lets say 2000 keys and user says 3 partitions with 500
194-
* passed in as boundary, the first 500 will goto P1, 501-1000 go to P2, 1001-1500 go to P3,
195-
* after that, next keys go to one partition at a time. So 1501 goes to P1, 1502 goes to P2,
196-
* 1503 goes to P3 and so on.
197-
*/
198-
def getPartition(key: Any): Int = {
199-
val partition = currPartition
200-
counts(partition) = counts(partition) + 1
201-
/*
202-
* Since we are filling up a partition before moving to next one (this helps in maintaining
203-
* order of keys, in certain cases, it is possible to end up with empty partitions, like
204-
* 3 partitions, 500 keys / partition and if rdd has 700 keys, 1 partition will be entirely
205-
* empty.
206-
*/
207-
if(counts(currPartition) >= keysPerPartition) {
208-
currPartition = (currPartition + 1) % numPartitions
209-
}
210-
partition
211-
}
212-
213-
override def equals(other: Any): Boolean = other match {
214-
case r: BoundaryPartitioner[_,_] =>
215-
(r.counts.sameElements(counts) && r.boundary == boundary
216-
&& r.currPartition == currPartition)
217-
case _ =>
218-
false
219-
}
220-
}

core/src/main/scala/org/apache/spark/TaskKilledException.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,11 @@
1717

1818
package org.apache.spark
1919

20+
import org.apache.spark.annotation.DeveloperApi
21+
2022
/**
21-
* Exception for a task getting killed.
23+
* :: DeveloperApi ::
24+
* Exception thrown when a task is explicitly killed (i.e., task failure is expected).
2225
*/
23-
private[spark] class TaskKilledException extends RuntimeException
26+
@DeveloperApi
27+
class TaskKilledException extends RuntimeException

core/src/test/scala/org/apache/spark/PartitioningSuite.scala

Lines changed: 0 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -66,40 +66,6 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet
6666
assert(descendingP4 != p4)
6767
}
6868

69-
test("BoundaryPartitioner equality") {
70-
// Make an RDD where all the elements are the same so that the partition range bounds
71-
// are deterministically all the same.
72-
val rdd = sc.parallelize(1.to(4000)).map(x => (x, x))
73-
74-
val p2 = new BoundaryPartitioner(2, rdd, 1000)
75-
val p4 = new BoundaryPartitioner(4, rdd, 1000)
76-
val anotherP4 = new BoundaryPartitioner(4, rdd)
77-
78-
assert(p2 === p2)
79-
assert(p4 === p4)
80-
assert(p2 != p4)
81-
assert(p4 != p2)
82-
assert(p4 === anotherP4)
83-
assert(anotherP4 === p4)
84-
}
85-
86-
test("BoundaryPartitioner getPartition") {
87-
val rdd = sc.parallelize(1.to(2000)).map(x => (x, x))
88-
val partitioner = new BoundaryPartitioner(4, rdd, 500)
89-
1.to(2000).map { element => {
90-
val partition = partitioner.getPartition(element)
91-
if (element <= 500) {
92-
assert(partition === 0)
93-
} else if (element > 501 && element <= 1000) {
94-
assert(partition === 1)
95-
} else if (element > 1001 && element <= 1500) {
96-
assert(partition === 2)
97-
} else if (element > 1501 && element <= 2000) {
98-
assert(partition === 3)
99-
}
100-
}}
101-
}
102-
10369
test("RangePartitioner getPartition") {
10470
val rdd = sc.parallelize(1.to(2000)).map(x => (x, x))
10571
// We have different behaviour of getPartition for partitions with less than 1000 and more than

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala

Lines changed: 11 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -25,13 +25,13 @@ import org.apache.spark.sql.catalyst.types._
2525

2626
object Optimizer extends RuleExecutor[LogicalPlan] {
2727
val batches =
28-
Batch("ConstantFolding", Once,
28+
Batch("ConstantFolding", FixedPoint(100),
2929
NullPropagation,
3030
ConstantFolding,
3131
BooleanSimplification,
3232
SimplifyFilters,
3333
SimplifyCasts) ::
34-
Batch("Filter Pushdown", Once,
34+
Batch("Filter Pushdown", FixedPoint(100),
3535
CombineFilters,
3636
PushPredicateThroughProject,
3737
PushPredicateThroughInnerJoin,
@@ -49,24 +49,27 @@ object Optimizer extends RuleExecutor[LogicalPlan] {
4949
*/
5050
object ColumnPruning extends Rule[LogicalPlan] {
5151
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
52+
// Eliminate attributes that are not needed to calculate the specified aggregates.
5253
case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty =>
53-
// Project away references that are not needed to calculate the required aggregates.
5454
a.copy(child = Project(a.references.toSeq, child))
5555

56+
// Eliminate unneeded attributes from either side of a Join.
5657
case Project(projectList, Join(left, right, joinType, condition)) =>
5758
// Collect the list of off references required either above or to evaluate the condition.
5859
val allReferences: Set[Attribute] =
5960
projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty)
60-
/** Applies a projection when the child is producing unnecessary attributes */
61+
62+
/** Applies a projection only when the child is producing unnecessary attributes */
6163
def prunedChild(c: LogicalPlan) =
62-
if ((allReferences.filter(c.outputSet.contains) -- c.outputSet).nonEmpty) {
64+
if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) {
6365
Project(allReferences.filter(c.outputSet.contains).toSeq, c)
6466
} else {
6567
c
6668
}
6769

6870
Project(projectList, Join(prunedChild(left), prunedChild(right), joinType, condition))
6971

72+
// Combine adjacent Projects.
7073
case Project(projectList1, Project(projectList2, child)) =>
7174
// Create a map of Aliases to their values from the child projection.
7275
// e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)).
@@ -83,6 +86,9 @@ object ColumnPruning extends Rule[LogicalPlan] {
8386
}).asInstanceOf[Seq[NamedExpression]]
8487

8588
Project(substitutedProjection, child)
89+
90+
// Eliminate no-op Projects
91+
case Project(projectList, child) if(child.output == projectList) => child
8692
}
8793
}
8894

0 commit comments

Comments
 (0)