Skip to content
Closed
Show file tree
Hide file tree
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 @@ -35,7 +35,7 @@
*/
public class SqlUseCatalog extends SqlCall {

public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("USE CATALOG", SqlKind.OTHER);
public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("USE CATALOG", SqlKind.OTHER_DDL);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why is this required?

Copy link
Contributor Author

@zjuwangg zjuwangg Nov 21, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Both validate(sqlNode: SqlNode) methods in FlinkPlannerImpl in flink-planner/blink-planner will validate row type, SqlKind.OTHER kind will fail the validation.

private final SqlIdentifier catalogName;

public SqlUseCatalog(SqlParserPos pos, SqlIdentifier catalogName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.apache.flink.table.operations.Operation;
import org.apache.flink.table.operations.QueryOperation;
import org.apache.flink.table.operations.TableSourceQueryOperation;
import org.apache.flink.table.operations.UseCatalogOperation;
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.table.operations.ddl.DropTableOperation;
import org.apache.flink.table.operations.utils.OperationTreeBuilder;
Expand Down Expand Up @@ -450,7 +451,7 @@ public void sqlUpdate(String stmt) {
if (operations.size() != 1) {
throw new TableException(
"Unsupported SQL query! sqlUpdate() only accepts a single SQL statement of type " +
"INSERT, CREATE TABLE, DROP TABLE");
"INSERT, CREATE TABLE, DROP TABLE, USE CATALOG");
}

Operation operation = operations.get(0);
Expand All @@ -473,10 +474,13 @@ public void sqlUpdate(String stmt) {
catalogManager.dropTable(
dropTableOperation.getTableIdentifier(),
dropTableOperation.isIfExists());
} else if (operation instanceof UseCatalogOperation) {
UseCatalogOperation useCatalogOperation = (UseCatalogOperation) operation;
catalogManager.setCurrentCatalog(useCatalogOperation.getCatalogName());
} else {
throw new TableException(
"Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of " +
"type INSERT, CREATE TABLE, DROP TABLE");
"type INSERT, CREATE TABLE, DROP TABLE, USE CATALOG");
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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.operations;

/**
* Operation to describe a USE CATALOG statement.
*/
public class UseCatalogOperation implements UseOperation {

private String catalogName;

public UseCatalogOperation(String catalogName) {
this.catalogName = catalogName;
}

public String getCatalogName() {
return catalogName;
}

@Override
public String asSummaryString() {
return String.format("USE CATALOGS %s", catalogName);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.operations;

import org.apache.flink.annotation.Internal;

/**
* An {@link Operation} that describes the catalog/database switch statements,
* e.g. USE CATALOG or USE [catalogName.]dataBaseName.
*
* <p>Different sub operations can represent their special meanings. For example, a
* use catalog operation means switching current catalog to another,
* while use database operation means switching current database.
*/
@Internal
public interface UseOperation extends Operation {
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.sql.parser.ddl.SqlDropTable;
import org.apache.flink.sql.parser.ddl.SqlTableColumn;
import org.apache.flink.sql.parser.ddl.SqlTableOption;
import org.apache.flink.sql.parser.ddl.SqlUseCatalog;
import org.apache.flink.sql.parser.dml.RichSqlInsert;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.TableSchema;
Expand All @@ -32,6 +33,7 @@
import org.apache.flink.table.catalog.UnresolvedIdentifier;
import org.apache.flink.table.operations.CatalogSinkModifyOperation;
import org.apache.flink.table.operations.Operation;
import org.apache.flink.table.operations.UseCatalogOperation;
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.table.operations.ddl.DropTableOperation;
import org.apache.flink.table.planner.calcite.FlinkPlannerImpl;
Expand Down Expand Up @@ -100,6 +102,8 @@ public static Optional<Operation> convert(
return Optional.of(converter.convertDropTable((SqlDropTable) validated));
} else if (validated instanceof RichSqlInsert) {
return Optional.of(converter.convertSqlInsert((RichSqlInsert) validated));
} else if (validated instanceof SqlUseCatalog) {
return Optional.of(converter.convertUseCatalog((SqlUseCatalog) validated));
} else if (validated.getKind().belongsTo(SqlKind.QUERY)) {
return Optional.of(converter.convertSqlQuery(validated));
} else {
Expand Down Expand Up @@ -177,6 +181,11 @@ private Operation convertSqlInsert(RichSqlInsert insert) {
insert.isOverwrite());
}

/** Convert use catalog statement. */
private Operation convertUseCatalog(SqlUseCatalog useCatalog) {
return new UseCatalogOperation(useCatalog.getCatalogName());
}

/** Fallback method for sql query. */
private Operation convertSqlQuery(SqlNode node) {
return toQueryOperation(flinkPlanner, node);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import org.apache.flink.table.module.ModuleManager;
import org.apache.flink.table.operations.CatalogSinkModifyOperation;
import org.apache.flink.table.operations.Operation;
import org.apache.flink.table.operations.UseCatalogOperation;
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.table.planner.calcite.CalciteParser;
import org.apache.flink.table.planner.calcite.FlinkPlannerImpl;
Expand Down Expand Up @@ -115,6 +116,16 @@ public void after() throws TableNotExistException {
catalog.dropTable(path2, true);
}

@Test
public void testUseCatalog() {
final String sql = "USE CATALOG cat1";
FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT);
Operation operation = parse(sql, planner, parser);
assert operation instanceof UseCatalogOperation;
assertEquals("cat1", ((UseCatalogOperation) operation).getCatalogName());
}

@Test
public void testCreateTable() {
final String sql = "CREATE TABLE tbl1 (\n" +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,18 +21,16 @@ package org.apache.flink.table.planner.catalog
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.api.internal.TableEnvironmentImpl
import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment, ValidationException}
import org.apache.flink.table.catalog.{CatalogFunctionImpl, ObjectPath}
import org.apache.flink.table.catalog.{CatalogFunctionImpl, GenericInMemoryCatalog, ObjectPath}
import org.apache.flink.table.planner.expressions.utils.Func0
import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory
import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc0
import org.apache.flink.types.Row

import org.junit.Assert.assertEquals
import org.junit.rules.ExpectedException
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
import org.junit.{Before, Ignore, Rule, Test}

import java.sql.Timestamp
import java.util

Expand Down Expand Up @@ -828,6 +826,16 @@ class CatalogTableITCase(isStreamingMode: Boolean) {
tableEnv.sqlUpdate("DROP TABLE IF EXISTS catalog1.database1.t1")
assert(tableEnv.listTables().sameElements(Array[String]("t1")))
}

@Test
def testUseCatalog(): Unit = {
tableEnv.registerCatalog("cat1", new GenericInMemoryCatalog("cat1"))
tableEnv.registerCatalog("cat2", new GenericInMemoryCatalog("cat2"))
tableEnv.sqlUpdate("use catalog cat1")
assertEquals("cat1", tableEnv.getCurrentCatalog)
tableEnv.sqlUpdate("use catalog cat2")
assertEquals("cat2", tableEnv.getCurrentCatalog)
}
}

object CatalogTableITCase {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.sql.parser.ddl.SqlDropTable;
import org.apache.flink.sql.parser.ddl.SqlTableColumn;
import org.apache.flink.sql.parser.ddl.SqlTableOption;
import org.apache.flink.sql.parser.ddl.SqlUseCatalog;
import org.apache.flink.sql.parser.dml.RichSqlInsert;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.TableSchema;
Expand All @@ -36,6 +37,7 @@
import org.apache.flink.table.operations.CatalogSinkModifyOperation;
import org.apache.flink.table.operations.Operation;
import org.apache.flink.table.operations.PlannerQueryOperation;
import org.apache.flink.table.operations.UseCatalogOperation;
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.table.operations.ddl.DropTableOperation;
import org.apache.flink.table.types.utils.TypeConversions;
Expand Down Expand Up @@ -102,6 +104,8 @@ public static Optional<Operation> convert(
throw new ValidationException("Partial inserts are not supported");
}
return Optional.of(converter.convertSqlInsert((RichSqlInsert) validated));
} else if (validated instanceof SqlUseCatalog) {
return Optional.of(converter.convertUseCatalog((SqlUseCatalog) validated));
} else if (validated.getKind().belongsTo(SqlKind.QUERY)) {
return Optional.of(converter.convertSqlQuery(validated));
} else {
Expand Down Expand Up @@ -189,6 +193,11 @@ private Operation convertSqlInsert(RichSqlInsert insert) {
insert.isOverwrite());
}

/** Convert use catalog statement. */
private Operation convertUseCatalog(SqlUseCatalog useCatalog) {
return new UseCatalogOperation(useCatalog.getCatalogName());
}

//~ Tools ------------------------------------------------------------------

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -468,7 +468,7 @@ abstract class TableEnvImpl(

if (operations.size != 1) throw new TableException(
"Unsupported SQL query! sqlUpdate() only accepts a single SQL statement of type " +
"INSERT, CREATE TABLE, DROP TABLE")
"INSERT, CREATE TABLE, DROP TABLE, USE CATALOG")

operations.get(0) match {
case op: CatalogSinkModifyOperation =>
Expand All @@ -485,9 +485,11 @@ abstract class TableEnvImpl(
catalogManager.dropTable(
dropTableOperation.getTableIdentifier,
dropTableOperation.isIfExists)
case useCatalogOperation: UseCatalogOperation =>
catalogManager.setCurrentCatalog(useCatalogOperation.getCatalogName)
case _ => throw new TableException(
"Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of " +
"type INSERT, CREATE TABLE, DROP TABLE")
"type INSERT, CREATE TABLE, DROP TABLE, USE CATALOG")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.flink.table.module.ModuleManager;
import org.apache.flink.table.operations.CatalogSinkModifyOperation;
import org.apache.flink.table.operations.Operation;
import org.apache.flink.table.operations.UseCatalogOperation;
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.table.planner.PlanningConfigurationBuilder;
import org.apache.flink.table.types.DataType;
Expand Down Expand Up @@ -113,6 +114,16 @@ public void after() throws TableNotExistException {
catalog.dropTable(path2, true);
}

@Test
public void testUseCatalog() {
final String sql = "USE CATALOG cat1";
FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql);
Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get();
assert operation instanceof UseCatalogOperation;
assertEquals("cat1", ((UseCatalogOperation) operation).getCatalogName());
}

@Test
public void testCreateTable() {
final String sql = "CREATE TABLE tbl1 (\n" +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -534,6 +534,16 @@ class CatalogTableITCase(isStreaming: Boolean) {
tableEnv.sqlUpdate("DROP TABLE IF EXISTS catalog1.database1.t1")
assert(tableEnv.listTables().sameElements(Array[String]("t1")))
}

@Test
def testUseCatalog(): Unit = {
tableEnv.registerCatalog("cat1", new GenericInMemoryCatalog("cat1"))
tableEnv.registerCatalog("cat2", new GenericInMemoryCatalog("cat2"))
tableEnv.sqlUpdate("use catalog cat1")
assertEquals("cat1", tableEnv.getCurrentCatalog)
tableEnv.sqlUpdate("use catalog cat2")
assertEquals("cat2", tableEnv.getCurrentCatalog)
}
}

object CatalogTableITCase {
Expand Down