16
16
*/
17
17
package org .apache .kafka .streams .processor .internals ;
18
18
19
+ import static java .nio .charset .StandardCharsets .UTF_8 ;
20
+ import static java .util .Objects .requireNonNull ;
21
+ import static org .apache .kafka .common .utils .Utils .getNullableSizePrefixedArray ;
22
+ import static org .apache .kafka .common .utils .Utils .getOptionalField ;
23
+
24
+ import java .nio .ByteBuffer ;
25
+ import java .util .Arrays ;
26
+ import java .util .Objects ;
19
27
import org .apache .kafka .common .header .Header ;
20
28
import org .apache .kafka .common .header .Headers ;
21
29
import org .apache .kafka .common .header .internals .RecordHeader ;
22
30
import org .apache .kafka .common .header .internals .RecordHeaders ;
23
31
import org .apache .kafka .streams .processor .RecordContext ;
24
32
import org .apache .kafka .streams .processor .api .RecordMetadata ;
25
33
26
- import java .nio .ByteBuffer ;
27
- import java .util .Objects ;
28
-
29
- import static java .nio .charset .StandardCharsets .UTF_8 ;
30
- import static java .util .Objects .requireNonNull ;
31
- import static org .apache .kafka .common .utils .Utils .getNullableSizePrefixedArray ;
32
-
33
34
public class ProcessorRecordContext implements RecordContext , RecordMetadata {
34
35
35
36
private final long timestamp ;
36
37
private final long offset ;
37
38
private final String topic ;
38
39
private final int partition ;
39
40
private final Headers headers ;
41
+ private byte [] sourceRawKey ;
42
+ private byte [] sourceRawValue ;
40
43
41
44
public ProcessorRecordContext (final long timestamp ,
42
45
final long offset ,
@@ -48,6 +51,24 @@ public ProcessorRecordContext(final long timestamp,
48
51
this .topic = topic ;
49
52
this .partition = partition ;
50
53
this .headers = Objects .requireNonNull (headers );
54
+ this .sourceRawKey = null ;
55
+ this .sourceRawValue = null ;
56
+ }
57
+
58
+ public ProcessorRecordContext (final long timestamp ,
59
+ final long offset ,
60
+ final int partition ,
61
+ final String topic ,
62
+ final Headers headers ,
63
+ final byte [] sourceRawKey ,
64
+ final byte [] sourceRawValue ) {
65
+ this .timestamp = timestamp ;
66
+ this .offset = offset ;
67
+ this .topic = topic ;
68
+ this .partition = partition ;
69
+ this .headers = Objects .requireNonNull (headers );
70
+ this .sourceRawKey = sourceRawKey ;
71
+ this .sourceRawValue = sourceRawValue ;
51
72
}
52
73
53
74
@ Override
@@ -75,6 +96,24 @@ public Headers headers() {
75
96
return headers ;
76
97
}
77
98
99
+ @ Override
100
+ public byte [] sourceRawKey () {
101
+ return sourceRawKey ;
102
+ }
103
+
104
+ @ Override
105
+ public byte [] sourceRawValue () {
106
+ return sourceRawValue ;
107
+ }
108
+
109
+ public void setSourceRawKey (final byte [] sourceRawKey ) {
110
+ this .sourceRawKey = sourceRawKey ;
111
+ }
112
+
113
+ public void setSourceRawValue (final byte [] sourceRawValue ) {
114
+ this .sourceRawValue = sourceRawValue ;
115
+ }
116
+
78
117
public long residentMemorySizeEstimate () {
79
118
long size = 0 ;
80
119
size += Long .BYTES ; // value.context.timestamp
@@ -124,6 +163,18 @@ public byte[] serialize() {
124
163
headerValuesBytes [i ] = valueBytes ;
125
164
}
126
165
166
+ if (sourceRawKey != null ) {
167
+ size += Character .BYTES ; // marker for sourceRawKey being present
168
+ size += Integer .BYTES ; // size of sourceRawKey
169
+ size += sourceRawKey .length ;
170
+ }
171
+
172
+ if (sourceRawValue != null ) {
173
+ size += Character .BYTES ; // marker for sourceRawValue being present
174
+ size += Integer .BYTES ; // size of sourceRawValue
175
+ size += sourceRawValue .length ;
176
+ }
177
+
127
178
final ByteBuffer buffer = ByteBuffer .allocate (size );
128
179
buffer .putLong (timestamp );
129
180
buffer .putLong (offset );
@@ -146,6 +197,18 @@ public byte[] serialize() {
146
197
}
147
198
}
148
199
200
+ if (sourceRawKey != null ) {
201
+ buffer .putChar ('k' );
202
+ buffer .putInt (sourceRawKey .length );
203
+ buffer .put (sourceRawKey );
204
+ }
205
+
206
+ if (sourceRawValue != null ) {
207
+ buffer .putChar ('v' );
208
+ buffer .putInt (sourceRawValue .length );
209
+ buffer .put (sourceRawValue );
210
+ }
211
+
149
212
return buffer .array ();
150
213
}
151
214
@@ -173,7 +236,15 @@ public static ProcessorRecordContext deserialize(final ByteBuffer buffer) {
173
236
headers = new RecordHeaders (headerArr );
174
237
}
175
238
176
- return new ProcessorRecordContext (timestamp , offset , partition , topic , headers );
239
+ final byte [] rawKey = getOptionalField ('k' , buffer ).orElse (null );
240
+ final byte [] rawValue = getOptionalField ('v' , buffer ).orElse (null );
241
+
242
+ return new ProcessorRecordContext (timestamp , offset , partition , topic , headers , rawKey , rawValue );
243
+ }
244
+
245
+ public void freeRawRecord () {
246
+ this .sourceRawKey = null ;
247
+ this .sourceRawValue = null ;
177
248
}
178
249
179
250
@ Override
@@ -189,7 +260,9 @@ public boolean equals(final Object o) {
189
260
offset == that .offset &&
190
261
partition == that .partition &&
191
262
Objects .equals (topic , that .topic ) &&
192
- Objects .equals (headers , that .headers );
263
+ Objects .equals (headers , that .headers ) &&
264
+ Arrays .equals (sourceRawKey , that .sourceRawKey ) &&
265
+ Arrays .equals (sourceRawValue , that .sourceRawValue );
193
266
}
194
267
195
268
/**
0 commit comments