-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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
KAFKA-16505: Fix source raw key and value in store caches #18739
base: trunk
Are you sure you want to change the base?
Changes from all commits
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 |
---|---|---|
|
@@ -259,6 +259,10 @@ public <K, V> void send(final String topic, | |
|
||
final ProducerRecord<byte[], byte[]> serializedRecord = new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes, headers); | ||
|
||
// As many records could be in-flight, | ||
// freeing raw records in the context to reduce memory pressure | ||
freeContext(context); | ||
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. The name of this method is a bit misleading. It basically frees the raw record within the context, not the whole context. What about calling it |
||
|
||
streamsProducer.send(serializedRecord, (metadata, exception) -> { | ||
try { | ||
// if there's already an exception record, skip logging offsets or new exceptions | ||
|
@@ -311,6 +315,12 @@ public <K, V> void send(final String topic, | |
}); | ||
} | ||
|
||
private static void freeContext(final InternalProcessorContext<Void, Void> context) { | ||
if (context != null && context.recordContext() != null) { | ||
context.recordContext().freeRawRecord(); | ||
} | ||
} | ||
|
||
private <K, V> void handleException(final ProductionExceptionHandler.SerializationExceptionOrigin origin, | ||
final String topic, | ||
final K key, | ||
|
@@ -388,7 +398,9 @@ private DefaultErrorHandlerContext errorHandlerContext(final InternalProcessorCo | |
recordContext.headers(), | ||
processorNodeId, | ||
taskId, | ||
recordContext.timestamp() | ||
recordContext.timestamp(), | ||
context.recordContext().sourceRawKey(), | ||
context.recordContext().sourceRawValue() | ||
) : | ||
new DefaultErrorHandlerContext( | ||
context, | ||
|
@@ -398,7 +410,9 @@ private DefaultErrorHandlerContext errorHandlerContext(final InternalProcessorCo | |
new RecordHeaders(), | ||
processorNodeId, | ||
taskId, | ||
-1L | ||
-1L, | ||
null, | ||
null | ||
); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,8 +23,22 @@ | |
|
||
public class StampedRecord extends Stamped<ConsumerRecord<?, ?>> { | ||
|
||
private final byte[] rawKey; | ||
private final byte[] rawValue; | ||
|
||
public StampedRecord(final ConsumerRecord<?, ?> record, final long timestamp) { | ||
super(record, timestamp); | ||
this.rawKey = null; | ||
this.rawValue = null; | ||
} | ||
|
||
public StampedRecord(final ConsumerRecord<?, ?> record, | ||
final long timestamp, | ||
final byte[] rawKey, | ||
final byte[] rawValue) { | ||
super(record, timestamp); | ||
this.rawKey = rawKey; | ||
this.rawValue = rawValue; | ||
} | ||
|
||
public String topic() { | ||
|
@@ -55,8 +69,26 @@ public Headers headers() { | |
return value.headers(); | ||
} | ||
|
||
public byte[] rawKey() { | ||
return rawKey; | ||
} | ||
|
||
public byte[] rawValue() { | ||
return rawValue; | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return value.toString() + ", timestamp = " + timestamp; | ||
} | ||
|
||
@Override | ||
public boolean equals(final Object other) { | ||
return super.equals(other); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return super.hashCode(); | ||
} | ||
Comment on lines
+84
to
+93
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. Why are those needed? |
||
} |
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.
You also need to add these info to the
serialize()
anddeserialize()
so that the buffer values also get the source record. Here it gets a bit tricky, because you need to consider the case where a serialized record context does not contain the source record because it was written by a version of Streams that has not yet had the source record in the context.