Skip to content

Commit 1ccd7cc

Browse files
committed
2 parents af9feb9 + ec1adec commit 1ccd7cc

File tree

171 files changed

+3971
-1360
lines changed

Some content is hidden

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

171 files changed

+3971
-1360
lines changed

CONTRIBUTING.md

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,12 @@
1+
## Contributing to Spark
2+
3+
Contributions via GitHub pull requests are gladly accepted from their original
4+
author. Along with any pull requests, please state that the contribution is
5+
your original work and that you license the work to the project under the
6+
project's open source license. Whether or not you state this explicitly, by
7+
submitting any copyrighted material via pull request, email, or other means
8+
you agree to license the material under the project's open source license and
9+
warrant that you have the legal authority to do so.
10+
11+
Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark)
12+
for more information.

README.md

Lines changed: 16 additions & 62 deletions
Original file line numberDiff line numberDiff line change
@@ -13,16 +13,19 @@ and Spark Streaming for stream processing.
1313
## Online Documentation
1414

1515
You can find the latest Spark documentation, including a programming
16-
guide, on the project webpage at <http://spark.apache.org/documentation.html>.
16+
guide, on the [project web page](http://spark.apache.org/documentation.html).
1717
This README file only contains basic setup instructions.
1818

1919
## Building Spark
2020

21-
Spark is built on Scala 2.10. To build Spark and its example programs, run:
21+
Spark is built using [Apache Maven](http://maven.apache.org/).
22+
To build Spark and its example programs, run:
2223

23-
./sbt/sbt assembly
24+
mvn -DskipTests clean package
2425

2526
(You do not need to do this if you downloaded a pre-built package.)
27+
More detailed documentation is available from the project site, at
28+
["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html).
2629

2730
## Interactive Scala Shell
2831

@@ -71,73 +74,24 @@ can be run using:
7174

7275
./dev/run-tests
7376

77+
Please see the guidance on how to
78+
[run all automated tests](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting).
79+
7480
## A Note About Hadoop Versions
7581

7682
Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported
7783
storage systems. Because the protocols have changed in different versions of
7884
Hadoop, you must build Spark against the same version that your cluster runs.
79-
You can change the version by setting `-Dhadoop.version` when building Spark.
80-
81-
For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop
82-
versions without YARN, use:
83-
84-
# Apache Hadoop 1.2.1
85-
$ sbt/sbt -Dhadoop.version=1.2.1 assembly
86-
87-
# Cloudera CDH 4.2.0 with MapReduce v1
88-
$ sbt/sbt -Dhadoop.version=2.0.0-mr1-cdh4.2.0 assembly
89-
90-
For Apache Hadoop 2.2.X, 2.1.X, 2.0.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions
91-
with YARN, also set `-Pyarn`:
92-
93-
# Apache Hadoop 2.0.5-alpha
94-
$ sbt/sbt -Dhadoop.version=2.0.5-alpha -Pyarn assembly
95-
96-
# Cloudera CDH 4.2.0 with MapReduce v2
97-
$ sbt/sbt -Dhadoop.version=2.0.0-cdh4.2.0 -Pyarn assembly
98-
99-
# Apache Hadoop 2.2.X and newer
100-
$ sbt/sbt -Dhadoop.version=2.2.0 -Pyarn assembly
101-
102-
When developing a Spark application, specify the Hadoop version by adding the
103-
"hadoop-client" artifact to your project's dependencies. For example, if you're
104-
using Hadoop 1.2.1 and build your application using SBT, add this entry to
105-
`libraryDependencies`:
106-
107-
"org.apache.hadoop" % "hadoop-client" % "1.2.1"
10885

109-
If your project is built with Maven, add this to your POM file's `<dependencies>` section:
110-
111-
<dependency>
112-
<groupId>org.apache.hadoop</groupId>
113-
<artifactId>hadoop-client</artifactId>
114-
<version>1.2.1</version>
115-
</dependency>
116-
117-
118-
## A Note About Thrift JDBC server and CLI for Spark SQL
119-
120-
Spark SQL supports Thrift JDBC server and CLI.
121-
See sql-programming-guide.md for more information about using the JDBC server and CLI.
122-
You can use those features by setting `-Phive` when building Spark as follows.
123-
124-
$ sbt/sbt -Phive assembly
86+
Please refer to the build documentation at
87+
["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version)
88+
for detailed guidance on building for a particular distribution of Hadoop, including
89+
building for particular Hive and Hive Thriftserver distributions. See also
90+
["Third Party Hadoop Distributions"](http://spark.apache.org/docs/latest/hadoop-third-party-distributions.html)
91+
for guidance on building a Spark application that works with a particular
92+
distribution.
12593

12694
## Configuration
12795

12896
Please refer to the [Configuration guide](http://spark.apache.org/docs/latest/configuration.html)
12997
in the online documentation for an overview on how to configure Spark.
130-
131-
132-
## Contributing to Spark
133-
134-
Contributions via GitHub pull requests are gladly accepted from their original
135-
author. Along with any pull requests, please state that the contribution is
136-
your original work and that you license the work to the project under the
137-
project's open source license. Whether or not you state this explicitly, by
138-
submitting any copyrighted material via pull request, email, or other means
139-
you agree to license the material under the project's open source license and
140-
warrant that you have the legal authority to do so.
141-
142-
Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark)
143-
for more information.

assembly/pom.xml

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,20 @@
8888

8989
<build>
9090
<plugins>
91+
<plugin>
92+
<groupId>org.apache.maven.plugins</groupId>
93+
<artifactId>maven-deploy-plugin</artifactId>
94+
<configuration>
95+
<skip>true</skip>
96+
</configuration>
97+
</plugin>
98+
<plugin>
99+
<groupId>org.apache.maven.plugins</groupId>
100+
<artifactId>maven-install-plugin</artifactId>
101+
<configuration>
102+
<skip>true</skip>
103+
</configuration>
104+
</plugin>
91105
<!-- Use the shade plugin to create a big JAR with all the dependencies -->
92106
<plugin>
93107
<groupId>org.apache.maven.plugins</groupId>

bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,6 @@ import org.scalatest.time.SpanSugar._
2424
import org.apache.spark._
2525
import org.apache.spark.storage.StorageLevel
2626

27-
import scala.language.postfixOps
28-
2927
class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable
3028
class TestMessage(val targetId: String) extends Message[String] with Serializable
3129

bin/compute-classpath.sh

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then
4343
echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\
4444
"classes ahead of assembly." >&2
4545
CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes"
46+
CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*"
4647
CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes"
4748
CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes"
4849
CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes"

bin/spark-class

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,7 @@ else
105105
exit 1
106106
fi
107107
fi
108-
JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q')
108+
JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q')
109109

110110
# Set JAVA_OPTS to be able to load native libraries and to set heap size
111111
if [ "$JAVA_VERSION" -ge 18 ]; then

core/pom.xml

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -351,6 +351,33 @@
351351
</execution>
352352
</executions>
353353
</plugin>
354+
<!--
355+
Copy guava to the build directory. This is needed to make the SPARK_PREPEND_CLASSES
356+
option work in compute-classpath.sh, since it would put the non-shaded Spark classes in
357+
the runtime classpath.
358+
-->
359+
<plugin>
360+
<groupId>org.apache.maven.plugins</groupId>
361+
<artifactId>maven-dependency-plugin</artifactId>
362+
<executions>
363+
<execution>
364+
<id>copy-dependencies</id>
365+
<phase>package</phase>
366+
<goals>
367+
<goal>copy-dependencies</goal>
368+
</goals>
369+
<configuration>
370+
<outputDirectory>${project.build.directory}</outputDirectory>
371+
<overWriteReleases>false</overWriteReleases>
372+
<overWriteSnapshots>false</overWriteSnapshots>
373+
<overWriteIfNewer>true</overWriteIfNewer>
374+
<useSubDirectoryPerType>true</useSubDirectoryPerType>
375+
<includeArtifactIds>guava</includeArtifactIds>
376+
<silent>true</silent>
377+
</configuration>
378+
</execution>
379+
</executions>
380+
</plugin>
354381
</plugins>
355382

356383
<resources>

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -162,7 +162,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging {
162162

163163
// always add the current user and SPARK_USER to the viewAcls
164164
private val defaultAclUsers = Set[String](System.getProperty("user.name", ""),
165-
Option(System.getenv("SPARK_USER")).getOrElse(""))
165+
Option(System.getenv("SPARK_USER")).getOrElse("")).filter(!_.isEmpty)
166166

167167
setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", ""))
168168
setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", ""))

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

Lines changed: 9 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -220,8 +220,14 @@ class SparkContext(config: SparkConf) extends Logging {
220220
new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf)
221221

222222
// Initialize the Spark UI, registering all associated listeners
223-
private[spark] val ui = new SparkUI(this)
224-
ui.bind()
223+
private[spark] val ui: Option[SparkUI] =
224+
if (conf.getBoolean("spark.ui.enabled", true)) {
225+
Some(new SparkUI(this))
226+
} else {
227+
// For tests, do not enable the UI
228+
None
229+
}
230+
ui.foreach(_.bind())
225231

226232
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
227233
val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf)
@@ -990,7 +996,7 @@ class SparkContext(config: SparkConf) extends Logging {
990996
/** Shut down the SparkContext. */
991997
def stop() {
992998
postApplicationEnd()
993-
ui.stop()
999+
ui.foreach(_.stop())
9941000
// Do this only if not stopped already - best case effort.
9951001
// prevent NPE if stopped more than once.
9961002
val dagSchedulerCopy = dagScheduler
@@ -1066,11 +1072,8 @@ class SparkContext(config: SparkConf) extends Logging {
10661072
val callSite = getCallSite
10671073
val cleanedFunc = clean(func)
10681074
logInfo("Starting job: " + callSite.shortForm)
1069-
val start = System.nanoTime
10701075
dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
10711076
resultHandler, localProperties.get)
1072-
logInfo(
1073-
"Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s")
10741077
rdd.doCheckpoint()
10751078
}
10761079

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

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -108,6 +108,14 @@ class SparkEnv (
108108
pythonWorkers.get(key).foreach(_.stopWorker(worker))
109109
}
110110
}
111+
112+
private[spark]
113+
def releasePythonWorker(pythonExec: String, envVars: Map[String, String], worker: Socket) {
114+
synchronized {
115+
val key = (pythonExec, envVars)
116+
pythonWorkers.get(key).foreach(_.releaseWorker(worker))
117+
}
118+
}
111119
}
112120

113121
object SparkEnv extends Logging {

0 commit comments

Comments
 (0)