-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
* Initial changes for Kafka3 support. * merged main and moving the common Kafka code to kafka-base module. * refactored common Kafka2 and Kafka3 code * revert maven kafka changes as per the PR comment. * Added integration test for Kafka3. * Changes as per the PR comments. * Fixed the compilation issue of last commit. * As per the PR comment.
- Loading branch information
Showing
32 changed files
with
1,930 additions
and
27 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
529 changes: 529 additions & 0 deletions
529
...test/java/org/apache/pinot/integration/tests/LLCRealtimeKafka3ClusterIntegrationTest.java
Large diffs are not rendered by default.
Oops, something went wrong.
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
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
79 changes: 79 additions & 0 deletions
79
pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/pom.xml
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,79 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
<!-- | ||
Licensed to the Apache Software Foundation (ASF) under one | ||
or more contributor license agreements. See the NOTICE file | ||
distributed with this work for additional information | ||
regarding copyright ownership. The ASF licenses this file | ||
to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
Unless required by applicable law or agreed to in writing, | ||
software distributed under the License is distributed on an | ||
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
KIND, either express or implied. See the License for the | ||
specific language governing permissions and limitations | ||
under the License. | ||
--> | ||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
<parent> | ||
<artifactId>pinot-stream-ingestion</artifactId> | ||
<groupId>org.apache.pinot</groupId> | ||
<version>1.3.0-SNAPSHOT</version> | ||
<relativePath>..</relativePath> | ||
</parent> | ||
|
||
<artifactId>pinot-kafka-3.0</artifactId> | ||
<name>Pinot Kafka 3.x</name> | ||
<url>https://pinot.apache.org/</url> | ||
<properties> | ||
<pinot.root>${basedir}/../../..</pinot.root> | ||
<shade.phase.prop>package</shade.phase.prop> | ||
<kafka.lib.version>3.8.0</kafka.lib.version> | ||
</properties> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.pinot</groupId> | ||
<artifactId>pinot-kafka-base</artifactId> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>org.apache.kafka</groupId> | ||
<artifactId>kafka-clients</artifactId> | ||
<version>${kafka.lib.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.kafka</groupId> | ||
<artifactId>kafka_${scala.compat.version}</artifactId> | ||
<version>${kafka.lib.version}</version> | ||
<exclusions> | ||
<exclusion> | ||
<groupId>org.apache.kafka</groupId> | ||
<artifactId>kafka-clients</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>commons-logging</groupId> | ||
<artifactId>commons-logging</artifactId> | ||
</exclusion> | ||
</exclusions> | ||
</dependency> | ||
<dependency> | ||
<groupId>com.fasterxml.jackson.module</groupId> | ||
<artifactId>jackson-module-scala_${scala.compat.version}</artifactId> | ||
</dependency> | ||
|
||
<!-- Test --> | ||
<dependency> | ||
<groupId>org.apache.pinot</groupId> | ||
<artifactId>pinot-kafka-base</artifactId> | ||
<type>test-jar</type> | ||
<scope>test</scope> | ||
</dependency> | ||
</dependencies> | ||
</project> |
53 changes: 53 additions & 0 deletions
53
...-kafka-3.0/src/main/java/org/apache/pinot/plugin/stream/kafka30/KafkaConsumerFactory.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,53 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.pinot.plugin.stream.kafka30; | ||
|
||
import org.apache.pinot.plugin.stream.kafka.KafkaConfigBackwardCompatibleUtils; | ||
import org.apache.pinot.spi.stream.PartitionGroupConsumer; | ||
import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; | ||
import org.apache.pinot.spi.stream.StreamConfig; | ||
import org.apache.pinot.spi.stream.StreamConsumerFactory; | ||
import org.apache.pinot.spi.stream.StreamMetadataProvider; | ||
|
||
|
||
public class KafkaConsumerFactory extends StreamConsumerFactory { | ||
|
||
@Override | ||
protected void init(StreamConfig streamConfig) { | ||
KafkaConfigBackwardCompatibleUtils.handleStreamConfig(streamConfig); | ||
super.init(streamConfig); | ||
} | ||
|
||
@Override | ||
public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { | ||
return new KafkaStreamMetadataProvider(clientId, _streamConfig, partition); | ||
} | ||
|
||
@Override | ||
public StreamMetadataProvider createStreamMetadataProvider(String clientId) { | ||
return new KafkaStreamMetadataProvider(clientId, _streamConfig); | ||
} | ||
|
||
@Override | ||
public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, | ||
PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { | ||
return new KafkaPartitionLevelConsumer(clientId, _streamConfig, | ||
partitionGroupConsumptionStatus.getPartitionGroupId()); | ||
} | ||
} |
110 changes: 110 additions & 0 deletions
110
...ain/java/org/apache/pinot/plugin/stream/kafka30/KafkaPartitionLevelConnectionHandler.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,110 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.pinot.plugin.stream.kafka30; | ||
|
||
import com.google.common.annotations.VisibleForTesting; | ||
import com.google.common.util.concurrent.Uninterruptibles; | ||
import java.io.IOException; | ||
import java.util.Collections; | ||
import java.util.Properties; | ||
import java.util.concurrent.TimeUnit; | ||
import org.apache.kafka.clients.consumer.Consumer; | ||
import org.apache.kafka.clients.consumer.ConsumerConfig; | ||
import org.apache.kafka.clients.consumer.KafkaConsumer; | ||
import org.apache.kafka.common.KafkaException; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.common.serialization.BytesDeserializer; | ||
import org.apache.kafka.common.serialization.StringDeserializer; | ||
import org.apache.kafka.common.utils.Bytes; | ||
import org.apache.pinot.plugin.stream.kafka.KafkaPartitionLevelStreamConfig; | ||
import org.apache.pinot.plugin.stream.kafka.KafkaSSLUtils; | ||
import org.apache.pinot.spi.stream.StreamConfig; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
|
||
/** | ||
* KafkaPartitionLevelConnectionHandler provides low level APIs to access Kafka partition level information. | ||
* E.g. partition counts, offsets per partition. | ||
* | ||
*/ | ||
public abstract class KafkaPartitionLevelConnectionHandler { | ||
|
||
private static final Logger LOGGER = LoggerFactory.getLogger(KafkaPartitionLevelConnectionHandler.class); | ||
protected final KafkaPartitionLevelStreamConfig _config; | ||
protected final String _clientId; | ||
protected final int _partition; | ||
protected final String _topic; | ||
protected final Consumer<String, Bytes> _consumer; | ||
protected final TopicPartition _topicPartition; | ||
|
||
public KafkaPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig, int partition) { | ||
_config = new KafkaPartitionLevelStreamConfig(streamConfig); | ||
_clientId = clientId; | ||
_partition = partition; | ||
_topic = _config.getKafkaTopicName(); | ||
Properties consumerProp = new Properties(); | ||
consumerProp.putAll(streamConfig.getStreamConfigsMap()); | ||
consumerProp.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, _config.getBootstrapHosts()); | ||
consumerProp.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); | ||
consumerProp.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, BytesDeserializer.class.getName()); | ||
if (_config.getKafkaIsolationLevel() != null) { | ||
consumerProp.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, _config.getKafkaIsolationLevel()); | ||
} | ||
consumerProp.put(ConsumerConfig.CLIENT_ID_CONFIG, _clientId); | ||
KafkaSSLUtils.initSSL(consumerProp); | ||
_consumer = createConsumer(consumerProp); | ||
_topicPartition = new TopicPartition(_topic, _partition); | ||
_consumer.assign(Collections.singletonList(_topicPartition)); | ||
} | ||
|
||
private Consumer<String, Bytes> createConsumer(Properties consumerProp) { | ||
// Creation of the KafkaConsumer can fail for multiple reasons including DNS issues. | ||
// We arbitrarily chose 5 retries with 2 seconds sleep in between retries. 10 seconds total felt | ||
// like a good balance of not waiting too long for a retry, but also not retrying too many times. | ||
int maxTries = 5; | ||
int tries = 0; | ||
while (true) { | ||
try { | ||
return new KafkaConsumer<>(consumerProp); | ||
} catch (KafkaException e) { | ||
tries++; | ||
if (tries >= maxTries) { | ||
LOGGER.error("Caught exception while creating Kafka consumer, giving up", e); | ||
throw e; | ||
} | ||
LOGGER.warn("Caught exception while creating Kafka consumer, retrying {}/{}", tries, maxTries, e); | ||
// We are choosing to sleepUniterruptibly here because other parts of the Kafka consumer code do this | ||
// as well. We don't want random interrupts to cause us to fail to create the consumer and have the table | ||
// stuck in ERROR state. | ||
Uninterruptibles.sleepUninterruptibly(2, TimeUnit.SECONDS); | ||
} | ||
} | ||
} | ||
|
||
public void close() | ||
throws IOException { | ||
_consumer.close(); | ||
} | ||
|
||
@VisibleForTesting | ||
public KafkaPartitionLevelStreamConfig getKafkaPartitionLevelStreamConfig() { | ||
return _config; | ||
} | ||
} |
Oops, something went wrong.