Skip to content

Commit

Permalink
Adding retries to update the metadata store instead of failure (#15141)
Browse files Browse the repository at this point in the history
Currently, If 2 tasks are consuming from the same partitions, try to publish the segment and update the metadata, the second task can fail because the end offset stored in the metadata store doesn't match with the start offset of the second task. We can fix this by retrying instead of failing.

AFAIK apart from the above issue, the metadata mismatch can happen in 2 scenarios:

- when we update the input topic name for the data source
- when we run 2 replicas of ingestion tasks(1 replica will publish and 1 will fail as the first replica has already updated the metadata).

Implemented the comparable function to compare the last committed end offset and new Sequence start offset. And return a specific error msg for this.

Add retry logic on indexers to retry for this specific error msg.

Updated the existing test case.
  • Loading branch information
Pankaj260100 authored Jan 10, 2024
1 parent 2938b8d commit 047c734
Show file tree
Hide file tree
Showing 10 changed files with 283 additions and 87 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,11 @@
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
import org.apache.druid.java.util.common.IAE;

public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<KafkaTopicPartition, Long>
import java.util.Comparator;

public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<KafkaTopicPartition, Long> implements Comparable<KafkaDataSourceMetadata>
{

@JsonCreator
Expand Down Expand Up @@ -58,4 +61,19 @@ protected SeekableStreamDataSourceMetadata<KafkaTopicPartition, Long> createConc
{
return new KafkaDataSourceMetadata(seekableStreamSequenceNumbers);
}

@Override
// This method is to compare KafkaDataSourceMetadata.
// It compares this and other SeekableStreamSequenceNumbers using naturalOrder comparator.
public int compareTo(KafkaDataSourceMetadata other)
{
if (!getClass().equals(other.getClass())) {
throw new IAE(
"Expected instance of %s, got %s",
this.getClass().getName(),
other.getClass().getName()
);
}
return getSeekableStreamSequenceNumbers().compareTo(other.getSeekableStreamSequenceNumbers(), Comparator.naturalOrder());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.druid.java.util.common.IAE;

import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
Expand Down Expand Up @@ -147,6 +148,38 @@ public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
}
}

@Override
public int compareTo(SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other, Comparator<SequenceOffsetType> comparator)
{
if (this.getClass() != other.getClass()) {
throw new IAE(
"Expected instance of %s, got %s",
this.getClass().getName(),
other.getClass().getName()
);
}

final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> otherStart =
(SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;

if (stream.equals(otherStart.stream)) {
//Same stream, compare the offset
boolean res = false;
for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : partitionSequenceNumberMap.entrySet()) {
PartitionIdType partitionId = entry.getKey();
SequenceOffsetType sequenceOffset = entry.getValue();
if (otherStart.partitionSequenceNumberMap.get(partitionId) != null && comparator.compare(sequenceOffset, otherStart.partitionSequenceNumberMap.get(partitionId)) > 0) {
res = true;
break;
}
}
if (res) {
return 1;
}
}
return 0;
}

@Override
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import com.fasterxml.jackson.annotation.JsonTypeInfo.Id;
import org.apache.druid.indexing.overlord.DataSourceMetadata;

import java.util.Comparator;
import java.util.Map;

@JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = SeekableStreamEndSequenceNumbers.class)
Expand Down Expand Up @@ -61,4 +62,11 @@ SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
);

/**
* Compare this and the other sequence offsets using comparator.
* Returns 1, if this sequence is ahead of the other.
* otherwise, Return 0
*/
int compareTo(SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> seekableStreamSequenceNumbers, Comparator<SequenceOffsetType> comparator);
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@

import javax.annotation.Nullable;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
Expand Down Expand Up @@ -161,6 +162,38 @@ public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
}
}

@Override
public int compareTo(SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other, Comparator<SequenceOffsetType> comparator)
{
if (this.getClass() != other.getClass()) {
throw new IAE(
"Expected instance of %s, got %s",
this.getClass().getName(),
other.getClass().getName()
);
}

final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> otherStart =
(SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;

if (stream.equals(otherStart.stream)) {
//Same stream, compare the offset
boolean res = false;
for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : partitionSequenceNumberMap.entrySet()) {
PartitionIdType partitionId = entry.getKey();
SequenceOffsetType sequenceOffset = entry.getValue();
if (otherStart.partitionSequenceNumberMap.get(partitionId) != null && comparator.compare(sequenceOffset, otherStart.partitionSequenceNumberMap.get(partitionId)) > 0) {
res = true;
break;
}
}
if (res) {
return 1;
}
}
return 0;
}

@Override
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,11 +151,7 @@ public void testFailTransactionalUpdateDataSourceMetadata() throws Exception
);

Assert.assertEquals(
SegmentPublishResult.fail(
"java.lang.RuntimeException: Inconsistent metadata state. " +
"This can happen if you update input topic in a spec without changing the supervisor name. " +
"Stored state: [null], Target state: [ObjectMetadata{theObject=[1]}]."
),
SegmentPublishResult.fail("java.lang.RuntimeException: Failed to update the metadata Store. The new start metadata is ahead of last commited end state."),
result
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.junit.Assert;
import org.junit.Test;

import java.util.Comparator;
import java.util.Map;

public class SeekableStreamEndSequenceNumbersTest
Expand Down Expand Up @@ -95,4 +96,40 @@ public void testConvertToStart()
endSequenceNumbers.asStartPartitions(true)
);
}

