Skip to content

Commit

Permalink
Add tests and reviews
Browse files Browse the repository at this point in the history
  • Loading branch information
raminqaf committed Jun 24, 2024
1 parent 7b5c0cb commit 727ceb2
Show file tree
Hide file tree
Showing 4 changed files with 71 additions and 38 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,15 @@

package com.bakdata.kafka;

import static org.apache.kafka.streams.StreamsConfig.APPLICATION_SERVER_CONFIG;

import java.util.Collections;
import java.util.Map;
import java.util.Optional;
import lombok.NonNull;
import lombok.Value;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.state.HostInfo;

/**
* Class for simplified access to configs provided by {@link StreamsConfig}
Expand Down Expand Up @@ -64,4 +68,22 @@ public String getBoostrapServers() {
public Map<String, Object> getKafkaProperties() {
return Collections.unmodifiableMap(this.kafkaProperties.originals());
}

/**
* Retrieves the host information based on the application server configuration.
*
* @return an {@code Optional} containing the {@link HostInfo} if the
* {@link StreamsConfig#APPLICATION_SERVER_CONFIG} is set; otherwise, an empty {@code Optional}.
*/
public Optional<HostInfo> getHostInfo() {
final String applicationServerConfig = this.kafkaProperties.getString(APPLICATION_SERVER_CONFIG);
return applicationServerConfig.isEmpty() ? Optional.empty()
: Optional.of(createHostInfo(applicationServerConfig));
}

private static HostInfo createHostInfo(final String applicationServerConfig) {
final String[] hostAndPort = applicationServerConfig.split(":");
return new HostInfo(hostAndPort[0], Integer.parseInt(hostAndPort[1]));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -48,27 +48,9 @@
public class RunningStreams {

@NonNull
StreamsConfig config;
ImprovedStreamsConfig config;
@NonNull
Topology topology;
@NonNull
KafkaStreams streams;

/**
* Retrieves the host information based on the application server configuration.
*
* @return an {@code Optional} containing the {@link HostInfo} if the
* {@link StreamsConfig#APPLICATION_SERVER_CONFIG} is set; otherwise, an empty {@code Optional}.
*/
public Optional<HostInfo> getHostInfo() {
final String applicationServerConfig = this.config.getString(APPLICATION_SERVER_CONFIG);
return applicationServerConfig.isEmpty() ? Optional.empty()
: Optional.of(createHostInfo(applicationServerConfig));
}

private static HostInfo createHostInfo(final String applicationServerConfig) {
final String[] hostAndPort = applicationServerConfig.split(":");
return new HostInfo(hostAndPort[0], Integer.parseInt(hostAndPort[1]));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ private void runStreams() {
log.info("Calling start hook");
final RunningStreams runningStreams = RunningStreams.builder()
.streams(this.streams)
.config(this.config)
.config(new ImprovedStreamsConfig(this.config))
.topology(this.topology)
.build();
this.executionOptions.onStart(runningStreams);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,43 +26,72 @@

import static org.assertj.core.api.Assertions.assertThat;

import java.util.List;
import java.util.Map;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.state.HostInfo;
import org.junit.jupiter.api.Test;

class RunningStreamsTest {
class ImprovedStreamsConfigTest {
@Test
void shouldGetAppId() {
final StreamsConfig config = new StreamsConfig(
Map.of(StreamsConfig.APPLICATION_ID_CONFIG, "test-app",
StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, List.of("broker1:9092", "broker2:9092", "broker3:9092")
)
);
assertThat(new ImprovedStreamsConfig(config).getAppId())
.isEqualTo("test-app");
}

@Test
void shouldGetBrokerAddress() {
final StreamsConfig config = new StreamsConfig(
Map.of(StreamsConfig.APPLICATION_ID_CONFIG, "test-app",
StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, List.of("broker1:9092", "broker2:9092", "broker3:9092")
)
);
assertThat(new ImprovedStreamsConfig(config).getBoostrapServers())
.isEqualTo("broker1:9092,broker2:9092,broker3:9092");
}

@Test
void shouldGetOriginalKafkaProperties() {
final StreamsConfig config = new StreamsConfig(
Map.of(StreamsConfig.APPLICATION_ID_CONFIG, "test-app",
StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "broker1:9092"
)
);
assertThat(new ImprovedStreamsConfig(config).getKafkaProperties())
.hasSize(2)
.anySatisfy((key, property) -> {
assertThat(key).isEqualTo(StreamsConfig.APPLICATION_ID_CONFIG);
assertThat(property).isEqualTo("test-app");
})
.anySatisfy((key, property) -> {
assertThat(key).isEqualTo(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
assertThat(property).isEqualTo("broker1:9092");
})
;
}

@Test
void shouldHaveHostInfoIfApplicationServiceIsConfigure() {
final Topology topology = new Topology().addSource("test", "test-topic");
final StreamsConfig config = new StreamsConfig(
Map.of(StreamsConfig.APPLICATION_ID_CONFIG, "test-app",
StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092",
StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "broker1:9092",
StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:9090"));
final RunningStreams runningStreams = RunningStreams.builder()
.config(config)
.topology(topology)
.streams(new KafkaStreams(topology, config))
.build();
assertThat(runningStreams.getHostInfo())
assertThat(new ImprovedStreamsConfig(config).getHostInfo())
.hasValue(new HostInfo("localhost", 9090));
}

@Test
void shouldReturnEmptyHostInfoIfApplicationServiceIsNotConfigure() {
final Topology topology = new Topology().addSource("test", "test-topic");
final StreamsConfig config = new StreamsConfig(
Map.of(StreamsConfig.APPLICATION_ID_CONFIG, "test-app",
StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"));
final RunningStreams runningStreams = RunningStreams.builder()
.config(config)
.topology(topology)
.streams(new KafkaStreams(topology, config))
.build();
assertThat(runningStreams.getHostInfo())
assertThat(new ImprovedStreamsConfig(config).getHostInfo())
.isEmpty();
}

}

0 comments on commit 727ceb2

Please sign in to comment.