-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-29665][SQL] Refine the TableProvider Interface #26868
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,50 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.internal.connector | ||
|
||
|
|
||
| import java.util | ||
|
|
||
| import org.apache.spark.sql.connector.catalog.{Table, TableProvider} | ||
| import org.apache.spark.sql.connector.expressions.Transform | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| // A simple version of `TableProvider` which doesn't support specified table schema/partitioning | ||
| // and treats table properties case-insensitively. This is private and only used in builtin sources. | ||
| trait SimpleTableProvider extends TableProvider { | ||
|
|
||
| def getTable(options: CaseInsensitiveStringMap): Table | ||
|
|
||
| private[this] var loadedTable: Table = _ | ||
|
||
| private def getOrLoadTable(options: CaseInsensitiveStringMap): Table = { | ||
| if (loadedTable == null) loadedTable = getTable(options) | ||
| loadedTable | ||
| } | ||
|
|
||
| override def inferSchema(options: CaseInsensitiveStringMap): StructType = { | ||
| getOrLoadTable(options).schema() | ||
| } | ||
|
|
||
| override def getTable( | ||
| schema: StructType, | ||
| partitioning: Array[Transform], | ||
| properties: util.Map[String, String]): Table = { | ||
| assert(partitioning.isEmpty) | ||
|
||
| getOrLoadTable(new CaseInsensitiveStringMap(properties)) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,8 +20,10 @@ package org.apache.spark.sql.execution.datasources.v2 | |
| import java.util.regex.Pattern | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, TableProvider} | ||
| import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, Table, TableProvider} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| private[sql] object DataSourceV2Utils extends Logging { | ||
|
|
||
|
|
@@ -57,4 +59,28 @@ private[sql] object DataSourceV2Utils extends Logging { | |
| case _ => Map.empty | ||
| } | ||
| } | ||
|
|
||
| def getTableFromProvider( | ||
| provider: TableProvider, | ||
| options: CaseInsensitiveStringMap, | ||
| userSpecifiedSchema: Option[StructType]): Table = { | ||
| userSpecifiedSchema match { | ||
| case Some(schema) => | ||
| if (provider.supportsExternalMetadata()) { | ||
|
||
| provider.getTable( | ||
| schema, | ||
| provider.inferPartitioning(options), | ||
| options.asCaseSensitiveMap()) | ||
| } else { | ||
| throw new UnsupportedOperationException( | ||
| s"${provider.getClass.getSimpleName} source does not support user-specified schema.") | ||
| } | ||
|
|
||
| case None => | ||
| provider.getTable( | ||
| provider.inferSchema(options), | ||
| provider.inferPartitioning(options), | ||
| options.asCaseSensitiveMap()) | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we use
CaseInsensitiveStringMapas table properties here? People can get the original case sensitive map easily viaasCaseSensitiveMap.cc @rdblue @brkyvz
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this needs to be case preserving, because it'll come from the metastore right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, let's keep it as it is.