Skip to content

Commit e50d94b

Browse files
committed
SPARK-24923: Add v2 CTAS and RTAS support.
This uses the catalog API introduced in SPARK-24252 to implement CTAS and RTAS plans.
1 parent 622180a commit e50d94b

File tree

12 files changed

+420
-95
lines changed

12 files changed

+420
-95
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NamedRelation.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,11 @@
1717

1818
package org.apache.spark.sql.catalyst.analysis
1919

20+
import org.apache.spark.sql.catalyst.expressions.AttributeReference
2021
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2122

2223
trait NamedRelation extends LogicalPlan {
2324
def name: String
25+
26+
def output: Seq[AttributeReference]
2427
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala

Lines changed: 33 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,8 @@
1717

1818
package org.apache.spark.sql.catalyst.plans.logical
1919

20-
import org.apache.spark.sql.catalyst.{AliasIdentifier}
20+
import org.apache.spark.sql.catalog.v2.{PartitionTransform, TableCatalog}
21+
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
2122
import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation}
2223
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
2324
import org.apache.spark.sql.catalyst.expressions._
@@ -384,6 +385,37 @@ object AppendData {
384385
}
385386
}
386387

388+
/**
389+
* Create a new table from a select query.
390+
*/
391+
case class CreateTableAsSelect(
392+
catalog: TableCatalog,
393+
table: TableIdentifier,
394+
partitioning: Seq[PartitionTransform],
395+
query: LogicalPlan,
396+
writeOptions: Map[String, String],
397+
ignoreIfExists: Boolean) extends LogicalPlan {
398+
399+
override def children: Seq[LogicalPlan] = Seq(query)
400+
override def output: Seq[Attribute] = Seq.empty
401+
override lazy val resolved = true
402+
}
403+
404+
/**
405+
* Replace a table with the results of a select query.
406+
*/
407+
case class ReplaceTableAsSelect(
408+
catalog: TableCatalog,
409+
table: TableIdentifier,
410+
partitioning: Seq[PartitionTransform],
411+
query: LogicalPlan,
412+
writeOptions: Map[String, String]) extends LogicalPlan {
413+
414+
override def children: Seq[LogicalPlan] = Seq(query)
415+
override def output: Seq[Attribute] = Seq.empty
416+
override lazy val resolved = true
417+
}
418+
387419
/**
388420
* Insert some data into a table. Note that this plan is unresolved and has to be replaced by the
389421
* concrete implementations during analysis.

sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@
2727
* provide data reading ability and scan the data from the data source.
2828
*/
2929
@InterfaceStability.Evolving
30-
public interface ReadSupport extends DataSourceV2 {
30+
public interface ReadSupport {
3131

3232
/**
3333
* Creates a {@link DataSourceReader} to scan the data from this data source.

sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@
2929
* provide data writing ability and save the data to the data source.
3030
*/
3131
@InterfaceStability.Evolving
32-
public interface WriteSupport extends DataSourceV2 {
32+
public interface WriteSupport {
3333

3434
/**
3535
* Creates an optional {@link DataSourceWriter} to save the data to this data source. Data

sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._
3737
import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource
3838
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
3939
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils
40-
import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport}
40+
import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, DataSourceV2Implicits, ReadSupport}
4141
import org.apache.spark.sql.types.{StringType, StructType}
4242
import org.apache.spark.unsafe.types.UTF8String
4343

@@ -191,6 +191,21 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
191191
"read files of Hive data source directly.")
192192
}
193193

194+
import DataSourceV2Implicits._
195+
196+
extraOptions.get("catalog") match {
197+
case Some(catalogName) if extraOptions.get(DataSourceOptions.TABLE_KEY).isDefined =>
198+
val catalog = sparkSession.catalog(catalogName).asTableCatalog
199+
val options = extraOptions.toMap
200+
val identifier = options.table.get
201+
202+
return Dataset.ofRows(sparkSession,
203+
DataSourceV2Relation.create(
204+
catalogName, identifier, catalog.loadTable(identifier), options))
205+
206+
case _ =>
207+
}
208+
194209
val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf)
195210
if (classOf[DataSourceV2].isAssignableFrom(cls)) {
196211
val ds = cls.newInstance().asInstanceOf[DataSourceV2]

sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala

Lines changed: 46 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import org.apache.spark.annotation.InterfaceStability
2525
import org.apache.spark.sql.catalyst.TableIdentifier
2626
import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedRelation}
2727
import org.apache.spark.sql.catalyst.catalog._
28-
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoTable, LogicalPlan}
28+
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, InsertIntoTable, LogicalPlan, ReplaceTableAsSelect}
2929
import org.apache.spark.sql.execution.SQLExecution
3030
import org.apache.spark.sql.execution.command.DDLUtils
3131
import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation}
@@ -236,6 +236,51 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
236236

237237
assertNotBucketed("save")
238238

239+
import DataSourceV2Implicits._
240+
241+
extraOptions.get("catalog") match {
242+
case Some(catalogName) if extraOptions.get(DataSourceOptions.TABLE_KEY).isDefined =>
243+
val catalog = df.sparkSession.catalog(catalogName).asTableCatalog
244+
val options = extraOptions.toMap
245+
val identifier = options.table.get
246+
val exists = catalog.tableExists(identifier)
247+
248+
(exists, mode) match {
249+
case (true, SaveMode.ErrorIfExists) =>
250+
throw new AnalysisException(s"Table already exists: ${identifier.quotedString}")
251+
252+
case (true, SaveMode.Overwrite) =>
253+
runCommand(df.sparkSession, "save") {
254+
ReplaceTableAsSelect(catalog, identifier, Seq.empty, df.logicalPlan, options)
255+
}
256+
257+
case (true, SaveMode.Append) =>
258+
val relation = DataSourceV2Relation.create(
259+
catalogName, identifier, catalog.loadTable(identifier), options)
260+
261+
runCommand(df.sparkSession, "save") {
262+
AppendData.byName(relation, df.logicalPlan)
263+
}
264+
265+
case (false, SaveMode.Append) =>
266+
throw new AnalysisException(s"Table does not exist: ${identifier.quotedString}")
267+
268+
case (false, SaveMode.ErrorIfExists) |
269+
(false, SaveMode.Ignore) |
270+
(false, SaveMode.Overwrite) =>
271+
272+
runCommand(df.sparkSession, "save") {
273+
CreateTableAsSelect(catalog, identifier, Seq.empty, df.logicalPlan, options,
274+
ignoreIfExists = mode == SaveMode.Ignore)
275+
}
276+
277+
case _ =>
278+
return // table exists and mode is ignore
279+
}
280+
281+
case _ =>
282+
}
283+
239284
val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf)
240285
if (classOf[DataSourceV2].isAssignableFrom(cls)) {
241286
val source = cls.newInstance().asInstanceOf[DataSourceV2]

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala

Lines changed: 53 additions & 62 deletions
Original file line numberDiff line numberDiff line change
@@ -17,20 +17,18 @@
1717

1818
package org.apache.spark.sql.execution.datasources.v2
1919

20-
import java.util.UUID
21-
22-
import scala.collection.JavaConverters._
23-
24-
import org.apache.spark.sql.{AnalysisException, SaveMode}
20+
import org.apache.spark.sql.catalog.v2.Table
2521
import org.apache.spark.sql.catalyst.TableIdentifier
2622
import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation}
2723
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
2824
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
2925
import org.apache.spark.sql.sources.DataSourceRegister
3026
import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, WriteSupport}
27+
import org.apache.spark.sql.sources.v2.DataSourceV2Implicits._
3128
import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsReportStatistics}
3229
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
3330
import org.apache.spark.sql.types.StructType
31+
import org.apache.spark.util.Utils
3432

3533
/**
3634
* A logical plan representing a data source v2 scan.
@@ -48,10 +46,10 @@ case class DataSourceV2Relation(
4846
userSpecifiedSchema: Option[StructType] = None)
4947
extends LeafNode with MultiInstanceRelation with NamedRelation with DataSourceV2StringFormat {
5048

51-
import DataSourceV2Relation._
49+
override def sourceName: String = source.name
5250

5351
override def name: String = {
54-
tableIdent.map(_.unquotedString).getOrElse(s"${source.name}:unknown")
52+
tableIdent.map(_.unquotedString).getOrElse(s"$sourceName:unknown")
5553
}
5654

5755
override def pushedFilters: Seq[Expression] = Seq.empty
@@ -62,7 +60,7 @@ case class DataSourceV2Relation(
6260

6361
def newWriter(): DataSourceWriter = source.createWriter(options, schema)
6462

65-
override def computeStats(): Statistics = newReader match {
63+
override def computeStats(): Statistics = newReader() match {
6664
case r: SupportsReportStatistics =>
6765
Statistics(sizeInBytes = r.getStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes))
6866
case _ =>
@@ -74,6 +72,43 @@ case class DataSourceV2Relation(
7472
}
7573
}
7674

75+
/**
76+
* A logical plan representing a data source v2 table.
77+
*
78+
* @param ident The table's TableIdentifier.
79+
* @param table The table.
80+
* @param output The output attributes of the table.
81+
* @param options The options for this scan or write.
82+
*/
83+
case class TableV2Relation(
84+
catalogName: String,
85+
ident: TableIdentifier,
86+
table: Table,
87+
output: Seq[AttributeReference],
88+
options: Map[String, String])
89+
extends LeafNode with MultiInstanceRelation with NamedRelation {
90+
91+
import org.apache.spark.sql.sources.v2.DataSourceV2Implicits._
92+
93+
override def name: String = ident.unquotedString
94+
95+
override def simpleString: String =
96+
s"RelationV2 $name ${Utils.truncatedString(output, "[", ", ", "]")}"
97+
98+
def newReader(): DataSourceReader = table.createReader(options)
99+
100+
override def computeStats(): Statistics = newReader() match {
101+
case r: SupportsReportStatistics =>
102+
Statistics(sizeInBytes = r.getStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes))
103+
case _ =>
104+
Statistics(sizeInBytes = conf.defaultSizeInBytes)
105+
}
106+
107+
override def newInstance(): TableV2Relation = {
108+
copy(output = output.map(_.newInstance()))
109+
}
110+
}
111+
77112
/**
78113
* A specialization of [[DataSourceV2Relation]] with the streaming bit set to true.
79114
*
@@ -88,6 +123,8 @@ case class StreamingDataSourceV2Relation(
88123
reader: DataSourceReader)
89124
extends LeafNode with MultiInstanceRelation with DataSourceV2StringFormat {
90125

126+
override def sourceName: String = source.name
127+
91128
override def isStreaming: Boolean = true
92129

93130
override def simpleString: String = "Streaming RelationV2 " + metadataString
@@ -116,68 +153,22 @@ case class StreamingDataSourceV2Relation(
116153
}
117154

118155
object DataSourceV2Relation {
119-
private implicit class SourceHelpers(source: DataSourceV2) {
120-
def asReadSupport: ReadSupport = {
121-
source match {
122-
case support: ReadSupport =>
123-
support
124-
case _ =>
125-
throw new AnalysisException(s"Data source is not readable: $name")
126-
}
127-
}
128-
129-
def asWriteSupport: WriteSupport = {
130-
source match {
131-
case support: WriteSupport =>
132-
support
133-
case _ =>
134-
throw new AnalysisException(s"Data source is not writable: $name")
135-
}
136-
}
137-
138-
def name: String = {
139-
source match {
140-
case registered: DataSourceRegister =>
141-
registered.shortName()
142-
case _ =>
143-
source.getClass.getSimpleName
144-
}
145-
}
146-
147-
def createReader(
148-
options: Map[String, String],
149-
userSpecifiedSchema: Option[StructType]): DataSourceReader = {
150-
val v2Options = new DataSourceOptions(options.asJava)
151-
userSpecifiedSchema match {
152-
case Some(s) =>
153-
asReadSupport.createReader(s, v2Options)
154-
case _ =>
155-
asReadSupport.createReader(v2Options)
156-
}
157-
}
158-
159-
def createWriter(
160-
options: Map[String, String],
161-
schema: StructType): DataSourceWriter = {
162-
val v2Options = new DataSourceOptions(options.asJava)
163-
asWriteSupport.createWriter(UUID.randomUUID.toString, schema, SaveMode.Append, v2Options).get
164-
}
165-
}
166-
167156
def create(
168157
source: DataSourceV2,
169158
options: Map[String, String],
170159
tableIdent: Option[TableIdentifier] = None,
171-
userSpecifiedSchema: Option[StructType] = None): DataSourceV2Relation = {
160+
userSpecifiedSchema: Option[StructType] = None): NamedRelation = {
172161
val reader = source.createReader(options, userSpecifiedSchema)
173-
val ident = tableIdent.orElse(tableFromOptions(options))
162+
val ident = tableIdent.orElse(options.table)
174163
DataSourceV2Relation(
175164
source, reader.readSchema().toAttributes, options, ident, userSpecifiedSchema)
176165
}
177166

178-
private def tableFromOptions(options: Map[String, String]): Option[TableIdentifier] = {
179-
options
180-
.get(DataSourceOptions.TABLE_KEY)
181-
.map(TableIdentifier(_, options.get(DataSourceOptions.DATABASE_KEY)))
167+
def create(
168+
catalogName: String,
169+
ident: TableIdentifier,
170+
table: Table,
171+
options: Map[String, String]): NamedRelation = {
172+
TableV2Relation(catalogName, ident, table, table.schema.toAttributes, options)
182173
}
183174
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.plans.physical
2626
import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
2727
import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeStageCodegenExec}
2828
import org.apache.spark.sql.execution.streaming.continuous._
29-
import org.apache.spark.sql.sources.v2.DataSourceV2
3029
import org.apache.spark.sql.sources.v2.reader._
3130
import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader
3231
import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -36,7 +35,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch
3635
*/
3736
case class DataSourceV2ScanExec(
3837
output: Seq[AttributeReference],
39-
@transient source: DataSourceV2,
38+
@transient sourceName: String,
4039
@transient options: Map[String, String],
4140
@transient pushedFilters: Seq[Expression],
4241
@transient reader: DataSourceReader)
@@ -52,7 +51,7 @@ case class DataSourceV2ScanExec(
5251
}
5352

5453
override def hashCode(): Int = {
55-
Seq(output, source, options).hashCode()
54+
Seq(output, sourceName, options).hashCode()
5655
}
5756

5857
override def outputPartitioning: physical.Partitioning = reader match {

0 commit comments

Comments
 (0)