Skip to content
Open
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 @@ -45,7 +45,13 @@ def excluded_methods(cls):
'getFactory',
'getTableFactory',
'getFunctionDefinitionFactory',
'listPartitionsByFilter'}
'listPartitionsByFilter',
'getConnection',
'dropConnection',
'connectionExists',
'listConnections',
'createConnection',
'alterConnection'}


class CatalogDatabaseAPICompletenessTests(PythonAPICompletenessTestCase, PyFlinkTestCase):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@
import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
import org.apache.flink.table.factories.ApiFactoryUtil;
import org.apache.flink.table.factories.CatalogStoreFactory;
import org.apache.flink.table.factories.DefaultConnectionFactory;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.factories.PlannerFactoryUtil;
import org.apache.flink.table.factories.TableFactoryUtil;
Expand Down Expand Up @@ -122,6 +123,7 @@
import org.apache.flink.table.resource.ResourceUri;
import org.apache.flink.table.secret.SecretStore;
import org.apache.flink.table.secret.SecretStoreFactory;
import org.apache.flink.table.secret.WritableSecretStore;
import org.apache.flink.table.types.AbstractDataType;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.utils.DataTypeUtils;
Expand Down Expand Up @@ -275,6 +277,11 @@ public static TableEnvironmentImpl create(EnvironmentSettings settings) {
final ResourceManager resourceManager =
new ResourceManager(settings.getConfiguration(), userClassLoader);
final ModuleManager moduleManager = new ModuleManager();
final WritableSecretStore writableSecretStore =
secretStore instanceof WritableSecretStore
? (WritableSecretStore) secretStore
: null;

final CatalogManager catalogManager =
CatalogManager.newBuilder()
.classLoader(userClassLoader)
Expand All @@ -297,6 +304,8 @@ public static TableEnvironmentImpl create(EnvironmentSettings settings) {
.sqlFactory(
settings.getSqlFactory()
.orElseGet(() -> DefaultSqlFactory.INSTANCE))
.connectionFactory(new DefaultConnectionFactory())
.writableSecretStore(writableSecretStore)
.build();

final FunctionCatalog functionCatalog =
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.catalog.exceptions.CatalogException;
import org.apache.flink.table.catalog.exceptions.ConnectionAlreadyExistException;
import org.apache.flink.table.catalog.exceptions.ConnectionNotExistException;
import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
Expand Down Expand Up @@ -59,6 +61,7 @@ public class GenericInMemoryCatalog extends AbstractCatalog {
private final Map<String, CatalogDatabase> databases;
private final Map<ObjectPath, CatalogBaseTable> tables;
private final Map<ObjectPath, CatalogModel> models;
private final Map<ObjectPath, CatalogConnection> connections;
private final Map<ObjectPath, CatalogFunction> functions;
private final Map<ObjectPath, Map<CatalogPartitionSpec, CatalogPartition>> partitions;

Expand All @@ -79,6 +82,7 @@ public GenericInMemoryCatalog(String name, String defaultDatabase) {
this.databases.put(defaultDatabase, new CatalogDatabaseImpl(new HashMap<>(), null));
this.tables = new LinkedHashMap<>();
this.models = new LinkedHashMap<>();
this.connections = new LinkedHashMap<>();
this.functions = new LinkedHashMap<>();
this.partitions = new LinkedHashMap<>();
this.tableStats = new LinkedHashMap<>();
Expand Down Expand Up @@ -453,6 +457,78 @@ public boolean modelExists(ObjectPath modelPath) {
return databaseExists(modelPath.getDatabaseName()) && models.containsKey(modelPath);
}

// ------ connections ------

@Override
public void createConnection(
ObjectPath connectionPath, CatalogConnection connection, boolean ignoreIfExists)
throws ConnectionAlreadyExistException, DatabaseNotExistException {
checkNotNull(connectionPath);
checkNotNull(connection);
if (!databaseExists(connectionPath.getDatabaseName())) {
throw new DatabaseNotExistException(getName(), connectionPath.getDatabaseName());
}
if (connectionExists(connectionPath)) {
if (!ignoreIfExists) {
throw new ConnectionAlreadyExistException(getName(), connectionPath);
}
} else {
connections.put(connectionPath, connection.copy());
}
}

@Override
public void alterConnection(
ObjectPath connectionPath, CatalogConnection newConnection, boolean ignoreIfNotExists)
throws ConnectionNotExistException {
checkNotNull(connectionPath);
checkNotNull(newConnection);

if (!connectionExists(connectionPath)) {
if (ignoreIfNotExists) {
return;
}
throw new ConnectionNotExistException(getName(), connectionPath);
}

connections.put(connectionPath, newConnection.copy());
}

@Override
public void dropConnection(ObjectPath connectionPath, boolean ignoreIfNotExists)
throws ConnectionNotExistException {
checkNotNull(connectionPath);
if (connectionExists(connectionPath)) {
connections.remove(connectionPath);
} else if (!ignoreIfNotExists) {
throw new ConnectionNotExistException(getName(), connectionPath);
}
}

@Override
public List<String> listConnections(String databaseName) throws DatabaseNotExistException {
return listObjectsUnderDatabase(connections, databaseName, k -> true);
}

@Override
public CatalogConnection getConnection(ObjectPath connectionPath)
throws ConnectionNotExistException {
checkNotNull(connectionPath);

if (!connectionExists(connectionPath)) {
throw new ConnectionNotExistException(getName(), connectionPath);
} else {
return connections.get(connectionPath).copy();
}
}

@Override
public boolean connectionExists(ObjectPath connectionPath) {
checkNotNull(connectionPath);
return databaseExists(connectionPath.getDatabaseName())
&& connections.containsKey(connectionPath);
}

// ------ functions ------

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

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.catalog.CatalogConnection;
import org.apache.flink.table.catalog.ObjectIdentifier;

/** When a connection is altered, a {@link AlterConnectionEvent} event will be created and fired. */
@PublicEvolving
public interface AlterConnectionEvent extends ConnectionModificationEvent {
ObjectIdentifier identifier();

CatalogConnection newConnection();

boolean ignoreIfNotExists();

static AlterConnectionEvent createEvent(
final CatalogContext context,
final ObjectIdentifier identifier,
final CatalogConnection newConnection,
final boolean ignoreIfNotExists) {
return new AlterConnectionEvent() {
@Override
public CatalogConnection newConnection() {
return newConnection;
}

@Override
public boolean ignoreIfNotExists() {
return ignoreIfNotExists;
}

@Override
public ObjectIdentifier identifier() {
return identifier;
}

@Override
public CatalogConnection connection() {
throw new IllegalStateException(
"There is no connection in AlterConnectionEvent, use identifier() instead.");
}

@Override
public boolean isTemporary() {
return false;
}

@Override
public CatalogContext context() {
return context;
}
};
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.catalog.listener;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.catalog.CatalogConnection;
import org.apache.flink.table.catalog.ObjectIdentifier;

/** Basic event for connection modification such as create, alter and drop. */
@PublicEvolving
public interface ConnectionModificationEvent extends CatalogModificationEvent {

ObjectIdentifier identifier();

CatalogConnection connection();

boolean isTemporary();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.catalog.listener;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.catalog.CatalogConnection;
import org.apache.flink.table.catalog.ObjectIdentifier;

/**
* When a connection is created, a {@link CreateConnectionEvent} event will be created and fired.
*/
@PublicEvolving
public interface CreateConnectionEvent extends ConnectionModificationEvent {
ObjectIdentifier identifier();

CatalogConnection connection();

boolean ignoreIfExists();

boolean isTemporary();

static CreateConnectionEvent createEvent(
final CatalogContext context,
final ObjectIdentifier identifier,
final CatalogConnection connection,
final boolean ignoreIfExists,
final boolean isTemporary) {
return new CreateConnectionEvent() {
@Override
public boolean ignoreIfExists() {
return ignoreIfExists;
}

@Override
public ObjectIdentifier identifier() {
return identifier;
}

@Override
public CatalogConnection connection() {
return connection;
}

@Override
public CatalogContext context() {
return context;
}

@Override
public boolean isTemporary() {
return isTemporary;
}
};
}
}
Loading