Skip to content

Commit a2bf756

Browse files
committed
Merge branch 'master' into stratified
2 parents 680b677 + e6f7bfc commit a2bf756

File tree

37 files changed

+512
-173
lines changed

37 files changed

+512
-173
lines changed

core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import scala.concurrent.Await
2626
import scala.concurrent.duration._
2727
import scala.language.postfixOps
2828
import scala.reflect.ClassTag
29+
import scala.util.control.NonFatal
2930

3031
import akka.actor._
3132
import akka.actor.OneForOneStrategy
@@ -768,6 +769,10 @@ class DAGScheduler(
768769
abortStage(stage, "Task not serializable: " + e.toString)
769770
runningStages -= stage
770771
return
772+
case NonFatal(e) => // Other exceptions, such as IllegalArgumentException from Kryo.
773+
abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}")
774+
runningStages -= stage
775+
return
771776
}
772777

773778
logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")")

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext {
6161
val acc : Accumulator[Int] = sc.accumulator(0)
6262

6363
val d = sc.parallelize(1 to 20)
64-
evaluating {d.foreach{x => acc.value = x}} should produce [Exception]
64+
an [Exception] should be thrownBy {d.foreach{x => acc.value = x}}
6565
}
6666

6767
test ("add value to collection accumulators") {
@@ -87,11 +87,11 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext {
8787
sc = new SparkContext("local[" + nThreads + "]", "test")
8888
val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]())
8989
val d = sc.parallelize(1 to maxI)
90-
evaluating {
90+
an [SparkException] should be thrownBy {
9191
d.foreach {
9292
x => acc.value += x
9393
}
94-
} should produce [SparkException]
94+
}
9595
resetSparkContext()
9696
}
9797
}

core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala

Lines changed: 19 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -27,45 +27,45 @@ import org.scalatest.Matchers
2727
class NextIteratorSuite extends FunSuite with Matchers {
2828
test("one iteration") {
2929
val i = new StubIterator(Buffer(1))
30-
i.hasNext should be === true
31-
i.next should be === 1
32-
i.hasNext should be === false
30+
i.hasNext should be (true)
31+
i.next should be (1)
32+
i.hasNext should be (false)
3333
intercept[NoSuchElementException] { i.next() }
3434
}
3535

3636
test("two iterations") {
3737
val i = new StubIterator(Buffer(1, 2))
38-
i.hasNext should be === true
39-
i.next should be === 1
40-
i.hasNext should be === true
41-
i.next should be === 2
42-
i.hasNext should be === false
38+
i.hasNext should be (true)
39+
i.next should be (1)
40+
i.hasNext should be (true)
41+
i.next should be (2)
42+
i.hasNext should be (false)
4343
intercept[NoSuchElementException] { i.next() }
4444
}
4545

4646
test("empty iteration") {
4747
val i = new StubIterator(Buffer())
48-
i.hasNext should be === false
48+
i.hasNext should be (false)
4949
intercept[NoSuchElementException] { i.next() }
5050
}
5151

5252
test("close is called once for empty iterations") {
5353
val i = new StubIterator(Buffer())
54-
i.hasNext should be === false
55-
i.hasNext should be === false
56-
i.closeCalled should be === 1
54+
i.hasNext should be (false)
55+
i.hasNext should be (false)
56+
i.closeCalled should be (1)
5757
}
5858

5959
test("close is called once for non-empty iterations") {
6060
val i = new StubIterator(Buffer(1, 2))
61-
i.next should be === 1
62-
i.next should be === 2
61+
i.next should be (1)
62+
i.next should be (2)
6363
// close isn't called until we check for the next element
64-
i.closeCalled should be === 0
65-
i.hasNext should be === false
66-
i.closeCalled should be === 1
67-
i.hasNext should be === false
68-
i.closeCalled should be === 1
64+
i.closeCalled should be (0)
65+
i.hasNext should be (false)
66+
i.closeCalled should be (1)
67+
i.hasNext should be (false)
68+
i.closeCalled should be (1)
6969
}
7070

7171
class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] {

core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala

Lines changed: 35 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -63,53 +63,53 @@ class SizeEstimatorSuite
6363
}
6464

6565
test("simple classes") {
66-
expectResult(16)(SizeEstimator.estimate(new DummyClass1))
67-
expectResult(16)(SizeEstimator.estimate(new DummyClass2))
68-
expectResult(24)(SizeEstimator.estimate(new DummyClass3))
69-
expectResult(24)(SizeEstimator.estimate(new DummyClass4(null)))
70-
expectResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
66+
assertResult(16)(SizeEstimator.estimate(new DummyClass1))
67+
assertResult(16)(SizeEstimator.estimate(new DummyClass2))
68+
assertResult(24)(SizeEstimator.estimate(new DummyClass3))
69+
assertResult(24)(SizeEstimator.estimate(new DummyClass4(null)))
70+
assertResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
7171
}
7272

