Skip to content
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

Updates Kafka configurations such that plugin has its own topic configurations #3551

Merged
merged 5 commits into from
Oct 31, 2023
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
*/
public class ByteCount {
private static final Pattern BYTE_PATTERN = Pattern.compile("^(?<value>\\d+\\.?\\d*)(?<unit>[a-z]+)?\\z");
private static final ByteCount ZERO_BYTES = new ByteCount(0);
private final long bytes;

private ByteCount(final long bytes) {
Expand Down Expand Up @@ -94,6 +95,10 @@ public static ByteCount parse(final String string) {
return new ByteCount(byteCount.longValue());
}

public static ByteCount zeroBytes() {
return ZERO_BYTES;
}

private static BigDecimal scaleToBytes(final BigDecimal value, final Unit unit) {
return value.multiply(BigDecimal.valueOf(unit.multiplier));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@

package org.opensearch.dataprepper.model.types;

import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import org.junit.jupiter.params.provider.ValueSource;
Expand All @@ -13,6 +14,7 @@
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.sameInstance;
import static org.junit.jupiter.api.Assertions.assertThrows;

class ByteCountTest {
Expand Down Expand Up @@ -145,4 +147,16 @@ void parse_returns_rounded_bytes_for_implicit_fractional_bytes(final String byte
assertThat(byteCount, notNullValue());
assertThat(byteCount.getBytes(), equalTo(expectedBytes));
}

@Test
void zeroBytes_returns_bytes_with_getBytes_equal_to_0() {
assertThat(ByteCount.zeroBytes(), notNullValue());
assertThat(ByteCount.zeroBytes().getBytes(), equalTo(0L));
}

@Test
void zeroBytes_returns_same_instance() {
assertThat(ByteCount.zeroBytes(), notNullValue());
assertThat(ByteCount.zeroBytes(), sameInstance(ByteCount.zeroBytes()));
}
}
Original file line number Diff line number Diff line change
@@ -1,3 +1,8 @@
/*
* Copyright OpenSearch Contributors
* SPDX-License-Identifier: Apache-2.0
*/

package org.opensearch.dataprepper.plugins.kafka.buffer;

import org.apache.commons.lang3.RandomStringUtils;
Expand All @@ -15,8 +20,6 @@
import org.opensearch.dataprepper.model.plugin.PluginFactory;
import org.opensearch.dataprepper.model.record.Record;
import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaBufferConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -50,7 +53,7 @@ public class KafkaBufferIT {
@Mock
private AcknowledgementSetManager acknowledgementSetManager;
@Mock
private TopicConfig topicConfig;
private BufferTopicConfig topicConfig;

private PluginMetrics pluginMetrics;
private String bootstrapServersCommaDelimited;
Expand All @@ -66,7 +69,7 @@ void setUp() {
String topicName = "buffer-" + RandomStringUtils.randomAlphabetic(5);
when(topicConfig.getName()).thenReturn(topicName);
when(topicConfig.getGroupId()).thenReturn("buffergroup-" + RandomStringUtils.randomAlphabetic(6));
when(topicConfig.isCreate()).thenReturn(true);
when(topicConfig.isCreateTopic()).thenReturn(true);
when(topicConfig.getSerdeFormat()).thenReturn(messageFormat);
when(topicConfig.getWorkers()).thenReturn(1);
when(topicConfig.getMaxPollInterval()).thenReturn(Duration.ofSeconds(5));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,9 @@
import org.opensearch.dataprepper.plugins.dlq.DlqProvider;
import org.opensearch.dataprepper.plugins.dlq.DlqWriter;
import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.ProducerTopicConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import java.io.IOException;
Expand Down Expand Up @@ -71,7 +70,7 @@ public class KafkaSinkAvroTypeIT {
private KafkaSinkConfig kafkaSinkConfig;

@Mock
private TopicConfig topicConfig;
private ProducerTopicConfig topicConfig;

private KafkaSink kafkaSink;

Expand Down Expand Up @@ -148,13 +147,12 @@ public void setup() throws RestClientException, IOException {
final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5);

topicConfig = mock(TopicConfig.class);
topicConfig = mock(ProducerTopicConfig.class);
when(topicConfig.getName()).thenReturn(testTopic);
when(topicConfig.getGroupId()).thenReturn(testGroup);
when(topicConfig.getWorkers()).thenReturn(1);
when(topicConfig.getSessionTimeOut()).thenReturn(Duration.ofSeconds(45));
when(topicConfig.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3));
when(topicConfig.getAutoCommit()).thenReturn(false);
when(topicConfig.getAutoOffsetReset()).thenReturn("earliest");
when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1));

Expand All @@ -172,7 +170,7 @@ public void TestPollRecordsAvroSASLPlainText() throws Exception {

final int numRecords = 1;
when(topicConfig.getConsumerMaxPollRecords()).thenReturn(numRecords);
when(topicConfig.isCreate()).thenReturn(false);
when(topicConfig.isCreateTopic()).thenReturn(false);
when(kafkaSinkConfig.getTopic()).thenReturn(topicConfig);

when(kafkaSinkConfig.getAuthConfig()).thenReturn(authConfig);
Expand Down Expand Up @@ -250,13 +248,8 @@ private void deleteTopic(AtomicBoolean created, String topicName) throws Interru
}

private void consumeTestMessages(List<Record<Event>> recList) {

props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG,
topicConfig.getCommitInterval().toSecondsPart());
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
topicConfig.getAutoOffsetReset());
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
topicConfig.getAutoCommit());
props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG,
topicConfig.getConsumerMaxPollRecords());
props.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,8 @@
import org.opensearch.dataprepper.plugins.dlq.DlqProvider;
import org.opensearch.dataprepper.plugins.dlq.DlqWriter;
import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.ProducerTopicConfig;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import java.time.Duration;
Expand All @@ -63,7 +62,7 @@ public class KafkaSinkJsonTypeIT {
private KafkaSinkConfig kafkaSinkConfig;

@Mock
private TopicConfig topicConfig;
private ProducerTopicConfig topicConfig;

private KafkaSink kafkaSink;

Expand Down Expand Up @@ -126,13 +125,12 @@ public void setup() {
final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5);

topicConfig = mock(TopicConfig.class);
topicConfig = mock(ProducerTopicConfig.class);
when(topicConfig.getName()).thenReturn(testTopic);
when(topicConfig.getGroupId()).thenReturn(testGroup);
when(topicConfig.getWorkers()).thenReturn(1);
when(topicConfig.getSessionTimeOut()).thenReturn(Duration.ofSeconds(45));
when(topicConfig.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3));
when(topicConfig.getAutoCommit()).thenReturn(false);
when(topicConfig.getAutoOffsetReset()).thenReturn("earliest");
when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1));
bootstrapServers = System.getProperty("tests.kafka.bootstrap_servers");
Expand All @@ -146,7 +144,7 @@ public void TestPollRecordsJsonSASLPlainText() throws Exception {

final int numRecords = 1;
when(topicConfig.getConsumerMaxPollRecords()).thenReturn(numRecords);
when(topicConfig.isCreate()).thenReturn(false);
when(topicConfig.isCreateTopic()).thenReturn(false);
when(kafkaSinkConfig.getTopic()).thenReturn(topicConfig);
when(kafkaSinkConfig.getAuthConfig()).thenReturn(authConfig);
kafkaSink = createObjectUnderTest();
Expand Down Expand Up @@ -223,13 +221,8 @@ private void configureJasConfForSASLPlainText() {
}

private void consumeTestMessages(List<Record<Event>> recList) {

props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG,
topicConfig.getCommitInterval().toSecondsPart());
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
topicConfig.getAutoOffsetReset());
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
topicConfig.getAutoCommit());
props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG,
topicConfig.getConsumerMaxPollRecords());
props.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,8 @@
import org.opensearch.dataprepper.plugins.dlq.DlqProvider;
import org.opensearch.dataprepper.plugins.dlq.DlqWriter;
import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.ProducerTopicConfig;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import java.time.Duration;
Expand All @@ -61,7 +60,7 @@ public class KafkaSinkPlainTextTypeIT {
private KafkaSinkConfig kafkaSinkConfig;

@Mock
private TopicConfig topicConfig;
private ProducerTopicConfig topicConfig;

private KafkaSink kafkaSink;

Expand Down Expand Up @@ -124,13 +123,12 @@ public void setup() {
final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5);

topicConfig = mock(TopicConfig.class);
topicConfig = mock(ProducerTopicConfig.class);
when(topicConfig.getName()).thenReturn(testTopic);
when(topicConfig.getGroupId()).thenReturn(testGroup);
when(topicConfig.getWorkers()).thenReturn(1);
when(topicConfig.getSessionTimeOut()).thenReturn(Duration.ofSeconds(45));
when(topicConfig.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3));
when(topicConfig.getAutoCommit()).thenReturn(false);
when(topicConfig.getAutoOffsetReset()).thenReturn("earliest");
when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1));
bootstrapServers = System.getProperty("tests.kafka.bootstrap_servers");
Expand All @@ -145,7 +143,7 @@ public void TestPollRecordsPlainText() throws Exception {

final int numRecords = 1;
when(topicConfig.getConsumerMaxPollRecords()).thenReturn(numRecords);
when(topicConfig.isCreate()).thenReturn(false);
when(topicConfig.isCreateTopic()).thenReturn(false);
when(kafkaSinkConfig.getTopic()).thenReturn(topicConfig);
when(kafkaSinkConfig.getAuthConfig()).thenReturn(authConfig);
kafkaSink = createObjectUnderTest();
Expand Down Expand Up @@ -221,13 +219,8 @@ private void deleteTopic(AtomicBoolean created, String topicName) throws Interru
}

