-
Notifications
You must be signed in to change notification settings - Fork 1k
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
feat: add connect templates and simplify JDBC source (MINOR) #3231
Changes from all commits
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 |
---|---|---|
@@ -0,0 +1,77 @@ | ||
/* | ||
* Copyright 2019 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.connect.supported; | ||
|
||
import com.google.common.collect.ImmutableMap; | ||
import io.confluent.ksql.connect.Connector; | ||
import java.util.EnumSet; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
import java.util.Optional; | ||
import java.util.function.Function; | ||
import java.util.stream.Collectors; | ||
import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; | ||
|
||
public enum Connectors implements SupportedConnector { | ||
|
||
JDBC(JdbcSource.JDBC_SOURCE_CLASS, new JdbcSource()) | ||
; | ||
|
||
public static final String CONNECTOR_CLASS = "connector.class"; | ||
|
||
private static final Map<String, SupportedConnector> CONNECTORS = ImmutableMap.copyOf( | ||
EnumSet.allOf(Connectors.class) | ||
.stream() | ||
.collect(Collectors.toMap( | ||
Connectors::getConnectorClass, | ||
Function.identity())) | ||
); | ||
|
||
private final String connectorClass; | ||
private final SupportedConnector supportedConnector; | ||
|
||
Connectors(final String connectorClass, final SupportedConnector supportedConnector) { | ||
this.connectorClass = Objects.requireNonNull(connectorClass, "connectorClass"); | ||
this.supportedConnector = Objects.requireNonNull(supportedConnector, "supportedConnector"); | ||
} | ||
|
||
public static Optional<Connector> from(final ConnectorInfo info) { | ||
final SupportedConnector connector = | ||
CONNECTORS.get(info.config().get(CONNECTOR_CLASS)); | ||
return connector == null ? Optional.empty() : connector.fromConnectInfo(info); | ||
} | ||
|
||
public static Map<String, String> resolve(final Map<String, String> configs) { | ||
final SupportedConnector connector = | ||
CONNECTORS.get(configs.get(CONNECTOR_CLASS)); | ||
return connector == null ? configs : connector.resolveConfigs(configs); | ||
|
||
} | ||
|
||
@Override | ||
public Optional<Connector> fromConnectInfo(final ConnectorInfo info) { | ||
return supportedConnector.fromConnectInfo(info); | ||
} | ||
|
||
@Override | ||
public Map<String, String> resolveConfigs(final Map<String, String> configs) { | ||
return supportedConnector.resolveConfigs(configs); | ||
} | ||
|
||
public String getConnectorClass() { | ||
return connectorClass; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
* Copyright 2019 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.connect.supported; | ||
|
||
import io.confluent.ksql.connect.Connector; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; | ||
|
||
/** | ||
* KSQL supports some "Blessed" connectors that we integrate well with. To be "blessed" | ||
* means that we can automatically import topics created by these connectors and that | ||
* we may provide templates that simplify configuration of these connectors. | ||
*/ | ||
public interface SupportedConnector { | ||
|
||
/** | ||
* Constructs a {@link Connector} from the configuration given to us by connect. | ||
*/ | ||
Optional<Connector> fromConnectInfo(ConnectorInfo info); | ||
|
||
/** | ||
* Resolves a template configuration into something that connect can understand. | ||
*/ | ||
Map<String, String> resolveConfigs(Map<String, String> configs); | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
|
||
import com.google.common.collect.Maps; | ||
import io.confluent.ksql.KsqlExecutionContext; | ||
import io.confluent.ksql.connect.supported.Connectors; | ||
import io.confluent.ksql.parser.tree.CreateConnector; | ||
import io.confluent.ksql.rest.entity.CreateConnectorEntity; | ||
import io.confluent.ksql.rest.entity.ErrorEntity; | ||
|
@@ -42,7 +43,10 @@ public static Optional<KsqlEntity> execute( | |
|
||
final ConnectResponse<ConnectorInfo> response = client.create( | ||
createConnector.getName(), | ||
Maps.transformValues(createConnector.getConfig(), l -> l.getValue().toString())); | ||
Connectors.resolve( | ||
Maps.transformValues( | ||
createConnector.getConfig(), | ||
l -> l != null ? l.getValue().toString() : null))); | ||
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. Why would this ever be null? 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. never... it's just that intelliJ complains when I don't check for it because transformValues has an annotation ( |
||
|
||
if (response.datum().isPresent()) { | ||
return Optional.of( | ||
|
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.
Why does this implement
SupportConnector
?