-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17338][SQL] add global temp view #14897
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 9 commits
919e41a
67e459a
2aba058
2732531
754d75b
6183400
a683fba
0a70679
88e684e
9ab7228
952f13c
39f5995
9f196e5
cbbe122
96833d5
fb96f1c
29e292a
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,121 @@ | ||
| /* | ||
| * 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.catalyst.catalog | ||
|
|
||
| import javax.annotation.concurrent.GuardedBy | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.sql.AnalysisException | ||
| import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.util.StringUtils | ||
|
|
||
|
|
||
| /** | ||
| * A thread-safe manager for global temporary views, providing atomic operations to manage them, | ||
| * e.g. create, update, remove, etc. | ||
| * | ||
| * Note that, the view name is always case-sensitive here, callers are responsible to format the | ||
| * view name w.r.t. case-sensitive config. | ||
| * | ||
| * @param database The system preserved virtual database that keeps all the global temporary views. | ||
| */ | ||
| class GlobalTempViewManager(val database: String) { | ||
|
|
||
| /** List of view definitions, mapping from view name to logical plan. */ | ||
| @GuardedBy("this") | ||
| private val viewDefinitions = new mutable.HashMap[String, LogicalPlan] | ||
|
|
||
| /** | ||
| * Returns the global view definition which matches the given name, or None if not found. | ||
| */ | ||
| def get(name: String): Option[LogicalPlan] = synchronized { | ||
| viewDefinitions.get(name) | ||
| } | ||
|
|
||
| /** | ||
| * Creates a global temp view, or issue an exception if the view already exists and | ||
| * `overrideIfExists` is false. | ||
| */ | ||
| def create( | ||
| name: String, | ||
| viewDefinition: LogicalPlan, | ||
| overrideIfExists: Boolean): Unit = synchronized { | ||
| if (!overrideIfExists && viewDefinitions.contains(name)) { | ||
| throw new TempTableAlreadyExistsException(name) | ||
| } | ||
| viewDefinitions.put(name, viewDefinition) | ||
| } | ||
|
|
||
| /** | ||
| * Updates the global temp view if it exists, returns true if updated, false otherwise. | ||
| */ | ||
| def update( | ||
| name: String, | ||
| viewDefinition: LogicalPlan): Boolean = synchronized { | ||
| if (viewDefinitions.contains(name)) { | ||
| viewDefinitions.put(name, viewDefinition) | ||
| true | ||
| } else { | ||
| false | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Removes the global temp view if it exists, returns true if removed, false otherwise. | ||
| */ | ||
| def remove(name: String): Boolean = synchronized { | ||
| viewDefinitions.remove(name).isDefined | ||
| } | ||
|
|
||
| /** | ||
| * Renames the global temp view if the source view exists and the destination view not exists, or | ||
| * issue an exception if the source view exists but the destination view already exists. Returns | ||
| * true if renamed, false otherwise. | ||
| */ | ||
| def rename(oldName: String, newName: String): Boolean = synchronized { | ||
| if (viewDefinitions.contains(oldName)) { | ||
| if (viewDefinitions.contains(newName)) { | ||
| throw new AnalysisException( | ||
| s"rename temporary view from '$oldName' to '$newName': destination view already exists") | ||
| } | ||
|
|
||
| val viewDefinition = viewDefinitions(oldName) | ||
| viewDefinitions.remove(oldName) | ||
| viewDefinitions.put(newName, viewDefinition) | ||
| true | ||
| } else { | ||
| false | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is reason that failing to rename has two behavior (when source does not exist, we return false. But, when destination already exists, we thrown an error)?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This behaviour comes from the previous temp view: If the source doesn't exist, try persisted table/view, if source exists but destination already exists, throw an error. |
||
| } | ||
|
|
||
| /** | ||
| * Lists the names of all global temporary views. | ||
| */ | ||
| def listViewNames(pattern: String): Seq[String] = synchronized { | ||
| StringUtils.filterPattern(viewDefinitions.keys.toSeq, pattern) | ||
| } | ||
|
|
||
| /** | ||
| * Clears all the global temporary views. | ||
| */ | ||
| def clear(): Unit = synchronized { | ||
| viewDefinitions.clear() | ||
| } | ||
| } | ||
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.
When do we use
updateand when do we usecreate?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.
CREATE VIEW and ALTER VIEW.
We can have a single API for them, but need to introduce a write mode: errorIfExists, overrideIfExists(if not exists, create one), updateIfExists(if not exists, do nothing).