7373
// NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors
7474
// (Sun vs IBM). Use a DummyString class to make tests deterministic.
7575
test("strings") {
76-
expectResult(40)(SizeEstimator.estimate(DummyString("")))
77-
expectResult(48)(SizeEstimator.estimate(DummyString("a")))
78-
expectResult(48)(SizeEstimator.estimate(DummyString("ab")))
79-
expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
76+
assertResult(40)(SizeEstimator.estimate(DummyString("")))
77+
assertResult(48)(SizeEstimator.estimate(DummyString("a")))
78+
assertResult(48)(SizeEstimator.estimate(DummyString("ab")))
79+
assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
8080
}
8181

8282
test("primitive arrays") {
83-
expectResult(32)(SizeEstimator.estimate(new Array[Byte](10)))
84-
expectResult(40)(SizeEstimator.estimate(new Array[Char](10)))
85-
expectResult(40)(SizeEstimator.estimate(new Array[Short](10)))
86-
expectResult(56)(SizeEstimator.estimate(new Array[Int](10)))
87-
expectResult(96)(SizeEstimator.estimate(new Array[Long](10)))
88-
expectResult(56)(SizeEstimator.estimate(new Array[Float](10)))
89-
expectResult(96)(SizeEstimator.estimate(new Array[Double](10)))
90-
expectResult(4016)(SizeEstimator.estimate(new Array[Int](1000)))
91-
expectResult(8016)(SizeEstimator.estimate(new Array[Long](1000)))
83+
assertResult(32)(SizeEstimator.estimate(new Array[Byte](10)))
84+
assertResult(40)(SizeEstimator.estimate(new Array[Char](10)))
85+
assertResult(40)(SizeEstimator.estimate(new Array[Short](10)))
86+
assertResult(56)(SizeEstimator.estimate(new Array[Int](10)))
87+
assertResult(96)(SizeEstimator.estimate(new Array[Long](10)))
88+
assertResult(56)(SizeEstimator.estimate(new Array[Float](10)))
89+
assertResult(96)(SizeEstimator.estimate(new Array[Double](10)))
90+
assertResult(4016)(SizeEstimator.estimate(new Array[Int](1000)))
91+
assertResult(8016)(SizeEstimator.estimate(new Array[Long](1000)))
9292
}
9393

9494
test("object arrays") {
9595
// Arrays containing nulls should just have one pointer per element
96-
expectResult(56)(SizeEstimator.estimate(new Array[String](10)))
97-
expectResult(56)(SizeEstimator.estimate(new Array[AnyRef](10)))
96+
assertResult(56)(SizeEstimator.estimate(new Array[String](10)))
97+
assertResult(56)(SizeEstimator.estimate(new Array[AnyRef](10)))
9898
// For object arrays with non-null elements, each object should take one pointer plus
9999
// however many bytes that class takes. (Note that Array.fill calls the code in its
100100
// second parameter separately for each object, so we get distinct objects.)
101-
expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)))
102-
expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)))
103-
expectResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)))
104-
expectResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)))
101+
assertResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)))
102+
assertResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)))
103+
assertResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)))
104+
assertResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)))
105105

106106
// Past size 100, our samples 100 elements, but we should still get the right size.
107-
expectResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)))
107+
assertResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)))
108108

109109
// If an array contains the *same* element many times, we should only count it once.
110110
val d1 = new DummyClass1
111-
expectResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object
112-
expectResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object
111+
assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object
112+
assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object
113113

114114
// Same thing with huge array containing the same element many times. Note that this won't
115115
// return exactly 4032 because it can't tell that *all* the elements will equal the first
@@ -127,10 +127,10 @@ class SizeEstimatorSuite
127127
val initialize = PrivateMethod[Unit]('initialize)
128128
SizeEstimator invokePrivate initialize()
129129

130-
expectResult(40)(SizeEstimator.estimate(DummyString("")))
131-
expectResult(48)(SizeEstimator.estimate(DummyString("a")))
132-
expectResult(48)(SizeEstimator.estimate(DummyString("ab")))
133-
expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
130+
assertResult(40)(SizeEstimator.estimate(DummyString("")))
131+
assertResult(48)(SizeEstimator.estimate(DummyString("a")))
132+
assertResult(48)(SizeEstimator.estimate(DummyString("ab")))
133+
assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
134134
resetOrClear("os.arch", arch)
135135
}
136136

