aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark
diff options
context:
space:
mode:
authorAhir Reddy <ahirreddy@gmail.com>2014-08-11 20:06:06 -0700
committerMichael Armbrust <michael@databricks.com>2014-08-11 20:06:25 -0700
commit8cb4e5b47b9b871bf4c0d93d0a747e55f66ca0ec (patch)
tree5bc7dff851dc97a3b9240ed7861516bb091ce696 /python/pyspark
parent8f6e2e9df41e7de22b1d1cbd524e20881f861dd0 (diff)
downloadspark-8cb4e5b47b9b871bf4c0d93d0a747e55f66ca0ec.tar.gz
spark-8cb4e5b47b9b871bf4c0d93d0a747e55f66ca0ec.tar.bz2
spark-8cb4e5b47b9b871bf4c0d93d0a747e55f66ca0ec.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 (cherry picked from commit 490ecfa20327a636289321ea447722aa32b81657) Signed-off-by: Michael Armbrust <michael@databricks.com>
Diffstat (limited to 'python/pyspark')
-rw-r--r--python/pyspark/sql.py14
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: