-
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(ksql-connect): poll connect-configs and auto register sources #3178
Merged
Merged
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
dbf3e03
feat(ksql-connect): poll connect-configs and auto register sources
agavra b07de44
refactor: flip config semantics of disable to enable
agavra 274ed86
fix: fix commit configs bug and add some logging
agavra 4686053
feat: no longer read from topic, instead poll connect
agavra 36c5ec3
fix: fix test issue NPE
agavra dff61f6
feat: address rohans comments
agavra 421911c
fix: fix findbug issue with await()
agavra File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
218 changes: 218 additions & 0 deletions
218
ksql-engine/src/main/java/io/confluent/ksql/connect/ConnectConfigService.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,218 @@ | ||
/* | ||
* 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; | ||
|
||
import com.google.common.annotations.VisibleForTesting; | ||
import com.google.common.collect.ImmutableList; | ||
import com.google.common.collect.ImmutableMap; | ||
import com.google.common.util.concurrent.AbstractExecutionThreadService; | ||
import com.google.common.util.concurrent.MoreExecutors; | ||
import io.confluent.ksql.services.ConnectClient; | ||
import io.confluent.ksql.services.ConnectClient.ConnectResponse; | ||
import io.confluent.ksql.util.KsqlConfig; | ||
import io.confluent.ksql.util.KsqlServerException; | ||
import java.time.Duration; | ||
import java.util.Collection; | ||
import java.util.HashSet; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
import java.util.Optional; | ||
import java.util.Set; | ||
import java.util.function.Function; | ||
import java.util.stream.Collectors; | ||
import org.apache.kafka.clients.consumer.ConsumerConfig; | ||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; | ||
import org.apache.kafka.clients.consumer.ConsumerRecords; | ||
import org.apache.kafka.clients.consumer.KafkaConsumer; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.common.errors.WakeupException; | ||
import org.apache.kafka.common.serialization.ByteArrayDeserializer; | ||
import org.apache.kafka.common.serialization.StringDeserializer; | ||
import org.apache.kafka.connect.runtime.ConnectorConfig; | ||
import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* The {@code ConnectConfigService} listens to the connect configuration topic, | ||
* which outputs messages whenever a new connector (or connector task) is started | ||
* in Connect. These messages contain information that is then passed to a | ||
* {@link ConnectPollingService} to digest and register with KSQL. | ||
* | ||
* <p>On startup, this service reads the connect configuration topic from the | ||
* beginning to make sure that it reconstructs the necessary state.</p> | ||
*/ | ||
final class ConnectConfigService extends AbstractExecutionThreadService { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(ConnectConfigService.class); | ||
private static final long POLL_TIMEOUT_S = 60; | ||
|
||
private final KsqlConfig ksqlConfig; | ||
private final String configsTopic; | ||
private final ConnectClient connectClient; | ||
private final ConnectPollingService pollingService; | ||
private final Function<Map<String, Object>, KafkaConsumer<String, byte[]>> consumerFactory; | ||
private final Function<ConnectorInfo, Optional<Connector>> connectorFactory; | ||
|
||
private Set<String> handledConnectors = new HashSet<>(); | ||
|
||
// not final because constructing a consumer is expensive and should be | ||
// done in startUp() | ||
private KafkaConsumer<String, byte[]> consumer; | ||
|
||
ConnectConfigService( | ||
final KsqlConfig ksqlConfig, | ||
final ConnectClient connectClient, | ||
final ConnectPollingService pollingService | ||
) { | ||
this( | ||
ksqlConfig, | ||
connectClient, | ||
pollingService, | ||
Connectors::fromConnectInfo, | ||
KafkaConsumer::new | ||
); | ||
} | ||
|
||
@VisibleForTesting | ||
ConnectConfigService( | ||
final KsqlConfig ksqlConfig, | ||
final ConnectClient connectClient, | ||
final ConnectPollingService pollingService, | ||
final Function<ConnectorInfo, Optional<Connector>> connectorFactory, | ||
final Function<Map<String, Object>, KafkaConsumer<String, byte[]>> consumerFactory | ||
) { | ||
this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); | ||
this.pollingService = Objects.requireNonNull(pollingService, "pollingService"); | ||
this.connectorFactory = Objects.requireNonNull(connectorFactory, "connectorFactory"); | ||
this.consumerFactory = Objects.requireNonNull(consumerFactory, "consumerFactory"); | ||
this.connectClient = Objects.requireNonNull(connectClient, "connectClient"); | ||
this.configsTopic = ksqlConfig.getString(KsqlConfig.CONNECT_CONFIGS_TOPIC_PROPERTY); | ||
|
||
addListener(new Listener() { | ||
@Override | ||
public void failed(final State from, final Throwable failure) { | ||
LOG.error("ConnectConfigService failed due to: ", failure); | ||
} | ||
}, MoreExecutors.directExecutor()); | ||
} | ||
|
||
@Override | ||
protected void startUp() { | ||
final String serviceId = ksqlConfig.getString(KsqlConfig.KSQL_SERVICE_ID_CONFIG); | ||
final Map<String, Object> consumerConfigs = ImmutableMap.<String, Object>builder() | ||
.putAll(ksqlConfig.getProducerClientConfigProps()) | ||
// don't create the config topic if it doesn't exist - this is also necessary | ||
// for some integration tests to pass | ||
.put(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, false) | ||
// set the group id to be the same as the service id to make sure that only one | ||
// KSQL server will subscribe to the topic and handle connectors (use this as | ||
// a form of poor man's leader election) | ||
.put(ConsumerConfig.GROUP_ID_CONFIG, serviceId) | ||
.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class) | ||
.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class) | ||
.build(); | ||
|
||
consumer = consumerFactory.apply(consumerConfigs); | ||
consumer.subscribe(ImmutableList.of(configsTopic), new ConsumerRebalanceListener() { | ||
@Override | ||
public void onPartitionsRevoked(final Collection<TopicPartition> partitions) { } | ||
|
||
@Override | ||
public void onPartitionsAssigned(final Collection<TopicPartition> partitions) { | ||
// if we were assigned the (only) connect-configs partition, we should rebuild | ||
// our entire state - this ensures that only a single server in the KSQL cluster | ||
// handles new connectors | ||
checkConnectors(); | ||
} | ||
}); | ||
} | ||
|
||
@Override | ||
protected void run() { | ||
while (isRunning()) { | ||
try { | ||
final ConsumerRecords<String, byte[]> records; | ||
records = consumer.poll(Duration.ofSeconds(POLL_TIMEOUT_S)); | ||
LOG.debug("Polled {} records from connect config topic", records.count()); | ||
|
||
if (!records.isEmpty()) { | ||
checkConnectors(); | ||
agavra marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
} catch (final WakeupException e) { | ||
if (isRunning()) { | ||
throw e; | ||
} | ||
} | ||
} | ||
} | ||
|
||
private void checkConnectors() { | ||
try { | ||
// something changed in the connect configuration topic - poll connect | ||
// and see if we need to update our connectors | ||
final ConnectResponse<List<String>> allConnectors = connectClient.connectors(); | ||
if (allConnectors.datum().isPresent()) { | ||
final List<String> toProcess = allConnectors.datum() | ||
.get() | ||
.stream() | ||
.filter(connectorName -> !handledConnectors.contains(connectorName)) | ||
.collect(Collectors.toList()); | ||
LOG.info("Was made aware of the following connectors: {}", toProcess); | ||
|
||
toProcess.forEach(this::handleConnector); | ||
} | ||
} catch (final KsqlServerException e) { | ||
LOG.warn("Failed to check the connectors due to some server error.", e); | ||
} | ||
} | ||
|
||
private void handleConnector(final String name) { | ||
try { | ||
final ConnectResponse<ConnectorInfo> describe = connectClient.describe(name); | ||
if (!describe.datum().isPresent()) { | ||
return; | ||
} | ||
|
||
handledConnectors.add(name); | ||
final Optional<Connector> connector = connectorFactory.apply(describe.datum().get()); | ||
if (connector.isPresent()) { | ||
LOG.info("Registering connector {}", connector); | ||
pollingService.addConnector(connector.get()); | ||
} else { | ||
LOG.warn("Ignoring unsupported connector {} ({})", | ||
name, describe.datum().get().config().get(ConnectorConfig.CONNECTOR_CLASS_CONFIG)); | ||
} | ||
} catch (final KsqlServerException e) { | ||
LOG.warn("Failed to describe connector {} due to some server error.", name, e); | ||
} | ||
} | ||
|
||
@Override | ||
protected void shutDown() { | ||
// this is called in the same thread as run() as it is not thread-safe | ||
consumer.close(); | ||
LOG.info("ConnectConfigService is down."); | ||
} | ||
|
||
@Override | ||
protected void triggerShutdown() { | ||
// this is called on a different thread from run() since it is thread-safe | ||
LOG.info("Shutting down ConnectConfigService."); | ||
consumer.wakeup(); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
What happens when connectors are terminated? Does Connect write such information to this topic? How do we deal with such scenarios where external systems create and terminate connectors?
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.
Good questions! Answers inline:
Nothing happens because connect doesn't write any such information to the topic, but more importantly it doesn't delete the topics that it already created so I'm not sure what the expected behavior would be.
In the future when we support SHOW CONNECTORS, they would not show up.
The beautiful thing about this design is that everything is decoupled, so whether the connector is created internally or externally, nothing changes!