aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/catalog.py
diff options
context:
space:
mode:
Diffstat (limited to 'python/pyspark/sql/catalog.py')
-rw-r--r--python/pyspark/sql/catalog.py27
1 files changed, 24 insertions, 3 deletions
diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py
index 30c7a3fe4f..253a750629 100644
--- a/python/pyspark/sql/catalog.py
+++ b/python/pyspark/sql/catalog.py
@@ -15,6 +15,7 @@
# limitations under the License.
#
+import warnings
from collections import namedtuple
from pyspark import since
@@ -138,7 +139,27 @@ class Catalog(object):
@since(2.0)
def createExternalTable(self, tableName, path=None, source=None, schema=None, **options):
- """Creates an external table based on the dataset in a data source.
+ """Creates a table based on the dataset in a data source.
+
+ It returns the DataFrame associated with the external table.
+
+ The data source is specified by the ``source`` and a set of ``options``.
+ If ``source`` is not specified, the default data source configured by
+ ``spark.sql.sources.default`` will be used.
+
+ Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and
+ created external table.
+
+ :return: :class:`DataFrame`
+ """
+ warnings.warn(
+ "createExternalTable is deprecated since Spark 2.2, please use createTable instead.",
+ DeprecationWarning)
+ return self.createTable(tableName, path, source, schema, **options)
+
+ @since(2.2)
+ def createTable(self, tableName, path=None, source=None, schema=None, **options):
+ """Creates a table based on the dataset in a data source.
It returns the DataFrame associated with the external table.
@@ -157,12 +178,12 @@ class Catalog(object):
source = self._sparkSession.conf.get(
"spark.sql.sources.default", "org.apache.spark.sql.parquet")
if schema is None:
- df = self._jcatalog.createExternalTable(tableName, source, options)
+ df = self._jcatalog.createTable(tableName, source, options)
else:
if not isinstance(schema, StructType):
raise TypeError("schema should be StructType")
scala_datatype = self._jsparkSession.parseDataType(schema.json())
- df = self._jcatalog.createExternalTable(tableName, source, scala_datatype, options)
+ df = self._jcatalog.createTable(tableName, source, scala_datatype, options)
return DataFrame(df, self._sparkSession._wrapped)
@since(2.0)