Skip to content

Commit 4fc8b4e

Browse files
committed
Merge branch 'master' of github.com:apache/spark into limit
2 parents 87b7d37 + 3d89043 commit 4fc8b4e

File tree

149 files changed

+2689
-461
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

149 files changed

+2689
-461
lines changed

.travis.yml

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,32 @@
1+
# Licensed to the Apache Software Foundation (ASF) under one or more
2+
# contributor license agreements. See the NOTICE file distributed with
3+
# this work for additional information regarding copyright ownership.
4+
# The ASF licenses this file to You under the Apache License, Version 2.0
5+
# (the "License"); you may not use this file except in compliance with
6+
# the License. You may obtain a copy of the License at
7+
#
8+
# http://www.apache.org/licenses/LICENSE-2.0
9+
#
10+
# Unless required by applicable law or agreed to in writing, software
11+
# distributed under the License is distributed on an "AS IS" BASIS,
12+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
# See the License for the specific language governing permissions and
14+
# limitations under the License.
15+
16+
language: scala
17+
scala:
18+
- "2.10.3"
19+
jdk:
20+
- oraclejdk7
21+
env:
22+
matrix:
23+
- TEST="scalastyle assembly/assembly"
24+
- TEST="catalyst/test sql/test streaming/test mllib/test graphx/test bagel/test"
25+
- TEST=hive/test
26+
cache:
27+
directories:
28+
- $HOME/.m2
29+
- $HOME/.ivy2
30+
- $HOME/.sbt
31+
script:
32+
- "sbt ++$TRAVIS_SCALA_VERSION $TEST"

bin/compute-classpath.sh

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,6 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
3636
# Hopefully we will find a way to avoid uber-jars entirely and deploy only the needed packages in
3737
# the future.
3838
if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then
39-
echo "Hive assembly found, including hive support. If this isn't desired run sbt hive/clean."
4039

