follow pep8 None should be compared using is or is not
http://legacy.python.org/dev/peps/pep-0008/ ## Programming Recommendations - Comparisons to singletons like None should always be done with is or is not, never the equality operators. Author: Ken Takagiwa <ken@Kens-MacBook-Pro.local> Closes #1422 from giwa/apache_master and squashes the following commits: 7b361f3 [Ken Takagiwa] follow pep8 None should be checked using is or is not
This commit is contained in:
parent
9c12de5092
commit
563acf5edf
|
@ -243,10 +243,10 @@ class CloudPickler(pickle.Pickler):
|
||||||
# if func is lambda, def'ed at prompt, is in main, or is nested, then
|
# if func is lambda, def'ed at prompt, is in main, or is nested, then
|
||||||
# we'll pickle the actual function object rather than simply saving a
|
# we'll pickle the actual function object rather than simply saving a
|
||||||
# reference (as is done in default pickler), via save_function_tuple.
|
# reference (as is done in default pickler), via save_function_tuple.
|
||||||
if islambda(obj) or obj.func_code.co_filename == '<stdin>' or themodule == None:
|
if islambda(obj) or obj.func_code.co_filename == '<stdin>' or themodule is None:
|
||||||
#Force server to import modules that have been imported in main
|
#Force server to import modules that have been imported in main
|
||||||
modList = None
|
modList = None
|
||||||
if themodule == None and not self.savedForceImports:
|
if themodule is None and not self.savedForceImports:
|
||||||
mainmod = sys.modules['__main__']
|
mainmod = sys.modules['__main__']
|
||||||
if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'):
|
if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'):
|
||||||
modList = list(mainmod.___pyc_forcedImports__)
|
modList = list(mainmod.___pyc_forcedImports__)
|
||||||
|
|
|
@ -30,7 +30,7 @@ u'My app'
|
||||||
u'local'
|
u'local'
|
||||||
>>> sc.appName
|
>>> sc.appName
|
||||||
u'My app'
|
u'My app'
|
||||||
>>> sc.sparkHome == None
|
>>> sc.sparkHome is None
|
||||||
True
|
True
|
||||||
|
|
||||||
>>> conf = SparkConf(loadDefaults=False)
|
>>> conf = SparkConf(loadDefaults=False)
|
||||||
|
@ -116,7 +116,7 @@ class SparkConf(object):
|
||||||
|
|
||||||
def setExecutorEnv(self, key=None, value=None, pairs=None):
|
def setExecutorEnv(self, key=None, value=None, pairs=None):
|
||||||
"""Set an environment variable to be passed to executors."""
|
"""Set an environment variable to be passed to executors."""
|
||||||
if (key != None and pairs != None) or (key == None and pairs == None):
|
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")
|
raise Exception("Either pass one key-value pair or a list of pairs")
|
||||||
elif key != None:
|
elif key != None:
|
||||||
self._jconf.setExecutorEnv(key, value)
|
self._jconf.setExecutorEnv(key, value)
|
||||||
|
|
|
@ -82,7 +82,7 @@ class RDDSampler(object):
|
||||||
return (num_arrivals - 1)
|
return (num_arrivals - 1)
|
||||||
|
|
||||||
def shuffle(self, vals):
|
def shuffle(self, vals):
|
||||||
if self._random == None:
|
if self._random is None:
|
||||||
self.initRandomGenerator(0) # this should only ever called on the master so
|
self.initRandomGenerator(0) # this should only ever called on the master so
|
||||||
# the split does not matter
|
# the split does not matter
|
||||||
|
|
||||||
|
|
|
@ -35,7 +35,7 @@ from pyspark.context import SparkContext
|
||||||
from pyspark.storagelevel import StorageLevel
|
from pyspark.storagelevel import StorageLevel
|
||||||
|
|
||||||
# this is the equivalent of ADD_JARS
|
# this is the equivalent of ADD_JARS
|
||||||
add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None
|
add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") is not None else None
|
||||||
|
|
||||||
if os.environ.get("SPARK_EXECUTOR_URI"):
|
if os.environ.get("SPARK_EXECUTOR_URI"):
|
||||||
SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])
|
SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])
|
||||||
|
@ -55,7 +55,7 @@ print("Using Python version %s (%s, %s)" % (
|
||||||
platform.python_build()[1]))
|
platform.python_build()[1]))
|
||||||
print("SparkContext available as sc.")
|
print("SparkContext available as sc.")
|
||||||
|
|
||||||
if add_files != None:
|
if add_files is not None:
|
||||||
print("Adding files: [%s]" % ", ".join(add_files))
|
print("Adding files: [%s]" % ", ".join(add_files))
|
||||||
|
|
||||||
# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
|
# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
|
||||||
|
|
Loading…
Reference in a new issue