|
16 | 16 | import java.time.Duration; |
17 | 17 | import java.util.Map; |
18 | 18 | import java.util.Optional; |
| 19 | +import org.apache.kafka.common.header.Headers; |
19 | 20 | import org.apache.kafka.common.serialization.Serde; |
20 | | -import org.apache.kafka.streams.StreamsMetrics; |
| 21 | +import org.apache.kafka.common.utils.Bytes; |
21 | 22 | import org.apache.kafka.streams.processor.Cancellable; |
| 23 | +import org.apache.kafka.streams.processor.CommitCallback; |
22 | 24 | import org.apache.kafka.streams.processor.PunctuationType; |
23 | 25 | import org.apache.kafka.streams.processor.Punctuator; |
| 26 | +import org.apache.kafka.streams.processor.StateRestoreCallback; |
24 | 27 | import org.apache.kafka.streams.processor.StateStore; |
25 | 28 | import org.apache.kafka.streams.processor.TaskId; |
| 29 | +import org.apache.kafka.streams.processor.To; |
26 | 30 | import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; |
27 | 31 | import org.apache.kafka.streams.processor.api.FixedKeyRecord; |
28 | 32 | import org.apache.kafka.streams.processor.api.ProcessorContext; |
29 | 33 | import org.apache.kafka.streams.processor.api.Record; |
30 | 34 | import org.apache.kafka.streams.processor.api.RecordMetadata; |
| 35 | +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; |
| 36 | +import org.apache.kafka.streams.processor.internals.ProcessorMetadata; |
| 37 | +import org.apache.kafka.streams.processor.internals.ProcessorNode; |
| 38 | +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; |
| 39 | +import org.apache.kafka.streams.processor.internals.RecordCollector; |
| 40 | +import org.apache.kafka.streams.processor.internals.StreamTask; |
| 41 | +import org.apache.kafka.streams.processor.internals.Task; |
| 42 | +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; |
| 43 | +import org.apache.kafka.streams.query.Position; |
| 44 | +import org.apache.kafka.streams.state.StoreBuilder; |
| 45 | +import org.apache.kafka.streams.state.internals.ThreadCache; |
31 | 46 |
|
32 | 47 | /** |
33 | 48 | * Basic wrapper around a {@link ProcessorContext}/{@link FixedKeyProcessorContext} |
34 | 49 | * that just delegates to the underlying context. |
35 | 50 | */ |
36 | 51 | @SuppressWarnings({"checkstyle:linelength", "checkstyle:overloadmethodsdeclarationorder"}) |
37 | | -public abstract class DelegatingProcessorContext<KOut, VOut, D extends ProcessorContext<KOut, VOut> & FixedKeyProcessorContext<KOut, VOut>> |
| 52 | +public abstract class DelegatingProcessorContext<KOut, VOut, D |
| 53 | + extends InternalProcessorContext<KOut, VOut> & FixedKeyProcessorContext<KOut, VOut>> |
38 | 54 | implements MergedProcessorContext<KOut, VOut> { |
39 | 55 |
|
40 | 56 | public abstract D delegate(); |
@@ -70,7 +86,7 @@ public File stateDir() { |
70 | 86 | } |
71 | 87 |
|
72 | 88 | @Override |
73 | | - public StreamsMetrics metrics() { |
| 89 | + public StreamsMetricsImpl metrics() { |
74 | 90 | return delegate().metrics(); |
75 | 91 | } |
76 | 92 |
|
@@ -138,4 +154,161 @@ public <K extends KOut, V extends VOut> void forward( |
138 | 154 | ) { |
139 | 155 | delegate().forward(record, childName); |
140 | 156 | } |
| 157 | + |
| 158 | + @Override |
| 159 | + public <T extends StateStore> T getStateStore(final StoreBuilder<T> builder) { |
| 160 | + return delegate().getStateStore(builder); |
| 161 | + } |
| 162 | + |
| 163 | + @Override |
| 164 | + public void setSystemTimeMs(final long timeMs) { |
| 165 | + delegate().setSystemTimeMs(timeMs); |
| 166 | + } |
| 167 | + |
| 168 | + @Override |
| 169 | + public ProcessorRecordContext recordContext() { |
| 170 | + return delegate().recordContext(); |
| 171 | + } |
| 172 | + |
| 173 | + @Override |
| 174 | + public void setRecordContext(final ProcessorRecordContext recordContext) { |
| 175 | + delegate().setRecordContext(recordContext); |
| 176 | + } |
| 177 | + |
| 178 | + @Override |
| 179 | + public void setCurrentNode(final ProcessorNode<?, ?, ?, ?> currentNode) { |
| 180 | + delegate().setCurrentNode(currentNode); |
| 181 | + } |
| 182 | + |
| 183 | + @Override |
| 184 | + public ProcessorNode<?, ?, ?, ?> currentNode() { |
| 185 | + return delegate().currentNode(); |
| 186 | + } |
| 187 | + |
| 188 | + @Override |
| 189 | + public ThreadCache cache() { |
| 190 | + return delegate().cache(); |
| 191 | + } |
| 192 | + |
| 193 | + @Override |
| 194 | + public void initialize() { |
| 195 | + delegate().initialize(); |
| 196 | + } |
| 197 | + |
| 198 | + @Override |
| 199 | + public void uninitialize() { |
| 200 | + delegate().uninitialize(); |
| 201 | + } |
| 202 | + |
| 203 | + @Override |
| 204 | + public Task.TaskType taskType() { |
| 205 | + return delegate().taskType(); |
| 206 | + } |
| 207 | + |
| 208 | + @Override |
| 209 | + public void transitionToActive( |
| 210 | + final StreamTask streamTask, |
| 211 | + final RecordCollector recordCollector, |
| 212 | + final ThreadCache newCache |
| 213 | + ) { |
| 214 | + delegate().transitionToActive(streamTask, recordCollector, newCache); |
| 215 | + } |
| 216 | + |
| 217 | + @Override |
| 218 | + public void transitionToStandby(final ThreadCache newCache) { |
| 219 | + delegate().transitionToStandby(newCache); |
| 220 | + } |
| 221 | + |
| 222 | + @Override |
| 223 | + public void registerCacheFlushListener( |
| 224 | + final String namespace, |
| 225 | + final ThreadCache.DirtyEntryFlushListener listener |
| 226 | + ) { |
| 227 | + delegate().registerCacheFlushListener(namespace, listener); |
| 228 | + } |
| 229 | + |
| 230 | + @Override |
| 231 | + public void logChange( |
| 232 | + final String storeName, |
| 233 | + final Bytes key, |
| 234 | + final byte[] value, |
| 235 | + final long timestamp, |
| 236 | + final Position position |
| 237 | + ) { |
| 238 | + delegate().logChange(storeName, key, value, timestamp, position); |
| 239 | + } |
| 240 | + |
| 241 | + @Override |
| 242 | + public String changelogFor(final String storeName) { |
| 243 | + return delegate().changelogFor(storeName); |
| 244 | + } |
| 245 | + |
| 246 | + @Override |
| 247 | + public void addProcessorMetadataKeyValue(final String key, final long value) { |
| 248 | + delegate().addProcessorMetadataKeyValue(key, value); |
| 249 | + } |
| 250 | + |
| 251 | + @Override |
| 252 | + public Long processorMetadataForKey(final String key) { |
| 253 | + return delegate().processorMetadataForKey(key); |
| 254 | + } |
| 255 | + |
| 256 | + @Override |
| 257 | + public void setProcessorMetadata(final ProcessorMetadata metadata) { |
| 258 | + delegate().setProcessorMetadata(metadata); |
| 259 | + } |
| 260 | + |
| 261 | + @Override |
| 262 | + public ProcessorMetadata getProcessorMetadata() { |
| 263 | + return delegate().getProcessorMetadata(); |
| 264 | + } |
| 265 | + |
| 266 | + @Override |
| 267 | + public void register(final StateStore store, final StateRestoreCallback stateRestoreCallback) { |
| 268 | + delegate().register(store, stateRestoreCallback); |
| 269 | + } |
| 270 | + |
| 271 | + @Override |
| 272 | + public <K, V> void forward(final K key, final V value) { |
| 273 | + delegate().forward(key, value); |
| 274 | + } |
| 275 | + |
| 276 | + @Override |
| 277 | + public <K, V> void forward(final K key, final V value, final To to) { |
| 278 | + delegate().forward(key, value, to); |
| 279 | + } |
| 280 | + |
| 281 | + @Override |
| 282 | + public String topic() { |
| 283 | + return delegate().topic(); |
| 284 | + } |
| 285 | + |
| 286 | + @Override |
| 287 | + public int partition() { |
| 288 | + return delegate().partition(); |
| 289 | + } |
| 290 | + |
| 291 | + @Override |
| 292 | + public long offset() { |
| 293 | + return delegate().offset(); |
| 294 | + } |
| 295 | + |
| 296 | + @Override |
| 297 | + public Headers headers() { |
| 298 | + return delegate().headers(); |
| 299 | + } |
| 300 | + |
| 301 | + @Override |
| 302 | + public long timestamp() { |
| 303 | + return delegate().timestamp(); |
| 304 | + } |
| 305 | + |
| 306 | + @Override |
| 307 | + public void register( |
| 308 | + final StateStore store, |
| 309 | + final StateRestoreCallback stateRestoreCallback, |
| 310 | + final CommitCallback commitCallback |
| 311 | + ) { |
| 312 | + delegate().register(store, stateRestoreCallback, commitCallback); |
| 313 | + } |
141 | 314 | } |
0 commit comments