Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-2157] Enable tight firewall rules for Spark #1777

Closed
wants to merge 44 commits into from

Conversation

andrewor14
Copy link
Contributor

The goal of this PR is to allow users of Spark to write tight firewall rules for their clusters. This is currently not possible because Spark uses random ports in many places, notably the communication between executors and drivers. The changes in this PR are based on top of @ash211's changes in #1107.

The list covered here may or may not be the complete set of port needed for Spark to operate perfectly. As of the latest commit there are no known sources of random ports used in inter-machine communication.

My spark-env.sh looks like this:

export SPARK_MASTER_PORT=6060
export SPARK_WORKER_PORT=7070
export SPARK_MASTER_WEBUI_PORT=9090
export SPARK_WORKER_WEBUI_PORT=9091

and my spark-defaults.conf looks like this:

spark.master spark://andrews-mbp:6060
spark.driver.port 5001
spark.fileserver.port 5011
spark.broadcast.port 5021
spark.replClassServer.port 5031
spark.blockManager.port 5041
spark.executor.port 5051

ash211 and others added 27 commits July 24, 2014 23:08
Fails when running master+worker+executor+shell on the same machine.  I think
the issue is that both the shell and the executor attempt to start a
ConnectionManager, which causes port conflicts.  Solution is to attempt and
increment on BindExceptions
- spark.fileserver.port
- spark.broadcast.port
- spark.replClassServer.port
- spark.blockManager.port
We were passing the Master/Worker UI names as base paths for Jetty
to use, which is obviously wrong in hindsight.
The existing documentation uses {master/worker}.ui.port, which doesn't
actually work because SparkConf doesn't pick up system properties that
are not prefixed by spark (SPARK-2857). We should at least document
a working alternative.
@SparkQA
Copy link

SparkQA commented Aug 5, 2014

QA tests have started for PR 1777. This patch merges cleanly.
View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17923/consoleFull

