-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat: spin up a connect worker embedded inside the KSQL JVM (#3241)
- Loading branch information
Showing
9 changed files
with
363 additions
and
5 deletions.
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
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,60 @@ | ||
# Sample configuration for a distributed Kafka Connect worker that uses Avro serialization and | ||
# integrates the the Schema Registry. This sample configuration assumes a local installation of | ||
# Confluent Platform with all services running on their default ports. | ||
|
||
# Bootstrap Kafka servers. If multiple servers are specified, they should be comma-separated. | ||
bootstrap.servers=localhost:9092 | ||
|
||
# The group ID is a unique identifier for the set of workers that form a single Kafka Connect | ||
# cluster | ||
group.id=ksql-connect-cluster | ||
|
||
# The converters specify the format of data in Kafka and how to translate it into Connect data. | ||
# Every Connect user will need to configure these based on the format they want their data in | ||
# when loaded from or stored into Kafka | ||
key.converter=io.confluent.connect.avro.AvroConverter | ||
key.converter.schema.registry.url=http://localhost:8081 | ||
value.converter=io.confluent.connect.avro.AvroConverter | ||
value.converter.schema.registry.url=http://localhost:8081 | ||
|
||
# Internal Storage Topics. | ||
# | ||
# Kafka Connect distributed workers store the connector and task configurations, connector offsets, | ||
# and connector statuses in three internal topics. These topics MUST be compacted. | ||
# When the Kafka Connect distributed worker starts, it will check for these topics and attempt to create them | ||
# as compacted topics if they don't yet exist, using the topic name, replication factor, and number of partitions | ||
# as specified in these properties, and other topic-specific settings inherited from your brokers' | ||
# auto-creation settings. If you need more control over these other topic-specific settings, you may want to | ||
# manually create these topics before starting Kafka Connect distributed workers. | ||
# | ||
# The following properties set the names of these three internal topics for storing configs, offsets, and status. | ||
config.storage.topic=ksql-connect-configs | ||
offset.storage.topic=ksql-connect-offsets | ||
status.storage.topic=ksql-connect-statuses | ||
|
||
# The following properties set the replication factor for the three internal topics, defaulting to 3 for each | ||
# and therefore requiring a minimum of 3 brokers in the cluster. Since we want the examples to run with | ||
# only a single broker, we set the replication factor here to just 1. That's okay for the examples, but | ||
# ALWAYS use a replication factor of AT LEAST 3 for production environments to reduce the risk of | ||
# losing connector offsets, configurations, and status. | ||
config.storage.replication.factor=1 | ||
offset.storage.replication.factor=1 | ||
status.storage.replication.factor=1 | ||
|
||
# The config storage topic must have a single partition, and this cannot be changed via properties. | ||
# Offsets for all connectors and tasks are written quite frequently and therefore the offset topic | ||
# should be highly partitioned; by default it is created with 25 partitions, but adjust accordingly | ||
# with the number of connector tasks deployed to a distributed worker cluster. Kafka Connect records | ||
# the status less frequently, and so by default the topic is created with 5 partitions. | ||
#offset.storage.partitions=25 | ||
#status.storage.partitions=5 | ||
|
||
# The offsets, status, and configurations are written to the topics using converters specified through | ||
# the following required properties. Most users will always want to use the JSON converter without schemas. | ||
# Offset and config data is never visible outside of Connect in this format. | ||
internal.key.converter=org.apache.kafka.connect.json.JsonConverter | ||
internal.value.converter=org.apache.kafka.connect.json.JsonConverter | ||
internal.key.converter.schemas.enable=false | ||
|
||
# fill this configuration in to use custom connectors | ||
# plugin.path= |
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
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
77 changes: 77 additions & 0 deletions
77
ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/ConnectExecutable.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,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.rest.server; | ||
|
||
import java.io.IOException; | ||
import java.net.BindException; | ||
import java.util.Collections; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
import org.apache.kafka.common.utils.Utils; | ||
import org.apache.kafka.connect.cli.ConnectDistributed; | ||
import org.apache.kafka.connect.errors.ConnectException; | ||
import org.apache.kafka.connect.runtime.Connect; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public final class ConnectExecutable implements Executable { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(ConnectExecutable.class); | ||
|
||
private final ConnectDistributed connectDistributed; | ||
private final Map<String, String> workerProps; | ||
private Connect connect; | ||
|
||
public static ConnectExecutable of(final String configFile) throws IOException { | ||
final Map<String, String> workerProps = !configFile.isEmpty() | ||
? Utils.propsToStringMap(Utils.loadProps(configFile)) | ||
: Collections.emptyMap(); | ||
|
||
return new ConnectExecutable(workerProps); | ||
} | ||
|
||
private ConnectExecutable(final Map<String, String> workerProps) { | ||
this.workerProps = Objects.requireNonNull(workerProps, "workerProps"); | ||
connectDistributed = new ConnectDistributed(); | ||
} | ||
|
||
@Override | ||
public void start() { | ||
try { | ||
connect = connectDistributed.startConnect(workerProps); | ||
} catch (final ConnectException e) { | ||
if (e.getCause() instanceof IOException && e.getCause().getCause() instanceof BindException) { | ||
LOG.warn("Cannot start a local connect instance because connect is running locally!", e); | ||
} else { | ||
throw e; | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
public void stop() { | ||
if (connect != null) { | ||
connect.stop(); | ||
} | ||
} | ||
|
||
@Override | ||
public void join() { | ||
if (connect != null) { | ||
connect.awaitStop(); | ||
} | ||
} | ||
} |
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
82 changes: 82 additions & 0 deletions
82
ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/MultiExecutable.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,82 @@ | ||
/* | ||
* 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.rest.server; | ||
|
||
import java.util.Objects; | ||
|
||
/** | ||
* {@code MultiExecutable} wraps multiple {@code Executable}s and ensures that when | ||
* an action is called all internal executables will perform the action, regardless | ||
* of whether or not previous executables succeeded. | ||
* | ||
* <p>The executables will be started, stopped and joined in the order that they | ||
* are supplied in {@link #of(Executable...)}.</p> | ||
*/ | ||
public final class MultiExecutable implements Executable { | ||
|
||
private final Executable[] executables; | ||
|
||
public static Executable of(final Executable... executables) { | ||
return new MultiExecutable(executables); | ||
} | ||
|
||
private MultiExecutable(final Executable... executables) { | ||
this.executables = Objects.requireNonNull(executables, "executables"); | ||
} | ||
|
||
@Override | ||
public void start() throws Exception { | ||
doAction(Executable::start); | ||
} | ||
|
||
@Override | ||
public void stop() throws Exception { | ||
doAction(Executable::stop); | ||
} | ||
|
||
@Override | ||
public void join() throws InterruptedException { | ||
doAction(Executable::join); | ||
} | ||
|
||
@SuppressWarnings("unchecked") | ||
private <T extends Exception> void doAction( | ||
final ExceptionalConsumer<Executable, T> action | ||
) throws T { | ||
|
||
T exception = null; | ||
for (final Executable executable : executables) { | ||
try { | ||
action.accept(executable); | ||
} catch (final Exception e) { | ||
if (exception == null) { | ||
exception = (T) e; | ||
} else { | ||
exception.addSuppressed(e); | ||
} | ||
} | ||
} | ||
|
||
if (exception != null) { | ||
throw exception; | ||
} | ||
} | ||
|
||
@FunctionalInterface | ||
private interface ExceptionalConsumer<I, T extends Exception> { | ||
void accept(I value) throws T; | ||
} | ||
} |
Oops, something went wrong.