-
Notifications
You must be signed in to change notification settings - Fork 114
[server] Global RT DIV: Max Age + Size-Based Sync #2302
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
base: main
Are you sure you want to change the base?
Changes from all commits
7d5bce1
3f1227f
f2db51a
ce0a3c1
d729032
bff6987
1533fa0
04f0952
2ccf50d
4b5f4e6
ac44289
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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; | ||
|
|
@@ -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}. | ||
|
|
@@ -210,26 +210,39 @@ private void setSegment(TopicType type, GUID guid, Segment segment) { | |
| /** | ||
| * Clone the vtSegments and LCVP to the destination PartitionTracker. May be called concurrently. | ||
| */ | ||
| public void cloneVtProducerStates(PartitionTracker destProducerTracker) { | ||
| public void cloneVtProducerStates(PartitionTracker destProducerTracker, long maxAgeInMs) { | ||
| long earliestAllowableTimestamp = maxAgeInMs == DISABLED ? DISABLED : new Date().getTime() - maxAgeInMs; | ||
| for (Map.Entry<GUID, Segment> entry: vtSegments.entrySet()) { | ||
| destProducerTracker.setSegment(PartitionTracker.VERSION_TOPIC, entry.getKey(), new Segment(entry.getValue())); | ||
| if (entry.getValue().getLastRecordTimestamp() >= earliestAllowableTimestamp) { | ||
| destProducerTracker.setSegment(PartitionTracker.VERSION_TOPIC, entry.getKey(), new Segment(entry.getValue())); | ||
| } else { | ||
| vtSegments.remove(entry.getKey()); // The state is eligible to be cleared. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Removing keys from a Map while iterating over it using standard for is a problem and will generally result in a ConcurrentModificationException. Can you check |
||
| } | ||
| } | ||
| destProducerTracker.updateLatestConsumedVtPosition(latestConsumedVtPosition.get()); | ||
| } | ||
|
|
||
| /** | ||
| * Clone the rtSegments to the destination PartitionTracker. Filter by brokerUrl. May be called concurrently. | ||
| */ | ||
| public void cloneRtProducerStates(PartitionTracker destProducerTracker, String brokerUrl) { | ||
| for (Map.Entry<String, VeniceConcurrentHashMap<GUID, Segment>> entry: rtSegments.entrySet()) { | ||
| if (!brokerUrl.isEmpty() && !brokerUrl.equals(entry.getKey())) { | ||
| continue; // filter by brokerUrl if specified | ||
| public void cloneRtProducerStates(PartitionTracker destProducerTracker, String brokerUrl, long maxAgeInMs) { | ||
| long earliestAllowableTimestamp = maxAgeInMs == DISABLED ? DISABLED : new Date().getTime() - maxAgeInMs; | ||
| for (Map.Entry<String, VeniceConcurrentHashMap<GUID, Segment>> broker2Segment: rtSegments.entrySet()) { | ||
| if (!brokerUrl.equals(broker2Segment.getKey())) { | ||
| continue; // filter by the specified brokerUrl | ||
| } | ||
| for (Map.Entry<GUID, Segment> rtEntry: entry.getValue().entrySet()) { | ||
| destProducerTracker.setSegment( | ||
| TopicType.of(TopicType.REALTIME_TOPIC_TYPE, entry.getKey()), | ||
| rtEntry.getKey(), | ||
| new Segment(rtEntry.getValue())); | ||
|
|
||
| final VeniceConcurrentHashMap<GUID, Segment> rtEntries = broker2Segment.getValue(); | ||
| for (Map.Entry<GUID, Segment> rtEntry: rtEntries.entrySet()) { | ||
| if (rtEntry.getValue().getLastRecordTimestamp() >= earliestAllowableTimestamp) { | ||
| TopicType realTimeTopicType = TopicType.of(TopicType.REALTIME_TOPIC_TYPE, broker2Segment.getKey()); | ||
| destProducerTracker.setSegment(realTimeTopicType, rtEntry.getKey(), new Segment(rtEntry.getValue())); | ||
| } else { | ||
| rtEntries.remove(rtEntry.getKey()); // The state is eligible to be cleared. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ditto. |
||
| } | ||
| } | ||
| if (broker2Segment.getValue().isEmpty()) { | ||
| rtSegments.remove(broker2Segment.getKey()); | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -711,7 +724,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; | ||
|
|
@@ -747,14 +760,15 @@ public void removeProducerState(TopicType type, GUID guid, OffsetRecord offsetRe | |
| } | ||
|
|
||
| @VisibleForTesting | ||
| Map<String, Map<GUID, Segment>> getAllRtSegmentsForTesting() { | ||
| return rtSegments.entrySet() | ||
| .stream() | ||
| .collect(Collectors.toMap(Map.Entry::getKey, entry -> Collections.unmodifiableMap(entry.getValue()))); | ||
| VeniceConcurrentHashMap<String, VeniceConcurrentHashMap<GUID, Segment>> getRtSegmentsForTesting() { | ||
| return rtSegments; | ||
| // return rtSegments.entrySet() | ||
| // .stream() | ||
| // .collect(Collectors.toMap(Map.Entry::getKey, entry -> Collections.unmodifiableMap(entry.getValue()))); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| Map<GUID, Segment> getVtSegmentsForTesting() { | ||
| VeniceConcurrentHashMap<GUID, Segment> getVtSegmentsForTesting() { | ||
| return vtSegments; | ||
| } | ||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think I have a question about this implementation if we use the running sum of the record sizes from pcs here. There seems to be a problem:
shouldSyncOffsetFromSnapshotis called in the consumer thread.pcs.processedRecordSizeSinceLastSyncis cleared later in the drainer thread when syncing offset.So, I imagine what could happen is that once size-based condition is triggered for one record in consumer thread, it will keep firing continuously for every following record for quite some time, until the first one record got synced in drainer thread. Because of that, we could have unnecessarily triggered a lot more sync offset operations.