Skip to content

Commit 7470901

Browse files
committed
reduce conflicts
2 parents afc7da5 + 554403f commit 7470901

File tree

141 files changed

+9627
-794
lines changed

Some content is hidden

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

141 files changed

+9627
-794
lines changed

README.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ To build Spark and its example programs, run:
2626

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

3131
## Interactive Scala Shell
3232

bin/compute-classpath.sh

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,8 +50,8 @@ fi
5050
if [ -n "$SPARK_PREPEND_CLASSES" ]; then
5151
echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\
5252
"classes ahead of assembly." >&2
53+
# Spark classes
5354
CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/classes"
54-
CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*"
5555
CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/classes"
5656
CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/classes"
5757
CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/classes"
@@ -63,6 +63,8 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then
6363
CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/classes"
6464
CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SPARK_SCALA_VERSION/classes"
6565
CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SPARK_SCALA_VERSION/classes"
66+
# Jars for shaded deps in their original form (copied here during build)
67+
CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*"
6668
fi
6769

6870
# Use spark-assembly jar from either RELEASE or assembly directory

core/pom.xml

Lines changed: 20 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -94,22 +94,35 @@
9494
<groupId>org.apache.curator</groupId>
9595
<artifactId>curator-recipes</artifactId>
9696
</dependency>
97+
98+
<!-- Jetty dependencies promoted to compile here so they are shaded
99+
and inlined into spark-core jar -->
97100
<dependency>
98101
<groupId>org.eclipse.jetty</groupId>
99102
<artifactId>jetty-plus</artifactId>
103+
<scope>compile</scope>
100104
</dependency>
101105
<dependency>
102106
<groupId>org.eclipse.jetty</groupId>
103107
<artifactId>jetty-security</artifactId>
108+
<scope>compile</scope>
104109
</dependency>
105110
<dependency>
106111
<groupId>org.eclipse.jetty</groupId>
107112
<artifactId>jetty-util</artifactId>
113+
<scope>compile</scope>
108114
</dependency>
109115
<dependency>
110116
<groupId>org.eclipse.jetty</groupId>
111117
<artifactId>jetty-server</artifactId>
118+
<scope>compile</scope>
112119
</dependency>
120+
<dependency>
121+
<groupId>org.eclipse.jetty</groupId>
122+
<artifactId>jetty-http</artifactId>
123+
<scope>compile</scope>
124+
</dependency>
125+
113126
<dependency>
114127
<groupId>org.apache.commons</groupId>
115128
<artifactId>commons-lang3</artifactId>
@@ -348,19 +361,24 @@
348361
<groupId>org.apache.maven.plugins</groupId>
349362
<artifactId>maven-dependency-plugin</artifactId>
350363
<executions>
364+
<!-- When using SPARK_PREPEND_CLASSES Spark classes compiled locally don't use
365+
shaded deps. So here we store jars in their original form which are added
366+
when the classpath is computed. -->
351367
<execution>
352368
<id>copy-dependencies</id>
353369
<phase>package</phase>
354370
<goals>
355371
<goal>copy-dependencies</goal>
356372
</goals>
357-
<configuration>
373+
<configuration>
358374
<outputDirectory>${project.build.directory}</outputDirectory>
359375
<overWriteReleases>false</overWriteReleases>
360376
<overWriteSnapshots>false</overWriteSnapshots>
361377
<overWriteIfNewer>true</overWriteIfNewer>
362378
<useSubDirectoryPerType>true</useSubDirectoryPerType>
363-
<includeArtifactIds>guava</includeArtifactIds>
379+
<includeArtifactIds>
380+
guava,jetty-io,jetty-http,jetty-plus,jetty-util,jetty-server
381+
</includeArtifactIds>
364382
<silent>true</silent>
365383
</configuration>
366384
</execution>

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

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@ import org.apache.spark.scheduler._
4949
* spark.dynamicAllocation.enabled - Whether this feature is enabled
5050
* spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
5151
* spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
52+
* spark.dynamicAllocation.initialExecutors - Number of executors to start with
5253
*
5354
* spark.dynamicAllocation.schedulerBacklogTimeout (M) -
5455
* If there are backlogged tasks for this duration, add new executors
@@ -70,9 +71,10 @@ private[spark] class ExecutorAllocationManager(
7071

7172
import ExecutorAllocationManager._
7273

73-
// Lower and upper bounds on the number of executors. These are required.
74-
private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
75-
private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
74+
// Lower and upper bounds on the number of executors.
75+
private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0)
76+
private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors",
77+
Integer.MAX_VALUE)
7678

