-
Notifications
You must be signed in to change notification settings - Fork 89
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
[compat] [server] [client] [test] Global RT DIV: Chunking Support #1385
base: main
Are you sure you want to change the base?
Changes from all commits
ae006b6
7b8754e
3f05108
c7d084f
ae605c1
20f333b
1f41050
fa893a8
317118f
c247f91
ecbdf7b
b0f5ef7
8c6e913
b69a883
d3ae029
6953f07
b27282d
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 |
---|---|---|
|
@@ -7,6 +7,8 @@ | |
import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.LEADER; | ||
import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.PAUSE_TRANSITION_FROM_STANDBY_TO_LEADER; | ||
import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.STANDBY; | ||
import static com.linkedin.davinci.validation.PartitionTracker.TopicType.REALTIME_TOPIC_TYPE; | ||
import static com.linkedin.davinci.validation.PartitionTracker.TopicType.VERSION_TOPIC_TYPE; | ||
import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; | ||
import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.START_OF_SEGMENT; | ||
import static com.linkedin.venice.pubsub.api.PubSubMessageHeaders.VENICE_LEADER_COMPLETION_STATE_HEADER; | ||
|
@@ -33,6 +35,7 @@ | |
import com.linkedin.davinci.store.view.VeniceViewWriter; | ||
import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; | ||
import com.linkedin.davinci.validation.PartitionTracker; | ||
import com.linkedin.davinci.validation.PartitionTracker.TopicType; | ||
import com.linkedin.venice.common.VeniceSystemStoreUtils; | ||
import com.linkedin.venice.compression.CompressionStrategy; | ||
import com.linkedin.venice.exceptions.VeniceException; | ||
|
@@ -1479,7 +1482,7 @@ protected void updateOffsetMetadataInOffsetRecord(PartitionConsumptionState part | |
upstreamTopic = versionTopic; | ||
} | ||
if (upstreamTopic.isRealTime()) { | ||
offsetRecord.resetUpstreamOffsetMap(partitionConsumptionState.getLatestProcessedUpstreamRTOffsetMap()); | ||
offsetRecord.mergeUpstreamOffsets(partitionConsumptionState.getLatestProcessedUpstreamRTOffsetMap()); | ||
} else { | ||
offsetRecord.setCheckpointUpstreamVersionTopicOffset( | ||
partitionConsumptionState.getLatestProcessedUpstreamVersionTopicOffset()); | ||
|
@@ -1619,7 +1622,6 @@ protected static void checkAndHandleUpstreamOffsetRewind( | |
int actualSchemaId = ByteUtils.readInt(actualValue, 0); | ||
Put put = (Put) envelope.payloadUnion; | ||
if (actualSchemaId == put.schemaId) { | ||
|
||
if (put.putValue.equals( | ||
ByteBuffer.wrap( | ||
actualValue, | ||
|
@@ -1904,6 +1906,10 @@ protected boolean shouldProcessRecord(PubSubMessage<KafkaKey, KafkaMessageEnvelo | |
} | ||
} | ||
} | ||
// Global RT DIV messages should be completely ignored when leader is consuming from remote version topic | ||
if (record.getKey().isGlobalRtDiv()) { | ||
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. Is it possible to have
|
||
return false; | ||
} | ||
} | ||
|
||
if (!Utils.resolveLeaderTopicFromPubSubTopic(pubSubTopicRepository, record.getTopicPartition().getPubSubTopic()) | ||
|
@@ -2281,18 +2287,17 @@ protected Iterable<PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long>> validate | |
Iterable<PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long>> records, | ||
String kafkaUrl, | ||
PubSubTopicPartition topicPartition) { | ||
PartitionConsumptionState partitionConsumptionState = | ||
partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); | ||
if (partitionConsumptionState == null) { | ||
final PartitionConsumptionState pcs = partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); | ||
if (pcs == null) { | ||
// The partition is likely unsubscribed, will skip these messages. | ||
LOGGER.warn( | ||
"No partition consumption state for store version: {}, partition:{}, will filter out all the messages", | ||
"No partition consumption state for store version: {}, partition: {}, will filter out all the messages", | ||
kafkaVersionTopic, | ||
topicPartition.getPartitionNumber()); | ||
return Collections.emptyList(); | ||
} | ||
boolean isEndOfPushReceived = partitionConsumptionState.isEndOfPushReceived(); | ||
if (!shouldProduceToVersionTopic(partitionConsumptionState)) { | ||
boolean isEndOfPushReceived = pcs.isEndOfPushReceived(); | ||
if (!shouldProduceToVersionTopic(pcs)) { | ||
return records; | ||
} | ||
/** | ||
|
@@ -2302,30 +2307,15 @@ protected Iterable<PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long>> validate | |
Iterator<PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long>> iter = records.iterator(); | ||
while (iter.hasNext()) { | ||
PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> record = iter.next(); | ||
boolean isRealTimeMsg = record.getTopicPartition().getPubSubTopic().isRealTime(); | ||
boolean isRealTimeTopic = record.getTopicPartition().getPubSubTopic().isRealTime(); | ||
try { | ||
/** | ||
* TODO: An improvement can be made to fail all future versions for fatal DIV exceptions after EOP. | ||
*/ | ||
if (!isGlobalRtDivEnabled) { | ||
validateMessage( | ||
PartitionTracker.VERSION_TOPIC, | ||
this.kafkaDataIntegrityValidatorForLeaders, | ||
record, | ||
isEndOfPushReceived, | ||
partitionConsumptionState); | ||
} else { | ||
validateMessage( | ||
PartitionTracker.TopicType.of( | ||
isRealTimeMsg | ||
? PartitionTracker.TopicType.REALTIME_TOPIC_TYPE | ||
: PartitionTracker.TopicType.VERSION_TOPIC_TYPE, | ||
kafkaUrl), | ||
this.kafkaDataIntegrityValidatorForLeaders, | ||
record, | ||
isEndOfPushReceived, | ||
partitionConsumptionState); | ||
} | ||
final TopicType topicType = (isGlobalRtDivEnabled) | ||
? TopicType.of(isRealTimeTopic ? REALTIME_TOPIC_TYPE : VERSION_TOPIC_TYPE, kafkaUrl) | ||
: PartitionTracker.VERSION_TOPIC; | ||
validateMessage(topicType, kafkaDataIntegrityValidatorForLeaders, record, isEndOfPushReceived, pcs); | ||
versionedDIVStats.recordSuccessMsg(storeName, versionNumber); | ||
} catch (FatalDataValidationException e) { | ||
if (!isEndOfPushReceived) { | ||
|
@@ -2342,7 +2332,7 @@ protected Iterable<PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long>> validate | |
"Skipping a duplicate record from: {} offset: {} for replica: {}", | ||
record.getTopicPartition(), | ||
record.getOffset(), | ||
partitionConsumptionState.getReplicaId()); | ||
pcs.getReplicaId()); | ||
iter.remove(); | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -12,6 +12,7 @@ | |
import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; | ||
import static com.linkedin.venice.LogMessages.KILLED_JOB_MESSAGE; | ||
import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.START_OF_SEGMENT; | ||
import static com.linkedin.venice.serialization.avro.AvroProtocolDefinition.GLOBAL_RT_DIV_STATE; | ||
import static com.linkedin.venice.utils.Utils.FATAL_DATA_VALIDATION_ERROR; | ||
import static com.linkedin.venice.utils.Utils.getReplicaId; | ||
import static java.util.Comparator.comparingInt; | ||
|
@@ -48,6 +49,7 @@ | |
import com.linkedin.venice.common.VeniceSystemStoreType; | ||
import com.linkedin.venice.common.VeniceSystemStoreUtils; | ||
import com.linkedin.venice.compression.CompressionStrategy; | ||
import com.linkedin.venice.compression.NoopCompressor; | ||
import com.linkedin.venice.compression.VeniceCompressor; | ||
import com.linkedin.venice.exceptions.DiskLimitExhaustedException; | ||
import com.linkedin.venice.exceptions.MemoryLimitExhaustedException; | ||
|
@@ -184,6 +186,8 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { | |
private static final int MAX_KILL_CHECKING_ATTEMPTS = 10; | ||
private static final int CHUNK_MANIFEST_SCHEMA_ID = | ||
AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion(); | ||
private static final int GLOBAL_RT_DIV_STATE_SCHEMA_ID = | ||
AvroProtocolDefinition.GLOBAL_RT_DIV_STATE.getCurrentProtocolVersion(); | ||
|
||
protected static final RedundantExceptionFilter REDUNDANT_LOGGING_FILTER = | ||
RedundantExceptionFilter.getRedundantExceptionFilter(); | ||
|
@@ -317,6 +321,7 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { | |
protected final IngestionNotificationDispatcher ingestionNotificationDispatcher; | ||
|
||
protected final ChunkAssembler chunkAssembler; | ||
protected final ChunkAssembler divChunkAssembler; | ||
private final Optional<ObjectCacheBackend> cacheBackend; | ||
private DaVinciRecordTransformer recordTransformer; | ||
|
||
|
@@ -467,6 +472,8 @@ public StoreIngestionTask( | |
new IngestionNotificationDispatcher(notifiers, kafkaVersionTopic, isCurrentVersion); | ||
this.missingSOPCheckExecutor.execute(() -> waitForStateVersion(kafkaVersionTopic)); | ||
this.chunkAssembler = new ChunkAssembler(storeName); | ||
this.divChunkAssembler = | ||
builder.getDivChunkAssembler() != null ? builder.getDivChunkAssembler() : new ChunkAssembler(storeName); | ||
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. It seems |
||
this.cacheBackend = cacheBackend; | ||
|
||
if (recordTransformerFunction != null) { | ||
|
@@ -1145,6 +1152,12 @@ private int handleSingleMessage( | |
record.getTopicPartition().getPartitionNumber(), | ||
partitionConsumptionStateMap.get(topicPartition.getPartitionNumber())); | ||
} | ||
} else if (record.getKey().isGlobalRtDiv()) { | ||
// TODO: This is a placeholder for the actual implementation. | ||
if (isGlobalRtDivEnabled) { | ||
processGlobalRtDivMessage(record); // This is a global realtime topic data integrity validator snapshot | ||
} | ||
return 0; | ||
} | ||
|
||
// This function may modify the original record in KME and it is unsafe to use the payload from KME directly after | ||
|
@@ -1189,6 +1202,28 @@ private int handleSingleMessage( | |
return record.getPayloadSize(); | ||
} | ||
|
||
void processGlobalRtDivMessage(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> record) { | ||
KafkaKey key = record.getKey(); | ||
KafkaMessageEnvelope value = record.getValue(); | ||
Put put = (Put) value.getPayloadUnion(); | ||
|
||
Object assembledObject = divChunkAssembler.bufferAndAssembleRecord( | ||
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. I saw this function is using in-memory storage engine as a temp storage, which will increase the memory usage, which is different from the regular data chunk handling, which would minimize the memory usage, and I think this is also what we agreed during the design review. |
||
record.getTopicPartition(), | ||
put.getSchemaId(), | ||
key.getKey(), | ||
put.getPutValue(), | ||
record.getOffset(), | ||
put.getSchemaId(), | ||
new NoopCompressor(), | ||
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. Shall we always use a certain compression algo for |
||
(valueBytes) -> GLOBAL_RT_DIV_STATE.getSerializer() | ||
.deserialize(ByteUtils.extractByteArray(valueBytes), GLOBAL_RT_DIV_STATE_SCHEMA_ID)); | ||
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. Shall we update |
||
|
||
if (assembledObject == null) { | ||
return; // the message value only contained one data chunk, so the Global RT DIV cannot yet be fully assembled | ||
} | ||
// TODO: We will add the code to process Global RT DIV message later in here. | ||
} | ||
|
||
/** | ||
* This function is in charge of producing the consumer records to the writer buffers maintained by {@link StoreBufferService}. | ||
* | ||
|
@@ -2397,6 +2432,12 @@ protected boolean shouldProcessRecord(PubSubMessage<KafkaKey, KafkaMessageEnvelo | |
return false; | ||
} | ||
|
||
// Global RT DIV messages should only be in version topics, not realtime topics. Skip it and log a warning. | ||
if (record.getKey().isGlobalRtDiv() && record.getTopic().isRealTime()) { | ||
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. What are the scenarios that the RT topic would contain |
||
LOGGER.warn("Skipping Global RT DIV message from realtime topic partition: {}", record.getTopicPartition()); | ||
return false; | ||
} | ||
|
||
if (partitionConsumptionState.isEndOfPushReceived() && partitionConsumptionState.isBatchOnly()) { | ||
KafkaKey key = record.getKey(); | ||
KafkaMessageEnvelope value = record.getValue(); | ||
|
@@ -3786,7 +3827,7 @@ private int processKafkaDataMessage( | |
private void waitReadyToProcessRecord(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> record) | ||
throws InterruptedException { | ||
KafkaMessageEnvelope kafkaValue = record.getValue(); | ||
if (record.getKey().isControlMessage() || kafkaValue == null) { | ||
if (record.getKey().isControlMessage() || record.getKey().isGlobalRtDiv() || kafkaValue == null) { | ||
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. Can |
||
return; | ||
} | ||
|
||
|
@@ -4561,4 +4602,8 @@ void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { | |
protected boolean isDaVinciClient() { | ||
return isDaVinciClient; | ||
} | ||
|
||
ChunkAssembler getDivChunkAssembler() { | ||
return this.divChunkAssembler; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -14,6 +14,7 @@ | |
import com.linkedin.davinci.storage.StorageService; | ||
import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; | ||
import com.linkedin.davinci.store.view.VeniceViewWriterFactory; | ||
import com.linkedin.davinci.utils.ChunkAssembler; | ||
import com.linkedin.venice.kafka.protocol.state.PartitionState; | ||
import com.linkedin.venice.meta.ReadOnlySchemaRepository; | ||
import com.linkedin.venice.meta.ReadOnlyStoreRepository; | ||
|
@@ -126,6 +127,7 @@ public static class Builder { | |
private PubSubTopicRepository pubSubTopicRepository; | ||
private Runnable runnableForKillIngestionTasksForNonCurrentVersions; | ||
private ExecutorService aaWCWorkLoadProcessingThreadPool; | ||
private ChunkAssembler divChunkAssembler; | ||
|
||
private interface Setter { | ||
void apply(); | ||
|
@@ -336,5 +338,13 @@ public Builder setAAWCWorkLoadProcessingThreadPool(ExecutorService executorServi | |
public ExecutorService getAAWCWorkLoadProcessingThreadPool() { | ||
return this.aaWCWorkLoadProcessingThreadPool; | ||
} | ||
|
||
public Builder setDivChunkAssembler(ChunkAssembler divChunkAssembler) { | ||
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.
|
||
return set(() -> this.divChunkAssembler = divChunkAssembler); | ||
} | ||
|
||
public ChunkAssembler getDivChunkAssembler() { | ||
return divChunkAssembler; | ||
} | ||
} | ||
} |
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.
Why do you do this change? Personally I think this makes less explicit about the offset map we are tracking
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.
Because I feel that
reset()
is not the right word for it. It's not clearing the state or replacing the map entirely. It's updating the existing map with the values of the new map and retaining the values missing from the new map.I can rename it to
updateUpstreamOffsets()
.