private void consumeTestMessages(List<Record<Event>> recList) {

props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG,
topicConfig.getCommitInterval().toSecondsPart());
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
topicConfig.getAutoOffsetReset());
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
topicConfig.getAutoCommit());
props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG,
topicConfig.getConsumerMaxPollRecords());
props.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,49 +5,45 @@

package org.opensearch.dataprepper.plugins.kafka.source;

import io.micrometer.core.instrument.Counter;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.StringSerializer;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
import org.opensearch.dataprepper.metrics.PluginMetrics;
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager;
import org.opensearch.dataprepper.model.buffer.Buffer;
import org.opensearch.dataprepper.model.record.Record;
import org.opensearch.dataprepper.model.configuration.PipelineDescription;
import org.opensearch.dataprepper.model.event.Event;
import org.opensearch.dataprepper.model.record.Record;
import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.ConsumerTopicConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType;
import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig;
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager;
import org.opensearch.dataprepper.model.configuration.PipelineDescription;

import static org.mockito.Mockito.when;
import org.mockito.Mock;
import static org.mockito.Mockito.mock;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.awaitility.Awaitility.await;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import io.micrometer.core.instrument.Counter;
import java.util.List;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.Iterator;

import java.time.Duration;
import java.time.Instant;

import org.apache.kafka.common.errors.SerializationException;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;
import static org.awaitility.Awaitility.await;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

