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

Minor cleanup of SeekableStreamSupervisor classes #17733

Merged
merged 4 commits into from
Feb 18, 2025
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -19,15 +19,12 @@

package org.apache.druid.indexing.rabbitstream;

import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.segment.indexing.IOConfig;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;

import java.util.Collections;

Expand All @@ -38,12 +35,9 @@ public class RabbitStreamIndexTaskIOConfigTest
public RabbitStreamIndexTaskIOConfigTest()
{
mapper = new DefaultObjectMapper();
mapper.registerModules((Iterable<Module>) new RabbitStreamIndexTaskModule().getJacksonModules());
mapper.registerModules(new RabbitStreamIndexTaskModule().getJacksonModules());
}

@Rule
public final ExpectedException exception = ExpectedException.none();

@Test
public void testSerdeWithDefaults() throws Exception
{
Expand All @@ -56,31 +50,29 @@ public void testSerdeWithDefaults() throws Exception
+ "}";

RabbitStreamIndexTaskIOConfig config = (RabbitStreamIndexTaskIOConfig) mapper.readValue(
mapper.writeValueAsString(
mapper.readValue(
jsonStr,
IOConfig.class)),
IOConfig.class);
mapper.writeValueAsString(mapper.readValue(jsonStr, IOConfig.class)),
IOConfig.class
);

Assert.assertNull(config.getTaskGroupId());
Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());

Assert.assertEquals("mystream", config.getStartSequenceNumbers().getStream());

Assert.assertEquals(Long.class,
config.getStartSequenceNumbers().getPartitionSequenceNumberMap().get("stream-1").getClass());
Assert.assertEquals(
ImmutableMap.of("stream-0", Long.valueOf(1), "stream-1", Long.valueOf(10)),
config.getStartSequenceNumbers().getPartitionSequenceNumberMap());
ImmutableMap.of("stream-0", 1L, "stream-1", 10L),
config.getStartSequenceNumbers().getPartitionSequenceNumberMap()
);

Assert.assertEquals("mystream", config.getEndSequenceNumbers().getStream());

Assert.assertEquals(
ImmutableMap.of("stream-0", Long.valueOf(15L), "stream-1", Long.valueOf(200L)),
config.getEndSequenceNumbers().getPartitionSequenceNumberMap());
ImmutableMap.of("stream-0", 15L, "stream-1", 200L),
config.getEndSequenceNumbers().getPartitionSequenceNumberMap()
);

Assert.assertTrue(config.isUseTransaction());
Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent());
Assert.assertNull("minimumMessageTime", config.getMinimumMessageTime());
Assert.assertEquals(config.getUri(), "rabbitmq-stream://localhost:5552");
Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,6 @@
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
import org.joda.time.DateTime;

Expand Down Expand Up @@ -91,12 +89,9 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<KafkaTopicPartitio
private static final Long NOT_SET = -1L;
private static final Long END_OF_PARTITION = Long.MAX_VALUE;

private final ServiceEmitter emitter;
private final DruidMonitorSchedulerConfig monitorSchedulerConfig;
private final Pattern pattern;
private volatile Map<KafkaTopicPartition, Long> latestSequenceFromStream;


private final KafkaSupervisorSpec spec;

public KafkaSupervisor(
Expand All @@ -122,8 +117,6 @@ public KafkaSupervisor(
);

this.spec = spec;
this.emitter = spec.getEmitter();
this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig();
this.pattern = getIoConfig().isMultiTopic() ? Pattern.compile(getIoConfig().getStream()) : null;
}

Expand Down Expand Up @@ -267,7 +260,7 @@ protected Map<KafkaTopicPartition, Long> getPartitionRecordLag()

if (!latestSequenceFromStream.keySet().equals(highestCurrentOffsets.keySet())) {
log.warn(
"Lag metric: Kafka partitions %s do not match task partitions %s",
"Kafka partitions[%s] do not match task partitions[%s]",
latestSequenceFromStream.keySet(),
highestCurrentOffsets.keySet()
);
Expand Down Expand Up @@ -307,9 +300,6 @@ private Map<KafkaTopicPartition, Long> getRecordLagPerPartitionInLatestSequences
}