4140
# Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost.
4241
DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}")

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

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -44,8 +44,9 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
4444
* Represents a dependency on the output of a shuffle stage.
4545
* @param rdd the parent RDD
4646
* @param partitioner partitioner used to partition the shuffle output
47-
* @param serializer [[Serializer]] to use. If set to null, the default serializer, as specified
48-
* by `spark.serializer` config option, will be used.
47+
* @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null,
48+
* the default serializer, as specified by `spark.serializer` config option, will
49+
* be used.
4950
*/
5051
class ShuffleDependency[K, V](
5152
@transient rdd: RDD[_ <: Product2[K, V]],

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

Lines changed: 19 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad
3535
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
3636
import org.apache.mesos.MesosNativeLibrary
3737

38+
import org.apache.spark.broadcast.Broadcast
3839
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
3940
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
4041
import org.apache.spark.rdd._
@@ -230,7 +231,7 @@ class SparkContext(
230231
postEnvironmentUpdate()
231232

232233
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
233-
val hadoopConfiguration = {
234+
val hadoopConfiguration: Configuration = {
234235
val env = SparkEnv.get
235236
val hadoopConf = SparkHadoopUtil.get.newConfiguration()
236237
// Explicitly check for S3 environment variables
@@ -630,7 +631,7 @@ class SparkContext(
630631
* standard mutable collections. So you can use this with mutable Map, Set, etc.
631632
*/
632633
def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T]
633-
(initialValue: R) = {
634+
(initialValue: R): Accumulable[R, T] = {
634635
val param = new GrowableAccumulableParam[R,T]
635636
new Accumulable(initialValue, param)
636637
}
@@ -640,7 +641,7 @@ class SparkContext(
640641
* [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions.
641642
* The variable will be sent to each cluster only once.
642643
*/
643-
def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal)
644+
def broadcast[T](value: T): Broadcast[T] = env.broadcastManager.newBroadcast[T](value, isLocal)
644645

645646
/**
646647
* Add a file to be downloaded with this Spark job on every node.
@@ -1126,7 +1127,7 @@ object SparkContext extends Logging {
11261127
implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd)
11271128

11281129
implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
1129-
rdd: RDD[(K, V)]) =
1130+
rdd: RDD[(K, V)]) =
11301131
new SequenceFileRDDFunctions(rdd)
11311132

11321133
implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag](
@@ -1163,27 +1164,33 @@ object SparkContext extends Logging {
11631164
}
11641165

11651166
// Helper objects for converting common types to Writable
1166-
private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = {
1167+
private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T)
1168+
: WritableConverter[T] = {
11671169
val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]]
11681170
new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W]))
11691171
}
11701172

1171-
implicit def intWritableConverter() = simpleWritableConverter[Int, IntWritable](_.get)
1173+
implicit def intWritableConverter(): WritableConverter[Int] =
1174+
simpleWritableConverter[Int, IntWritable](_.get)
11721175

1173-
implicit def longWritableConverter() = simpleWritableConverter[Long, LongWritable](_.get)
1176+
implicit def longWritableConverter(): WritableConverter[Long] =
1177+
simpleWritableConverter[Long, LongWritable](_.get)
11741178

1175-
implicit def doubleWritableConverter() = simpleWritableConverter[Double, DoubleWritable](_.get)
1179+
implicit def doubleWritableConverter(): WritableConverter[Double] =
1180+
simpleWritableConverter[Double, DoubleWritable](_.get)
11761181

1177-
implicit def floatWritableConverter() = simpleWritableConverter[Float, FloatWritable](_.get)
1182+
implicit def floatWritableConverter(): WritableConverter[Float] =
1183+
simpleWritableConverter[Float, FloatWritable](_.get)
11781184

1179-
implicit def booleanWritableConverter() =
1185+
implicit def booleanWritableConverter(): WritableConverter[Boolean] =
11801186
simpleWritableConverter[Boolean, BooleanWritable](_.get)
11811187

1182-
implicit def bytesWritableConverter() = {
1188+
implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = {
11831189
simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes)
11841190
}
11851191

1186-
implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString)
1192+
implicit def stringWritableConverter(): WritableConverter[String] =
1193+
simpleWritableConverter[String, Text](_.toString)
11871194

11881195
implicit def writableWritableConverter[T <: Writable]() =
11891196
new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T])

core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -391,19 +391,24 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
391391
/**
392392
* Save this RDD as a text file, using string representations of elements.
393393
*/
394-
def saveAsTextFile(path: String) = rdd.saveAsTextFile(path)
394+
def saveAsTextFile(path: String): Unit = {
395+
rdd.saveAsTextFile(path)
396+
}
395397

396398

397399
/**
398400
* Save this RDD as a compressed text file, using string representations of elements.
399401
*/
400-
def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) =
402+
def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit = {
401403
rdd.saveAsTextFile(path, codec)
404+
}
402405

403406
/**
404407
* Save this RDD as a SequenceFile of serialized objects.
405408
*/
406-
def saveAsObjectFile(path: String) = rdd.saveAsObjectFile(path)
409+
def saveAsObjectFile(path: String): Unit = {
410+
rdd.saveAsObjectFile(path)
411+
}
407412

408413
/**
409414
* Creates tuples of the elements in this RDD by applying `f`.
@@ -420,7 +425,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
420425
* executed on this RDD. It is strongly recommended that this RDD is persisted in
421426
* memory, otherwise saving it on a file will require recomputation.
422427
*/
423-
def checkpoint() = rdd.checkpoint()
428+
def checkpoint(): Unit = {
429+
rdd.checkpoint()
430+
}
424431

425432
/**
426433
* Return whether this RDD has been checkpointed or not
@@ -481,7 +488,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
481488
/**
482489
* Returns the maximum element from this RDD as defined by the specified
483490
* Comparator[T].
484-
* @params comp the comparator that defines ordering
491+
* @param comp the comparator that defines ordering
485492
* @return the maximum of the RDD
486493
* */
487494
def max(comp: Comparator[T]): T = {
@@ -491,7 +498,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
491498
/**
492499
* Returns the minimum element from this RDD as defined by the specified
493500
* Comparator[T].
494-
* @params comp the comparator that defines ordering
501+
* @param comp the comparator that defines ordering
495502
* @return the minimum of the RDD
496503
* */
497504
def min(comp: Comparator[T]): T = {

core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -463,7 +463,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
463463
sc.setCheckpointDir(dir)
464464
}
465465

466-
def getCheckpointDir = JavaUtils.optionToOptional(sc.getCheckpointDir)
466+
def getCheckpointDir: Optional[String] = JavaUtils.optionToOptional(sc.getCheckpointDir)
467467

468468
protected def checkpointFile[T](path: String): JavaRDD[T] = {
469469
implicit val ctag: ClassTag[T] = fakeClassTag

core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -112,5 +112,5 @@ private[spark] class ClientArguments(args: Array[String]) {
112112
}
113113

114114
object ClientArguments {
115-
def isValidJarUrl(s: String) = s.matches("(.+):(.+)jar")
115+
def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar")
116116
}

core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import scala.collection.JavaConversions._
3232
* Contains util methods to interact with Hadoop from Spark.
3333
*/
3434
class SparkHadoopUtil {
35-
val conf = newConfiguration()
35+
val conf: Configuration = newConfiguration()
3636
UserGroupInformation.setConfiguration(conf)
3737

3838
def runAsUser(user: String)(func: () => Unit) {

core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark.deploy.master
2020
import scala.collection.JavaConversions._
2121

2222
import akka.serialization.Serialization
23+
import org.apache.curator.framework.CuratorFramework
2324
import org.apache.zookeeper.CreateMode
2425

2526
import org.apache.spark.{Logging, SparkConf}
@@ -29,7 +30,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
2930
with Logging
3031
{
3132
val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
32-
val zk = SparkCuratorUtil.newClient(conf)
33+
val zk: CuratorFramework = SparkCuratorUtil.newClient(conf)
3334

3435
SparkCuratorUtil.mkdir(zk, WORKING_DIR)
3536

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -112,11 +112,10 @@ private[spark] class Executor(
112112
}
113113
}
114114

115-
// Create our ClassLoader and set it on this thread
115+
// Create our ClassLoader
116116
// do this after SparkEnv creation so can access the SecurityManager
117117
private val urlClassLoader = createClassLoader()
118118
private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader)
119-
Thread.currentThread.setContextClassLoader(replClassLoader)
120119

121120
// Akka's message frame size. If task result is bigger than this, we use the block manager
122121
// to send the result back.
@@ -294,7 +293,7 @@ private[spark] class Executor(
294293
* created by the interpreter to the search path
295294
*/
296295
private def createClassLoader(): ExecutorURLClassLoader = {
297-
val loader = this.getClass.getClassLoader
296+
val loader = Thread.currentThread().getContextClassLoader
298297

299298
// For each of the jars in the jarSet, add them to the class loader.
300299
// We assume each of the files has already been fetched.

0 commit comments

Comments
 (0)