1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
|
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark
import org.scalatest.FunSuite
import org.apache.hadoop.io.BytesWritable
class SparkContextSuite extends FunSuite with LocalSparkContext {
/** Allows system properties to be changed in tests */
private def withSystemProperty[T](property: String, value: String)(block: => T): T = {
val originalValue = System.getProperty(property)
try {
System.setProperty(property, value)
block
} finally {
if (originalValue == null) {
System.clearProperty(property)
} else {
System.setProperty(property, originalValue)
}
}
}
test("Only one SparkContext may be active at a time") {
// Regression test for SPARK-4180
withSystemProperty("spark.driver.allowMultipleContexts", "false") {
val conf = new SparkConf().setAppName("test").setMaster("local")
sc = new SparkContext(conf)
// A SparkContext is already running, so we shouldn't be able to create a second one
intercept[SparkException] { new SparkContext(conf) }
// After stopping the running context, we should be able to create a new one
resetSparkContext()
sc = new SparkContext(conf)
}
}
test("Can still construct a new SparkContext after failing to construct a previous one") {
withSystemProperty("spark.driver.allowMultipleContexts", "false") {
// This is an invalid configuration (no app name or master URL)
intercept[SparkException] {
new SparkContext(new SparkConf())
}
// Even though those earlier calls failed, we should still be able to create a new context
sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test"))
}
}
test("Check for multiple SparkContexts can be disabled via undocumented debug option") {
withSystemProperty("spark.driver.allowMultipleContexts", "true") {
var secondSparkContext: SparkContext = null
try {
val conf = new SparkConf().setAppName("test").setMaster("local")
sc = new SparkContext(conf)
secondSparkContext = new SparkContext(conf)
} finally {
Option(secondSparkContext).foreach(_.stop())
}
}
}
test("BytesWritable implicit conversion is correct") {
// Regression test for SPARK-3121
val bytesWritable = new BytesWritable()
val inputArray = (1 to 10).map(_.toByte).toArray
bytesWritable.set(inputArray, 0, 10)
bytesWritable.set(inputArray, 0, 5)
val converter = SparkContext.bytesWritableConverter()
val byteArray = converter.convert(bytesWritable)
assert(byteArray.length === 5)
bytesWritable.set(inputArray, 0, 0)
val byteArray2 = converter.convert(bytesWritable)
assert(byteArray2.length === 0)
}
}
|