|
19 | 19 | import sys |
20 | 20 | import warnings |
21 | 21 | from functools import reduce |
| 22 | +from threading import RLock |
22 | 23 |
|
23 | 24 | if sys.version >= '3': |
24 | 25 | basestring = unicode = str |
@@ -58,16 +59,98 @@ def toDF(self, schema=None, sampleRatio=None): |
58 | 59 |
|
59 | 60 |
|
60 | 61 | class SparkSession(object): |
61 | | - """Main entry point for Spark SQL functionality. |
| 62 | + """The entry point to programming Spark with the Dataset and DataFrame API. |
62 | 63 |
|
63 | 64 | A SparkSession can be used create :class:`DataFrame`, register :class:`DataFrame` as |
64 | 65 | tables, execute SQL over tables, cache tables, and read parquet files. |
| 66 | + To create a SparkSession, use the following builder pattern: |
| 67 | +
|
| 68 | + >>> spark = SparkSession.builder \ |
| 69 | + .master("local") \ |
| 70 | + .appName("Word Count") \ |
| 71 | + .config("spark.some.config.option", "some-value") \ |
| 72 | + .getOrCreate() |
65 | 73 |
|
66 | 74 | :param sparkContext: The :class:`SparkContext` backing this SparkSession. |
67 | 75 | :param jsparkSession: An optional JVM Scala SparkSession. If set, we do not instantiate a new |
68 | 76 | SparkSession in the JVM, instead we make all calls to this object. |
69 | 77 | """ |
70 | 78 |
|
| 79 | + class Builder(object): |
| 80 | + """Builder for :class:`SparkSession`. |
| 81 | + """ |
| 82 | + |
| 83 | + _lock = RLock() |
| 84 | + _options = {} |
| 85 | + |
| 86 | + @since(2.0) |
| 87 | + def config(self, key=None, value=None, conf=None): |
| 88 | + """Sets a config option. Options set using this method are automatically propagated to |
| 89 | + both :class:`SparkConf` and :class:`SparkSession`'s own configuration. |
| 90 | +
|
| 91 | + For an existing SparkConf, use `conf` parameter. |
| 92 | + >>> from pyspark.conf import SparkConf |
| 93 | + >>> SparkSession.builder.config(conf=SparkConf()) |
| 94 | + <pyspark.sql.session... |
| 95 | +
|
| 96 | + For a (key, value) pair, you can omit parameter names. |
| 97 | + >>> SparkSession.builder.config("spark.some.config.option", "some-value") |
| 98 | + <pyspark.sql.session... |
| 99 | +
|
| 100 | + :param key: a key name string for configuration property |
| 101 | + :param value: a value for configuration property |
| 102 | + :param conf: an instance of :class:`SparkConf` |
| 103 | + """ |
| 104 | + with self._lock: |
| 105 | + if conf is None: |
| 106 | + self._options[key] = str(value) |
| 107 | + else: |
| 108 | + for (k, v) in conf.getAll(): |
| 109 | + self._options[k] = v |
| 110 | + return self |
| 111 | + |
| 112 | + @since(2.0) |
| 113 | + def master(self, master): |
| 114 | + """Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" |
| 115 | + to run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone |
| 116 | + cluster. |
| 117 | +
|
| 118 | + :param master: a url for spark master |
| 119 | + """ |
| 120 | + return self.config("spark.master", master) |
| 121 | + |
| 122 | + @since(2.0) |
| 123 | + def appName(self, name): |
| 124 | + """Sets a name for the application, which will be shown in the Spark web UI. |
| 125 | +
|
| 126 | + :param name: an application name |
| 127 | + """ |
| 128 | + return self.config("spark.app.name", name) |
| 129 | + |
| 130 | + @since(2.0) |
| 131 | + def enableHiveSupport(self): |
| 132 | + """Enables Hive support, including connectivity to a persistent Hive metastore, support |
| 133 | + for Hive serdes, and Hive user-defined functions. |
| 134 | + """ |
| 135 | + return self.config("spark.sql.catalogImplementation", "hive") |
| 136 | + |
| 137 | + @since(2.0) |
| 138 | + def getOrCreate(self): |
| 139 | + """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a new |
| 140 | + one based on the options set in this builder. |
| 141 | + """ |
| 142 | + with self._lock: |
| 143 | + from pyspark.conf import SparkConf |
| 144 | + from pyspark.context import SparkContext |
| 145 | + from pyspark.sql.context import SQLContext |
| 146 | + sparkConf = SparkConf() |
| 147 | + for key, value in self._options.items(): |
| 148 | + sparkConf.set(key, value) |
| 149 | + sparkContext = SparkContext.getOrCreate(sparkConf) |
| 150 | + return SQLContext.getOrCreate(sparkContext).sparkSession |
| 151 | + |
| 152 | + builder = Builder() |
| 153 | + |
71 | 154 | _instantiatedContext = None |
72 | 155 |
|
73 | 156 | @ignore_unicode_prefix |
@@ -445,6 +528,12 @@ def read(self): |
445 | 528 | """ |
446 | 529 | return DataFrameReader(self._wrapped) |
447 | 530 |
|
| 531 | + @since(2.0) |
| 532 | + def stop(self): |
| 533 | + """Stop the underlying :class:`SparkContext`. |
| 534 | + """ |
| 535 | + self._sc.stop() |
| 536 | + |
448 | 537 |
|
449 | 538 | def _test(): |
450 | 539 | import os |
|
0 commit comments