twalthr commented on code in PR #22986:
URL: https://github.com/apache/flink/pull/22986#discussion_r1263667523
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java:
##########
@@ -73,9 +73,9 @@ public final class CatalogManager implements CatalogRegistry {
private final Map<ObjectIdentifier, CatalogBaseTable> temporaryTables;
// The name of the current catalog and database
- private String currentCatalogName;
+ @Nullable private String currentCatalogName;
Review Comment:
very very nit: But could we agree on using this order?
```
private @Nullable String currentCatalogName;
```
The nullability should be next to the data type everywhere, not next to the
visibility
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java:
##########
@@ -681,8 +695,30 @@ private boolean permanentDatabaseExists(String
catalogName, String databaseName)
*/
public ObjectIdentifier qualifyIdentifier(UnresolvedIdentifier identifier)
{
return ObjectIdentifier.of(
- identifier.getCatalogName().orElseGet(this::getCurrentCatalog),
-
identifier.getDatabaseName().orElseGet(this::getCurrentDatabase),
+ identifier
+ .getCatalogName()
+ .orElseGet(
+ () -> {
+ final String currentCatalog =
getCurrentCatalog();
+ if
(StringUtils.isNullOrWhitespaceOnly(currentCatalog)) {
+ throw new ValidationException(
+ "Default catalog has not been
set. Please use fully"
+ + " qualified
identifiers.");
+ }
+ return currentCatalog;
+ }),
+ identifier
+ .getDatabaseName()
+ .orElseGet(
+ () -> {
+ final String currentDatabase =
getCurrentDatabase();
+ if
(StringUtils.isNullOrWhitespaceOnly(currentDatabase)) {
+ throw new ValidationException(
+ "Default database has not been
set. Please use fully"
Review Comment:
```
"A current database has not been set. Please use a fully qualified
identifier (such as 'my_database.my_table' or
'my_catalog.my_database.my_table') or set a current database using 'USE
my_database'."
```
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java:
##########
@@ -324,9 +329,18 @@ public String getCurrentDatabase() {
* @see CatalogManager#setCurrentCatalog(String)
*/
public void setCurrentDatabase(String databaseName) {
+ if (databaseName == null) {
+ this.currentDatabaseName = null;
+ return;
+ }
+
checkArgument(
!StringUtils.isNullOrWhitespaceOnly(databaseName),
- "The database name cannot be null or empty.");
+ "The database name cannot be empty.");
+
+ if (currentCatalogName == null) {
+ throw new CatalogException("Default catalog has not been set.");
Review Comment:
It's not about the "default catalog" here but the "current catalog".
##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/UnknownCatalogITCase.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.flink.table.planner.catalog;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for no default catalog and/or database. */
+public class UnknownCatalogITCase {
+
+ public static final String BUILTIN_CATALOG = "cat";
+ private static final String BUILTIN_DATABASE = "db";
+ public static final EnvironmentSettings ENVIRONMENT_SETTINGS =
+ EnvironmentSettings.newInstance()
+ .inStreamingMode()
+ .withBuiltInCatalogName(BUILTIN_CATALOG)
+ .withBuiltInDatabaseName(BUILTIN_DATABASE)
+ .build();
+
+ @Test
+ public void testUnsetCatalogWithFullyQualified() throws Exception {
+ TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS);
+
+ tEnv.useCatalog(null);
+ final String tablePath = String.format("%s.%s.%s", BUILTIN_CATALOG,
BUILTIN_DATABASE, "tb");
+ registerTable(tEnv, tablePath);
+
+ Table table = tEnv.sqlQuery(String.format("SELECT * FROM %s",
tablePath));
+
+
assertThat(CollectionUtil.iteratorToList(table.execute().collect()).toString())
+ .isEqualTo("[+I[1, a], +I[2, b]]");
+ }
+
+ @Test
+ public void testUnsetCatalogWithSingleIdentifier() throws Exception {
+ TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS);
+
+ tEnv.useCatalog(null);
+ final String tableName = "tb";
+ final String tablePath =
+ String.format("%s.%s.%s", BUILTIN_CATALOG, BUILTIN_DATABASE,
tableName);
+ registerTable(tEnv, tablePath);
+
+ assertThatThrownBy(() -> tEnv.sqlQuery("SELECT * FROM " + tableName))
+ .isInstanceOf(ValidationException.class)
+ .hasMessageContaining(String.format("Object '%s' not found",
tableName));
+ }
+
+ @Test
+ public void testUsingUnknownDatabaseWithDatabaseQualified() throws
Exception {
+ TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS);
+ tEnv.useDatabase(null);
+
+ final String tableName = "tb";
+ final String tablePath =
+ String.format("%s.%s.%s", BUILTIN_CATALOG, BUILTIN_DATABASE,
tableName);
+ registerTable(tEnv, tablePath);
+
+ Table table =
+ tEnv.sqlQuery(String.format("SELECT * FROM %s.%s",
BUILTIN_DATABASE, tableName));
+
+
assertThat(CollectionUtil.iteratorToList(table.execute().collect()).toString())
+ .isEqualTo("[+I[1, a], +I[2, b]]");
+ }
+
+ @Test
+ public void testUsingUnknownDatabaseWithSingleIdentifier() throws
Exception {
+ TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS);
+ tEnv.useDatabase(null);
+
+ final String tableName = "tb";
+ final String tablePath =
+ String.format("%s.%s.%s", BUILTIN_CATALOG, BUILTIN_DATABASE,
tableName);
+ registerTable(tEnv, tablePath);
+
+ assertThatThrownBy(() -> tEnv.sqlQuery("SELECT * FROM " + tableName))
+ .isInstanceOf(ValidationException.class)
+ .hasMessageContaining(String.format("Object '%s' not found",
tableName));
+ }
+
+ @Test
+ public void testUnsetCatalogWithAlterTable() throws Exception {
+ TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS);
+
+ tEnv.useCatalog(null);
+ final String tableName = "tb";
+ final String tablePath =
+ String.format("%s.%s.%s", BUILTIN_CATALOG, BUILTIN_DATABASE,
tableName);
+ registerTable(tEnv, tablePath);
+
+ assertThatThrownBy(
+ () ->
+ tEnv.executeSql(
+ String.format("ALTER TABLE %s ADD (f
STRING)", tableName)))
+ .isInstanceOf(ValidationException.class)
+ .hasMessage(
+ "Default catalog has not been set. Please use fully
qualified"
+ + " identifiers.");
+ }
+
+ @Test
+ public void testUnsetDatabaseWithAlterTable() throws Exception {
Review Comment:
Add a test for a catalog that return null for the `getDefaultDatabase()`
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java:
##########
@@ -681,8 +695,30 @@ private boolean permanentDatabaseExists(String
catalogName, String databaseName)
*/
public ObjectIdentifier qualifyIdentifier(UnresolvedIdentifier identifier)
{
return ObjectIdentifier.of(
- identifier.getCatalogName().orElseGet(this::getCurrentCatalog),
-
identifier.getDatabaseName().orElseGet(this::getCurrentDatabase),
+ identifier
+ .getCatalogName()
+ .orElseGet(
+ () -> {
+ final String currentCatalog =
getCurrentCatalog();
+ if
(StringUtils.isNullOrWhitespaceOnly(currentCatalog)) {
+ throw new ValidationException(
+ "Default catalog has not been
set. Please use fully"
Review Comment:
```
"A current catalog has not been set. Please use a fully qualified identifier
(such as 'my_catalog.my_database.my_table') or set a current catalog using 'USE
CATALOG my_catalog'."
```
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java:
##########
@@ -283,17 +283,22 @@ public String getCurrentCatalog() {
* @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier)
*/
public void setCurrentCatalog(String catalogName) throws
CatalogNotExistException {
Review Comment:
add `setCurrentCatalog(@Nullable String catalogName)`
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java:
##########
@@ -324,9 +329,18 @@ public String getCurrentDatabase() {
* @see CatalogManager#setCurrentCatalog(String)
*/
public void setCurrentDatabase(String databaseName) {
Review Comment:
` setCurrentDatabase(@Nullable String databaseName) `
##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/UnknownCatalogITCase.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.flink.table.planner.catalog;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for no default catalog and/or database. */
+public class UnknownCatalogITCase {
Review Comment:
We don't need an ITCase here. A plan test should be sufficient.
##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/UnknownCatalogITCase.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.flink.table.planner.catalog;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for no default catalog and/or database. */
+public class UnknownCatalogITCase {
+
+ public static final String BUILTIN_CATALOG = "cat";
+ private static final String BUILTIN_DATABASE = "db";
+ public static final EnvironmentSettings ENVIRONMENT_SETTINGS =
+ EnvironmentSettings.newInstance()
+ .inStreamingMode()
+ .withBuiltInCatalogName(BUILTIN_CATALOG)
+ .withBuiltInDatabaseName(BUILTIN_DATABASE)
+ .build();
+
+ @Test
+ public void testUnsetCatalogWithFullyQualified() throws Exception {
+ TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS);
+
+ tEnv.useCatalog(null);
Review Comment:
Add `@Nullable` annotation to this and the other method. And a JavaDoc.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]