@Override
// suppress use of CollectionUtils.mapValues() since the valueMapper function is dependent on map key here
@SuppressWarnings("SSBasedInspection")
// Used while generating Supervisor lag reports per task
protected Map<KafkaTopicPartition, Long> getRecordLagPerPartition(Map<KafkaTopicPartition, Long> currentOffsets)
{
if (latestSequenceFromStream == null || currentOffsets == null) {
Expand All @@ -333,6 +323,7 @@ protected Map<KafkaTopicPartition, Long> getRecordLagPerPartition(Map<KafkaTopic
@Override
protected Map<KafkaTopicPartition, Long> getTimeLagPerPartition(Map<KafkaTopicPartition, Long> currentOffsets)
{
// Currently not supported
return null;
}

Expand Down Expand Up @@ -389,6 +380,11 @@ public LagStats computeLagStats()
return computeLags(partitionRecordLag);
}

/**
* Fetches the latest offsets from the Kafka stream and updates the map
* {@link #latestSequenceFromStream}. The actual lag is computed lazily in
* {@link #getPartitionRecordLag}.
*/
@Override
protected void updatePartitionLagFromStream()
{
Expand All @@ -410,9 +406,6 @@ protected void updatePartitionLagFromStream()

recordSupplier.seekToLatest(partitions);

// this method isn't actually computing the lag, just fetching the latests offsets from the stream. This is
// because we currently only have record lag for kafka, which can be lazily computed by subtracting the highest
// task offsets from the latest offsets from the stream when it is needed
latestSequenceFromStream =
partitions.stream().collect(Collectors.toMap(StreamPartition::getPartitionId, recordSupplier::getPosition));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
package org.apache.druid.indexing.kafka;

import com.fasterxml.jackson.databind.JsonMappingException;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
Expand All @@ -42,7 +41,7 @@ public class KafkaIOConfigTest
public KafkaIOConfigTest()
{
mapper = new DefaultObjectMapper();
mapper.registerModules((Iterable<Module>) new KafkaIndexTaskModule().getJacksonModules());
mapper.registerModules(new KafkaIndexTaskModule().getJacksonModules());
}

@Rule
Expand Down Expand Up @@ -78,8 +77,8 @@ public void testSerdeWithDefaults() throws Exception
200L), config.getEndSequenceNumbers().getPartitionSequenceNumberMap());
Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
Assert.assertTrue(config.isUseTransaction());
Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent());
Assert.assertFalse("maximumMessageTime", config.getMaximumMessageTime().isPresent());
Assert.assertNull("minimumMessageTime", config.getMinimumMessageTime());
Assert.assertNull("maximumMessageTime", config.getMaximumMessageTime());
Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
}

Expand Down Expand Up @@ -113,8 +112,8 @@ public void testSerdeWithDefaultsAndSequenceNumbers() throws Exception
200L), config.getEndSequenceNumbers().getPartitionSequenceNumberMap());
Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
Assert.assertTrue(config.isUseTransaction());
Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent());
Assert.assertFalse("maximumMessageTime", config.getMaximumMessageTime().isPresent());
Assert.assertNull("minimumMessageTime", config.getMinimumMessageTime());
Assert.assertNull("maximumMessageTime", config.getMaximumMessageTime());
Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
}

Expand Down Expand Up @@ -151,8 +150,8 @@ public void testSerdeWithNonDefaults() throws Exception
200L), config.getEndSequenceNumbers().getPartitionSequenceNumberMap());
Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
Assert.assertFalse(config.isUseTransaction());
Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get());
Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get());
Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime());
Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime());
Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonMappingException;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.error.DruidException;
Expand Down Expand Up @@ -51,7 +50,7 @@ public class KafkaSupervisorIOConfigTest
public KafkaSupervisorIOConfigTest()
{
mapper = new DefaultObjectMapper();
mapper.registerModules((Iterable<Module>) new KafkaIndexTaskModule().getJacksonModules());
mapper.registerModules(new KafkaIndexTaskModule().getJacksonModules());
}

@Rule
Expand Down
Loading
Loading