@Test
public void testCompareToWithTrueResult()
{
final String stream = "theStream";
final Map<Integer, Long> offsetMap1 = ImmutableMap.of(1, 5L, 2, 6L);
final SeekableStreamEndSequenceNumbers<Integer, Long> partitions1 = new SeekableStreamEndSequenceNumbers<>(
stream,
offsetMap1
);

final Map<Integer, Long> offsetMap2 = ImmutableMap.of(1, 4L, 2, 4L);
final SeekableStreamEndSequenceNumbers<Integer, Long> partitions2 = new SeekableStreamEndSequenceNumbers<>(
stream,
offsetMap2
);
Assert.assertEquals(1, partitions1.compareTo(partitions2, Comparator.naturalOrder()));
}

@Test
public void testCompareToWithFalseResult()
{
final String stream = "theStream";
final Map<Integer, Long> offsetMap1 = ImmutableMap.of(1, 3L, 2, 2L);
final SeekableStreamEndSequenceNumbers<Integer, Long> partitions1 = new SeekableStreamEndSequenceNumbers<>(
stream,
offsetMap1
);

final Map<Integer, Long> offsetMap2 = ImmutableMap.of(1, 4L, 2, 4L);
final SeekableStreamEndSequenceNumbers<Integer, Long> partitions2 = new SeekableStreamEndSequenceNumbers<>(
stream,
offsetMap2
);
Assert.assertEquals(0, partitions1.compareTo(partitions2, Comparator.naturalOrder()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.junit.Assert;
import org.junit.Test;

import java.util.Comparator;
import java.util.Map;

public class SeekableStreamStartSequenceNumbersTest
Expand Down Expand Up @@ -74,4 +75,44 @@ public void testSerde() throws Exception
OBJECT_MAPPER.convertValue(asMap.get("partitionOffsetMap"), new TypeReference<Map<Integer, Long>>() {})
);
}

@Test
public void testCompareToWithTrueResult()
{
final String stream = "theStream";
final Map<Integer, Long> offsetMap1 = ImmutableMap.of(1, 5L, 2, 6L);
final SeekableStreamStartSequenceNumbers<Integer, Long> partitions1 = new SeekableStreamStartSequenceNumbers<>(
stream,
offsetMap1,
ImmutableSet.of(6)
);

final Map<Integer, Long> offsetMap2 = ImmutableMap.of(1, 4L, 2, 4L);
final SeekableStreamStartSequenceNumbers<Integer, Long> partitions2 = new SeekableStreamStartSequenceNumbers<>(
stream,
offsetMap2,
ImmutableSet.of(6)
);
Assert.assertEquals(1, partitions1.compareTo(partitions2, Comparator.naturalOrder()));
}

@Test
public void testCompareToWithFalseResult()
{
final String stream = "theStream";
final Map<Integer, Long> offsetMap1 = ImmutableMap.of(1, 3L, 2, 2L);
final SeekableStreamStartSequenceNumbers<Integer, Long> partitions1 = new SeekableStreamStartSequenceNumbers<>(
stream,
offsetMap1,
ImmutableSet.of(6)
);

final Map<Integer, Long> offsetMap2 = ImmutableMap.of(1, 4L, 2, 4L);
final SeekableStreamStartSequenceNumbers<Integer, Long> partitions2 = new SeekableStreamStartSequenceNumbers<>(
stream,
offsetMap2,
ImmutableSet.of(6)
);
Assert.assertEquals(0, partitions1.compareTo(partitions2, Comparator.naturalOrder()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2404,18 +2404,37 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle(
}

final boolean startMetadataMatchesExisting;
int startMetadataGreaterThanExisting = 0;

if (oldCommitMetadataFromDb == null) {
startMetadataMatchesExisting = startMetadata.isValidStart();
startMetadataGreaterThanExisting = 1;
} else {
// Checking against the last committed metadata.
// If the new start sequence number is greater than the end sequence number of last commit compareTo() function will return 1,
// 0 in all other cases. It might be because multiple tasks are publishing the sequence at around same time.
if (startMetadata instanceof Comparable) {
startMetadataGreaterThanExisting = ((Comparable) startMetadata.asStartMetadata()).compareTo(oldCommitMetadataFromDb.asStartMetadata());
}

// Converting the last one into start metadata for checking since only the same type of metadata can be matched.
// Even though kafka/kinesis indexing services use different sequenceNumber types for representing
// start and end sequenceNumbers, the below conversion is fine because the new start sequenceNumbers are supposed
// to be same with end sequenceNumbers of the last commit.
startMetadataMatchesExisting = startMetadata.asStartMetadata().matches(oldCommitMetadataFromDb.asStartMetadata());
}

if (startMetadataGreaterThanExisting == 1 && !startMetadataMatchesExisting) {
// Offset stored in StartMetadata is Greater than the last commited metadata,
// Then retry multiple task might be trying to publish the segment for same partitions.
log.info("Failed to update the metadata Store. The new start metadata: [%s] is ahead of last commited end state: [%s].",
startMetadata,
oldCommitMetadataFromDb);
return new DataStoreMetadataUpdateResult(true, false,
"Failed to update the metadata Store. The new start metadata is ahead of last commited end state."
);
}

if (!startMetadataMatchesExisting) {
// Not in the desired start state.
return new DataStoreMetadataUpdateResult(true, false, StringUtils.format(
Expand Down
Loading

0 comments on commit 047c734

Please sign in to comment.