diff options
author | Yin Huai <yhuai@databricks.com> | 2015-05-23 09:48:20 -0700 |
---|---|---|
committer | Yin Huai <yhuai@databricks.com> | 2015-05-23 09:48:20 -0700 |
commit | 2b7e63585d61be2dab78b70af3867cda3983d5b1 (patch) | |
tree | 845de9eb688867035c27923f690d9151f142dbe3 /sql/core | |
parent | a4df0f2d84ff24318b139db534521141d9d4d593 (diff) | |
download | spark-2b7e63585d61be2dab78b70af3867cda3983d5b1.tar.gz spark-2b7e63585d61be2dab78b70af3867cda3983d5b1.tar.bz2 spark-2b7e63585d61be2dab78b70af3867cda3983d5b1.zip |
[SPARK-7654] [SQL] Move insertInto into reader/writer interface.
This one continues the work of https://github.com/apache/spark/pull/6216.
Author: Yin Huai <yhuai@databricks.com>
Author: Reynold Xin <rxin@databricks.com>
Closes #6366 from yhuai/insert and squashes the following commits:
3d717fb [Yin Huai] Use insertInto to handle the casue when table exists and Append is used for saveAsTable.
56d2540 [Yin Huai] Add PreWriteCheck to HiveContext's analyzer.
c636e35 [Yin Huai] Remove unnecessary empty lines.
cf83837 [Yin Huai] Move insertInto to write. Also, remove the partition columns from InsertIntoHadoopFsRelation.
0841a54 [Reynold Xin] Removed experimental tag for deprecated methods.
33ed8ef [Reynold Xin] [SPARK-7654][SQL] Move insertInto into reader/writer interface.
Diffstat (limited to 'sql/core')
8 files changed, 99 insertions, 66 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 3ec1c4a2f1..f968577bc5 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 @@ -1395,28 +1395,6 @@ class DataFrame private[sql]( def write: DataFrameWriter = new DataFrameWriter(this) /** - * :: Experimental :: - * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. - * @group output - * @since 1.3.0 - */ - @Experimental - def insertInto(tableName: String, overwrite: Boolean): Unit = { - sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite, ifNotExists = false)).toRdd - } - - /** - * :: Experimental :: - * Adds the rows from this RDD to the specified table. - * Throws an exception if the table already exists. - * @group output - * @since 1.3.0 - */ - @Experimental - def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false) - - /** * Returns the content of the [[DataFrame]] as a RDD of JSON strings. * @group rdd * @since 1.3.0 @@ -1551,13 +1529,7 @@ class DataFrame private[sql]( */ @deprecated("Use write.mode(mode).saveAsTable(tableName)", "1.4.0") def saveAsTable(tableName: String, mode: SaveMode): Unit = { - if (sqlContext.catalog.tableExists(Seq(tableName)) && mode == SaveMode.Append) { - // If table already exists and the save mode is Append, - // we will just call insertInto to append the contents of this DataFrame. - insertInto(tableName, overwrite = false) - } else { - write.mode(mode).saveAsTable(tableName) - } + write.mode(mode).saveAsTable(tableName) } /** @@ -1713,9 +1685,29 @@ class DataFrame private[sql]( write.format(source).mode(mode).options(options).save() } + + /** + * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. + * @group output + */ + @deprecated("Use write.mode(SaveMode.Append|SaveMode.Overwrite).saveAsTable(tableName)", "1.4.0") + def insertInto(tableName: String, overwrite: Boolean): Unit = { + write.mode(if (overwrite) SaveMode.Overwrite else SaveMode.Append).insertInto(tableName) + } + + /** + * Adds the rows from this RDD to the specified table. + * Throws an exception if the table already exists. + * @group output + */ + @deprecated("Use write.mode(SaveMode.Append).saveAsTable(tableName)", "1.4.0") + def insertInto(tableName: String): Unit = { + write.mode(SaveMode.Append).insertInto(tableName) + } + //////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////// - // End of eeprecated methods + // End of deprecated methods //////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 381c10f48f..b44d4c86ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -95,20 +95,6 @@ class DataFrameReader private[sql](sqlContext: SQLContext) { } /** - * Specifies the input partitioning. If specified, the underlying data source does not need to - * discover the data partitioning scheme, and thus can speed up very large inputs. - * - * This is only applicable for Parquet at the moment. - * - * @since 1.4.0 - */ - @scala.annotation.varargs - def partitionBy(colNames: String*): DataFrameReader = { - this.partitioningColumns = Option(colNames) - this - } - - /** * Loads input in as a [[DataFrame]], for data sources that require a path (e.g. data backed by * a local or distributed file system). * @@ -128,7 +114,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) { val resolved = ResolvedDataSource( sqlContext, userSpecifiedSchema = userSpecifiedSchema, - partitionColumns = partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), + partitionColumns = Array.empty[String], provider = source, options = extraOptions.toMap) DataFrame(sqlContext, LogicalRelation(resolved.relation)) @@ -300,6 +286,4 @@ class DataFrameReader private[sql](sqlContext: SQLContext) { private var extraOptions = new scala.collection.mutable.HashMap[String, String] - private var partitioningColumns: Option[Seq[String]] = None - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index f2e721d4db..5548b26cb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql import java.util.Properties import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.jdbc.{JDBCWriteDetails, JdbcUtils} import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} @@ -149,21 +151,65 @@ final class DataFrameWriter private[sql](df: DataFrame) { } /** + * Inserts the content of the [[DataFrame]] to the specified table. It requires that + * the schema of the [[DataFrame]] is the same as the schema of the table. + * + * Because it inserts data to an existing table, format or options will be ignored. + * + * @since 1.4.0 + */ + def insertInto(tableName: String): Unit = { + val partitions = + partitioningColumns.map(_.map(col => col -> (None: Option[String])).toMap) + val overwrite = (mode == SaveMode.Overwrite) + df.sqlContext.executePlan(InsertIntoTable( + UnresolvedRelation(Seq(tableName)), + partitions.getOrElse(Map.empty[String, Option[String]]), + df.logicalPlan, + overwrite, + ifNotExists = false)).toRdd + } + + /** * Saves the content of the [[DataFrame]] as the specified table. * + * In the case the table already exists, behavior of this function depends on the + * save mode, specified by the `mode` function (default to throwing an exception). + * When `mode` is `Overwrite`, the schema of the [[DataFrame]] does not need to be + * the same as that of the existing table. + * When `mode` is `Append`, the schema of the [[DataFrame]] need to be + * the same as that of the existing table, and format or options will be ignored. + * * @since 1.4.0 */ def saveAsTable(tableName: String): Unit = { - val cmd = - CreateTableUsingAsSelect( - tableName, - source, - temporary = false, - partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), - mode, - extraOptions.toMap, - df.logicalPlan) - df.sqlContext.executePlan(cmd).toRdd + if (df.sqlContext.catalog.tableExists(tableName :: Nil) && mode != SaveMode.Overwrite) { + mode match { + case SaveMode.Ignore => + // Do nothing + + case SaveMode.ErrorIfExists => + throw new AnalysisException(s"Table $tableName already exists.") + + case SaveMode.Append => + // If it is Append, we just ask insertInto to handle it. We will not use insertInto + // to handle saveAsTable with Overwrite because saveAsTable can change the schema of + // the table. But, insertInto with Overwrite requires the schema of data be the same + // the schema of the table. + insertInto(tableName) + } + } else { + val cmd = + CreateTableUsingAsSelect( + tableName, + source, + temporary = false, + partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), + mode, + extraOptions.toMap, + df.logicalPlan) + df.sqlContext.executePlan(cmd).toRdd + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index c45c431438..70a220cc43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -129,7 +129,7 @@ private[parquet] object RowReadSupport { } /** - * A `parquet.hadoop.api.WriteSupport` for Row ojects. + * A `parquet.hadoop.api.WriteSupport` for Row objects. */ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index c03649d00b..dacd967cff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -105,10 +105,9 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: HadoopFsRelation), part, query, overwrite, false) if part.isEmpty => + l @ LogicalRelation(t: HadoopFsRelation), part, query, overwrite, false) => val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append - execution.ExecutedCommand( - InsertIntoHadoopFsRelation(t, query, Array.empty[String], mode)) :: Nil + execution.ExecutedCommand(InsertIntoHadoopFsRelation(t, query, mode)) :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 498f7538d4..c3674a8c76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -61,7 +61,6 @@ private[sql] case class InsertIntoDataSource( private[sql] case class InsertIntoHadoopFsRelation( @transient relation: HadoopFsRelation, @transient query: LogicalPlan, - partitionColumns: Array[String], mode: SaveMode) extends RunnableCommand { @@ -100,6 +99,7 @@ private[sql] case class InsertIntoHadoopFsRelation( relation.schema, needsConversion = false) + val partitionColumns = relation.partitionColumns.fieldNames if (partitionColumns.isEmpty) { insert(new DefaultWriterContainer(relation, job), df) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 5e723122ee..ca30b8e746 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -335,7 +335,6 @@ private[sql] object ResolvedDataSource { InsertIntoHadoopFsRelation( r, project, - partitionColumns.toArray, mode)).toRdd r case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index ab33125b74..a3fd7f13b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -35,9 +35,9 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - // We are inserting into an InsertableRelation. + // We are inserting into an InsertableRelation or HadoopFsRelation. case i @ InsertIntoTable( - l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite, ifNotExists) => { + l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation), _, child, _, _) => { // First, make sure the data to be inserted have the same number of fields with the // schema of the relation. if (l.output.size != child.output.size) { @@ -101,7 +101,20 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } } - case logical.InsertIntoTable(LogicalRelation(_: HadoopFsRelation), _, _, _, _) => // OK + case logical.InsertIntoTable(LogicalRelation(r: HadoopFsRelation), part, _, _, _) => + // We need to make sure the partition columns specified by users do match partition + // columns of the relation. + val existingPartitionColumns = r.partitionColumns.fieldNames.toSet + val specifiedPartitionColumns = part.keySet + if (existingPartitionColumns != specifiedPartitionColumns) { + failAnalysis(s"Specified partition columns " + + s"(${specifiedPartitionColumns.mkString(", ")}) " + + s"do not match the partition columns of the table. Please use " + + s"(${existingPartitionColumns.mkString(", ")}) as the partition columns.") + } else { + // OK + } + case logical.InsertIntoTable(l: LogicalRelation, _, _, _, _) => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") |