diff --git a/config/ksql-server.properties b/config/ksql-server.properties
index f03ea402223b..1437b05945de 100644
--- a/config/ksql-server.properties
+++ b/config/ksql-server.properties
@@ -47,6 +47,7 @@ ksql.logging.processing.stream.auto.create=true
# The set of Kafka brokers to bootstrap Kafka cluster information from:
bootstrap.servers=localhost:9092
+ksql.connect.polling.enable=true
# Uncomment and complete the following to enable KSQL's integration to the Confluent Schema Registry:
#ksql.schema.registry.url=?
diff --git a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java
index dd68786fa7af..53f8b575feb9 100644
--- a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java
+++ b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java
@@ -31,6 +31,7 @@ public final class CreateConfigs {
public static final String WINDOW_TYPE_PROPERTY = "WINDOW_TYPE";
public static final String WINDOW_SIZE_PROPERTY = "WINDOW_SIZE";
public static final String AVRO_SCHEMA_ID = "AVRO_SCHEMA_ID";
+ public static final String SOURCE_CONNECTOR = "SOURCE_CONNECTOR";
private static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(
@@ -64,6 +65,14 @@ public final class CreateConfigs {
null,
Importance.LOW,
"Undocumented feature"
+ ).define(
+ SOURCE_CONNECTOR,
+ Type.STRING,
+ null,
+ Importance.LOW,
+ "Indicates that this source was created by a connector with the given name. This "
+ + "is useful for understanding which sources map to which connectors and will "
+ + "be automatically populated for connectors."
);
static {
diff --git a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java
index 299646930304..7f463e70a673 100644
--- a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java
+++ b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java
@@ -65,6 +65,10 @@ public class KsqlConfig extends AbstractConfig {
public static final String CONNECT_URL_PROPERTY = "ksql.connect.registry.url";
+ public static final String CONNECT_POLLING_ENABLE_PROPERTY = "ksql.connect.polling.enable";
+
+ public static final String CONNECT_CONFIGS_TOPIC_PROPERTY = "ksql.connect.configs.topic";
+
public static final String KSQL_ENABLE_UDFS = "ksql.udfs.enabled";
public static final String KSQL_EXT_DIR = "ksql.extension.dir";
@@ -158,6 +162,7 @@ public class KsqlConfig extends AbstractConfig {
public static final String DEFAULT_SCHEMA_REGISTRY_URL = "http://localhost:8081";
public static final String DEFAULT_CONNECT_URL = "http://localhost:8083";
+ public static final String DEFAULT_CONNECT_CONFIGS_TOPIC = "connect-configs";
public static final String KSQL_STREAMS_PREFIX = "ksql.streams.";
@@ -390,6 +395,7 @@ private static ConfigDef configDef(final ConfigGeneration generation) {
return generation == ConfigGeneration.CURRENT ? CURRENT_DEF : LEGACY_DEF;
}
+ // CHECKSTYLE_RULES.OFF: MethodLength
private static ConfigDef buildConfigDef(final ConfigGeneration generation) {
final ConfigDef configDef = new ConfigDef()
.define(
@@ -439,6 +445,19 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) {
DEFAULT_CONNECT_URL,
Importance.MEDIUM,
"The URL for the connect deployment, defaults to http://localhost:8083"
+ ).define(
+ CONNECT_POLLING_ENABLE_PROPERTY,
+ Type.BOOLEAN,
+ false,
+ Importance.LOW,
+ "A value of false for this configuration will disable automatically importing sources "
+ + "from connectors into KSQL."
+ ).define(
+ CONNECT_CONFIGS_TOPIC_PROPERTY ,
+ ConfigDef.Type.STRING,
+ DEFAULT_CONNECT_CONFIGS_TOPIC,
+ Importance.LOW,
+ "The name for the connect configuration topic, defaults to 'connect-configs'"
).define(
KSQL_ENABLE_UDFS,
ConfigDef.Type.BOOLEAN,
@@ -539,6 +558,7 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) {
}
return configDef;
}
+ // CHECKSTYLE_RULES.ON: MethodLength
private static final class ConfigValue {
final ConfigItem configItem;
diff --git a/ksql-engine/src/main/java/io/confluent/ksql/connect/ConnectConfigService.java b/ksql-engine/src/main/java/io/confluent/ksql/connect/ConnectConfigService.java
new file mode 100644
index 000000000000..65539f1fa326
--- /dev/null
+++ b/ksql-engine/src/main/java/io/confluent/ksql/connect/ConnectConfigService.java
@@ -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.
+ *
+ *
On startup, this service reads the connect configuration topic from the
+ * beginning to make sure that it reconstructs the necessary state.
+ */
+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