public class ConfluentKafkaProducerConsumerIT {
@Mock
Expand Down Expand Up @@ -78,7 +74,7 @@ public class ConfluentKafkaProducerConsumerIT {
private PlainTextAuthConfig plainTextAuthConfig;

private KafkaSource kafkaSource;
private TopicConfig topicConfig;
private ConsumerTopicConfig topicConfig;
private Counter counter;
private List<Record> receivedRecords;

Expand Down Expand Up @@ -129,7 +125,7 @@ public void setup() {
topicName = System.getProperty("tests.kafka.topic_name");
username = System.getProperty("tests.kafka.username");
password = System.getProperty("tests.kafka.password");
topicConfig = mock(TopicConfig.class);
topicConfig = mock(ConsumerTopicConfig.class);
when(topicConfig.getName()).thenReturn(topicName);
when(topicConfig.getGroupId()).thenReturn("testGroupConf");
when(topicConfig.getWorkers()).thenReturn(1);
Expand All @@ -147,7 +143,7 @@ public void setup() {
when(topicConfig.getFetchMaxWait()).thenReturn(500);
when(topicConfig.getMaxPartitionFetchBytes()).thenReturn(1024L*1024);
when(topicConfig.getReconnectBackoff()).thenReturn(Duration.ofSeconds(10));
when(sourceConfig.getTopics()).thenReturn(List.of(topicConfig));
when(sourceConfig.getTopics()).thenReturn((List) List.of(topicConfig));
when(sourceConfig.getBootstrapServers()).thenReturn(List.of(bootstrapServers));
encryptionConfig = mock(EncryptionConfig.class);
when(sourceConfig.getEncryptionConfig()).thenReturn(encryptionConfig);
Expand Down
Loading
Loading