@@ -84,7 +84,8 @@ private[spark] class Executor(
// Initialize Spark environment (using system properties read above)
private val env = {
if (!isLocal) {
val _env = SparkEnv.create(conf, executorId, slaveHostname, 0,
val port = conf.getInt("spark.executor.env.port", 0) // TODO: document this
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is probably a better name for this. I just don't know what else to call it.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what about just spark.executor.port (ala spark.driver.port)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There's already a spark.executor.port, and these two overlap

asfgit pushed a commit that referenced this pull request Aug 6, 2014
This is failing my tests in #1777. @tdas

Author: Andrew Or <andrewor14@gmail.com>

Closes #1803 from andrewor14/fix-flaky-streaming-test and squashes the following commits:

ea11a03 [Andrew Or] Catch all exceptions caused by BindExceptions
54a0ca0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-flaky-streaming-test
664095c [Andrew Or] Tone down bind exception message
af3ddc9 [Andrew Or] Handle port collisions in flume polling test
tdas pushed a commit to tdas/spark that referenced this pull request Aug 21, 2014
This is failing my tests in apache#1777. @tdas

Author: Andrew Or <andrewor14@gmail.com>

Closes apache#1803 from andrewor14/fix-flaky-streaming-test and squashes the following commits:

ea11a03 [Andrew Or] Catch all exceptions caused by BindExceptions
54a0ca0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-flaky-streaming-test
664095c [Andrew Or] Tone down bind exception message
af3ddc9 [Andrew Or] Handle port collisions in flume polling test
asfgit pushed a commit that referenced this pull request Aug 21, 2014
This is failing my tests in #1777. @tdas

Author: Andrew Or <andrewor14@gmail.com>

Closes #1803 from andrewor14/fix-flaky-streaming-test and squashes the following commits:

ea11a03 [Andrew Or] Catch all exceptions caused by BindExceptions
54a0ca0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-flaky-streaming-test
664095c [Andrew Or] Tone down bind exception message
af3ddc9 [Andrew Or] Handle port collisions in flume polling test
asfgit pushed a commit that referenced this pull request Aug 26, 2014
As of #1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001.
```
14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001.
```
This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1810 from andrewor14/service-name and squashes the following commits:

8c459ed [Andrew Or] Use a common variable for driver/executor actor system names
3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor
921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name
c8c6a62 [Andrew Or] Do not include hyphens in actor name
1c1b42e [Andrew Or] Avoid spaces in akka system name
f644b55 [Andrew Or] Use more specific service name
asfgit pushed a commit that referenced this pull request Aug 26, 2014
As of #1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001.
```
14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001.
```
This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1810 from andrewor14/service-name and squashes the following commits:

8c459ed [Andrew Or] Use a common variable for driver/executor actor system names
3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor
921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name
c8c6a62 [Andrew Or] Do not include hyphens in actor name
1c1b42e [Andrew Or] Avoid spaces in akka system name
f644b55 [Andrew Or] Use more specific service name

(cherry picked from commit b21ae5b)
Signed-off-by: Andrew Or <andrewor14@gmail.com>
kayousterhout pushed a commit to kayousterhout/spark-1 that referenced this pull request Aug 27, 2014
As of apache#1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001.
```
14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001.
```
This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality.

Author: Andrew Or <andrewor14@gmail.com>

Closes apache#1810 from andrewor14/service-name and squashes the following commits:

8c459ed [Andrew Or] Use a common variable for driver/executor actor system names
3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor
921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name
c8c6a62 [Andrew Or] Do not include hyphens in actor name
1c1b42e [Andrew Or] Avoid spaces in akka system name
f644b55 [Andrew Or] Use more specific service name
xiliu82 pushed a commit to xiliu82/spark that referenced this pull request Sep 4, 2014
The goal of this PR is to allow users of Spark to write tight firewall rules for their clusters. This is currently not possible because Spark uses random ports in many places, notably the communication between executors and drivers. The changes in this PR are based on top of ash211's changes in apache#1107.

The list covered here may or may not be the complete set of port needed for Spark to operate perfectly. However, as of the latest commit there are no known sources of random ports (except in tests). I have not documented a few of the more obscure configs.

My spark-env.sh looks like this:
```
export SPARK_MASTER_PORT=6060
export SPARK_WORKER_PORT=7070
export SPARK_MASTER_WEBUI_PORT=9090
export SPARK_WORKER_WEBUI_PORT=9091
```
and my spark-defaults.conf looks like this:
```
spark.master spark://andrews-mbp:6060
spark.driver.port 5001
spark.fileserver.port 5011
spark.broadcast.port 5021
spark.replClassServer.port 5031
spark.blockManager.port 5041
spark.executor.port 5051
```

Author: Andrew Or <andrewor14@gmail.com>
Author: Andrew Ash <andrew@andrewash.com>

Closes apache#1777 from andrewor14/configure-ports and squashes the following commits:

621267b [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
8a6b820 [Andrew Or] Use a random UI port during tests
7da0493 [Andrew Or] Fix tests
523c30e [Andrew Or] Add test for isBindCollision
b97b02a [Andrew Or] Minor fixes
c22ad00 [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
93d359f [Andrew Or] Executors connect to wrong port when collision occurs
d502e5f [Andrew Or] Handle port collisions when creating Akka systems
a2dd05c [Andrew Or] Patrick's comment nit
86461e2 [Andrew Or] Remove spark.executor.env.port and spark.standalone.client.port
1d2d5c6 [Andrew Or] Fix ports for standalone cluster mode
cb3be88 [Andrew Or] Various doc fixes (broken link, format etc.)
e837cde [Andrew Or] Remove outdated TODOs
bfbab28 [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
de1b207 [Andrew Or] Update docs to reflect new ports
b565079 [Andrew Or] Add spark.ports.maxRetries
2551eb2 [Andrew Or] Remove spark.worker.watcher.port
151327a [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
9868358 [Andrew Or] Add a few miscellaneous ports
6016e77 [Andrew Or] Add spark.executor.port
8d836e6 [Andrew Or] Also document SPARK_{MASTER/WORKER}_WEBUI_PORT
4d9e6f3 [Andrew Or] Fix super subtle bug
3f8e51b [Andrew Or] Correct erroneous docs...
e111d08 [Andrew Or] Add names for UI services
470f38c [Andrew Or] Special case non-"Address already in use" exceptions
1d7e408 [Andrew Or] Treat 0 ports specially + return correct ConnectionManager port
ba32280 [Andrew Or] Minor fixes
6b550b0 [Andrew Or] Assorted fixes
73fbe89 [Andrew Or] Move start service logic to Utils
ec676f4 [Andrew Or] Merge branch 'SPARK-2157' of github.com:ash211/spark into configure-ports
038a579 [Andrew Ash] Trust the server start function to report the port the service started on
7c5bdc4 [Andrew Ash] Fix style issue
0347aef [Andrew Ash] Unify port fallback logic to a single place
24a4c32 [Andrew Ash] Remove type on val to match surrounding style
9e4ad96 [Andrew Ash] Reformat for style checker
5d84e0e [Andrew Ash] Document new port configuration options
066dc7a [Andrew Ash] Fix up HttpServer port increments
cad16da [Andrew Ash] Add fallover increment logic for HttpServer
c5a0568 [Andrew Ash] Fix ConnectionManager to retry with increment
b80d2fd [Andrew Ash] Make Spark's block manager port configurable
17c79bb [Andrew Ash] Add a configuration option for spark-shell's class server
f34115d [Andrew Ash] SPARK-1176 Add port configuration for HttpBroadcast
49ee29b [Andrew Ash] SPARK-1174 Add port configuration for HttpFileServer
1c0981a [Andrew Ash] Make port in HttpServer configurable
xiliu82 pushed a commit to xiliu82/spark that referenced this pull request Sep 4, 2014
This is failing my tests in apache#1777. @tdas

Author: Andrew Or <andrewor14@gmail.com>

Closes apache#1803 from andrewor14/fix-flaky-streaming-test and squashes the following commits:

ea11a03 [Andrew Or] Catch all exceptions caused by BindExceptions
54a0ca0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-flaky-streaming-test
664095c [Andrew Or] Tone down bind exception message
af3ddc9 [Andrew Or] Handle port collisions in flume polling test
xiliu82 pushed a commit to xiliu82/spark that referenced this pull request Sep 4, 2014
As of apache#1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001.
```
14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001.
```
This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality.

Author: Andrew Or <andrewor14@gmail.com>

Closes apache#1810 from andrewor14/service-name and squashes the following commits:

8c459ed [Andrew Or] Use a common variable for driver/executor actor system names
3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor
921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name
c8c6a62 [Andrew Or] Do not include hyphens in actor name
1c1b42e [Andrew Or] Avoid spaces in akka system name
f644b55 [Andrew Or] Use more specific service name
asfgit pushed a commit that referenced this pull request Jan 13, 2015
https://issues.apache.org/jira/browse/SPARK-5006

I think the issue is produced in #1777.

Not digging mesos's backend yet. Maybe should add same logic either.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits:

8cdf96d [WangTao] indent thing
2d86d65 [WangTaoTheTonic] fix line length
7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor
61a370d [WangTaoTheTonic] some minor fixes
bc6e1ec [WangTaoTheTonic] rebase
67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments
f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg
29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable
396c226 [WangTaoTheTonic] make the grammar more like scala
191face [WangTaoTheTonic] invalid value name
62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work
asfgit pushed a commit that referenced this pull request Jan 21, 2015
https://issues.apache.org/jira/browse/SPARK-5006

I think the issue is produced in #1777.

Not digging mesos's backend yet. Maybe should add same logic either.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits:

8cdf96d [WangTao] indent thing
2d86d65 [WangTaoTheTonic] fix line length
7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor
61a370d [WangTaoTheTonic] some minor fixes
bc6e1ec [WangTaoTheTonic] rebase
67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments
f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg
29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable
396c226 [WangTaoTheTonic] make the grammar more like scala
191face [WangTaoTheTonic] invalid value name
62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work

Conflicts:
	external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
ckadner pushed a commit to ckadner/bahir_from_spark_8301fad that referenced this pull request Jun 6, 2016
https://issues.apache.org/jira/browse/SPARK-5006

I think the issue is produced in apache/spark#1777.

Not digging mesos's backend yet. Maybe should add same logic either.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits:

8cdf96d [WangTao] indent thing
2d86d65 [WangTaoTheTonic] fix line length
7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor
61a370d [WangTaoTheTonic] some minor fixes
bc6e1ec [WangTaoTheTonic] rebase
67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments
f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg
29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable
396c226 [WangTaoTheTonic] make the grammar more like scala
191face [WangTaoTheTonic] invalid value name
62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants