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

Adding retries to update the metadata store instead of failure #15141

Merged
merged 21 commits into from
Jan 10, 2024
Merged
Show file tree
Hide file tree
Changes from 19 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 @@ -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 @@
{
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)
Pankaj260100 marked this conversation as resolved.
Show resolved Hide resolved
{
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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you also please add a check to compare the partitions to account for repartitioning?

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) {
Pankaj260100 marked this conversation as resolved.
Show resolved Hide resolved
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.
Copy link
Contributor

@AmatyaAvadhanula AmatyaAvadhanula Oct 30, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please elaborate what a sequence being ahead of the other means?

I think that it means that the partition-wise sequence number of the first is greater than or equal to the other's with a strict greater than condition for at least one of the partitions. WDYT?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AmatyaAvadhanula, We will only retry when the first is greater than the other's; in case both are equal, this will not fail to publish, So no point in retry, right?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't it possible that when there are 10 partitions, 6 have strictly greater sequence numbers while the remaining 4 have equal sequence numbers because no new data was added to them?
I think we should retry in this case as well

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AmatyaAvadhanula, This case is covered. We will retry when atleast one partition sequence number is strictly greater than the other. We will compare all partition offsets, and if we find one of the partition offsets greater than the other offset, we update the res variable as true. And retry in that case.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was wondering if it's possible that one partition is strictly greater but the other is strictly lower

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey @AmatyaAvadhanula, as per our discussion I have added a check to verify task partitions are contained within the set in the metadata total set.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have reverted this: the test case where a new kafka partition gets added and wants to publish for the first time will fail as it's not in the old committed offset.

* 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 @@ -2054,18 +2054,37 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle(
}

final boolean startMetadataMatchesExisting;
int startMetadataGreaterThanExisting = 0;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this be a boolean instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we can have a boolean. I have implemented the Comparable for compareTo() method and it returns int. so, I didn't change it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

comareTo() function returns +1, -1 and 0 for greaterThan, lessThan and Equal repectively.


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
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.RetryUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
Expand All @@ -61,6 +62,7 @@
import java.util.Objects;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ExecutionException;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import java.util.stream.Stream;
Expand Down Expand Up @@ -595,7 +597,9 @@
final Object callerMetadata = metadata == null
? null
: ((AppenderatorDriverMetadata) metadata).getCallerMetadata();

return executor.submit(
() -> RetryUtils.retry(
() -> {
try {
final ImmutableSet<DataSegment> ourSegments = ImmutableSet.copyOf(pushedAndTombstones);
Expand Down Expand Up @@ -652,19 +656,16 @@
segmentsAndCommitMetadata.getSegments().forEach(dataSegmentKiller::killQuietly);
}
} else {
log.errorSegments(ourSegments, "Failed to publish segments");
if (publishResult.getErrorMsg() != null && publishResult.getErrorMsg().contains(("Failed to update the metadata Store. The new start metadata is ahead of last commited end state."))) {
throw new ISE(publishResult.getErrorMsg());
}
// Our segments aren't active. Publish failed for some reason. Clean them up and then throw an error.
segmentsAndCommitMetadata.getSegments().forEach(dataSegmentKiller::killQuietly);
Pankaj260100 marked this conversation as resolved.
Show resolved Hide resolved

if (publishResult.getErrorMsg() != null) {
log.errorSegments(ourSegments, "Failed to publish segments");
throw new ISE(
"Failed to publish segments because of [%s]",
publishResult.getErrorMsg()
);
} else {
log.errorSegments(ourSegments, "Failed to publish segments");
throw new ISE("Failed to publish segments");
throw new ISE("Failed to publish segments because of [%s]", publishResult.getErrorMsg());
}
throw new ISE("Failed to publish segments");
}
}
}
Expand All @@ -675,12 +676,19 @@
segmentsAndCommitMetadata.getSegments(),
"Failed publish, not removing segments"
);
Throwables.propagateIfPossible(e);
throw new RuntimeException(e);
if (e.getMessage() != null && e.getMessage().contains("Failed to update the metadata Store. The new start metadata is ahead of last commited end state.")) {
// we can recover from this error.
throw new ExecutionException(e);
Pankaj260100 marked this conversation as resolved.
Show resolved Hide resolved
} else {
Throwables.propagateIfPossible(e);
Fixed Show fixed Hide fixed
throw new RuntimeException(e);
}
}

return segmentsAndCommitMetadata;
}
},
e -> (e instanceof ExecutionException),
RetryUtils.DEFAULT_MAX_TRIES
)
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -891,10 +891,7 @@ public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException
new ObjectMetadata(ImmutableMap.of("foo", "bar")),
new ObjectMetadata(ImmutableMap.of("foo", "baz"))
);
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={foo=bar}}]."), result1);
Assert.assertEquals(SegmentPublishResult.fail("java.lang.RuntimeException: Failed to update the metadata Store. The new start metadata is ahead of last commited end state."), result1);

// Should only be tried once.
Assert.assertEquals(1, metadataUpdateCounter.get());
Expand Down
Loading