Skip to content
Open
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 @@ -1097,13 +1097,19 @@ protected void syncConsumedUpstreamRTOffsetMapIfNeeded(
Map<String, PubSubPosition> upstreamStartPositionByPubSubUrl) {
// Update in-memory consumedUpstreamRTOffsetMap in case no RT record is consumed after the subscription
final PubSubTopic leaderTopic = pcs.getOffsetRecord().getLeaderTopic(pubSubTopicRepository);
final PubSubTopicPartition leaderTopicPartition = pcs.getSourceTopicPartition(leaderTopic);
if (leaderTopic != null && leaderTopic.isRealTime()) {
upstreamStartPositionByPubSubUrl.forEach((kafkaURL, upstreamStartPosition) -> {
PubSubPosition latestConsumedRtPosition =
getLatestConsumedUpstreamPositionForHybridOffsetLagMeasurement(pcs, kafkaURL);
// update latest consumed RT position if incoming upstream start position is greater
// than the latest consumed RT position
if (upstreamStartPosition.getNumericOffset() > latestConsumedRtPosition.getNumericOffset()) {
if ((PubSubSymbolicPosition.EARLIEST.equals(latestConsumedRtPosition)
&& !PubSubSymbolicPosition.EARLIEST.equals(upstreamStartPosition))
|| getTopicManager(kafkaURL).diffPosition(
Utils.createPubSubTopicPartitionFromLeaderTopicPartition(kafkaURL, leaderTopicPartition),
upstreamStartPosition,
latestConsumedRtPosition) > 0) {
updateLatestConsumedRtPositions(pcs, kafkaURL, upstreamStartPosition);
}
});
Expand Down Expand Up @@ -1583,7 +1589,8 @@ protected static void checkAndHandleUpstreamOffsetRewind(
final PubSubPosition newUpstreamPosition,
final PubSubPosition previousUpstreamPosition,
LeaderFollowerStoreIngestionTask ingestionTask) {
if (newUpstreamPosition.getNumericOffset() >= previousUpstreamPosition.getNumericOffset()) {
if (PubSubSymbolicPosition.EARLIEST.equals(previousUpstreamPosition)
|| topicManager.diffPosition(pubSubTopicPartition, newUpstreamPosition, previousUpstreamPosition) >= 0) {
return; // Rewind did not happen
}
if (!ingestionTask.isHybridMode()) {
Expand Down Expand Up @@ -1988,7 +1995,8 @@ protected boolean shouldProcessRecord(DefaultPubSubMessage record) {
}

PubSubPosition lastProcessedVtPos = partitionConsumptionState.getLatestProcessedVtPosition();
if (lastProcessedVtPos.getNumericOffset() >= record.getPosition().getNumericOffset()) {
if (!PubSubSymbolicPosition.EARLIEST.equals(lastProcessedVtPos) && topicManagerRepository.getLocalTopicManager()
.diffPosition(record.getTopicPartition(), lastProcessedVtPos, record.getPosition()) >= 0) {
String message = partitionConsumptionState.getLeaderFollowerState() + " replica: "
+ partitionConsumptionState.getReplicaId() + " had already processed the record";
if (!REDUNDANT_LOGGING_FILTER.isRedundantException(message)) {
Expand Down Expand Up @@ -3730,7 +3738,7 @@ void loadGlobalRtDiv(int partition, String brokerUrl) {
// If the GlobalRtDivState is not present, it could be acceptable if this could be the first leader to be elected
// Object not existing could be problematic if this isn't the first leader (detected via nonzero leaderPosition)
PubSubPosition leaderPosition = pcs.getLeaderPosition(brokerUrl, false);
if (leaderPosition.getNumericOffset() > 0) {
if (!PubSubSymbolicPosition.EARLIEST.equals(leaderPosition)) {
LOGGER.warn(
"Unable to retrieve Global RT DIV from storage engine for topic-partition: {} brokerUrl: {} leaderPosition: {}",
topicPartition,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4727,10 +4727,11 @@ protected PubSubPosition extractUpstreamPosition(DefaultPubSubMessage consumerRe
return PubSubSymbolicPosition.EARLIEST;
}
LeaderMetadata leaderMetadataFooter = consumerRecord.getValue().leaderMetadataFooter;

// always return upstreamOffset instead of upstreamPubSubPosition
// till we fix all the issues in offset to pubsubPosition migration
return PubSubUtil.fromKafkaOffset(leaderMetadataFooter.upstreamOffset);
return deserializePositionWithOffsetFallback(
pubSubContext.getPubSubPositionDeserializer(),
consumerRecord.getTopicPartition(),
leaderMetadataFooter.upstreamPubSubPosition,
leaderMetadataFooter.upstreamOffset);
}

// extract the upstream cluster id from the given consumer record's leader metadata.
Expand Down Expand Up @@ -5048,7 +5049,7 @@ PubSubPosition deserializePositionWithOffsetFallback(
try {
final PubSubPosition position = pubSubPositionDeserializer.toPosition(wireFormatBytes);
// Guard against regressions: honor the caller-provided minimum offset.
if (offset > 0 && position.getNumericOffset() < offset) {
if (position.getNumericOffset() < offset) {
Copy link

Copilot AI Oct 28, 2025

Choose a reason for hiding this comment

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

Removing the offset > 0 check means that this condition will now fire even when offset is 0 or negative. This could be intentional for handling edge cases, but it changes the behavior for positions at offset 0. Consider documenting why the guard was removed or verifying that positions at offset 0 should be subject to this fallback logic.

Suggested change
if (position.getNumericOffset() < offset) {
if (offset > 0 && position.getNumericOffset() < offset) {

Copilot uses AI. Check for mistakes.
String context = String.format(" for: %s/%s", topicPartition, versionTopic);
if (!REDUNDANT_LOGGING_FILTER.isRedundantException(context)) {
LOGGER.warn(
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
package com.linkedin.davinci.validation;

import static com.linkedin.davinci.validation.DataIntegrityValidator.DISABLED;
import static com.linkedin.venice.pubsub.PubSubUtil.getPubSubPositionString;

import com.linkedin.venice.annotation.Threadsafe;
import com.linkedin.venice.annotation.VisibleForTesting;
Expand Down Expand Up @@ -901,9 +900,8 @@ private String generateMessage(
if (consumerRecord.getValue().leaderMetadataFooter != null) {
sb.append("; LeaderMetadata { upstream position: ")
.append(
getPubSubPositionString(
pubSubPositionDeserializer,
consumerRecord.getValue().leaderMetadataFooter.upstreamPubSubPosition))
pubSubPositionDeserializer
.toPosition(consumerRecord.getValue().leaderMetadataFooter.upstreamPubSubPosition))
Comment on lines +903 to +904
Copy link

Copilot AI Oct 28, 2025

Choose a reason for hiding this comment

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

Potential NullPointerException when upstreamPubSubPosition is null. The toPosition method throws an IllegalArgumentException when passed a null ByteBuffer. Consider adding a null check or using a defensive fallback similar to other parts of the codebase: upstreamPubSubPosition == null ? \"-\" : pubSubPositionDeserializer.toPosition(upstreamPubSubPosition)

Suggested change
pubSubPositionDeserializer
.toPosition(consumerRecord.getValue().leaderMetadataFooter.upstreamPubSubPosition))
consumerRecord.getValue().leaderMetadataFooter.upstreamPubSubPosition == null
? "-"
: pubSubPositionDeserializer.toPosition(consumerRecord.getValue().leaderMetadataFooter.upstreamPubSubPosition))

Copilot uses AI. Check for mistakes.
.append("; upstream pub sub cluster ID: ")
.append(consumerRecord.getValue().leaderMetadataFooter.upstreamKafkaClusterId)
.append("; producer host name: ")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2343,7 +2343,7 @@ public void testSubscribeCompletedPartition(AAConfig aaConfig) throws Exception
ArgumentCaptor<PubSubPosition> positionCaptor = ArgumentCaptor.forClass(PubSubPosition.class);
verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS))
.completed(eq(topic), eq(PARTITION_FOO), positionCaptor.capture(), eq("STANDBY"));
assertEquals(positionCaptor.getValue().getNumericOffset(), p100.getNumericOffset());
assertEquals(positionCaptor.getValue(), p100);
}, aaConfig);
config.setBeforeStartingConsumption(
() -> doReturn(getOffsetRecord(p100, true, pubSubContext)).when(mockStorageMetadataService)
Expand All @@ -2364,7 +2364,7 @@ public void testSubscribeCompletedPartitionUnsubscribe(AAConfig aaConfig) throws
ArgumentCaptor<PubSubPosition> positionCaptor = ArgumentCaptor.forClass(PubSubPosition.class);
verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS))
.completed(eq(topic), eq(PARTITION_FOO), positionCaptor.capture(), eq("STANDBY"));
assertEquals(positionCaptor.getValue().getNumericOffset(), p100.getNumericOffset());
assertEquals(positionCaptor.getValue(), p100);
verify(aggKafkaConsumerService, timeout(TEST_TIMEOUT_MS))
.batchUnsubscribeConsumerFor(pubSubTopic, Collections.singleton(fooTopicPartition));
verify(aggKafkaConsumerService, never()).unsubscribeConsumerFor(pubSubTopic, barTopicPartition);
Expand Down Expand Up @@ -2397,7 +2397,7 @@ public void testCompleteCalledWhenUnsubscribeAfterBatchPushDisabled(AAConfig aaC
ArgumentCaptor<PubSubPosition> positionCaptor = ArgumentCaptor.forClass(PubSubPosition.class);
verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS))
.completed(eq(topic), eq(PARTITION_FOO), positionCaptor.capture(), eq("STANDBY"));
assertEquals(positionCaptor.getValue().getNumericOffset(), p10.getNumericOffset());
assertEquals(positionCaptor.getValue(), p10);
}, aaConfig);
config.setBeforeStartingConsumption(() -> {
Store mockStore = mock(Store.class);
Expand Down Expand Up @@ -2593,11 +2593,12 @@ public void testDataValidationCheckPointing(SortedInput sortedInput, AAConfig aa
long offset = entry.getValue().getInternalOffset();
LOGGER.info("Verifying completed was called for partition {} and offset {} or greater.", partition, offset);

ArgumentCaptor<PubSubPosition> positionCaptor = ArgumentCaptor.forClass(PubSubPosition.class);
ArgumentCaptor<InMemoryPubSubPosition> positionCaptor =
ArgumentCaptor.forClass(InMemoryPubSubPosition.class);
verify(mockLogNotifier, timeout(LONG_TEST_TIMEOUT).atLeastOnce())
.completed(eq(topic), eq(partition), positionCaptor.capture(), eq("STANDBY"));
PubSubPosition completedPosition = positionCaptor.getValue();
assertTrue(completedPosition.getNumericOffset() >= offset);
InMemoryPubSubPosition completedPosition = positionCaptor.getValue();
assertTrue(completedPosition.getInternalOffset() >= offset);
});

// After this, all asynchronous processing should be finished, so there's no need for time outs anymore.
Expand Down Expand Up @@ -4704,14 +4705,14 @@ public void testOffsetSyncBeforeGracefulShutDown(AAConfig aaConfig) throws Excep
} else {
// If the pcs is non-null, then we perform additional checks to ensure that it was not synced
Assert.assertEquals(
pcs.getLatestProcessedVtPosition().getNumericOffset(),
p0.getNumericOffset(),
pcs.getLatestProcessedVtPosition(),
p0,
"pcs.getLatestProcessedLocalVersionTopicOffset() for PARTITION_FOO is expected to be zero!");
OffsetRecord offsetRecord = pcs.getOffsetRecord();
assertNotNull(offsetRecord);
Assert.assertEquals(
offsetRecord.getCheckpointedLocalVtPosition().getNumericOffset(),
p0.getNumericOffset(),
offsetRecord.getCheckpointedLocalVtPosition(),
p0,
"offsetRecord.getCheckpointedLocalVtPosition() for PARTITION_FOO is expected to be zero!");
}

Expand All @@ -4723,14 +4724,14 @@ public void testOffsetSyncBeforeGracefulShutDown(AAConfig aaConfig) throws Excep
assertNotNull(offsetRecord);
Assert.assertEquals(pcs.getLatestProcessedVtPosition(), p2); // PCS updated
// offsetRecord hasn't been updated yet
Assert.assertEquals(offsetRecord.getCheckpointedLocalVtPosition().getNumericOffset(), p0.getNumericOffset());
Assert.assertEquals(offsetRecord.getCheckpointedLocalVtPosition(), p0);
storeIngestionTaskUnderTest.close();

// Verify the OffsetRecord is synced up with pcs and get persisted only once during shutdown
verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS).times(1)).put(eq(topic), eq(PARTITION_FOO), any());
Assert.assertEquals(
offsetRecord.getCheckpointedLocalVtPosition().getNumericOffset(),
p2.getNumericOffset(),
offsetRecord.getCheckpointedLocalVtPosition(),
p2,
"offsetRecord.getCheckpointedLocalVtPosition() for PARTITION_FOO is expected to be 2!");

// Verify that the underlying storage engine sync function is invoked.
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
package com.linkedin.venice;

import static com.linkedin.venice.chunking.ChunkKeyValueTransformer.KeyType.WITH_VALUE_CHUNK;
import static com.linkedin.venice.pubsub.PubSubUtil.getPubSubPositionString;

import com.github.luben.zstd.Zstd;
import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper;
Expand Down Expand Up @@ -506,9 +505,7 @@ private void logRecordMetadata(DefaultPubSubMessage record) {
producerMetadata.messageTimestamp,
producerMetadata.logicalTimestamp,
leaderMetadata == null ? "-" : leaderMetadata.hostName,
leaderMetadata == null
? "-"
: getPubSubPositionString(pubSubPositionDeserializer, leaderMetadata.upstreamPubSubPosition),
leaderMetadata == null ? "-" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition),
Copy link

Copilot AI Oct 28, 2025

Choose a reason for hiding this comment

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

Potential NullPointerException when leaderMetadata.upstreamPubSubPosition is null. The toPosition method throws an IllegalArgumentException when passed a null ByteBuffer. Add a null check for upstreamPubSubPosition or use a nested ternary to handle this case: leaderMetadata == null || leaderMetadata.upstreamPubSubPosition == null ? \"-\" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition)

Suggested change
leaderMetadata == null ? "-" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition),
(leaderMetadata == null || leaderMetadata.upstreamPubSubPosition == null) ? "-" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition),

Copilot uses AI. Check for mistakes.
leaderMetadata == null ? "-" : leaderMetadata.upstreamKafkaClusterId,
chunkMetadata);
} catch (Exception e) {
Expand Down Expand Up @@ -659,9 +656,7 @@ static String constructTopicSwitchLog(
producerMetadata.messageTimestamp,
producerMetadata.logicalTimestamp,
leaderMetadata == null ? "-" : leaderMetadata.hostName,
leaderMetadata == null
? "-"
: getPubSubPositionString(pubSubPositionDeserializer, leaderMetadata.upstreamPubSubPosition),
leaderMetadata == null ? "-" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition),
Copy link

Copilot AI Oct 28, 2025

Choose a reason for hiding this comment

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

Potential NullPointerException when leaderMetadata.upstreamPubSubPosition is null. The toPosition method throws an IllegalArgumentException when passed a null ByteBuffer. Add a null check for upstreamPubSubPosition or use a nested ternary to handle this case: leaderMetadata == null || leaderMetadata.upstreamPubSubPosition == null ? \"-\" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition)

Suggested change
leaderMetadata == null ? "-" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition),
leaderMetadata == null || leaderMetadata.upstreamPubSubPosition == null ? "-" : pubSubPositionDeserializer.toPosition(leaderMetadata.upstreamPubSubPosition),

Copilot uses AI. Check for mistakes.
leaderMetadata == null ? "-" : leaderMetadata.upstreamKafkaClusterId);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
import com.linkedin.venice.pubsub.PubSubContext;
import com.linkedin.venice.pubsub.PubSubPositionDeserializer;
import com.linkedin.venice.pubsub.PubSubTopicRepository;
import com.linkedin.venice.pubsub.PubSubUtil;
import com.linkedin.venice.pubsub.api.PubSubPosition;
import com.linkedin.venice.pubsub.api.PubSubSymbolicPosition;
import com.linkedin.venice.pubsub.api.PubSubTopic;
Expand Down Expand Up @@ -128,7 +127,9 @@ public String getPreviousStatusesEntry(CharSequence key) {
}

public PubSubPosition getCheckpointedLocalVtPosition() {
return PubSubUtil.fromKafkaOffset(this.partitionState.offset);
return deserializePositionWithOffsetFallback(
this.partitionState.lastProcessedVersionTopicPubSubPosition,
this.partitionState.offset);
}

public void checkpointLocalVtPosition(PubSubPosition vtPosition) {
Expand All @@ -141,7 +142,9 @@ public void checkpointLocalVtPosition(PubSubPosition vtPosition) {
}

public PubSubPosition getCheckpointedRemoteVtPosition() {
return PubSubUtil.fromKafkaOffset(this.partitionState.upstreamVersionTopicOffset);
return deserializePositionWithOffsetFallback(
this.partitionState.upstreamVersionTopicPubSubPosition,
this.partitionState.upstreamVersionTopicOffset);
}

public void checkpointRemoteVtPosition(PubSubPosition remoteVtPosition) {
Expand Down Expand Up @@ -300,11 +303,12 @@ public PubSubTopic getLeaderTopic(PubSubTopicRepository pubSubTopicRepository) {
*/
public PubSubPosition getCheckpointedRtPosition(String pubSubBrokerAddress) {
Long offset = partitionState.upstreamOffsetMap.get(pubSubBrokerAddress);
ByteBuffer wfBuffer = partitionState.upstreamRealTimeTopicPubSubPositionMap.get(pubSubBrokerAddress);
if (offset == null) {
// If the offset is not set, return EARLIEST symbolic position.
return PubSubSymbolicPosition.EARLIEST;
}
return PubSubUtil.fromKafkaOffset(offset);
return deserializePositionWithOffsetFallback(wfBuffer, offset);
}

public void checkpointRtPosition(String pubSubBrokerAddress, PubSubPosition leaderPosition) {
Expand Down Expand Up @@ -333,8 +337,9 @@ public void cloneRtPositionCheckpoints(@Nonnull Map<String, PubSubPosition> chec
checkpointUpstreamPositionsReceiver.clear();
for (Map.Entry<String, Long> offsetEntry: partitionState.upstreamOffsetMap.entrySet()) {
String pubSubBrokerAddress = offsetEntry.getKey();
ByteBuffer wfBuffer = partitionState.upstreamRealTimeTopicPubSubPositionMap.get(pubSubBrokerAddress);
checkpointUpstreamPositionsReceiver
.put(pubSubBrokerAddress, PubSubUtil.fromKafkaOffset(offsetEntry.getValue()));
.put(pubSubBrokerAddress, deserializePositionWithOffsetFallback(wfBuffer, offsetEntry.getValue()));
}
}
}
Expand Down Expand Up @@ -523,7 +528,7 @@ PubSubPosition deserializePositionWithOffsetFallback(ByteBuffer wireFormatBytes,
final PubSubPosition position = pubSubPositionDeserializer.toPosition(wireFormatBytes);

// Guard against regressions: honor the caller-provided minimum offset.
if (offset > 0 && position.getNumericOffset() < offset) {
if (position.getNumericOffset() < offset) {
Copy link

Copilot AI Oct 28, 2025

Choose a reason for hiding this comment

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

Removing the offset > 0 check means that this condition will now fire even when offset is 0 or negative. This could be intentional for handling edge cases, but it changes the behavior for positions at offset 0. Consider documenting why the guard was removed or verifying that positions at offset 0 should be subject to this fallback logic.

Copilot uses AI. Check for mistakes.
LOGGER.info(
"Deserialized position: {} is behind the provided offset: {}. Using offset-based position.",
position.getNumericOffset(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -307,12 +307,4 @@ public static PubSubPosition parsePositionWireFormat(
throw new IllegalArgumentException("Invalid base64 encoded bytes in position wire format string", e);
}
}

public static String getPubSubPositionString(
PubSubPositionDeserializer pubSubPositionDeserializer,
ByteBuffer pubSubPosition) {
return (pubSubPosition == null || !pubSubPosition.hasRemaining())
? "<EMPTY>"
: pubSubPositionDeserializer.toPosition(pubSubPosition).toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -563,61 +563,4 @@ public void testParsePositionWireFormatInvalidInputs() {
() -> PubSubUtil.parsePositionWireFormat("1:invalid_base64!", deserializer));
expectThrows(IllegalArgumentException.class, () -> PubSubUtil.parsePositionWireFormat("1:", deserializer));
}

@Test
public void testGetPubSubPositionString() {
PubSubPositionDeserializer deserializer = PubSubPositionDeserializer.DEFAULT_DESERIALIZER;

// Case 1: Null ByteBuffer should return "<EMPTY>"
String result = PubSubUtil.getPubSubPositionString(deserializer, null);
assertEquals(result, "<EMPTY>", "Null ByteBuffer should return <EMPTY>");

// Case 2: Empty ByteBuffer (no remaining bytes) should return "<EMPTY>"
ByteBuffer emptyBuffer = ByteBuffer.allocate(0);
result = PubSubUtil.getPubSubPositionString(deserializer, emptyBuffer);
assertEquals(result, "<EMPTY>", "Empty ByteBuffer should return <EMPTY>");

// Case 3: ByteBuffer with position fully consumed (no remaining) should return "<EMPTY>"
ApacheKafkaOffsetPosition position = ApacheKafkaOffsetPosition.of(100L);
ByteBuffer consumedBuffer = position.toWireFormatBuffer();
// Consume all bytes by moving position to limit
consumedBuffer.position(consumedBuffer.limit());
result = PubSubUtil.getPubSubPositionString(deserializer, consumedBuffer);
assertEquals(result, "<EMPTY>", "ByteBuffer with no remaining bytes should return <EMPTY>");

// Case 4: Valid ByteBuffer with ApacheKafkaOffsetPosition data should deserialize and return toString()
ApacheKafkaOffsetPosition validPosition = ApacheKafkaOffsetPosition.of(12345L);
ByteBuffer validBuffer = validPosition.toWireFormatBuffer();
result = PubSubUtil.getPubSubPositionString(deserializer, validBuffer);
assertEquals(
result,
validPosition.toString(),
"Valid ByteBuffer should deserialize and return position's toString()");

// Case 5: Symbolic positions - EARLIEST
ByteBuffer earliestBuffer = PubSubSymbolicPosition.EARLIEST.toWireFormatBuffer();
result = PubSubUtil.getPubSubPositionString(deserializer, earliestBuffer);
assertEquals(
result,
PubSubSymbolicPosition.EARLIEST.toString(),
"Should correctly deserialize EARLIEST symbolic position");

// Case 6: Symbolic positions - LATEST
ByteBuffer latestBuffer = PubSubSymbolicPosition.LATEST.toWireFormatBuffer();
result = PubSubUtil.getPubSubPositionString(deserializer, latestBuffer);
assertEquals(
result,
PubSubSymbolicPosition.LATEST.toString(),
"Should correctly deserialize LATEST symbolic position");

// Case 7: ByteBuffer with partial position data followed by limit
ApacheKafkaOffsetPosition partialPosition = ApacheKafkaOffsetPosition.of(777L);
byte[] wireFormatBytes = partialPosition.toWireFormatBytes();
// Create a buffer with extra capacity but only set limit to actual data
ByteBuffer bufferWithLimit = ByteBuffer.allocate(wireFormatBytes.length + 10);
bufferWithLimit.put(wireFormatBytes);
bufferWithLimit.flip(); // Set limit to current position and reset position to 0
result = PubSubUtil.getPubSubPositionString(deserializer, bufferWithLimit);
assertEquals(result, partialPosition.toString(), "Should correctly handle ByteBuffer with limit set");
}
}
Loading