7779
// How long there must be backlogged tasks for before an addition is triggered
7880
private val schedulerBacklogTimeout = conf.getLong(
@@ -132,10 +134,10 @@ private[spark] class ExecutorAllocationManager(
132134
*/
133135
private def validateSettings(): Unit = {
134136
if (minNumExecutors < 0 || maxNumExecutors < 0) {
135-
throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
137+
throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be positive!")
136138
}
137-
if (minNumExecutors == 0 || maxNumExecutors == 0) {
138-
throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!")
139+
if (maxNumExecutors == 0) {
140+
throw new SparkException("spark.dynamicAllocation.maxExecutors cannot be 0!")
139141
}
140142
if (minNumExecutors > maxNumExecutors) {
141143
throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ private[spark] class HttpFileServer(
3636
var serverUri : String = null
3737

3838
def initialize() {
39-
baseDir = Utils.createTempDir()
39+
baseDir = Utils.createTempDir(Utils.getLocalDir(conf), "httpd")
4040
fileDir = new File(baseDir, "files")
4141
jarDir = new File(baseDir, "jars")
4242
fileDir.mkdir()

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

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package org.apache.spark
1919

2020
import java.io.File
2121

22+
import org.eclipse.jetty.server.ssl.SslSocketConnector
2223
import org.eclipse.jetty.util.security.{Constraint, Password}
2324
import org.eclipse.jetty.security.authentication.DigestAuthenticator
2425
import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService}
@@ -72,7 +73,10 @@ private[spark] class HttpServer(
7273
*/
7374
private def doStart(startPort: Int): (Server, Int) = {
7475
val server = new Server()
75-
val connector = new SocketConnector
76+
77+
val connector = securityManager.fileServerSSLOptions.createJettySslContextFactory()
78+
.map(new SslSocketConnector(_)).getOrElse(new SocketConnector)
79+
7680
connector.setMaxIdleTime(60 * 1000)
7781
connector.setSoLingerTime(-1)
7882
connector.setPort(startPort)
@@ -149,13 +153,14 @@ private[spark] class HttpServer(
149153
}
150154

151155
/**
152-
* Get the URI of this HTTP server (http://host:port)
156+
* Get the URI of this HTTP server (http://host:port or https://host:port)
153157
*/
154158
def uri: String = {
155159
if (server == null) {
156160
throw new ServerStateException("Server is not started")
157161
} else {
158-
"http://" + Utils.localIpAddress + ":" + port
162+
val scheme = if (securityManager.fileServerSSLOptions.enabled) "https" else "http"
163+
s"$scheme://${Utils.localIpAddress}:$port"
159164
}
160165
}
161166
}
Lines changed: 178 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,178 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark
19+
20+
import java.io.File
21+
22+
import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory}
23+
import org.eclipse.jetty.util.ssl.SslContextFactory
24+
25+
/**
26+
* SSLOptions class is a common container for SSL configuration options. It offers methods to
27+
* generate specific objects to configure SSL for different communication protocols.
28+
*
29+
* SSLOptions is intended to provide the maximum common set of SSL settings, which are supported
30+
* by the protocol, which it can generate the configuration for. Since Akka doesn't support client
31+
* authentication with SSL, SSLOptions cannot support it either.
32+
*
33+
* @param enabled enables or disables SSL; if it is set to false, the rest of the
34+
* settings are disregarded
35+
* @param keyStore a path to the key-store file
36+
* @param keyStorePassword a password to access the key-store file
37+
* @param keyPassword a password to access the private key in the key-store
38+
* @param trustStore a path to the trust-store file
39+
* @param trustStorePassword a password to access the trust-store file
40+
* @param protocol SSL protocol (remember that SSLv3 was compromised) supported by Java
41+
* @param enabledAlgorithms a set of encryption algorithms to use
42+
*/
43+
private[spark] case class SSLOptions(
44+
enabled: Boolean = false,
45+
keyStore: Option[File] = None,
46+
keyStorePassword: Option[String] = None,
47+
keyPassword: Option[String] = None,
48+
trustStore: Option[File] = None,
49+
trustStorePassword: Option[String] = None,
50+
protocol: Option[String] = None,
51+
enabledAlgorithms: Set[String] = Set.empty) {
52+
53+
/**
54+
* Creates a Jetty SSL context factory according to the SSL settings represented by this object.
55+
*/
56+
def createJettySslContextFactory(): Option[SslContextFactory] = {
57+
if (enabled) {
58+
val sslContextFactory = new SslContextFactory()
59+
60+
keyStore.foreach(file => sslContextFactory.setKeyStorePath(file.getAbsolutePath))
61+
trustStore.foreach(file => sslContextFactory.setTrustStore(file.getAbsolutePath))
62+
keyStorePassword.foreach(sslContextFactory.setKeyStorePassword)
63+
trustStorePassword.foreach(sslContextFactory.setTrustStorePassword)
64+
keyPassword.foreach(sslContextFactory.setKeyManagerPassword)
65+
protocol.foreach(sslContextFactory.setProtocol)
66+
sslContextFactory.setIncludeCipherSuites(enabledAlgorithms.toSeq: _*)
67+
68+
Some(sslContextFactory)
69+
} else {
70+
None
71+
}
72+
}
73+
74+
/**
75+
* Creates an Akka configuration object which contains all the SSL settings represented by this
76+
* object. It can be used then to compose the ultimate Akka configuration.
77+
*/
78+
def createAkkaConfig: Option[Config] = {
79+
import scala.collection.JavaConversions._
80+
if (enabled) {
81+
Some(ConfigFactory.empty()
82+
.withValue("akka.remote.netty.tcp.security.key-store",
83+
ConfigValueFactory.fromAnyRef(keyStore.map(_.getAbsolutePath).getOrElse("")))
84+
.withValue("akka.remote.netty.tcp.security.key-store-password",
85+
ConfigValueFactory.fromAnyRef(keyStorePassword.getOrElse("")))
86+
.withValue("akka.remote.netty.tcp.security.trust-store",
87+
ConfigValueFactory.fromAnyRef(trustStore.map(_.getAbsolutePath).getOrElse("")))
88+
.withValue("akka.remote.netty.tcp.security.trust-store-password",
89+
ConfigValueFactory.fromAnyRef(trustStorePassword.getOrElse("")))
90+
.withValue("akka.remote.netty.tcp.security.key-password",
91+
ConfigValueFactory.fromAnyRef(keyPassword.getOrElse("")))
92+
.withValue("akka.remote.netty.tcp.security.random-number-generator",
93+
ConfigValueFactory.fromAnyRef(""))
94+
.withValue("akka.remote.netty.tcp.security.protocol",
95+
ConfigValueFactory.fromAnyRef(protocol.getOrElse("")))
96+
.withValue("akka.remote.netty.tcp.security.enabled-algorithms",
97+
ConfigValueFactory.fromIterable(enabledAlgorithms.toSeq))
98+
.withValue("akka.remote.netty.tcp.enable-ssl",
99+
ConfigValueFactory.fromAnyRef(true)))
100+
} else {
101+
None
102+
}
103+
}
104+
105+
/** Returns a string representation of this SSLOptions with all the passwords masked. */
106+
override def toString: String = s"SSLOptions{enabled=$enabled, " +
107+
s"keyStore=$keyStore, keyStorePassword=${keyStorePassword.map(_ => "xxx")}, " +
108+
s"trustStore=$trustStore, trustStorePassword=${trustStorePassword.map(_ => "xxx")}, " +
109+
s"protocol=$protocol, enabledAlgorithms=$enabledAlgorithms}"
110+
111+
}
112+
113+
private[spark] object SSLOptions extends Logging {
114+
115+
/** Resolves SSLOptions settings from a given Spark configuration object at a given namespace.
116+
*
117+
* The following settings are allowed:
118+
* $ - `[ns].enabled` - `true` or `false`, to enable or disable SSL respectively
119+
* $ - `[ns].keyStore` - a path to the key-store file; can be relative to the current directory
120+
* $ - `[ns].keyStorePassword` - a password to the key-store file
121+
* $ - `[ns].keyPassword` - a password to the private key
122+
* $ - `[ns].trustStore` - a path to the trust-store file; can be relative to the current
123+
* directory
124+
* $ - `[ns].trustStorePassword` - a password to the trust-store file
125+
* $ - `[ns].protocol` - a protocol name supported by a particular Java version
126+
* $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers
127+
*
128+
* For a list of protocols and ciphers supported by particular Java versions, you may go to
129+
* [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle
130+
* blog page]].
131+
*
132+
* You can optionally specify the default configuration. If you do, for each setting which is
133+
* missing in SparkConf, the corresponding setting is used from the default configuration.
134+
*
135+
* @param conf Spark configuration object where the settings are collected from
136+
* @param ns the namespace name
137+
* @param defaults the default configuration
138+
* @return [[org.apache.spark.SSLOptions]] object
139+
*/
140+
def parse(conf: SparkConf, ns: String, defaults: Option[SSLOptions] = None): SSLOptions = {
141+
val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled))
142+
143+
val keyStore = conf.getOption(s"$ns.keyStore").map(new File(_))
144+
.orElse(defaults.flatMap(_.keyStore))
145+
146+
val keyStorePassword = conf.getOption(s"$ns.keyStorePassword")
147+
.orElse(defaults.flatMap(_.keyStorePassword))
148+
149+
val keyPassword = conf.getOption(s"$ns.keyPassword")
150+
.orElse(defaults.flatMap(_.keyPassword))
151+
152+
val trustStore = conf.getOption(s"$ns.trustStore").map(new File(_))
153+
.orElse(defaults.flatMap(_.trustStore))
154+
155+
val trustStorePassword = conf.getOption(s"$ns.trustStorePassword")
156+
.orElse(defaults.flatMap(_.trustStorePassword))
157+
158+
val protocol = conf.getOption(s"$ns.protocol")
159+
.orElse(defaults.flatMap(_.protocol))
160+
161+
val enabledAlgorithms = conf.getOption(s"$ns.enabledAlgorithms")
162+
.map(_.split(",").map(_.trim).filter(_.nonEmpty).toSet)
163+
.orElse(defaults.map(_.enabledAlgorithms))
164+
.getOrElse(Set.empty)
165+
166+
new SSLOptions(
167+
enabled,
168+
keyStore,
169+
keyStorePassword,
170+
keyPassword,
171+
trustStore,
172+
trustStorePassword,
173+
protocol,
174+
enabledAlgorithms)
175+
}
176+
177+
}
178+

0 commit comments

Comments
 (0)