Skip to content

Commit 034453e

Browse files
authored
[server] Global RT DIV: Fix VT DIV Sync Order (#2340)
1. Fix VT DIV Sync Order 2. Accommodate Delete Values in `shouldSyncOffsetFromSnapshot()` 3. Skip Global RT DIV in Repush
1 parent 394ce61 commit 034453e

File tree

14 files changed

+466
-72
lines changed

14 files changed

+466
-72
lines changed

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AbstractStoreBufferService.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,5 +41,6 @@ public abstract CompletableFuture<Void> execSyncOffsetCommandAsync(
4141
public abstract void execSyncOffsetFromSnapshotAsync(
4242
PubSubTopicPartition topicPartition,
4343
PartitionTracker vtDivSnapshot,
44+
CompletableFuture<Void> lastRecordPersistedFuture,
4445
StoreIngestionTask ingestionTask) throws InterruptedException;
4546
}

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java

Lines changed: 56 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -2485,19 +2485,8 @@ protected DelegateConsumerRecordResult delegateConsumerRecord(
24852485
// Update the latest consumed VT position (LCVP) since we're consuming from the version topic
24862486
if (isGlobalRtDivEnabled()) {
24872487
getConsumerDiv().updateLatestConsumedVtPosition(partition, consumerRecord.getPosition());
2488-
2489-
if (shouldSyncOffsetFromSnapshot(consumerRecord, partitionConsumptionState)) {
2490-
PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(getVersionTopic(), partition);
2491-
PartitionTracker vtDiv = consumerDiv.cloneVtProducerStates(partition); // has latest consumed VT position
2492-
storeBufferService.execSyncOffsetFromSnapshotAsync(topicPartition, vtDiv, this);
2493-
// TODO: remove. this is a temporary log for debugging while the feature is in its infancy
2494-
int partitionStateMapSize = vtDiv.getPartitionStates(PartitionTracker.VERSION_TOPIC).size();
2495-
LOGGER.info(
2496-
"event=globalRtDiv Syncing LCVP for OffsetRecord topic-partition: {} position: {} size: {}",
2497-
topicPartition,
2498-
consumerRecord.getPosition(),
2499-
partitionStateMapSize);
2500-
}
2488+
// Only after the current message is queued to drainer
2489+
// The Offset Record's LCVP may be synced in syncOffsetFromSnapshotIfNeeded()
25012490
}
25022491

25032492
/**
@@ -2788,16 +2777,48 @@ protected DelegateConsumerRecordResult delegateConsumerRecord(
27882777
}
27892778
}
27902779

2780+
void syncOffsetFromSnapshotIfNeeded(DefaultPubSubMessage record, PubSubTopicPartition topicPartition) {
2781+
int partition = topicPartition.getPartitionNumber();
2782+
if (!isGlobalRtDivEnabled() || !shouldSyncOffsetFromSnapshot(record, getPartitionConsumptionState(partition))) {
2783+
return; // without Global RT DIV enabled, the offset record is synced in the drainer in syncOffset()
2784+
}
2785+
2786+
PartitionConsumptionState pcs = getPartitionConsumptionState(topicPartition.getPartitionNumber());
2787+
if (pcs == null || pcs.getLastQueuedRecordPersistedFuture() == null) {
2788+
LOGGER.warn(
2789+
"event=globalRtDiv No PCS or lastRecordPersistedFuture found for topic-partition: {}. "
2790+
+ "Will not sync OffsetRecord without waiting for any record to be persisted",
2791+
topicPartition);
2792+
return;
2793+
}
2794+
2795+
try {
2796+
PartitionTracker vtDiv = consumerDiv.cloneVtProducerStates(partition); // has latest consumed VT position
2797+
CompletableFuture<Void> lastFuture = pcs.getLastQueuedRecordPersistedFuture();
2798+
storeBufferService.execSyncOffsetFromSnapshotAsync(topicPartition, vtDiv, lastFuture, this);
2799+
2800+
// TODO: remove. this is a temporary log for debugging while the feature is in its infancy
2801+
LOGGER.info(
2802+
"event=globalRtDiv Syncing LCVP for OffsetRecord topic-partition: {} position: {} size: {}",
2803+
topicPartition,
2804+
record.getPosition(),
2805+
vtDiv.getPartitionStates(PartitionTracker.VERSION_TOPIC).size());
2806+
} catch (InterruptedException e) {
2807+
LOGGER.error("event=globalRtDiv Unable to sync Offset Record to update the latest consumed vt position", e);
2808+
}
2809+
}
2810+
27912811
/**
2792-
* Followers should sync the VT DIV to the OffsetRecord if the consumer sees a Global RT DIV message
2793-
* (sync only once for a Global RT DIV, which can either be one singular message or multiple chunks + one manifest.
2794-
* thus, the condition is to check that it's not a chunk) or if it sees a non-segment control message.
2812+
* Followers should sync the VT DIV to the OffsetRecord if the consumer sees a non-segment control message or a
2813+
* Global RT DIV message.
2814+
* Each Global RT DIV sync will create one singular Put or multiple Puts (chunks + one manifest + Deletes). Thus
2815+
* if we want to sync only once, checking if it's a singular Put or the manifest Put should only trigger once.
27952816
*/
27962817
boolean shouldSyncOffsetFromSnapshot(DefaultPubSubMessage consumerRecord, PartitionConsumptionState pcs) {
27972818
if (consumerRecord.getKey().isGlobalRtDiv()) {
2798-
Put put = (Put) consumerRecord.getValue().getPayloadUnion();
2799-
if (put.getSchemaId() != CHUNK_SCHEMA_ID) {
2800-
return true;
2819+
Object payloadUnion = consumerRecord.getValue().getPayloadUnion();
2820+
if (payloadUnion instanceof Put && ((Put) payloadUnion).getSchemaId() != CHUNK_SCHEMA_ID) {
2821+
return true; // Global RT DIV message can be multiple chunks + deletes, only sync on one Put (manifest or value)
28012822
}
28022823
}
28032824
return isNonSegmentControlMessage(consumerRecord, null);
@@ -3538,19 +3559,19 @@ private byte[] createGlobalRtDivValueBytes(
35383559
}
35393560

35403561
private LeaderProducerCallback createGlobalRtDivCallback(
3541-
DefaultPubSubMessage previousMessage,
3542-
PartitionConsumptionState partitionConsumptionState,
3562+
DefaultPubSubMessage prevMessage,
3563+
PartitionConsumptionState pcs,
35433564
int partition,
35443565
String brokerUrl,
35453566
long beforeProcessingRecordTimestampNs,
3546-
LeaderProducedRecordContext context,
3567+
LeaderProducedRecordContext prevContext,
35473568
byte[] keyBytes,
35483569
byte[] valueBytes,
35493570
PubSubTopicPartition topicPartition,
35503571
PartitionTracker vtDiv) {
35513572
final int schemaId = AvroProtocolDefinition.GLOBAL_RT_DIV_STATE.getCurrentProtocolVersion();
35523573
KafkaKey divKey = new KafkaKey(MessageType.GLOBAL_RT_DIV, keyBytes);
3553-
KafkaMessageEnvelope divEnvelope = getVeniceWriter(partitionConsumptionState).get()
3574+
KafkaMessageEnvelope divEnvelope = getVeniceWriter(pcs).get()
35543575
.getKafkaMessageEnvelope(
35553576
MessageType.PUT,
35563577
false,
@@ -3567,25 +3588,23 @@ private LeaderProducerCallback createGlobalRtDivCallback(
35673588
divKey,
35683589
divEnvelope,
35693590
topicPartition,
3570-
previousMessage.getPosition(),
3591+
prevMessage.getPosition(),
35713592
System.currentTimeMillis(),
35723593
divKey.getKeyLength() + valueBytes.length);
3573-
LeaderProducerCallback divCallback = createProducerCallback(
3574-
divMessage,
3575-
partitionConsumptionState,
3576-
LeaderProducedRecordContext
3577-
.newPutRecord(context.getConsumedKafkaClusterId(), context.getConsumedPosition(), keyBytes, put),
3578-
partition,
3579-
brokerUrl,
3580-
beforeProcessingRecordTimestampNs);
3581-
3582-
// After producing RT DIV to local VT, the VT DIV should be sent to the drainer to sync to the OffsetRecord
3583-
divCallback.setOnCompletionFunction(produceResult -> {
3594+
LeaderProducedRecordContext context = LeaderProducedRecordContext
3595+
.newPutRecord(prevContext.getConsumedKafkaClusterId(), prevContext.getConsumedPosition(), keyBytes, put);
3596+
LeaderProducerCallback divCallback =
3597+
createProducerCallback(divMessage, pcs, context, partition, brokerUrl, beforeProcessingRecordTimestampNs);
3598+
3599+
// After producing the RT DIV to local VT and LeaderProducerCallback.onCompletion() enqueuing RT DIV in the drainer,
3600+
// the VT DIV should be sent to the drainer to persist the LCVP onto disk by syncing the OffsetRecord
3601+
divCallback.setOnCompletionCallback(produceResult -> {
35843602
try {
3603+
CompletableFuture<Void> lastRecordPersistedFuture = context.getPersistedToDBFuture();
35853604
vtDiv.updateLatestConsumedVtPosition(produceResult.getPubSubPosition()); // LCVP = produced position in local VT
3586-
storeBufferService.execSyncOffsetFromSnapshotAsync(topicPartition, vtDiv, this);
3605+
storeBufferService.execSyncOffsetFromSnapshotAsync(topicPartition, vtDiv, lastRecordPersistedFuture, this);
35873606
} catch (InterruptedException e) {
3588-
LOGGER.error("Failed to sync VT DIV to OffsetRecord for replica: {}", topicPartition, e);
3607+
LOGGER.error("event=globalRtDiv Failed to sync VT DIV to OffsetRecord for replica: {}", topicPartition, e);
35893608
}
35903609
});
35913610

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderProducerCallback.java

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,8 @@ public class LeaderProducerCallback implements ChunkAwareCallback {
2727
private static final RedundantExceptionFilter REDUNDANT_LOGGING_FILTER =
2828
RedundantExceptionFilter.getRedundantExceptionFilter();
2929
private static final Consumer<PubSubProduceResult> NO_OP = produceResult -> {};
30-
private Consumer<PubSubProduceResult> onCompletionFunction = NO_OP;
30+
private Consumer<PubSubProduceResult> onCompletionFunction = NO_OP; // ran before onCompletion() runs
31+
private Consumer<PubSubProduceResult> onCompletionCallback = NO_OP; // ran after onCompletion() runs
3132

3233
protected static final ChunkedValueManifestSerializer CHUNKED_VALUE_MANIFEST_SERIALIZER =
3334
new ChunkedValueManifestSerializer(false);
@@ -208,6 +209,7 @@ public void onCompletion(PubSubProduceResult produceResult, Exception e) {
208209
LatencyUtils.getElapsedTimeFromMsToMs(currentTimeForMetricsMs),
209210
currentTimeForMetricsMs);
210211
}
212+
this.onCompletionCallback.accept(produceResult);
211213
} catch (Exception oe) {
212214
boolean endOfPushReceived = partitionConsumptionState.isEndOfPushReceived();
213215
LOGGER.error(
@@ -357,6 +359,10 @@ public void setOnCompletionFunction(Consumer<PubSubProduceResult> onCompletionFu
357359
this.onCompletionFunction = onCompletionFunction;
358360
}
359361

362+
public void setOnCompletionCallback(Consumer<PubSubProduceResult> onCompletionCallback) {
363+
this.onCompletionCallback = onCompletionCallback;
364+
}
365+
360366
// Visible for VeniceWriter unit test.
361367
public PartitionConsumptionState getPartitionConsumptionState() {
362368
return partitionConsumptionState;

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/SeparatedStoreBufferService.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -100,8 +100,10 @@ public CompletableFuture<Void> execSyncOffsetCommandAsync(
100100
public void execSyncOffsetFromSnapshotAsync(
101101
PubSubTopicPartition topicPartition,
102102
PartitionTracker vtDivSnapshot,
103+
CompletableFuture<Void> lastRecordPersistedFuture,
103104
StoreIngestionTask ingestionTask) throws InterruptedException {
104-
getDelegate(ingestionTask).execSyncOffsetFromSnapshotAsync(topicPartition, vtDivSnapshot, ingestionTask);
105+
getDelegate(ingestionTask)
106+
.execSyncOffsetFromSnapshotAsync(topicPartition, vtDivSnapshot, lastRecordPersistedFuture, ingestionTask);
105107
}
106108

107109
@Override

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreBufferService.java

Lines changed: 22 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -336,12 +336,19 @@ public CompletableFuture<Void> execSyncOffsetCommandAsync(
336336
return syncOffsetCmd.getCmdExecutedFuture();
337337
}
338338

339+
/**
340+
* lastRecordPersistedFuture indicates whether the last record was persisted successfully and will have two sources.
341+
* 1. From the follower code path, it is lastQueuedRecordPersistedFuture from the PCS set in putConsumeRecord().
342+
* 2. From the leader side, it is the persistedToDBFuture from the LeaderProducedRecordContext from when the
343+
* LeaderProducerCallback was created.
344+
*/
339345
public void execSyncOffsetFromSnapshotAsync(
340346
PubSubTopicPartition topicPartition,
341347
PartitionTracker vtDivSnapshot,
348+
CompletableFuture<Void> lastRecordPersistedFuture,
342349
StoreIngestionTask ingestionTask) throws InterruptedException {
343350
DefaultPubSubMessage fakeRecord = new FakePubSubMessage(topicPartition);
344-
SyncVtDivNode syncDivNode = new SyncVtDivNode(fakeRecord, vtDivSnapshot, ingestionTask);
351+
SyncVtDivNode syncDivNode = new SyncVtDivNode(fakeRecord, vtDivSnapshot, lastRecordPersistedFuture, ingestionTask);
345352
getDrainerForConsumerRecord(fakeRecord, topicPartition.getPartitionNumber()).put(syncDivNode);
346353
}
347354

@@ -688,20 +695,31 @@ protected int getBaseClassOverhead() {
688695
/**
689696
* Allows the ConsumptionTask to command the Drainer to sync the VT DIV to the OffsetRecord.
690697
*/
691-
private static class SyncVtDivNode extends QueueNode {
698+
static class SyncVtDivNode extends QueueNode {
692699
private static final int PARTIAL_CLASS_OVERHEAD = getClassOverhead(SyncVtDivNode.class);
693700

694-
private PartitionTracker vtDivSnapshot;
701+
private final PartitionTracker vtDivSnapshot;
702+
private final CompletableFuture<Void> lastRecordPersistedFuture;
695703

696704
public SyncVtDivNode(
697705
DefaultPubSubMessage consumerRecord,
698706
PartitionTracker vtDivSnapshot,
707+
CompletableFuture<Void> lastRecordPersistedFuture,
699708
StoreIngestionTask ingestionTask) {
700709
super(consumerRecord, ingestionTask, StringUtils.EMPTY, 0);
701710
this.vtDivSnapshot = vtDivSnapshot;
711+
this.lastRecordPersistedFuture = lastRecordPersistedFuture;
702712
}
703713

704714
public void execute() {
715+
if (!lastRecordPersistedFuture.isDone() || lastRecordPersistedFuture.isCompletedExceptionally()) {
716+
LOGGER.warn(
717+
"event=globalRtDiv Skipping SyncVtDivNode for {} because preceding record failed (done={} exception={})",
718+
getConsumerRecord().getTopicPartition(),
719+
lastRecordPersistedFuture.isDone(),
720+
lastRecordPersistedFuture.isCompletedExceptionally());
721+
return;
722+
}
705723
getIngestionTask().updateAndSyncOffsetFromSnapshot(vtDivSnapshot, getConsumerRecord().getTopicPartition());
706724
}
707725

@@ -845,7 +863,7 @@ public void run() {
845863
}
846864
}
847865

848-
private static class FakePubSubMessage implements DefaultPubSubMessage {
866+
static class FakePubSubMessage implements DefaultPubSubMessage {
849867
private static final int SHALLOW_CLASS_OVERHEAD = ClassSizeEstimator.getClassOverhead(FakePubSubMessage.class);
850868
private final PubSubTopicPartition topicPartition;
851869

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1325,6 +1325,10 @@ private int handleSingleMessage(
13251325
elapsedTimeForPuttingIntoQueue.setValue(
13261326
elapsedTimeForPuttingIntoQueue.getValue() + LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS));
13271327
}
1328+
1329+
// Intentionally not protecting against exceptions thrown by putConsumerRecord()
1330+
// Only sync OffsetRecord if the message that triggered the sync was successfully enqueued into the drainer
1331+
syncOffsetFromSnapshotIfNeeded(record, topicPartition); // latest consumed VT position (LCVP) in offset record
13281332
break;
13291333
case PRODUCED_TO_KAFKA:
13301334
case SKIPPED_MESSAGE:
@@ -2806,6 +2810,8 @@ boolean shouldSendGlobalRtDiv(DefaultPubSubMessage record, PartitionConsumptionS
28062810
return syncBytesInterval > 0 && (getConsumedBytesSinceLastSync().getOrDefault(brokerUrl, 0L) >= syncBytesInterval);
28072811
}
28082812

2813+
abstract void syncOffsetFromSnapshotIfNeeded(DefaultPubSubMessage record, PubSubTopicPartition topicPartition);
2814+
28092815
/**
28102816
* Update the offset metadata in OffsetRecord in the following cases:
28112817
* 1. A ControlMessage other than Start_of_Segment and End_of_Segment is processed

clients/da-vinci-client/src/main/java/com/linkedin/davinci/validation/PartitionTracker.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -190,7 +190,7 @@ public Map<CharSequence, ProducerPartitionState> getPartitionStates(TopicType ty
190190
.stream()
191191
.collect(
192192
Collectors.toMap(
193-
entry -> GuidUtils.getHexFromGuid(entry.getKey()),
193+
entry -> GuidUtils.guidToUtf8(entry.getKey()),
194194
entry -> entry.getValue().toProducerPartitionState()));
195195
}
196196

clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java

Lines changed: 15 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@
4545
import com.linkedin.venice.compression.VeniceCompressor;
4646
import com.linkedin.venice.exceptions.VeniceTimeoutException;
4747
import com.linkedin.venice.kafka.protocol.ControlMessage;
48+
import com.linkedin.venice.kafka.protocol.Delete;
4849
import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope;
4950
import com.linkedin.venice.kafka.protocol.LeaderMetadata;
5051
import com.linkedin.venice.kafka.protocol.ProducerMetadata;
@@ -113,6 +114,8 @@
113114

114115
public class LeaderFollowerStoreIngestionTaskTest {
115116
private static final PubSubTopicRepository TOPIC_REPOSITORY = new PubSubTopicRepository();
117+
private static final int GLOBAL_RT_DIV_VERSION =
118+
AvroProtocolDefinition.GLOBAL_RT_DIV_STATE.getCurrentProtocolVersion();
116119

117120
private Store mockStore;
118121
private LeaderFollowerStoreIngestionTask leaderFollowerStoreIngestionTask;
@@ -605,8 +608,7 @@ public void testSendGlobalRtDivMessage() throws InterruptedException, IOExceptio
605608
byte[] valueBytes = ByteUtils.extractByteArray(compressor.decompress(ByteBuffer.wrap(compressedBytes)));
606609
InternalAvroSpecificSerializer<GlobalRtDivState> serializer =
607610
leaderFollowerStoreIngestionTask.globalRtDivStateSerializer;
608-
GlobalRtDivState globalRtDiv =
609-
serializer.deserialize(valueBytes, AvroProtocolDefinition.GLOBAL_RT_DIV_STATE.getCurrentProtocolVersion());
611+
GlobalRtDivState globalRtDiv = serializer.deserialize(valueBytes, GLOBAL_RT_DIV_VERSION);
610612
assertNotNull(globalRtDiv);
611613

612614
// Verify the callback has DivSnapshot (VT + RT DIV)
@@ -618,19 +620,20 @@ public void testSendGlobalRtDivMessage() throws InterruptedException, IOExceptio
618620
assertEquals(callbackPayload.getPartition(), partition);
619621
assertTrue(callbackPayload.getValue().payloadUnion instanceof Put);
620622
Put put = (Put) callbackPayload.getValue().payloadUnion;
621-
assertEquals(put.getSchemaId(), AvroProtocolDefinition.GLOBAL_RT_DIV_STATE.getCurrentProtocolVersion());
623+
assertEquals(put.getSchemaId(), GLOBAL_RT_DIV_VERSION);
622624
assertNotNull(put.getPutValue());
623625

624626
// Verify that completing the future from put() causes execSyncOffsetFromSnapshotAsync to be called
625627
// and that produceResult should override the LCVP of the VT DIV sent to the drainer
626-
verify(mockStoreBufferService, never()).execSyncOffsetFromSnapshotAsync(any(), any(), any());
628+
verify(mockStoreBufferService, never()).execSyncOffsetFromSnapshotAsync(any(), any(), any(), any());
627629
PubSubProduceResult produceResult = mock(PubSubProduceResult.class);
628630
PubSubPosition specificPosition = InMemoryPubSubPosition.of(11L);
629631
when(produceResult.getPubSubPosition()).thenReturn(specificPosition);
630632
when(produceResult.getSerializedSize()).thenReturn(keyBytes.length + put.putValue.remaining());
631633
callback.onCompletion(produceResult, null);
632634
ArgumentCaptor<PartitionTracker> vtDivCaptor = ArgumentCaptor.forClass(PartitionTracker.class);
633-
verify(mockStoreBufferService, times(1)).execSyncOffsetFromSnapshotAsync(any(), vtDivCaptor.capture(), any());
635+
verify(mockStoreBufferService, times(1))
636+
.execSyncOffsetFromSnapshotAsync(any(), vtDivCaptor.capture(), any(), any());
634637
assertEquals(vtDivCaptor.getValue().getLatestConsumedVtPosition(), specificPosition);
635638
}
636639

@@ -668,17 +671,22 @@ public void testShouldSyncOffsetFromSnapshot() throws InterruptedException {
668671
Put mockPut = mock(Put.class);
669672
KafkaMessageEnvelope mockKme = globalRtDivMessage.getValue();
670673

671-
// The method should only return true for non-chunk Global RT DIV messages
674+
// The method should only return true for non-chunked Global RT DIV messages
672675
assertFalse(mockIngestionTask.shouldSyncOffsetFromSnapshot(globalRtDivMessage, mockPartitionConsumptionState));
673676
doReturn(true).when(mockKey).isGlobalRtDiv();
674677
doReturn(mockPut).when(mockKme).getPayloadUnion();
675-
doReturn(AvroProtocolDefinition.GLOBAL_RT_DIV_STATE.getCurrentProtocolVersion()).when(mockPut).getSchemaId();
678+
doReturn(GLOBAL_RT_DIV_VERSION).when(mockPut).getSchemaId();
676679
assertTrue(mockIngestionTask.shouldSyncOffsetFromSnapshot(globalRtDivMessage, mockPartitionConsumptionState));
677680
doReturn(AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion()).when(mockPut).getSchemaId();
678681
assertFalse(mockIngestionTask.shouldSyncOffsetFromSnapshot(globalRtDivMessage, mockPartitionConsumptionState));
679682
doReturn(AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion()).when(mockPut).getSchemaId();
680683
assertTrue(mockIngestionTask.shouldSyncOffsetFromSnapshot(globalRtDivMessage, mockPartitionConsumptionState));
681684

685+
// The method should not error when a non-Put value is passed in
686+
Delete mockDelete = mock(Delete.class);
687+
doReturn(mockDelete).when(mockKme).getPayloadUnion();
688+
assertFalse(mockIngestionTask.shouldSyncOffsetFromSnapshot(globalRtDivMessage, mockPartitionConsumptionState));
689+
682690
// Set up Control Message
683691
final DefaultPubSubMessage nonSegmentControlMessage = getMockMessage(2).getMessage();
684692
mockKey = nonSegmentControlMessage.getKey();

0 commit comments

Comments
 (0)