diff --git a/airbyte-integrations/connectors/source-kafka/Dockerfile b/airbyte-integrations/connectors/source-kafka/Dockerfile new file mode 100644 index 000000000000..47adceed70bc --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/Dockerfile @@ -0,0 +1,12 @@ +FROM airbyte/integration-base-java:dev + +WORKDIR /airbyte + +ENV APPLICATION source-kafka + +COPY build/distributions/${APPLICATION}*.tar ${APPLICATION}.tar + +RUN tar xf ${APPLICATION}.tar --strip-components=1 + +LABEL io.airbyte.name=airbyte/source-kafka +LABEL io.airbyte.version=0.1.0 diff --git a/airbyte-integrations/connectors/source-kafka/README.md b/airbyte-integrations/connectors/source-kafka/README.md new file mode 100644 index 000000000000..ba2d9ad6ea27 --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/README.md @@ -0,0 +1,45 @@ +# Kafka Source + +This is the repository for the Kafka source connector. +For information about how to use this connector within Airbyte, see [the documentation](https://docs.airbyte.io/integrations/sources/kafka). + +## Local development + +### Prerequisites +* If you are using Python for connector development, minimal required version `= 3.7.0` +* Valid credentials (see the "Create credentials section for instructions) +TODO: _which languages and tools does a user need to develop on this connector? add them to the bullet list above_ + +### Iteration +TODO: _which commands should a developer use to run this connector locally?_ + +### Testing +#### Unit Tests +TODO: _how can a user run unit tests?_ + +#### Integration Tests +TODO: _how can a user run integration tests?_ +_this section is currently under construction -- please reach out to us on Slack for help with setting up Airbyte's standard test suite_ + + +### Locally running the connector docker image + +First, make sure you build the latest Docker image: +``` +docker build . -t airbyte/kafka:dev +``` + +Then run any of the connector commands as follows: +``` +docker run --rm airbyte/source-kafka:dev spec +docker run --rm -v $(pwd)/secrets:/secrets airbyte/source-kafka:dev check --config /secrets/config.json +docker run --rm -v $(pwd)/secrets:/secrets airbyte/source-kafka:dev discover --config /secrets/config.json +docker run --rm -v $(pwd)/secrets:/secrets -v $(pwd)/sample_files:/sample_files airbyte/source-kafka:dev read --config /secrets/config.json --catalog /sample_files/configured_catalog.json +``` + +#### Create credentials +**If you are a community contributor**, follow the instructions in the [documentation](https://docs.airbyte.io/integrations/sources/kafka) +to generate the necessary credentials. Then create a file `secrets/config.json` conforming to the `spec.json` file. `secrets` is gitignored by default. + +**If you are an Airbyte core member**, copy the credentials from Lastpass under the secret name `source kafka test creds` +and place them into `secrets/config.json`. diff --git a/airbyte-integrations/connectors/source-kafka/build.gradle b/airbyte-integrations/connectors/source-kafka/build.gradle new file mode 100644 index 000000000000..5fcb74099344 --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/build.gradle @@ -0,0 +1,24 @@ +plugins { + id 'application' + id 'airbyte-docker' + id 'airbyte-integration-test-java' +} + +application { + mainClass = 'io.airbyte.integrations.source.kafka.KafkaSource' +} + +dependencies { + implementation project(':airbyte-config:models') + implementation project(':airbyte-protocol:models') + implementation project(':airbyte-integrations:bases:base-java') + + implementation 'org.apache.kafka:kafka-clients:2.8.0' + implementation 'org.apache.kafka:connect-json:2.8.0' + + integrationTestJavaImplementation project(':airbyte-integrations:bases:standard-source-test') + integrationTestJavaImplementation project(':airbyte-integrations:connectors:source-kafka') + integrationTestJavaImplementation "org.testcontainers:kafka:1.15.3" + + implementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs) +} diff --git a/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaProtocol.java b/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaProtocol.java new file mode 100644 index 000000000000..32dcdeb1b387 --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaProtocol.java @@ -0,0 +1,33 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.kafka; + +public enum KafkaProtocol { + + PLAINTEXT, + SASL_PLAINTEXT, + SASL_SSL + +} diff --git a/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaSource.java b/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaSource.java new file mode 100644 index 000000000000..cdb4879282e5 --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaSource.java @@ -0,0 +1,145 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.kafka; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.AbstractIterator; +import com.google.common.collect.Lists; +import io.airbyte.commons.util.AutoCloseableIterator; +import io.airbyte.commons.util.AutoCloseableIterators; +import io.airbyte.integrations.BaseConnector; +import io.airbyte.integrations.base.*; +import io.airbyte.protocol.models.*; +import io.airbyte.protocol.models.AirbyteConnectionStatus.Status; +import java.time.Duration; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.*; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaSource extends BaseConnector implements Source { + + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaSource.class); + + public KafkaSource() {} + + @Override + public AirbyteConnectionStatus check(JsonNode config) { + try { + final String testTopic = config.has("test_topic") ? config.get("test_topic").asText() : ""; + if (!testTopic.isBlank()) { + final KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.getKafkaSourceConfig(config); + final KafkaConsumer consumer = kafkaSourceConfig.getCheckConsumer(); + consumer.subscribe(Pattern.compile(testTopic)); + consumer.listTopics(); + consumer.close(); + LOGGER.info("Successfully connected to Kafka brokers for topic '{}'.", config.get("test_topic").asText()); + } + return new AirbyteConnectionStatus().withStatus(Status.SUCCEEDED); + } catch (Exception e) { + LOGGER.error("Exception attempting to connect to the Kafka brokers: ", e); + return new AirbyteConnectionStatus() + .withStatus(Status.FAILED) + .withMessage("Could not connect to the Kafka brokers with provided configuration. \n" + e.getMessage()); + } + } + + @Override + public AirbyteCatalog discover(JsonNode config) throws Exception { + + Set topicsToSubscribe = KafkaSourceConfig.getKafkaSourceConfig(config).getTopicsToSubscribe(); + List streams = topicsToSubscribe.stream().map(topic -> CatalogHelpers + .createAirbyteStream(topic, Field.of("value", JsonSchemaPrimitive.STRING)) + .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))) + .collect(Collectors.toList()); + return new AirbyteCatalog().withStreams(streams); + } + + @Override + public AutoCloseableIterator read(JsonNode config, ConfiguredAirbyteCatalog catalog, JsonNode state) throws Exception { + final AirbyteConnectionStatus check = check(config); + if (check.getStatus().equals(AirbyteConnectionStatus.Status.FAILED)) { + throw new RuntimeException("Unable establish a connection: " + check.getMessage()); + } + + final KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.getKafkaSourceConfig(config); + final KafkaConsumer consumer = kafkaSourceConfig.getConsumer(); + List> recordsList = new ArrayList<>(); + + int retry = config.has("repeated_calls") ? config.get("repeated_calls").intValue() : 0; + int pollCount = 0; + while (true) { + final ConsumerRecords consumerRecords = consumer.poll(Duration.of(100, ChronoUnit.MILLIS)); + if (consumerRecords.count() == 0) { + pollCount++; + if (pollCount > retry) { + break; + } + } + + consumerRecords.forEach(record -> { + LOGGER.info("Consumer Record: key - {}, value - {}, partition - {}, offset - {}", + record.key(), record.value(), record.partition(), record.offset()); + recordsList.add(record); + }); + consumer.commitAsync(); + } + consumer.close(); + Iterator> iterator = recordsList.iterator(); + + return AutoCloseableIterators.fromIterator(new AbstractIterator<>() { + + @Override + protected AirbyteMessage computeNext() { + if (iterator.hasNext()) { + ConsumerRecord record = iterator.next(); + return new AirbyteMessage() + .withType(AirbyteMessage.Type.RECORD) + .withRecord(new AirbyteRecordMessage() + .withStream(record.topic()) + .withEmittedAt(Instant.now().toEpochMilli()) + .withData(record.value())); + } + + return endOfData(); + } + + }); + } + + public static void main(String[] args) throws Exception { + final Source source = new KafkaSource(); + LOGGER.info("Starting source: {}", KafkaSource.class); + new IntegrationRunner(source).run(args); + LOGGER.info("Completed source: {}", KafkaSource.class); + } + +} diff --git a/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaSourceConfig.java b/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaSourceConfig.java new file mode 100644 index 000000000000..b0904434ab5b --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/src/main/java/io/airbyte/integrations/source/kafka/KafkaSourceConfig.java @@ -0,0 +1,160 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.kafka; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableMap; +import io.airbyte.commons.json.Jsons; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.connect.json.JsonDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaSourceConfig { + + protected static final Logger LOGGER = LoggerFactory.getLogger(KafkaSourceConfig.class); + private static KafkaSourceConfig instance; + private final JsonNode config; + private KafkaConsumer consumer; + private Set topicsToSubscribe; + + private KafkaSourceConfig(JsonNode config) { + this.config = config; + } + + public static KafkaSourceConfig getKafkaSourceConfig(JsonNode config) { + if (instance == null) { + instance = new KafkaSourceConfig(config); + } + return instance; + } + + private KafkaConsumer buildKafkaConsumer(JsonNode config) { + final Map props = new HashMap<>(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.get("bootstrap_servers").asText()); + props.put(ConsumerConfig.GROUP_ID_CONFIG, + config.has("group_id") ? config.get("group_id").asText() : null); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + config.has("max_poll_records") ? config.get("max_poll_records").intValue() : null); + props.putAll(propertiesByProtocol(config)); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, + config.has("client_id") ? config.get("client_id").asText() : null); + props.put(ConsumerConfig.CLIENT_DNS_LOOKUP_CONFIG, config.get("client_dns_lookup").asText()); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, config.get("enable_auto_commit").booleanValue()); + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, + config.has("auto_commit_interval_ms") ? config.get("auto_commit_interval_ms").intValue() : null); + props.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, + config.has("retry_backoff_ms") ? config.get("retry_backoff_ms").intValue() : null); + props.put(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, + config.has("request_timeout_ms") ? config.get("request_timeout_ms").intValue() : null); + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, + config.has("receive_buffer_bytes") ? config.get("receive_buffer_bytes").intValue() : null); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, JsonDeserializer.class.getName()); + + final Map filteredProps = props.entrySet().stream() + .filter(entry -> entry.getValue() != null && !entry.getValue().toString().isBlank()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + return new KafkaConsumer<>(filteredProps); + } + + private Map propertiesByProtocol(JsonNode config) { + JsonNode protocolConfig = config.get("protocol"); + LOGGER.info("Kafka protocol config: {}", protocolConfig.toString()); + final KafkaProtocol protocol = KafkaProtocol.valueOf(protocolConfig.get("security_protocol").asText().toUpperCase()); + final ImmutableMap.Builder builder = ImmutableMap.builder() + .put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, protocol.toString()); + + switch (protocol) { + case PLAINTEXT -> {} + case SASL_SSL, SASL_PLAINTEXT -> { + builder.put(SaslConfigs.SASL_JAAS_CONFIG, config.get("sasl_jaas_config").asText()); + builder.put(SaslConfigs.SASL_MECHANISM, config.get("sasl_mechanism").asText()); + } + default -> throw new RuntimeException("Unexpected Kafka protocol: " + Jsons.serialize(protocol)); + } + + return builder.build(); + } + + public KafkaConsumer getConsumer() { + if (consumer != null) { + return consumer; + } + consumer = buildKafkaConsumer(config); + + JsonNode subscription = config.get("subscription"); + LOGGER.info("Kafka subscribe method: {}", subscription.toString()); + switch (subscription.get("subscription_type").asText()) { + case "subscribe" -> { + String topicPattern = subscription.get("topic_pattern").asText(); + consumer.subscribe(Pattern.compile(topicPattern)); + topicsToSubscribe = consumer.listTopics().keySet().stream() + .filter(topic -> topic.matches(topicPattern)) + .collect(Collectors.toSet()); + } + case "assign" -> { + topicsToSubscribe = new HashSet<>(); + String topicPartitions = subscription.get("topic_partitions").asText(); + String[] topicPartitionsStr = topicPartitions.replaceAll("\\s+", "").split(","); + List topicPartitionList = Arrays.stream(topicPartitionsStr).map(topicPartition -> { + String[] pair = topicPartition.split(":"); + topicsToSubscribe.add(pair[0]); + return new TopicPartition(pair[0], Integer.parseInt(pair[1])); + }).collect(Collectors.toList()); + LOGGER.info("Topic-partition list: {}", topicPartitionList); + consumer.assign(topicPartitionList); + } + } + return consumer; + } + + public Set getTopicsToSubscribe() { + if (topicsToSubscribe == null) { + getConsumer(); + } + return topicsToSubscribe; + } + + public KafkaConsumer getCheckConsumer() { + return buildKafkaConsumer(config); + } + +} diff --git a/airbyte-integrations/connectors/source-kafka/src/main/resources/spec.json b/airbyte-integrations/connectors/source-kafka/src/main/resources/spec.json new file mode 100644 index 000000000000..21668c0b89e8 --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/src/main/resources/spec.json @@ -0,0 +1,216 @@ +{ + "documentationUrl": "https://docs.airbyte.io/integrations/sources/kafka", + "supportsIncremental": true, + "supportsNormalization": false, + "supportsDBT": false, + "supported_source_sync_modes": ["append"], + "connectionSpecification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "Kafka Source Spec", + "type": "object", + "required": [ + "bootstrap_servers", + "subscription", + "protocol" + ], + "additionalProperties": false, + "properties": { + "bootstrap_servers": { + "title": "Bootstrap servers", + "description": "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping—this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form host1:port1,host2:port2,.... Since these servers are just used for the initial connection to discover the full cluster membership (which may change dynamically), this list need not contain the full set of servers (you may want more than one, though, in case a server is down).", + "type": "string", + "examples": ["kafka-broker1:9092,kafka-broker2:9092"] + }, + "subscription": { + "title": "Subscribe method", + "type": "object", + "description": "You can choose to manually assign a list of partitions, or subscribe to all topics matching specified pattern to get dynamically assigned partitions", + "oneOf": [ + { + "title": "Manually assign a list of partitions", + "required": ["subscription_type", "topic_partitions"], + "properties": { + "subscription_type": { + "description": "Manually assign a list of partitions to this consumer. This interface does not allow for incremental assignment and will replace the previous assignment (if there is one).\nIf the given list of topic partitions is empty, it is treated the same as unsubscribe().", + "type": "string", + "const": "assign", + "enum": ["assign"], + "default": "assign" + }, + "topic_partitions": { + "title": "List of topic:partition pairs", + "type": "string", + "examples": ["sample.topic:0, sample.topic:1"] + } + } + }, + { + "title": "Subscribe to all topics matching specified pattern", + "required": ["subscription_type","topic_pattern"], + "properties": { + "subscription_type": { + "description": "Topic pattern from which the records will be read.", + "type": "string", + "const": "subscribe", + "enum": ["subscribe"], + "default": "subscribe" + }, + "topic_pattern": { + "title": "Topic pattern", + "type": "string", + "examples": ["sample.topic"] + } + } + } + ] + }, + "test_topic": { + "title": "Test topic", + "description": "Topic to test if Airbyte can consume messages.", + "type": "string", + "examples": ["test.topic"] + }, + "group_id": { + "title": "Group ID", + "description": "Group id.", + "type": "string", + "examples": ["group.id"] + }, + "max_poll_records": { + "title": "Max poll records", + "description": "The maximum number of records returned in a single call to poll(). Note, that max_poll_records does not impact the underlying fetching behavior. The consumer will cache the records from each fetch request and returns them incrementally from each poll.", + "type": "integer", + "default": 500 + }, + "protocol": { + "title": "Protocol", + "type": "object", + "description": "Protocol used to communicate with brokers.", + "oneOf": [ + { + "title": "PLAINTEXT", + "required": ["security_protocol"], + "properties": { + "security_protocol": { + "type": "string", + "enum": ["PLAINTEXT"], + "default": "PLAINTEXT" + } + } + }, + { + "title": "SASL PLAINTEXT", + "required": [ + "security_protocol", + "sasl_mechanism", + "sasl_jaas_config" + ], + "properties": { + "security_protocol": { + "type": "string", + "enum": ["SASL_PLAINTEXT"], + "default": "SASL_PLAINTEXT" + }, + "sasl_mechanism": { + "title": "SASL mechanism", + "description": "SASL mechanism used for client connections. This may be any mechanism for which a security provider is available.", + "type": "string", + "default": "PLAIN", + "enum": ["PLAIN"] + }, + "sasl_jaas_config": { + "title": "SASL JAAS config", + "description": "JAAS login context parameters for SASL connections in the format used by JAAS configuration files.", + "type": "string", + "default": "", + "airbyte_secret": true + } + } + }, + { + "title": "SASL SSL", + "required": [ + "security_protocol", + "sasl_mechanism", + "sasl_jaas_config" + ], + "properties": { + "security_protocol": { + "type": "string", + "enum": ["SASL_SSL"], + "default": "SASL_SSL" + }, + "sasl_mechanism": { + "title": "SASL mechanism", + "description": "SASL mechanism used for client connections. This may be any mechanism for which a security provider is available.", + "type": "string", + "default": "GSSAPI", + "enum": ["GSSAPI", "OAUTHBEARER", "SCRAM-SHA-256"] + }, + "sasl_jaas_config": { + "title": "SASL JAAS config", + "description": "JAAS login context parameters for SASL connections in the format used by JAAS configuration files.", + "type": "string", + "default": "", + "airbyte_secret": true + } + } + } + ] + }, + "client_id": { + "title": "Client ID", + "description": "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.", + "type": "string", + "examples": ["airbyte-consumer"] + }, + "enable_auto_commit": { + "title": "Enable auto commit", + "description": "If true the consumer's offset will be periodically committed in the background.", + "type": "boolean", + "default": true + }, + "auto_commit_interval_ms": { + "title": "Auto commit interval ms", + "description": "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if enable.auto.commit is set to true.", + "type": "integer", + "default": 5000 + }, + "client_dns_lookup": { + "title": "Client DNS lookup", + "description": "Controls how the client uses DNS lookups. If set to use_all_dns_ips, connect to each returned IP address in sequence until a successful connection is established. After a disconnection, the next IP is used. Once all IPs have been used once, the client resolves the IP(s) from the hostname again. If set to resolve_canonical_bootstrap_servers_only, resolve each bootstrap address into a list of canonical names. After the bootstrap phase, this behaves the same as use_all_dns_ips. If set to default (deprecated), attempt to connect to the first IP address returned by the lookup, even if the lookup returns multiple IP addresses.", + "type": "string", + "default": "use_all_dns_ips", + "enum": [ + "default", + "use_all_dns_ips", + "resolve_canonical_bootstrap_servers_only" + ] + }, + "retry_backoff_ms": { + "title": "Retry backoff ms", + "description": "The amount of time to wait before attempting to retry a failed request to a given topic partition. This avoids repeatedly sending requests in a tight loop under some failure scenarios.", + "type": "integer", + "default": 100 + }, + "request_timeout_ms": { + "title": "Request timeout ms", + "description": "The configuration controls the maximum amount of time the client will wait for the response of a request. If the response is not received before the timeout elapses the client will resend the request if necessary or fail the request if retries are exhausted.", + "type": "integer", + "default": 30000 + }, + "receive_buffer_bytes": { + "title": "Receive buffer bytes", + "description": "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data. If the value is -1, the OS default will be used.", + "type": "integer", + "default": 32768 + }, + "repeated_calls": { + "title": "Repeated calls", + "description": "The number of repeated calls to poll() if no messages were received.", + "type": "integer", + "default": 3 + } + } + } +} diff --git a/airbyte-integrations/connectors/source-kafka/src/test-integration/java/io/airbyte/integrations/source/kafka/KafkaSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-kafka/src/test-integration/java/io/airbyte/integrations/source/kafka/KafkaSourceAcceptanceTest.java new file mode 100644 index 000000000000..f4b23ec6b743 --- /dev/null +++ b/airbyte-integrations/connectors/source-kafka/src/test-integration/java/io/airbyte/integrations/source/kafka/KafkaSourceAcceptanceTest.java @@ -0,0 +1,100 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.kafka; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.ImmutableMap; +import io.airbyte.commons.jackson.MoreMappers; +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.resources.MoreResources; +import io.airbyte.integrations.standardtest.source.SourceAcceptanceTest; +import io.airbyte.integrations.standardtest.source.TestDestinationEnv; +import io.airbyte.protocol.models.*; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +public class KafkaSourceAcceptanceTest extends SourceAcceptanceTest { + + private static final ObjectMapper mapper = MoreMappers.initMapper(); + private static final String TOPIC_NAME = "test.topic"; + + private static KafkaContainer KAFKA; + + @Override + protected String getImageName() { + return "airbyte/source-kafka:dev"; + } + + @Override + protected JsonNode getConfig() { + ObjectNode stubProtocolConfig = mapper.createObjectNode(); + stubProtocolConfig.put("security_protocol", KafkaProtocol.PLAINTEXT.toString()); + + return Jsons.jsonNode(ImmutableMap.builder() + .put("bootstrap_servers", KAFKA.getBootstrapServers()) + .put("topic_pattern", TOPIC_NAME) + .build()); + } + + @Override + protected void setupEnvironment(TestDestinationEnv environment) throws Exception { + KAFKA = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.0")); + KAFKA.start(); + } + + @Override + protected void tearDown(TestDestinationEnv testEnv) { + KAFKA.close(); + } + + @Override + protected ConnectorSpecification getSpec() throws Exception { + return Jsons.deserialize(MoreResources.readResource("spec.json"), ConnectorSpecification.class); + } + + @Override + protected ConfiguredAirbyteCatalog getConfiguredCatalog() throws Exception { + return CatalogHelpers.createConfiguredAirbyteCatalog( + TOPIC_NAME, + null, + Field.of("value", JsonSchemaPrimitive.STRING)); + } + + @Override + protected JsonNode getState() throws Exception { + return Jsons.jsonNode(new HashMap<>()); + } + + @Override + protected List getRegexTests() throws Exception { + return Collections.emptyList(); + } + +} diff --git a/docs/SUMMARY.md b/docs/SUMMARY.md index 2010d444ce97..47cb33cf795f 100644 --- a/docs/SUMMARY.md +++ b/docs/SUMMARY.md @@ -71,6 +71,7 @@ * [Intercom](integrations/sources/intercom.md) * [Iterable](integrations/sources/iterable.md) * [Jira](integrations/sources/jira.md) + * [Kafka](integrations/sources/kafka.md) * [Klaviyo](integrations/sources/klaviyo.md) * [Kustomer](integrations/sources/kustomer.md) * [Lever Hiring](integrations/sources/lever-hiring.md) diff --git a/docs/integrations/sources/kafka.md b/docs/integrations/sources/kafka.md new file mode 100644 index 000000000000..d4e2379541fd --- /dev/null +++ b/docs/integrations/sources/kafka.md @@ -0,0 +1,76 @@ +# Kafka + +## Overview + +The Airbyte Kafka source allows you to sync data from Kafka. Each Kafka topic is written to the corresponding stream. + +### Sync overview + +#### Output schema + +Each Kafka topic will be output into a stream. + +Currently, this connector only reads data with JSON format. More formats (e.g. Apache Avro) will be supported in +the future. + +#### Features + +| Feature | Supported?\(Yes/No\) | Notes | +| :--- | :--- | :--- | +| Full Refresh Sync | Yes | | +| Incremental - Append Sync | Yes | | +| Namespaces | No | | + +## Getting started + +### Requirements + +To use the Kafka source, you'll need: + +* A Kafka cluster 1.0 or above. + +### Setup guide + +#### Network Access + +Make sure your Kafka brokers can be accessed by Airbyte. + +#### **Permissions** + +Airbyte should be allowed to read messages from topics, and these topics should be created before reading from Kafka. + +#### Target topics + +You can determine the topics from which messages are read via the `topic_pattern` configuration parameter. +Messages can be read from a hardcoded, pre-defined topic. + +To read all messages from a single hardcoded topic, enter its name in the `topic_pattern` field +e.g: setting `topic_pattern` to `my-topic-name` will read all messages from that topic. + +You can determine the topic partitions from which messages are read via the `topic_partitions` configuration parameter. + +### Setup the Kafka destination in Airbyte + +You should now have all the requirements needed to configure Kafka as a destination in the UI. You can configure the +following parameters on the Kafka destination (though many of these are optional or have default values): + +* **Bootstrap servers** +* **Topic pattern** +* **Topic partition** +* **Test topic** +* **Group ID** +* **Max poll records** +* **SASL JAAS config** +* **SASL mechanism** +* **Client ID** +* **Enable auto commit** +* **Auto commit interval ms** +* **Client DNS lookup** +* **Retry backoff ms** +* **Request timeout ms** +* **Receive buffer bytes** +* **Repeated calls** + +More info about this can be found in the [Kafka consumer configs documentation site](https://kafka.apache.org/documentation/#consumerconfigs). + +## Changelog