From 2a6213adb33dfc4a1e8bb28d66aeb8321040e05e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 5 Feb 2015 13:39:30 -0800 Subject: [PATCH] Update API names. --- .../org/apache/spark/sql/DataFrame.scala | 32 ++++++---- .../org/apache/spark/sql/DataFrameImpl.scala | 19 ++++-- .../org/apache/spark/sql/SQLContext.scala | 59 +++++++++++++++++++ 3 files changed, 94 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 15b6344948961..5d8ed5779f402 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -546,25 +546,24 @@ trait DataFrame extends RDDApi[Row] { /** * :: Experimental :: - * Creates a table from the the contents of this DataFrame based on a given data source - * and a set of options. + * Creates a table from the the contents of this DataFrame based on a set of options. + * It will use the default data source configured in `spark.sql.source.default`. * This will fail if the table already exists. * * Note that this currently only works with DataFrames that are created from a HiveContext as * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then - * be the target of an `insertInto`. + * be the target of an insertInto`. */ @Experimental def saveAsTable( tableName: String, - dataSourceName: String, - options: (String, String)*): Unit + options: java.util.Map[String, String]): Unit /** * :: Experimental :: - * Creates a table from the the contents of this DataFrame based on a set of options. - * It will use the default data source configured in spark.sql.source.default. + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source and a set of options. * This will fail if the table already exists. * * Note that this currently only works with DataFrames that are created from a HiveContext as @@ -575,12 +574,14 @@ trait DataFrame extends RDDApi[Row] { @Experimental def saveAsTable( tableName: String, - options: java.util.Map[String, String]): Unit + dataSourceName: String, + options: (String, String)*): Unit /** * :: Experimental :: - * Creates a table from the the contents of this DataFrame based on a given data source and - * a set of options. This will fail if the table already exists. + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source and a set of options. + * This will fail if the table already exists. * * Note that this currently only works with DataFrames that are created from a HiveContext as * there is no notion of a persisted catalog in a standard SQL context. Instead you can write @@ -625,6 +626,17 @@ trait DataFrame extends RDDApi[Row] { @Experimental def save(path: String, dataSourceName: String, options: (String, String)*): Unit + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path based on the given data source + * and a set of options. + */ + @Experimental + def save( + path: String, + dataSourceName: String, + options: java.util.Map[String, String]): Unit + /** * :: Experimental :: * Saves the contents of this DataFrame based on the given data source and a set of options. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 839f9567d9d1f..b77e3d0546641 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -311,16 +311,16 @@ private[sql] class DataFrameImpl protected[sql]( override def saveAsTable( tableName: String, - dataSourceName: String, - options: (String, String)*): Unit = { - saveAsTable(tableName, dataSourceName, Map.empty[String, String]) + options: java.util.Map[String, String]): Unit = { + val dataSourceName = sqlContext.conf.defaultDataSourceName + saveAsTable(tableName, dataSourceName, options) } override def saveAsTable( tableName: String, - options: java.util.Map[String, String]): Unit = { - val dataSourceName = sqlContext.conf.defaultDataSourceName - saveAsTable(tableName, dataSourceName, options) + dataSourceName: String, + options: (String, String)*): Unit = { + saveAsTable(tableName, dataSourceName, Map.empty[String, String]) } override def saveAsTable( @@ -360,6 +360,13 @@ private[sql] class DataFrameImpl protected[sql]( save(dataSourceName, "path" -> path, options:_*) } + override def save( + path: String, + dataSourceName: String, + options: java.util.Map[String, String]): Unit = { + save(path, dataSourceName, options.toSeq:_*) + } + override def save( dataSourceName: String, option: (String, String), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c875798153b6b..e760c14139fe7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -420,6 +420,19 @@ class SQLContext(@transient val sparkContext: SparkContext) load(dataSourceName, ("path", path), options:_*) } + /** + * :: Experimental :: + * Loads a dataset from the given path as a DataFrame based on a given data source and + * a set of options. + */ + @Experimental + def load( + path: String, + dataSourceName: String, + options: java.util.Map[String, String]): DataFrame = { + load(path, dataSourceName, options.toSeq:_*) + } + /** * :: Experimental :: * Loads a dataset based on a given data source and a set of options. @@ -455,6 +468,52 @@ class SQLContext(@transient val sparkContext: SparkContext) DataFrame(this, LogicalRelation(resolved.relation)) } + @Experimental + def createDataSource(tableName: String, options: (String, String)*): Unit = { + val dataSourceName = conf.defaultDataSourceName + createDataSource(tableName, dataSourceName, options.toMap) + } + + @Experimental + def createDataSource( + tableName: String, + options: java.util.Map[String, String]): Unit = { + val dataSourceName = conf.defaultDataSourceName + createDataSource(tableName, dataSourceName, options.toMap) + } + + @Experimental + def createDataSource( + tableName: String, + dataSourceName: String, + options: (String, String)*): Unit = { + createDataSource(tableName, dataSourceName, options.toMap) + } + + @Experimental + def createDataSource( + tableName: String, + dataSourceName: String, + options: java.util.Map[String, String]): Unit = { + createDataSource(tableName, dataSourceName, options.toMap) + } + + @Experimental + def createDataSource( + tableName: String, + dataSourceName: String, + options: Map[String, String]): Unit = { + val cmd = + CreateTableUsing( + tableName, + userSpecifiedSchema = None, + dataSourceName, + temporary = false, + options, + allowExisting = false) + executePlan(cmd).toRdd + } + /** * :: Experimental :: * Construct an RDD representing the database table accessible via JDBC URL