Skip to content
Draft
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 @@ -2457,19 +2457,8 @@ protected DelegateConsumerRecordResult delegateConsumerRecord(
// Update the latest consumed VT position (LCVP) since we're consuming from the version topic
if (isGlobalRtDivEnabled()) {
getConsumerDiv().updateLatestConsumedVtPosition(partition, consumerRecord.getPosition());

if (shouldSyncOffsetFromSnapshot(consumerRecord, partitionConsumptionState)) {
PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(getVersionTopic(), partition);
PartitionTracker vtDiv = consumerDiv.cloneVtProducerStates(partition); // has latest consumed VT position
storeBufferService.execSyncOffsetFromSnapshotAsync(topicPartition, vtDiv, this);
// TODO: remove. this is a temporary log for debugging while the feature is in its infancy
int partitionStateMapSize = vtDiv.getPartitionStates(PartitionTracker.VERSION_TOPIC).size();
LOGGER.info(
"event=globalRtDiv Syncing LCVP for OffsetRecord topic-partition: {} position: {} size: {}",
topicPartition,
consumerRecord.getPosition(),
partitionStateMapSize);
}
// Only after the current message is queued to drainer
// The Offset Record's LCVP may be synced in syncOffsetFromSnapshotIfNeeded()
}

/**
Expand Down Expand Up @@ -2760,6 +2749,26 @@ protected DelegateConsumerRecordResult delegateConsumerRecord(
}
}

void syncOffsetFromSnapshotIfNeeded(DefaultPubSubMessage record, PubSubTopicPartition topicPartition) {
int partition = topicPartition.getPartitionNumber();
if (!isGlobalRtDivEnabled() || !shouldSyncOffsetFromSnapshot(record, getPartitionConsumptionState(partition))) {
return; // without Global RT DIV enabled, the offset record is synced in the drainer in syncOffset()
}

try {
PartitionTracker vtDiv = consumerDiv.cloneVtProducerStates(partition); // has latest consumed VT position
storeBufferService.execSyncOffsetFromSnapshotAsync(topicPartition, vtDiv, this);
// TODO: remove. this is a temporary log for debugging while the feature is in its infancy
LOGGER.info(
"event=globalRtDiv Syncing LCVP for OffsetRecord topic-partition: {} position: {} size: {}",
topicPartition,
record.getPosition(),
vtDiv.getPartitionStates(PartitionTracker.VERSION_TOPIC).size());
} catch (InterruptedException e) {
LOGGER.error("Unable to sync offset", e);
}
}

/**
* Followers should sync the VT DIV to the OffsetRecord if the consumer sees a Global RT DIV message
* (sync only once for a Global RT DIV, which can either be one singular message or multiple chunks + one manifest.
Expand All @@ -2771,8 +2780,13 @@ boolean shouldSyncOffsetFromSnapshot(DefaultPubSubMessage consumerRecord, Partit
if (put.getSchemaId() != CHUNK_SCHEMA_ID) {
return true;
}
} else if (isNonSegmentControlMessage(consumerRecord, null)) {
return true; // sync when processing most control messages
}
return isNonSegmentControlMessage(consumerRecord, null);

// must be greater than the interval in shouldSendGlobalRtDiv() to not interfere
final long syncBytesInterval = getSyncBytesInterval(pcs); // size-based sync condition
return syncBytesInterval > 0 && (pcs.getProcessedRecordSizeSinceLastSync() >= 2 * syncBytesInterval);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1287,6 +1287,8 @@ private int handleSingleMessage(
elapsedTimeForPuttingIntoQueue.setValue(
elapsedTimeForPuttingIntoQueue.getValue() + LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS));
}

syncOffsetFromSnapshotIfNeeded(record, topicPartition); // latest consumed VT position (LCVP) in offset record
break;
case PRODUCED_TO_KAFKA:
case SKIPPED_MESSAGE:
Expand Down Expand Up @@ -1865,6 +1867,10 @@ protected void updateOffsetMetadataAndSyncOffset(@Nonnull PartitionConsumptionSt
}

protected void updateOffsetMetadataAndSyncOffset(DataIntegrityValidator div, @Nonnull PartitionConsumptionState pcs) {
if (isGlobalRtDivEnabled()) {
LOGGER.info("Skipping updateOffsetMetadataAndSyncOffset() because Global RT DIV is enabled.");
return;
}
/**
* Offset metadata and producer states must be updated at the same time in OffsetRecord; otherwise, one checkpoint
* could be ahead of the other.
Expand Down Expand Up @@ -2745,6 +2751,8 @@ boolean shouldSendGlobalRtDiv(DefaultPubSubMessage record, PartitionConsumptionS
return syncBytesInterval > 0 && (getConsumedBytesSinceLastSync().getOrDefault(brokerUrl, 0L) >= syncBytesInterval);
}

abstract void syncOffsetFromSnapshotIfNeeded(DefaultPubSubMessage record, PubSubTopicPartition topicPartition);

/**
* Update the offset metadata in OffsetRecord in the following cases:
* 1. A ControlMessage other than Start_of_Segment and End_of_Segment is processed
Expand Down Expand Up @@ -2833,7 +2841,11 @@ private void syncOffset(PartitionConsumptionState pcs) {
// TODO: update
String msg = "Offset synced for replica: " + pcs.getReplicaId() + " - localVtOffset: {}";
if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) {
LOGGER.info(msg, offsetRecord.getCheckpointedLocalVtPosition());
LOGGER.info(
msg,
(isGlobalRtDivEnabled())
? offsetRecord.getLatestConsumedVtPosition()
: offsetRecord.getCheckpointedLocalVtPosition());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ public void setPartitionState(
PartitionTracker.TopicType type,
int partition,
Map<CharSequence, ProducerPartitionState> producerPartitionStateMap) {
registerPartition(partition).setPartitionState(type, producerPartitionStateMap, DISABLED);
registerPartition(partition).setPartitionState(type, producerPartitionStateMap, this.maxAgeInMs);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,15 +153,15 @@ Segment getSegment(TopicType type, GUID guid) {
}

public void setPartitionState(TopicType type, OffsetRecord offsetRecord, long maxAgeInMs) {
long minimumRequiredRecordProducerTimestamp =
maxAgeInMs == DISABLED ? DISABLED : offsetRecord.getMaxMessageTimeInMs() - maxAgeInMs;
setPartitionState(type, offsetRecord.getProducerPartitionStateMap(), minimumRequiredRecordProducerTimestamp);
long earliestAllowableTimestamp =
maxAgeInMs == DISABLED ? DISABLED : offsetRecord.calculateLatestMessageTimeInMs() - maxAgeInMs;
setPartitionState(type, offsetRecord.getProducerPartitionStateMap(), earliestAllowableTimestamp);
}

public void setPartitionState(
TopicType type,
Map<CharSequence, ProducerPartitionState> producerPartitionStateMap,
long minimumRequiredRecordProducerTimestamp) {
long earliestAllowableTimestamp) {
Iterator<Map.Entry<CharSequence, ProducerPartitionState>> iterator =
producerPartitionStateMap.entrySet().iterator();
Map.Entry<CharSequence, ProducerPartitionState> entry;
Expand All @@ -171,7 +171,7 @@ public void setPartitionState(
entry = iterator.next();
producerGuid = GuidUtils.getGuidFromCharSequence(entry.getKey());
producerPartitionState = entry.getValue();
if (producerPartitionState.messageTimestamp >= minimumRequiredRecordProducerTimestamp) {
if (producerPartitionState.messageTimestamp >= earliestAllowableTimestamp) {
/**
* This {@link producerPartitionState} is eligible to be retained, so we'll set the state in the
* {@link PartitionTracker}.
Expand All @@ -190,7 +190,7 @@ public Map<CharSequence, ProducerPartitionState> getPartitionStates(TopicType ty
.stream()
.collect(
Collectors.toMap(
entry -> GuidUtils.getHexFromGuid(entry.getKey()),
entry -> GuidUtils.guidToUtf8(entry.getKey()),
entry -> entry.getValue().toProducerPartitionState()));
}

Expand All @@ -212,6 +212,7 @@ private void setSegment(TopicType type, GUID guid, Segment segment) {
*/
public void cloneVtProducerStates(PartitionTracker destProducerTracker) {
for (Map.Entry<GUID, Segment> entry: vtSegments.entrySet()) {
entry.getValue().getLastRecordProducerTimestamp();
destProducerTracker.setSegment(PartitionTracker.VERSION_TOPIC, entry.getKey(), new Segment(entry.getValue()));
}
destProducerTracker.updateLatestConsumedVtPosition(latestConsumedVtPosition.get());
Expand Down Expand Up @@ -711,7 +712,7 @@ public void checkMissingMessage(
}

void clearExpiredStateAndUpdateOffsetRecord(TopicType type, OffsetRecord offsetRecord, long maxAgeInMs) {
long minimumRequiredRecordProducerTimestamp = offsetRecord.getMaxMessageTimeInMs() - maxAgeInMs;
long minimumRequiredRecordProducerTimestamp = offsetRecord.calculateLatestMessageTimeInMs() - maxAgeInMs;
int numberOfClearedGUIDs = 0;
Iterator<Map.Entry<GUID, Segment>> iterator = getSegments(type).entrySet().iterator();
Map.Entry<GUID, Segment> entry;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -692,6 +692,31 @@ public void testShouldSyncOffsetFromSnapshot() throws InterruptedException {
doReturn(ControlMessageType.START_OF_SEGMENT.getValue()).when(mockControlMessage).getControlMessageType();
assertFalse(
mockIngestionTask.shouldSyncOffsetFromSnapshot(nonSegmentControlMessage, mockPartitionConsumptionState));

// Mock the getSyncBytesInterval method to return a specific value
doReturn(1000L).when(mockIngestionTask).getSyncBytesInterval(any());

// Create a mock message that is neither a Global RT DIV nor a control message
final DefaultPubSubMessage regularMessage = getMockMessage(3).getMessage();
KafkaKey regularMockKey = regularMessage.getKey();
doReturn(false).when(regularMockKey).isGlobalRtDiv();
doReturn(false).when(regularMockKey).isControlMessage();

// Test case 1: When processedRecordSizeSinceLastSync is less than 2*syncBytesInterval
doReturn(1500L).when(mockPartitionConsumptionState).getProcessedRecordSizeSinceLastSync();
assertFalse(mockIngestionTask.shouldSyncOffsetFromSnapshot(regularMessage, mockPartitionConsumptionState));

// Test case 2: When processedRecordSizeSinceLastSync is equal to 2*syncBytesInterval
doReturn(2000L).when(mockPartitionConsumptionState).getProcessedRecordSizeSinceLastSync();
assertTrue(mockIngestionTask.shouldSyncOffsetFromSnapshot(regularMessage, mockPartitionConsumptionState));

// Test case 3: When processedRecordSizeSinceLastSync is greater than 2*syncBytesInterval
doReturn(2500L).when(mockPartitionConsumptionState).getProcessedRecordSizeSinceLastSync();
assertTrue(mockIngestionTask.shouldSyncOffsetFromSnapshot(regularMessage, mockPartitionConsumptionState));

// Test case 4: When syncBytesInterval is 0 (disabled)
doReturn(0L).when(mockIngestionTask).getSyncBytesInterval(any());
assertFalse(mockIngestionTask.shouldSyncOffsetFromSnapshot(regularMessage, mockPartitionConsumptionState));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -378,8 +378,8 @@ private static DefaultPubSubMessage buildRecord(
messageEnvelope.leaderMetadataFooter.upstreamPubSubPosition = PubSubSymbolicPosition.EARLIEST.toWireFormatBuffer();
messageEnvelope.payloadUnion = payload;

if (offsetRecord != null && offsetRecord.getMaxMessageTimeInMs() < brokerTimestamp) {
when(offsetRecord.getMaxMessageTimeInMs()).thenReturn(brokerTimestamp);
if (offsetRecord != null && offsetRecord.calculateLatestMessageTimeInMs() < brokerTimestamp) {
when(offsetRecord.calculateLatestMessageTimeInMs()).thenReturn(brokerTimestamp);
}

return new ImmutablePubSubMessage(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,11 @@ public static String getCharSequenceFromGuid(GUID guid) {
return new String(guid.bytes(), CHARSET); // TODO: Optimize this. It's probably expensive...
}

public static CharSequence guidToUtf8(GUID guid) {
/** TODO: Consider replacing with {@link GuidUtils#getUtf8FromGuid(GUID)}, which might be more efficient. */
return new Utf8(getCharSequenceFromGuid(guid));
}

public static Utf8 getUtf8FromGuid(GUID guid) {
/** Adapted from {@link StringCoding#encodeUTF8(byte, byte[], boolean)} */
byte[] val = guid.bytes();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.io.ByteBufferToHexFormatJsonEncoder;
import org.apache.avro.io.Encoder;
import org.apache.avro.util.Utf8;
import org.apache.commons.lang.Validate;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
Expand Down Expand Up @@ -176,12 +175,12 @@ public void setLastCheckpointTimestamp(long timestamp) {
/**
* @return the last messageTimeStamp across all producers tracked by this OffsetRecord
*/
public long getMaxMessageTimeInMs() {
long maxMessageTimestamp = -1;
for (ProducerPartitionState state: this.partitionState.producerStates.values()) {
maxMessageTimestamp = Math.max(maxMessageTimestamp, state.messageTimestamp);
}
return maxMessageTimestamp;
public long calculateLatestMessageTimeInMs() {
return this.partitionState.producerStates.values()
.stream()
.mapToLong(ProducerPartitionState::getMessageTimestamp)
.max()
.orElse(-1);
}

public long getLatestProducerProcessingTimeInMs() {
Expand All @@ -201,11 +200,11 @@ public boolean isEndOfPushReceived() {
}

public synchronized void setProducerPartitionState(GUID producerGuid, ProducerPartitionState state) {
this.partitionState.producerStates.put(guidToUtf8(producerGuid), state);
this.partitionState.producerStates.put(GuidUtils.guidToUtf8(producerGuid), state);
}

public synchronized void removeProducerPartitionState(GUID producerGuid) {
this.partitionState.producerStates.remove(guidToUtf8(producerGuid));
this.partitionState.producerStates.remove(GuidUtils.guidToUtf8(producerGuid));
}

public synchronized Map<CharSequence, ProducerPartitionState> getProducerPartitionStateMap() {
Expand All @@ -218,30 +217,30 @@ public synchronized void setRealtimeTopicProducerState(
ProducerPartitionState state) {
partitionState.getRealtimeTopicProducerStates()
.computeIfAbsent(kafkaUrl, url -> new VeniceConcurrentHashMap<>())
.put(guidToUtf8(producerGuid), state);
.put(GuidUtils.guidToUtf8(producerGuid), state);
}

public synchronized void removeRealTimeTopicProducerState(String kafkaUrl, GUID producerGuid) {
if (partitionState.getRealtimeTopicProducerStates().get(kafkaUrl) == null) {
return;
}
partitionState.getRealtimeTopicProducerStates().get(kafkaUrl).remove(guidToUtf8(producerGuid));
partitionState.getRealtimeTopicProducerStates().get(kafkaUrl).remove(GuidUtils.guidToUtf8(producerGuid));
}

public synchronized ProducerPartitionState getRealTimeProducerState(String kafkaUrl, GUID producerGuid) {
Map<CharSequence, ProducerPartitionState> map = partitionState.getRealtimeTopicProducerStates().get(kafkaUrl);
if (map == null) {
return null;
}
return map.get(guidToUtf8(producerGuid));
return map.get(GuidUtils.guidToUtf8(producerGuid));
}

private Map<String, Map<CharSequence, ProducerPartitionState>> getRealTimeProducerState() {
return partitionState.getRealtimeTopicProducerStates();
}

public synchronized ProducerPartitionState getProducerPartitionState(GUID producerGuid) {
return getProducerPartitionStateMap().get(guidToUtf8(producerGuid));
return getProducerPartitionStateMap().get(GuidUtils.guidToUtf8(producerGuid));
}

public void setDatabaseInfo(Map<String, String> databaseInfo) {
Expand Down Expand Up @@ -387,22 +386,11 @@ public void setKeyUrnCompressionDict(KeyUrnCompressionDict keyUrnCompressionDict
this.partitionState.keyUrnCompressionDict = keyUrnCompressionDict;
}

/**
* It may be useful to cache this mapping. TODO: Explore GC tuning later.
*
* @param guid to be converted
* @return a {@link Utf8} instance corresponding to the {@link GUID} that was passed in
*/
CharSequence guidToUtf8(GUID guid) {
/** TODO: Consider replacing with {@link GuidUtils#getUtf8FromGuid(GUID)}, which might be more efficient. */
return new Utf8(GuidUtils.getCharSequenceFromGuid(guid));
}

@Override
public String toString() {
return "OffsetRecord{" + "localVtPosition=" + getCheckpointedLocalVtPosition() + ", remoteVtPosition="
+ getCheckpointedRemoteVtPosition() + ", rtPositions=" + getPartitionUpstreamPositionString() + ", leaderTopic="
+ getLeaderTopic() + ", offsetLag=" + getOffsetLag() + ", eventTimeEpochMs=" + getMaxMessageTimeInMs()
+ getLeaderTopic() + ", offsetLag=" + getOffsetLag() + ", eventTimeEpochMs=" + calculateLatestMessageTimeInMs()
+ ", latestProducerProcessingTimeInMs=" + getLatestProducerProcessingTimeInMs() + ", isEndOfPushReceived="
+ isEndOfPushReceived() + ", databaseInfo=" + getDatabaseInfo() + ", realTimeProducerState="
+ getRealTimeProducerState() + ", recordTransformerClassHash=" + getRecordTransformerClassHash()
Expand Down