@@ -142,10 +142,10 @@ class SizeEstimatorSuite
142142
val initialize = PrivateMethod[Unit]('initialize)
143143
SizeEstimator invokePrivate initialize()
144144

145-
expectResult(56)(SizeEstimator.estimate(DummyString("")))
146-
expectResult(64)(SizeEstimator.estimate(DummyString("a")))
147-
expectResult(64)(SizeEstimator.estimate(DummyString("ab")))
148-
expectResult(72)(SizeEstimator.estimate(DummyString("abcdefgh")))
145+
assertResult(56)(SizeEstimator.estimate(DummyString("")))
146+
assertResult(64)(SizeEstimator.estimate(DummyString("a")))
147+
assertResult(64)(SizeEstimator.estimate(DummyString("ab")))
148+
assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh")))
149149

150150
resetOrClear("os.arch", arch)
151151
resetOrClear("spark.test.useCompressedOops", oops)

docs/programming-guide.md

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -975,8 +975,8 @@ for details.
975975
<td> Return an array with the first <i>n</i> elements of the dataset. Note that this is currently not executed in parallel. Instead, the driver program computes all the elements. </td>
976976
</tr>
977977
<tr>
978-
<td> <b>takeSample</b>(<i>withReplacement</i>, <i>num</i>, <i>seed</i>) </td>
979-
<td> Return an array with a random sample of <i>num</i> elements of the dataset, with or without replacement, using the given random number generator seed. </td>
978+
<td> <b>takeSample</b>(<i>withReplacement</i>, <i>num</i>, [<i>seed</i>]) </td>
979+
<td> Return an array with a random sample of <i>num</i> elements of the dataset, with or without replacement, optionally pre-specifying a random number generator seed.</td>
980980
</tr>
981981
<tr>
982982
<td> <b>takeOrdered</b>(<i>n</i>, <i>[ordering]</i>) </td>

docs/running-on-yarn.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes
5555
</tr>
5656
<tr>
5757
<td><code>spark.yarn.max.executor.failures</code></td>
58-
<td>2*numExecutors</td>
58+
<td>numExecutors * 2, with minimum of 3</td>
5959
<td>
6060
The maximum number of executor failures before failing the application.
6161
</td>

ec2/spark_ec2.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -279,6 +279,7 @@ def launch_cluster(conn, opts, cluster_name):
279279
master_group.authorize(src_group=slave_group)
280280
master_group.authorize('tcp', 22, 22, '0.0.0.0/0')
281281
master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0')
282+
master_group.authorize('tcp', 18080, 18080, '0.0.0.0/0')
282283
master_group.authorize('tcp', 19999, 19999, '0.0.0.0/0')
283284
master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0')
284285
master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0')

examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.examples.mllib
1919

20+
import scala.collection.mutable
21+
2022
import com.esotericsoftware.kryo.Kryo
2123
import org.apache.log4j.{Level, Logger}
2224
import scopt.OptionParser
@@ -41,6 +43,7 @@ object MovieLensALS {
4143
class ALSRegistrator extends KryoRegistrator {
4244
override def registerClasses(kryo: Kryo) {
4345
kryo.register(classOf[Rating])
46+
kryo.register(classOf[mutable.BitSet])
4447
}
4548
}
4649

python/pyspark/sql.py

Lines changed: 14 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -152,10 +152,12 @@ def jsonFile(self, path):
152152
>>> ofn.close()
153153
>>> srdd = sqlCtx.jsonFile(jsonFile)
154154
>>> sqlCtx.registerRDDAsTable(srdd, "table1")
155-
>>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1")
156-
>>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}},
157-
... {"f1": 2, "f2": "row2", "f3":{"field4":22}},
158-
... {"f1": 3, "f2": "row3", "f3":{"field4":33}}]
155+
>>> srdd2 = sqlCtx.sql(
156+
... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1")
157+
>>> srdd2.collect() == [
158+
... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None},
159+
... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]},
160+
... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}]
159161
True
160162
"""
161163
jschema_rdd = self._ssql_ctx.jsonFile(path)
@@ -167,10 +169,12 @@ def jsonRDD(self, rdd):
167169
168170
>>> srdd = sqlCtx.jsonRDD(json)
169171
>>> sqlCtx.registerRDDAsTable(srdd, "table1")
170-
>>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1")
171-
>>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}},
172-
... {"f1": 2, "f2": "row2", "f3":{"field4":22}},
173-
... {"f1": 3, "f2": "row3", "f3":{"field4":33}}]
172+
>>> srdd2 = sqlCtx.sql(
173+
... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1")
174+
>>> srdd2.collect() == [
175+
... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None},
176+
... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]},
177+
... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}]
174178
True
175179
"""
176180
def func(split, iterator):
@@ -492,8 +496,8 @@ def _test():
492496
globs['rdd'] = sc.parallelize([{"field1" : 1, "field2" : "row1"},
493497
{"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}])
494498
jsonStrings = ['{"field1": 1, "field2": "row1", "field3":{"field4":11}}',
495-
'{"field1" : 2, "field2": "row2", "field3":{"field4":22}}',
496-
'{"field1" : 3, "field2": "row3", "field3":{"field4":33}}']
499+
'{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]}, "field6":[{"field7": "row2"}]}',
500+
'{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}']
497501
globs['jsonStrings'] = jsonStrings
498502
globs['json'] = sc.parallelize(jsonStrings)
499503
globs['nestedRdd1'] = sc.parallelize([

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -118,6 +118,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
118118
protected val UNCACHE = Keyword("UNCACHE")
119119
protected val UNION = Keyword("UNION")
120120
protected val WHERE = Keyword("WHERE")
121+
protected val INTERSECT = Keyword("INTERSECT")
121122
protected val EXCEPT = Keyword("EXCEPT")
122123

123124

@@ -140,6 +141,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
140141
protected lazy val query: Parser[LogicalPlan] = (
141142
select * (
142143
UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } |
144+
INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } |
143145
EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} |
144146
UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) }
145147
)

0 commit comments

Comments
 (0)