Skip to content

Commit

Permalink
Changing the logic to retry in few scenarios
Browse files Browse the repository at this point in the history
  • Loading branch information
Pankaj260100 committed Oct 12, 2023
1 parent 277aaa5 commit f7220c1
Show file tree
Hide file tree
Showing 11 changed files with 230 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,12 @@ public boolean matches(DataSourceMetadata other)
return equals(other);
}

@Override
public int compareTo(DataSourceMetadata o)
{
return -1;
}

@Override
public DataSourceMetadata plus(DataSourceMetadata other)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,18 @@ public boolean matches(DataSourceMetadata other)
return plus(other).equals(other.plus(this));
}

@Override
public int compareTo(DataSourceMetadata other)
{
if (!getClass().equals(other.getClass())) {
return -1;
}
final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that =
(SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;

return seekableStreamSequenceNumbers.compareTo(that.seekableStreamSequenceNumbers);
}

@Override
public DataSourceMetadata plus(DataSourceMetadata other)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicReference;

/**
* Represents the end sequenceNumber per partition of a sequence. Note that end sequenceNumbers are always
Expand Down Expand Up @@ -147,6 +148,40 @@ public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
}
}

@Override
public int compareTo(SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other)
{
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
AtomicReference<Boolean> res = new AtomicReference<>(false);
partitionSequenceNumberMap.forEach(
(partitionId, sequenceOffset) -> {
if (otherStart.partitionSequenceNumberMap.get(partitionId) != null && Long.parseLong(String.valueOf(sequenceOffset)) > Long.parseLong(String.valueOf(otherStart.partitionSequenceNumberMap.get(partitionId)))) {
res.set(true);
}
}
);
if (res.get()) {
return 1;
}
return -1;
} else {
// Different streams
return -1;
}
}

@Override
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
@Type(name = "start", value = SeekableStreamStartSequenceNumbers.class),
@Type(name = "end", value = SeekableStreamEndSequenceNumbers.class)
})
public interface SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>
public interface SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> extends Comparable<SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>>
{
/**
* Returns the stream/topic name.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;

/**
* Represents the start sequenceNumber per partition of a sequence. This class keeps an additional set of
Expand Down Expand Up @@ -161,6 +162,40 @@ public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
}
}

@Override
public int compareTo(SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other)
{
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
AtomicReference<Boolean> res = new AtomicReference<>(false);
partitionSequenceNumberMap.forEach(
(partitionId, sequenceOffset) -> {
if (otherStart.partitionSequenceNumberMap.get(partitionId) != null && Long.parseLong(String.valueOf(sequenceOffset)) > Long.parseLong(String.valueOf(otherStart.partitionSequenceNumberMap.get(partitionId)))) {
res.set(true);
}
}
);
if (res.get()) {
return 1;
}
return -1;
} else {
// Different streams
return -1;
}
}

@Override
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,17 +29,24 @@
import org.apache.druid.indexing.overlord.ObjectMetadata;
import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.indexing.overlord.TaskLockbox;
import org.apache.druid.indexing.overlord.TimeChunkLockRequest;
import org.apache.druid.indexing.overlord.config.TaskLockConfig;
import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.assertj.core.api.Assertions;
import org.easymock.EasyMock;
import org.hamcrest.CoreMatchers;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;

public class SegmentTransactionalInsertActionTest
{
Expand Down Expand Up @@ -134,6 +141,105 @@ public void testTransactionalUpdateDataSourceMetadata() throws Exception
);
}

@Test
public void testTransactionalUpdateDataSourceMetadataWithRecoverFromMetadataMismatch() throws Exception
{
final Task task1 = NoopTask.create();
final TaskLockbox taskLockbox1 = new TaskLockbox(actionTestKit.getTaskStorage(), actionTestKit.getMetadataStorageCoordinator());
taskLockbox1.add(task1);
taskLockbox1.lock(task1, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task1, INTERVAL, null), 5000);

final Task task2 = NoopTask.create();
final TaskLockbox taskLockbox2 = new TaskLockbox(actionTestKit.getTaskStorage(), actionTestKit.getMetadataStorageCoordinator());
taskLockbox2.add(task2);
taskLockbox2.lock(task2, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task2, INTERVAL, null), 5000);

final TaskLockConfig taskLockConfig = new TaskLockConfig()
{
@Override
public boolean isBatchSegmentAllocation()
{
return true;
}

@Override
public long getBatchAllocationWaitTime()
{
return 10L;
}
};
TaskActionToolbox taskActionToolbox = actionTestKit.getTaskActionToolbox();

// Task1 and Task2 tries to publish segment1 and segment2 for same partition at around same time.
// With different start and end offsets. Segment2 -> {1 - 2}, Segment1 -> {null - 1}
Future<SegmentPublishResult> result2Future = CompletableFuture.supplyAsync(() -> {
return SegmentTransactionalInsertAction.appendAction(
ImmutableSet.of(SEGMENT2),
new ObjectMetadata(ImmutableList.of(1)),
new ObjectMetadata(ImmutableList.of(2))
).perform(
task2,
new TaskActionToolbox(
taskLockbox2,
taskActionToolbox.getTaskStorage(),
taskActionToolbox.getIndexerMetadataStorageCoordinator(),
new SegmentAllocationQueue(
taskLockbox2,
taskLockConfig,
taskActionToolbox.getIndexerMetadataStorageCoordinator(),
taskActionToolbox.getEmitter(),
ScheduledExecutors::fixed
),
taskActionToolbox.getEmitter(),
EasyMock.createMock(SupervisorManager.class),
taskActionToolbox.getJsonMapper()
)
);
});


Future<SegmentPublishResult> result1Future = CompletableFuture.supplyAsync(() -> {
return SegmentTransactionalInsertAction.appendAction(
ImmutableSet.of(SEGMENT1),
new ObjectMetadata(null),
new ObjectMetadata(ImmutableList.of(1))
).perform(
task1,
new TaskActionToolbox(
taskLockbox1,
taskActionToolbox.getTaskStorage(),
taskActionToolbox.getIndexerMetadataStorageCoordinator(),
new SegmentAllocationQueue(
taskLockbox1,
taskLockConfig,
taskActionToolbox.getIndexerMetadataStorageCoordinator(),
taskActionToolbox.getEmitter(),
ScheduledExecutors::fixed
),
taskActionToolbox.getEmitter(),
EasyMock.createMock(SupervisorManager.class),
taskActionToolbox.getJsonMapper()
)
);
});

SegmentPublishResult result2 = result2Future.get();
SegmentPublishResult result1 = result1Future.get();

Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(SEGMENT1)), result1);
Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(SEGMENT2)), result2);

Assertions.assertThat(
actionTestKit.getMetadataStorageCoordinator()
.retrieveUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE)
).containsExactlyInAnyOrder(SEGMENT1, SEGMENT2);

Assert.assertEquals(
new ObjectMetadata(ImmutableList.of(2)),
actionTestKit.getMetadataStorageCoordinator().retrieveDataSourceMetadata(DATA_SOURCE)
);
}

@Test
public void testTransactionalDropSegments() throws Exception
{
Expand Down Expand Up @@ -185,9 +291,8 @@ 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]}]."
"org.apache.druid.metadata.RetryTransactionException: Failed to update the metadata Store. " +
"The new start metadata: [ObjectMetadata{theObject=[1]}] is ahead of last commited end state: [null]."
),
result
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,11 @@ public TaskLockbox getTaskLockbox()
return taskLockbox;
}

public TaskStorage getTaskStorage()
{
return taskStorage;
}

public IndexerMetadataStorageCoordinator getMetadataStorageCoordinator()
{
return metadataStorageCoordinator;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "object", value = ObjectMetadata.class)
})
public interface DataSourceMetadata
public interface DataSourceMetadata extends Comparable<DataSourceMetadata>
{
/**
* Returns true if this instance should be considered a valid starting point for a new dataSource that has
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,12 @@ public boolean matches(DataSourceMetadata other)
return equals(other);
}

@Override
public int compareTo(DataSourceMetadata o)
{
return -1;
}

@Override
public DataSourceMetadata plus(DataSourceMetadata other)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1552,18 +1552,35 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle(
}

final boolean startMetadataMatchesExisting;
final int startMetadataGreaterThanExisting;

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,
// -1 in all other cases. It might be because multiple tasks are publishing the sequence at around same time.
startMetadataGreaterThanExisting = 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.

return new DataStoreMetadataUpdateResult(true, true, StringUtils.format(
"Failed to update the metadata Store. The new start metadata: [%s] is ahead of last commited end state: [%s].",
startMetadata,
oldCommitMetadataFromDb
));
}

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 @@ -680,13 +680,12 @@ 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("org.apache.druid.metadata.RetryTransactionException: Failed to update the metadata Store. " +
"The new start metadata: [ObjectMetadata{theObject={foo=bar}}] is ahead of last commited end state: [null]."), result1);

// Should only be tried once.
Assert.assertEquals(1, metadataUpdateCounter.get());
// The code will retry for this test case as well, So it will be equal to total retries available which is 2.
Assert.assertEquals(2, metadataUpdateCounter.get());
}

@Test
Expand Down

0 comments on commit f7220c1

Please sign in to comment.