diff options
author | Ahir Reddy <ahirreddy@gmail.com> | 2014-08-11 20:06:06 -0700 |
---|---|---|
committer | Michael Armbrust <michael@databricks.com> | 2014-08-11 20:06:06 -0700 |
commit | 490ecfa20327a636289321ea447722aa32b81657 (patch) | |
tree | ad6e24977f71e128ec5caab222440e0c8bb6c5bd /python | |
parent | 6fab941b65f0cb6c9b32e0f8290d76889cda6a87 (diff) | |
download | spark-490ecfa20327a636289321ea447722aa32b81657.tar.gz spark-490ecfa20327a636289321ea447722aa32b81657.tar.bz2 spark-490ecfa20327a636289321ea447722aa32b81657.zip |
[SPARK-2844][SQL] Correctly set JVM HiveContext if it is passed into Python HiveContext constructor
https://issues.apache.org/jira/browse/SPARK-2844
Author: Ahir Reddy <ahirreddy@gmail.com>
Closes #1768 from ahirreddy/python-hive-context-fix and squashes the following commits:
7972d3b [Ahir Reddy] Correctly set JVM HiveContext if it is passed into Python HiveContext constructor
Diffstat (limited to 'python')
-rw-r--r-- | python/pyspark/sql.py | 14 |
1 files changed, 14 insertions, 0 deletions
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 950e275adb..36040463e6 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -912,6 +912,8 @@ class SQLContext: """Create a new SQLContext. @param sparkContext: The SparkContext to wrap. + @param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new + SQLContext in the JVM, instead we make all calls to this object. >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.inferSchema(srdd) # doctest: +IGNORE_EXCEPTION_DETAIL @@ -1315,6 +1317,18 @@ class HiveContext(SQLContext): It supports running both SQL and HiveQL commands. """ + def __init__(self, sparkContext, hiveContext=None): + """Create a new HiveContext. + + @param sparkContext: The SparkContext to wrap. + @param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new + HiveContext in the JVM, instead we make all calls to this object. + """ + SQLContext.__init__(self, sparkContext) + + if hiveContext: + self._scala_HiveContext = hiveContext + @property def _ssql_ctx(self): try: |