aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/conf.py
blob: 924da3eecf214cc5abb45642f339dc1bf927ea07 (plain) (blame)
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
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
#
# 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.
#

"""
>>> from pyspark.conf import SparkConf
>>> from pyspark.context import SparkContext
>>> conf = SparkConf()
>>> conf.setMaster("local").setAppName("My app")
<pyspark.conf.SparkConf object at ...>
>>> conf.get("spark.master")
u'local'
>>> conf.get("spark.app.name")
u'My app'
>>> sc = SparkContext(conf=conf)
>>> sc.master
u'local'
>>> sc.appName
u'My app'
>>> sc.sparkHome is None
True

>>> conf = SparkConf(loadDefaults=False)
>>> conf.setSparkHome("/path")
<pyspark.conf.SparkConf object at ...>
>>> conf.get("spark.home")
u'/path'
>>> conf.setExecutorEnv("VAR1", "value1")
<pyspark.conf.SparkConf object at ...>
>>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")])
<pyspark.conf.SparkConf object at ...>
>>> conf.get("spark.executorEnv.VAR1")
u'value1'
>>> print(conf.toDebugString())
spark.executorEnv.VAR1=value1
spark.executorEnv.VAR3=value3
spark.executorEnv.VAR4=value4
spark.home=/path
>>> sorted(conf.getAll(), key=lambda p: p[0])
[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), \
(u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')]
"""

__all__ = ['SparkConf']

import sys
import re

if sys.version > '3':
    unicode = str
    __doc__ = re.sub(r"(\W|^)[uU](['])", r'\1\2', __doc__)


class SparkConf(object):

    """
    Configuration for a Spark application. Used to set various Spark
    parameters as key-value pairs.

    Most of the time, you would create a SparkConf object with
    C{SparkConf()}, which will load values from C{spark.*} Java system
    properties as well. In this case, any parameters you set directly on
    the C{SparkConf} object take priority over system properties.

    For unit tests, you can also call C{SparkConf(false)} to skip
    loading external settings and get the same configuration no matter
    what the system properties are.

    All setter methods in this class support chaining. For example,
    you can write C{conf.setMaster("local").setAppName("My app")}.

    Note that once a SparkConf object is passed to Spark, it is cloned
    and can no longer be modified by the user.
    """

    def __init__(self, loadDefaults=True, _jvm=None, _jconf=None):
        """
        Create a new Spark configuration.

        :param loadDefaults: whether to load values from Java system
               properties (True by default)
        :param _jvm: internal parameter used to pass a handle to the
               Java VM; does not need to be set by users
        :param _jconf: Optionally pass in an existing SparkConf handle
               to use its parameters
        """
        if _jconf:
            self._jconf = _jconf
        else:
            from pyspark.context import SparkContext
            SparkContext._ensure_initialized()
            _jvm = _jvm or SparkContext._jvm
            self._jconf = _jvm.SparkConf(loadDefaults)

    def set(self, key, value):
        """Set a configuration property."""
        self._jconf.set(key, unicode(value))
        return self

    def setIfMissing(self, key, value):
        """Set a configuration property, if not already set."""
        if self.get(key) is None:
            self.set(key, value)
        return self

    def setMaster(self, value):
        """Set master URL to connect to."""
        self._jconf.setMaster(value)
        return self

    def setAppName(self, value):
        """Set application name."""
        self._jconf.setAppName(value)
        return self

    def setSparkHome(self, value):
        """Set path where Spark is installed on worker nodes."""
        self._jconf.setSparkHome(value)
        return self

    def setExecutorEnv(self, key=None, value=None, pairs=None):
        """Set an environment variable to be passed to executors."""
        if (key is not None and pairs is not None) or (key is None and pairs is None):
            raise Exception("Either pass one key-value pair or a list of pairs")
        elif key is not None:
            self._jconf.setExecutorEnv(key, value)
        elif pairs is not None:
            for (k, v) in pairs:
                self._jconf.setExecutorEnv(k, v)
        return self

    def setAll(self, pairs):
        """
        Set multiple parameters, passed as a list of key-value pairs.

        :param pairs: list of key-value pairs to set
        """
        for (k, v) in pairs:
            self._jconf.set(k, v)
        return self

    def get(self, key, defaultValue=None):
        """Get the configured value for some key, or return a default otherwise."""
        if defaultValue is None:   # Py4J doesn't call the right get() if we pass None
            if not self._jconf.contains(key):
                return None
            return self._jconf.get(key)
        else:
            return self._jconf.get(key, defaultValue)

    def getAll(self):
        """Get all values as a list of key-value pairs."""
        pairs = []
        for elem in self._jconf.getAll():
            pairs.append((elem._1(), elem._2()))
        return pairs

    def contains(self, key):
        """Does this configuration contain a given key?"""
        return self._jconf.contains(key)

    def toDebugString(self):
        """
        Returns a printable version of the configuration, as a list of
        key=value pairs, one per line.
        """
        return self._jconf.toDebugString()


def _test():
    import doctest
    (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS)
    if failure_count:
        exit(-1)


if __name__ == "__main__":
    _test()