Skip to content

Commit

Permalink
feat: Support IF EXISTS keyword on DROP CONNECTOR (#6067)
Browse files Browse the repository at this point in the history
* feat: Support IF EXISTS keyword on DROP CONNECTOR

* fix: review comments

Co-authored-by: Zara Lim <zlim@zara-lim-mbp16.attlocal.net>
  • Loading branch information
Zara Lim and Zara Lim committed Aug 24, 2020
1 parent a147c50 commit 2c99df9
Show file tree
Hide file tree
Showing 10 changed files with 168 additions and 13 deletions.
7 changes: 5 additions & 2 deletions docs/developer-guide/ksqldb-reference/drop-connector.md
Expand Up @@ -13,11 +13,14 @@ Synopsis
--------

```sql
DROP CONNECTOR connector_name;
DROP CONNECTOR [IF EXISTS] connector_name;
```

Description
-----------

Drop a connector and delete it from the {{ site.kconnect }} cluster. The
topics associated with this cluster are not deleted by this command.
topics associated with this cluster are not deleted by this command.

If the IF EXISTS clause is present, the statement doesn't fail if the
connector doesn't exist.
Expand Up @@ -42,6 +42,7 @@
import io.confluent.ksql.cli.console.table.builder.TablesListTableBuilder;
import io.confluent.ksql.cli.console.table.builder.TopicDescriptionTableBuilder;
import io.confluent.ksql.cli.console.table.builder.TypeListTableBuilder;
import io.confluent.ksql.cli.console.table.builder.WarningEntityTableBuilder;
import io.confluent.ksql.model.WindowType;
import io.confluent.ksql.query.QueryError;
import io.confluent.ksql.rest.ApiJsonMapper;
Expand Down Expand Up @@ -81,6 +82,7 @@
import io.confluent.ksql.rest.entity.TablesList;
import io.confluent.ksql.rest.entity.TopicDescription;
import io.confluent.ksql.rest.entity.TypeList;
import io.confluent.ksql.rest.entity.WarningEntity;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.util.CmdLineUtil;
import io.confluent.ksql.util.HandlerMaps;
Expand Down Expand Up @@ -173,6 +175,8 @@ public class Console implements Closeable {
tablePrinter(TypeList.class, TypeListTableBuilder::new))
.put(ErrorEntity.class,
tablePrinter(ErrorEntity.class, ErrorEntityTableBuilder::new))
.put(WarningEntity.class,
tablePrinter(WarningEntity.class, WarningEntityTableBuilder::new))
.build();

private static <T extends KsqlEntity> Handler1<KsqlEntity, Console> tablePrinter(
Expand Down
@@ -0,0 +1,29 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.cli.console.table.builder;

import io.confluent.ksql.cli.console.table.Table;
import io.confluent.ksql.rest.entity.WarningEntity;

public class WarningEntityTableBuilder implements TableBuilder<WarningEntity> {
@Override
public Table buildTable(final WarningEntity entity) {
return new Table.Builder()
.withColumnHeaders("Message")
.withRow(entity.getMessage())
.build();
}
}
Expand Up @@ -77,7 +77,7 @@ statement
| INSERT INTO sourceName (columns)? VALUES values #insertValues
| DROP STREAM (IF EXISTS)? sourceName (DELETE TOPIC)? #dropStream
| DROP TABLE (IF EXISTS)? sourceName (DELETE TOPIC)? #dropTable
| DROP CONNECTOR identifier #dropConnector
| DROP CONNECTOR (IF EXISTS)? identifier #dropConnector
| EXPLAIN (statement | identifier) #explain
| CREATE TYPE identifier AS type #registerType
| DROP TYPE (IF EXISTS)? identifier #dropType
Expand Down
Expand Up @@ -396,6 +396,7 @@ public Node visitDropStream(final SqlBaseParser.DropStreamContext context) {
public Node visitDropConnector(final DropConnectorContext context) {
return new DropConnector(
getLocation(context),
context.EXISTS() != null,
ParserUtil.getIdentifierText(context.identifier())
);
}
Expand Down
Expand Up @@ -15,19 +15,31 @@

package io.confluent.ksql.parser.tree;

import static com.google.common.base.MoreObjects.toStringHelper;

import io.confluent.ksql.parser.NodeLocation;
import java.util.Objects;
import java.util.Optional;

public class DropConnector extends Statement {

private final boolean ifExists;
private final String connectorName;

public DropConnector(final Optional<NodeLocation> location, final String connectorName) {
public DropConnector(
final Optional<NodeLocation> location,
final boolean ifExists,
final String connectorName
) {
super(location);
this.ifExists = ifExists;
this.connectorName = connectorName;
}

public boolean getIfExists() {
return ifExists;
}

public String getConnectorName() {
return connectorName;
}
Expand All @@ -41,18 +53,19 @@ public boolean equals(final Object o) {
return false;
}
final DropConnector that = (DropConnector) o;
return Objects.equals(connectorName, that.connectorName);
return Objects.equals(connectorName, that.connectorName) && ifExists == that.ifExists;
}

@Override
public int hashCode() {
return Objects.hash(connectorName);
return Objects.hash(ifExists, connectorName);
}

@Override
public String toString() {
return "DropConnector{"
+ "connectorName='" + connectorName + '\''
+ '}';
return toStringHelper(this)
.add("ifExists", ifExists)
.add("connectorName", connectorName)
.toString();
}
}
Expand Up @@ -21,10 +21,12 @@
import io.confluent.ksql.rest.entity.DropConnectorEntity;
import io.confluent.ksql.rest.entity.ErrorEntity;
import io.confluent.ksql.rest.entity.KsqlEntity;
import io.confluent.ksql.rest.entity.WarningEntity;
import io.confluent.ksql.services.ConnectClient.ConnectResponse;
import io.confluent.ksql.services.ServiceContext;
import io.confluent.ksql.statement.ConfiguredStatement;
import java.util.Optional;
import org.apache.hc.core5.http.HttpStatus;

public final class DropConnectorExecutor {

Expand All @@ -37,11 +39,17 @@ public static Optional<KsqlEntity> execute(
final ServiceContext serviceContext
) {
final String connectorName = statement.getStatement().getConnectorName();
final boolean ifExists = statement.getStatement().getIfExists();
final ConnectResponse<String> response =
serviceContext.getConnectClient().delete(connectorName);

if (response.error().isPresent()) {
return Optional.of(new ErrorEntity(statement.getStatementText(), response.error().get()));
if (ifExists && response.httpCode() == HttpStatus.SC_NOT_FOUND) {
return Optional.of(new WarningEntity(statement.getStatementText(),
"Connector '" + connectorName + "' does not exist."));
} else {
return Optional.of(new ErrorEntity(statement.getStatementText(), response.error().get()));
}
}

return Optional.of(new DropConnectorEntity(statement.getStatementText(), connectorName));
Expand Down
Expand Up @@ -30,6 +30,7 @@
import io.confluent.ksql.rest.entity.DropConnectorEntity;
import io.confluent.ksql.rest.entity.ErrorEntity;
import io.confluent.ksql.rest.entity.KsqlEntity;
import io.confluent.ksql.rest.entity.WarningEntity;
import io.confluent.ksql.services.ConnectClient;
import io.confluent.ksql.services.ConnectClient.ConnectResponse;
import io.confluent.ksql.services.ServiceContext;
Expand All @@ -48,15 +49,25 @@ public class DropConnectorExecutorTest {

private static final KsqlConfig CONFIG = new KsqlConfig(ImmutableMap.of());

private static final DropConnector CREATE_CONNECTOR = new DropConnector(Optional.empty(), "foo");
private static final DropConnector DROP_CONNECTOR =
new DropConnector(Optional.empty(), false, "foo");
private static final DropConnector DROP_CONNECTOR_IF_EXISTS =
new DropConnector(Optional.empty(), true, "foo");

private static final ConfiguredStatement<DropConnector> DROP_CONNECTOR_CONFIGURED =
ConfiguredStatement.of(
PreparedStatement.of(
"DROP CONNECTOR \"foo\"",
CREATE_CONNECTOR),
DROP_CONNECTOR),
ImmutableMap.of(),
CONFIG);
private static final ConfiguredStatement<DropConnector> DROP_CONNECTOR_IF_EXISTS_CONFIGURED =
ConfiguredStatement.of(
PreparedStatement.of(
"DROP CONNECTOR \"foo\"",
DROP_CONNECTOR_IF_EXISTS),
ImmutableMap.of(),
CONFIG);

@Mock
private ServiceContext serviceContext;
Expand Down Expand Up @@ -106,10 +117,28 @@ public void shouldReturnErrorEntityOnError() {
// When:
final Optional<KsqlEntity> entity = DropConnectorExecutor
.execute(DROP_CONNECTOR_CONFIGURED, mock(SessionProperties.class), null, serviceContext);
final Optional<KsqlEntity> entityIfExists = DropConnectorExecutor
.execute(DROP_CONNECTOR_IF_EXISTS_CONFIGURED, mock(SessionProperties.class), null, serviceContext);

// Then:
assertThat("Expected non-empty response", entity.isPresent());
assertThat(entity.get(), instanceOf(ErrorEntity.class));
assertThat("Expected non-empty response", entityIfExists.isPresent());
assertThat(entityIfExists.get(), instanceOf(ErrorEntity.class));
}

@Test
public void shouldReturnWarningIfNotExist() {
// Given:
when(connectClient.delete(anyString()))
.thenReturn(ConnectResponse.failure("Danger Mouse!", HttpStatus.SC_NOT_FOUND));

// When:
final Optional<KsqlEntity> entity = DropConnectorExecutor
.execute(DROP_CONNECTOR_IF_EXISTS_CONFIGURED, mock(SessionProperties.class), null, serviceContext);

// Then:
assertThat("Expected non-empty response", entity.isPresent());
assertThat(entity.get(), instanceOf(WarningEntity.class));
}
}
Expand Up @@ -48,7 +48,8 @@
@JsonSubTypes.Type(value = ConnectorList.class, name = "connector_list"),
@JsonSubTypes.Type(value = ConnectorDescription.class, name = "connector_description"),
@JsonSubTypes.Type(value = TypeList.class, name = "type_list"),
@JsonSubTypes.Type(value = ErrorEntity.class, name = "error_entity")
@JsonSubTypes.Type(value = ErrorEntity.class, name = "error_entity"),
@JsonSubTypes.Type(value = WarningEntity.class, name = "warning_entity")
})
public abstract class KsqlEntity {
private final String statementText;
Expand Down
@@ -0,0 +1,67 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.rest.entity;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.errorprone.annotations.Immutable;

import java.util.Objects;

@JsonIgnoreProperties(ignoreUnknown = true)
@Immutable
public class WarningEntity extends KsqlEntity {
private final String message;

@JsonCreator
public WarningEntity(
@JsonProperty("statementText") final String statementText,
@JsonProperty("message") final String message
) {
super(statementText);
this.message = Objects.requireNonNull(message, "message");
}

public String getMessage() {
return message;
}

@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}

final WarningEntity that = (WarningEntity) o;
return Objects.equals(message, that.message);
}

@Override
public int hashCode() {
return Objects.hash(message);
}

@Override
public String toString() {
return "WarningEntity{"
+ "message='" + message + '\''
+ '}';
}
}

0 comments on commit 2c99df9

Please sign in to comment.