Skip to content

Commit

Permalink
Updates Kafka configurations such that plugin has its own topic confi…
Browse files Browse the repository at this point in the history
…gurations (opensearch-project#3551)

Updates Kafka buffer configurations to disallow configurations which are not needed - e.g. schema and the serde_format for the topic configuration. As part of this change, I also split the TopicConfig into three distinct interfaces and classes. This allows each plugin to either accept a configuration or provide a value of the plugin's own choosing. Also adds copyright headers to all files modified as part of this commit.

Renamed is_topic_create to create_topic. Also made this a boolean internally instead of Boolean since it will have a value.

Adds a zeroBytes() static method to ByteCount as a convenience.

Signed-off-by: David Venable <[email protected]>
  • Loading branch information
dlvenable authored Oct 31, 2023
1 parent cb765d8 commit 7323c9e
Show file tree
Hide file tree
Showing 52 changed files with 1,118 additions and 688 deletions.
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,14 +36,12 @@
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.SchemaConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicProducerConfig;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand Down Expand Up @@ -71,7 +69,7 @@ public class KafkaSinkAvroTypeIT {
private KafkaSinkConfig kafkaSinkConfig;

@Mock
private TopicConfig topicConfig;
private TopicProducerConfig topicConfig;

private KafkaSink kafkaSink;

Expand Down Expand Up @@ -145,18 +143,10 @@ public void setup() throws RestClientException, IOException {
when(kafkaSinkConfig.getSerdeFormat()).thenReturn(MessageFormat.AVRO.toString());
when(kafkaSinkConfig.getPartitionKey()).thenReturn("test-${name}");

final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5);

topicConfig = mock(TopicConfig.class);
topicConfig = mock(TopicProducerConfig.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");
when(kafkaSinkConfig.getBootstrapServers()).thenReturn(Collections.singletonList(bootstrapServers));
Expand All @@ -171,8 +161,7 @@ public void TestPollRecordsAvroSASLPlainText() throws Exception {
configureJasConfForSASLPlainText();

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,16 +239,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());
final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
props.put(ConsumerConfig.GROUP_ID_CONFIG, testGroup);
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
StringDeserializer.class);
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,10 @@
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.TopicProducerConfig;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand All @@ -63,7 +61,7 @@ public class KafkaSinkJsonTypeIT {
private KafkaSinkConfig kafkaSinkConfig;

@Mock
private TopicConfig topicConfig;
private TopicProducerConfig topicConfig;

private KafkaSink kafkaSink;

Expand Down Expand Up @@ -123,18 +121,10 @@ public void setup() {
when(kafkaSinkConfig.getSerdeFormat()).thenReturn(MessageFormat.JSON.toString());
when(kafkaSinkConfig.getPartitionKey()).thenReturn("test-${name}");

final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5);

topicConfig = mock(TopicConfig.class);
topicConfig = mock(TopicProducerConfig.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");
when(kafkaSinkConfig.getBootstrapServers()).thenReturn(Collections.singletonList(bootstrapServers));
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
Expand All @@ -145,8 +135,7 @@ public void TestPollRecordsJsonSASLPlainText() throws Exception {
configureJasConfForSASLPlainText();

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,16 +212,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());
final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
props.put(ConsumerConfig.GROUP_ID_CONFIG, testGroup);
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
StringDeserializer.class);
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,12 +32,10 @@
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.TopicProducerConfig;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand All @@ -61,7 +59,7 @@ public class KafkaSinkPlainTextTypeIT {
private KafkaSinkConfig kafkaSinkConfig;

@Mock
private TopicConfig topicConfig;
private TopicProducerConfig topicConfig;

private KafkaSink kafkaSink;

Expand Down Expand Up @@ -121,18 +119,10 @@ public void setup() {
when(kafkaSinkConfig.getSerdeFormat()).thenReturn(MessageFormat.PLAINTEXT.toString());
when(kafkaSinkConfig.getPartitionKey()).thenReturn("test-${name}");

final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);
testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5);

topicConfig = mock(TopicConfig.class);
topicConfig = mock(TopicProducerConfig.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");
when(kafkaSinkConfig.getBootstrapServers()).thenReturn(Collections.singletonList(bootstrapServers));
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
Expand All @@ -144,8 +134,7 @@ public void TestPollRecordsPlainText() throws Exception {
configureJasConfForSASLPlainText();

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,16 +210,9 @@ private void deleteTopic(AtomicBoolean created, String topicName) throws Interru
}

private void consumeTestMessages(List<Record<Event>> recList) {
final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5);

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());
props.put(ConsumerConfig.GROUP_ID_CONFIG, testGroup);
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
StringDeserializer.class);
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
Expand Down
Loading

0 comments on commit 7323c9e

Please sign in to comment.