-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-48668][SQL] Support ALTER NAMESPACE ... UNSET PROPERTIES in v2 #47038
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 5 commits
12f8f71
48e3b44
363ad49
7ddf703
1f3ed06
960f32a
0748ea4
53a9a25
33fbc7c
0d09a65
b3b60ac
0e925c6
87b0bcb
f5efc77
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 |
|---|---|---|
|
|
@@ -53,24 +53,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) | |
| case CurrentNamespace => | ||
| ResolvedNamespace(currentCatalog, catalogManager.currentNamespace.toImmutableArraySeq) | ||
| case UnresolvedNamespace(Seq(), fetchMetadata) => | ||
| resolveNamespace(currentCatalog, Seq.empty[String], fetchMetadata) | ||
| ResolveCatalogs.resolveNamespace(currentCatalog, Seq.empty[String], fetchMetadata) | ||
| case UnresolvedNamespace(CatalogAndNamespace(catalog, ns), fetchMetadata) => | ||
| resolveNamespace(catalog, ns, fetchMetadata) | ||
| } | ||
|
|
||
| private def resolveNamespace( | ||
| catalog: CatalogPlugin, | ||
| ns: Seq[String], | ||
| fetchMetadata: Boolean): ResolvedNamespace = { | ||
| catalog match { | ||
| case supportsNS: SupportsNamespaces if fetchMetadata => | ||
| ResolvedNamespace( | ||
| catalog, | ||
| ns, | ||
| supportsNS.loadNamespaceMetadata(ns.toArray).asScala.toMap) | ||
| case _ => | ||
| ResolvedNamespace(catalog, ns) | ||
| } | ||
| ResolveCatalogs.resolveNamespace(catalog, ns, fetchMetadata) | ||
| } | ||
|
|
||
| private def resolveVariableName(nameParts: Seq[String]): ResolvedIdentifier = { | ||
|
|
@@ -98,3 +83,20 @@ class ResolveCatalogs(val catalogManager: CatalogManager) | |
| } | ||
| } | ||
| } | ||
|
|
||
| object ResolveCatalogs { | ||
| def resolveNamespace( | ||
|
||
| catalog: CatalogPlugin, | ||
| ns: Seq[String], | ||
| fetchMetadata: Boolean): ResolvedNamespace = { | ||
| catalog match { | ||
| case supportsNS: SupportsNamespaces if fetchMetadata => | ||
| ResolvedNamespace( | ||
| catalog, | ||
| ns, | ||
| supportsNS.loadNamespaceMetadata(ns.toArray).asScala.toMap) | ||
| case _ => | ||
| ResolvedNamespace(catalog, ns) | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2659,12 +2659,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat | |
| } | ||
|
|
||
| def unsetNonExistentPropertiesError( | ||
| properties: Seq[String], table: TableIdentifier): Throwable = { | ||
| properties: Seq[String], nameParts: Seq[String]): Throwable = { | ||
|
||
| new AnalysisException( | ||
| errorClass = "UNSET_NONEXISTENT_PROPERTIES", | ||
| messageParameters = Map( | ||
| "properties" -> properties.map(toSQLId).mkString(", "), | ||
| "table" -> toSQLId(table.nameParts)) | ||
| "relationId" -> toSQLId(nameParts)) | ||
| ) | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -1098,4 +1098,24 @@ class SparkSqlAstBuilder extends AstBuilder { | |||||
|
|
||||||
| (ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER)) | ||||||
| } | ||||||
|
|
||||||
|
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. I have to put the parse process of |
||||||
| /** | ||||||
| * Create a [[UnsetNamespacePropertiesCommand]] command. | ||||||
| * | ||||||
| * For example: | ||||||
|
||||||
| * For example: | |
| * Expected format: |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} | |
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} | ||
| import org.apache.spark.sql.catalyst.plans.QueryPlan | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, SupervisingCommand} | ||
| import org.apache.spark.sql.catalyst.trees.LeafLike | ||
| import org.apache.spark.sql.catalyst.trees.{LeafLike, UnaryLike} | ||
| import org.apache.spark.sql.connector.ExternalCommandRunner | ||
| import org.apache.spark.sql.execution.{CommandExecutionMode, ExplainMode, LeafExecNode, SparkPlan, UnaryExecNode} | ||
| import org.apache.spark.sql.execution.metric.SQLMetric | ||
|
|
@@ -51,6 +51,7 @@ trait RunnableCommand extends Command { | |
| } | ||
|
|
||
| trait LeafRunnableCommand extends RunnableCommand with LeafLike[LogicalPlan] | ||
| trait UnaryRunnableCommand extends RunnableCommand with UnaryLike[LogicalPlan] | ||
|
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. Because |
||
|
|
||
| /** | ||
| * A physical operator that executes the run method of a `RunnableCommand` and | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,70 @@ | ||
| /* | ||
| * 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.execution.command | ||
|
|
||
| import org.apache.spark.SparkThrowable | ||
| import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifier} | ||
| import org.apache.spark.sql.catalyst.parser.ParseException | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.execution.SparkSqlParser | ||
| import org.apache.spark.sql.test.SharedSparkSession | ||
|
|
||
| class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedSparkSession { | ||
|
|
||
| private lazy val parser = new SparkSqlParser() | ||
|
|
||
| private def parseException(sqlText: String): SparkThrowable = { | ||
| intercept[ParseException](sql(sqlText).collect()) | ||
| } | ||
|
|
||
| private def parsePlan(sqlText: String): LogicalPlan = { | ||
| parser.parsePlan(sqlText) | ||
| } | ||
|
|
||
| test("unset namespace properties") { | ||
| Seq("DATABASE", "SCHEMA", "NAMESPACE").foreach { nsToken => | ||
| Seq("PROPERTIES", "DBPROPERTIES").foreach { propToken => | ||
| comparePlans( | ||
| parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a', 'b', 'c')"), | ||
| UnsetNamespacePropertiesCommand( | ||
| UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = false)) | ||
|
|
||
| comparePlans( | ||
| parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')"), | ||
| UnsetNamespacePropertiesCommand( | ||
| UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = true)) | ||
|
|
||
| comparePlans( | ||
| parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a')"), | ||
| UnsetNamespacePropertiesCommand( | ||
| UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a"), ifExists = false)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("property values must not be set") { | ||
| val sql = "ALTER NAMESPACE my_db UNSET PROPERTIES('key_without_value', 'key_with_value'='x')" | ||
| checkError( | ||
| exception = parseException(sql), | ||
| errorClass = "_LEGACY_ERROR_TEMP_0035", | ||
| parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"), | ||
| context = ExpectedContext( | ||
| fragment = sql, | ||
| start = 0, | ||
| stop = 80)) | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
| @@ -0,0 +1,127 @@ | ||||||||||||||||||||||||||||||||||||||||||||
| /* | ||||||||||||||||||||||||||||||||||||||||||||
| * 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.execution.command | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.{AnalysisException, QueryTest} | ||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.catalyst.parser.ParseException | ||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces} | ||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.errors.DataTypeErrors.toSQLId | ||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.internal.SQLConf | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| /** | ||||||||||||||||||||||||||||||||||||||||||||
| * This base suite contains unified tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command | ||||||||||||||||||||||||||||||||||||||||||||
| * that check V1 and V2 table catalogs. The tests that cannot run for all supported catalogs are | ||||||||||||||||||||||||||||||||||||||||||||
| * located in more specific test suites: | ||||||||||||||||||||||||||||||||||||||||||||
| * | ||||||||||||||||||||||||||||||||||||||||||||
| * - V2 table catalog tests: | ||||||||||||||||||||||||||||||||||||||||||||
| * `org.apache.spark.sql.execution.command.v2.AlterNamespaceUnsetPropertiesSuite` | ||||||||||||||||||||||||||||||||||||||||||||
| * - V1 table catalog tests: | ||||||||||||||||||||||||||||||||||||||||||||
| * `org.apache.spark.sql.execution.command.v1.AlterNamespaceUnsetPropertiesSuiteBase` | ||||||||||||||||||||||||||||||||||||||||||||
| * - V1 In-Memory catalog: | ||||||||||||||||||||||||||||||||||||||||||||
| * `org.apache.spark.sql.execution.command.v1.AlterNamespaceUnsetPropertiesSuite` | ||||||||||||||||||||||||||||||||||||||||||||
| * - V1 Hive External catalog: | ||||||||||||||||||||||||||||||||||||||||||||
| * `org.apache.spark.sql.hive.execution.command.AlterNamespaceUnsetPropertiesSuite` | ||||||||||||||||||||||||||||||||||||||||||||
| */ | ||||||||||||||||||||||||||||||||||||||||||||
| trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { | ||||||||||||||||||||||||||||||||||||||||||||
| override val command = "ALTER NAMESPACE ... UNSET PROPERTIES" | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| protected def namespace: String | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| protected def getProperties(namespace: String): String = { | ||||||||||||||||||||||||||||||||||||||||||||
| val propsRow = sql(s"DESCRIBE NAMESPACE EXTENDED $namespace") | ||||||||||||||||||||||||||||||||||||||||||||
| .toDF("key", "value") | ||||||||||||||||||||||||||||||||||||||||||||
| .where("key like 'Properties%'") | ||||||||||||||||||||||||||||||||||||||||||||
| .collect() | ||||||||||||||||||||||||||||||||||||||||||||
| assert(propsRow.length == 1) | ||||||||||||||||||||||||||||||||||||||||||||
| propsRow(0).getString(1) | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| test("namespace does not exist") { | ||||||||||||||||||||||||||||||||||||||||||||
| val ns = "not_exist" | ||||||||||||||||||||||||||||||||||||||||||||
| val e = intercept[AnalysisException] { | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"ALTER NAMESPACE $catalog.$ns UNSET PROPERTIES ('d')") | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| checkError(e, | ||||||||||||||||||||||||||||||||||||||||||||
| errorClass = "SCHEMA_NOT_FOUND", | ||||||||||||||||||||||||||||||||||||||||||||
| parameters = Map("schemaName" -> s"`$ns`")) | ||||||||||||||||||||||||||||||||||||||||||||
|
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. just noticed one thing: why does the error message omit the catalog name? For table not found, the error message contains the user-given table name in the SQL statement.
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. Let me investigate it
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.
From the user's perspective (when he uses multiple
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. Also, do we also need to fix similar scenarios (when spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala Lines 257 to 267 in 4c99c4d
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. @cloud-fan Initial PR, just attach |
||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| test("basic test") { | ||||||||||||||||||||||||||||||||||||||||||||
| val ns = s"$catalog.$namespace" | ||||||||||||||||||||||||||||||||||||||||||||
| withNamespace(ns) { | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"CREATE NAMESPACE $ns") | ||||||||||||||||||||||||||||||||||||||||||||
| assert(getProperties(ns) === "") | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')") | ||||||||||||||||||||||||||||||||||||||||||||
| assert(getProperties(ns) === "((a,a), (b,b), (c,c))") | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") | ||||||||||||||||||||||||||||||||||||||||||||
| assert(getProperties(ns) === "((a,a), (c,c))") | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| checkError( | ||||||||||||||||||||||||||||||||||||||||||||
| exception = intercept[AnalysisException] { | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") | ||||||||||||||||||||||||||||||||||||||||||||
| }, | ||||||||||||||||||||||||||||||||||||||||||||
| errorClass = "UNSET_NONEXISTENT_PROPERTIES", | ||||||||||||||||||||||||||||||||||||||||||||
| parameters = Map("properties" -> "`b`", "relationId" -> toSQLId(namespace))) | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES IF EXISTS ('b')") | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| test("test reserved properties") { | ||||||||||||||||||||||||||||||||||||||||||||
| import SupportsNamespaces._ | ||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ | ||||||||||||||||||||||||||||||||||||||||||||
| val ns = s"$catalog.$namespace" | ||||||||||||||||||||||||||||||||||||||||||||
| withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { | ||||||||||||||||||||||||||||||||||||||||||||
| CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => | ||||||||||||||||||||||||||||||||||||||||||||
| withNamespace(ns) { | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"CREATE NAMESPACE $ns") | ||||||||||||||||||||||||||||||||||||||||||||
| val sqlText = s"ALTER NAMESPACE $ns UNSET PROPERTIES ('$key')" | ||||||||||||||||||||||||||||||||||||||||||||
| checkErrorMatchPVals( | ||||||||||||||||||||||||||||||||||||||||||||
| exception = intercept[ParseException] { | ||||||||||||||||||||||||||||||||||||||||||||
| sql(sqlText) | ||||||||||||||||||||||||||||||||||||||||||||
| }, | ||||||||||||||||||||||||||||||||||||||||||||
| errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", | ||||||||||||||||||||||||||||||||||||||||||||
| parameters = Map( | ||||||||||||||||||||||||||||||||||||||||||||
| "property" -> key, | ||||||||||||||||||||||||||||||||||||||||||||
| "msg" -> ".*"), | ||||||||||||||||||||||||||||||||||||||||||||
| sqlState = None, | ||||||||||||||||||||||||||||||||||||||||||||
| context = ExpectedContext( | ||||||||||||||||||||||||||||||||||||||||||||
| fragment = sqlText, | ||||||||||||||||||||||||||||||||||||||||||||
| start = 0, | ||||||||||||||||||||||||||||||||||||||||||||
| stop = 37 + ns.length + key.length) | ||||||||||||||||||||||||||||||||||||||||||||
| ) | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { | ||||||||||||||||||||||||||||||||||||||||||||
| CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => | ||||||||||||||||||||||||||||||||||||||||||||
| withNamespace(ns) { | ||||||||||||||||||||||||||||||||||||||||||||
| // Set the location explicitly because v2 catalog may not set the default location. | ||||||||||||||||||||||||||||||||||||||||||||
| // Without this, `meta.get(key)` below may return null. | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"CREATE NAMESPACE $ns LOCATION 'tmp/prop_test'") | ||||||||||||||||||||||||||||||||||||||||||||
| assert(getProperties(ns) === "") | ||||||||||||||||||||||||||||||||||||||||||||
| sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('$key')") | ||||||||||||||||||||||||||||||||||||||||||||
| assert(getProperties(ns) === "", s"$key is a reserved namespace property and ignored") | ||||||||||||||||||||||||||||||||||||||||||||
| val meta = spark.sessionState.catalogManager.catalog(catalog) | ||||||||||||||||||||||||||||||||||||||||||||
| .asNamespaceCatalog.loadNamespaceMetadata(namespace.split('.')) | ||||||||||||||||||||||||||||||||||||||||||||
| assert(!meta.get(key).contains("foo"), | ||||||||||||||||||||||||||||||||||||||||||||
| "reserved properties should not have side effects") | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||
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 want to reuse it in the
namespacesceneThere 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.
but namespace is not a relation... maybe just
table or namespace <name>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.
Yea, okay.