Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import org.apache.spark.sql.internal.connector.V1Function
import org.apache.spark.sql.types.{DataType, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.Utils

/**
* A [[TableCatalog]] that translates calls to the v1 SessionCatalog.
Expand Down Expand Up @@ -158,9 +159,9 @@ class V2SessionCatalog(catalog: SessionCatalog)
catalog.refreshTable(ident.asTableIdentifier)
}

override def createTable(
private def createTable0(
ident: Identifier,
columns: Array[Column],
schema: StructType,
partitions: Array[Transform],
properties: util.Map[String, String]): Table = {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
Expand All @@ -178,7 +179,6 @@ class V2SessionCatalog(catalog: SessionCatalog)
CatalogTableType.MANAGED
}

val schema = CatalogV2Util.v2ColumnsToStructType(columns)
val (newSchema, newPartitions) = DataSourceV2Utils.getTableProvider(provider, conf) match {
// If the provider does not support external metadata, users should not be allowed to
// specify custom schema when creating the data source table, since the schema will not
Expand Down Expand Up @@ -255,7 +255,18 @@ class V2SessionCatalog(catalog: SessionCatalog)
schema: StructType,
partitions: Array[Transform],
properties: util.Map[String, String]): Table = {
throw QueryCompilationErrors.createTableDeprecatedError()
if (Utils.isTesting) {
throw QueryCompilationErrors.createTableDeprecatedError()
}
createTable0(ident, schema, partitions, properties)
}

override def createTable(
ident: Identifier,
columns: Array[Column],
partitions: Array[Transform],
properties: util.Map[String, String]): Table = {
createTable0(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties)
}

private def toOptions(properties: Map[String, String]): Map[String, String] = {
Expand Down